From d41375048df0050fae5d15f44b165a77ce896f1e Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Mon, 7 Oct 2019 12:08:53 +0800 Subject: [PATCH 001/746] [FLINK-14310][runtime] Get ExecutionVertexID from ExecutionVertex rather than creating new instances --- .../failover/AdaptedRestartPipelinedRegionStrategyNG.java | 2 +- .../failover/adapter/DefaultFailoverTopology.java | 3 +-- .../ExecutionGraphToInputsLocationsRetrieverAdapter.java | 2 +- .../adapter/ExecutionGraphToSchedulingTopologyAdapter.java | 2 +- .../ExecutionGraphToSchedulingTopologyAdapterTest.java | 4 ++-- 5 files changed, 6 insertions(+), 7 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java index 164cd91e8a06..eb43a2c35499 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java @@ -268,7 +268,7 @@ private ExecutionVertex getExecutionVertex(final ExecutionVertexID vertexID) { } private ExecutionVertexID getExecutionVertexID(final ExecutionVertex vertex) { - return new ExecutionVertexID(vertex.getJobvertexId(), vertex.getParallelSubtaskIndex()); + return vertex.getID(); } private List sortVerticesTopologically(final Set vertices) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverTopology.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverTopology.java index 7f832f0bfb85..e27d2134276d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverTopology.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverTopology.java @@ -24,7 +24,6 @@ import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverTopology; import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverVertex; -import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; import java.util.ArrayList; import java.util.Collection; @@ -56,7 +55,7 @@ public DefaultFailoverTopology(ExecutionGraph executionGraph) { final Map failoverVertexMap = new IdentityHashMap<>(); for (ExecutionVertex vertex : executionGraph.getAllExecutionVertices()) { final DefaultFailoverVertex failoverVertex = new DefaultFailoverVertex( - new ExecutionVertexID(vertex.getJobvertexId(), vertex.getParallelSubtaskIndex()), + vertex.getID(), vertex.getTaskNameWithSubtaskIndex()); this.failoverVertices.add(failoverVertex); failoverVertexMap.put(vertex, failoverVertex); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionGraphToInputsLocationsRetrieverAdapter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionGraphToInputsLocationsRetrieverAdapter.java index 392f00e0309c..96f75459d60f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionGraphToInputsLocationsRetrieverAdapter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionGraphToInputsLocationsRetrieverAdapter.java @@ -57,7 +57,7 @@ public Collection> getConsumedResultPartitionsProd List producers = new ArrayList<>(inputEdges.length); for (ExecutionEdge inputEdge : inputEdges) { ExecutionVertex producer = inputEdge.getSource().getProducer(); - producers.add(new ExecutionVertexID(producer.getJobvertexId(), producer.getParallelSubtaskIndex())); + producers.add(producer.getID()); } resultPartitionProducers.add(producers); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/ExecutionGraphToSchedulingTopologyAdapter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/ExecutionGraphToSchedulingTopologyAdapter.java index 9b377d82b18e..94c835c5a578 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/ExecutionGraphToSchedulingTopologyAdapter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/ExecutionGraphToSchedulingTopologyAdapter.java @@ -107,7 +107,7 @@ private static DefaultSchedulingExecutionVertex generateSchedulingExecutionVerte List producedPartitions) { DefaultSchedulingExecutionVertex schedulingVertex = new DefaultSchedulingExecutionVertex( - new ExecutionVertexID(vertex.getJobvertexId(), vertex.getParallelSubtaskIndex()), + vertex.getID(), producedPartitions, new ExecutionStateSupplier(vertex), vertex.getInputDependencyConstraint()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/ExecutionGraphToSchedulingTopologyAdapterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/ExecutionGraphToSchedulingTopologyAdapterTest.java index 5a925dae462c..ab72e2c209f3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/ExecutionGraphToSchedulingTopologyAdapterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/ExecutionGraphToSchedulingTopologyAdapterTest.java @@ -181,7 +181,7 @@ private static void assertPartitionsEquals( // since deep equality is verified later in the main loop // this DOES rely on an implicit assumption that the vertices objects returned by the topology are // identical to those stored in the partition - ExecutionVertexID originalId = new ExecutionVertexID(originalConsumer.getJobvertexId(), originalConsumer.getParallelSubtaskIndex()); + ExecutionVertexID originalId = originalConsumer.getID(); assertTrue(adaptedConsumers.stream().anyMatch(adaptedConsumer -> adaptedConsumer.getId().equals(originalId))); } } @@ -203,7 +203,7 @@ private static void assertVertexEquals( ExecutionVertex originalVertex, SchedulingExecutionVertex adaptedVertex) { assertEquals( - new ExecutionVertexID(originalVertex.getJobvertexId(), originalVertex.getParallelSubtaskIndex()), + originalVertex.getID(), adaptedVertex.getId()); assertEquals(originalVertex.getInputDependencyConstraint(), adaptedVertex.getInputDependencyConstraint()); } From 93833b34a7c79f866e605b74a4d7f98a965c0e80 Mon Sep 17 00:00:00 2001 From: tianchen92 Date: Tue, 8 Oct 2019 01:47:52 +0800 Subject: [PATCH 002/746] [FLINK-10437][rest] Mark WebOptions#ADDRESS as fallback key --- .../main/java/org/apache/flink/configuration/RestOptions.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java index 6c718afeca8c..733e5bf6f1b6 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java @@ -38,7 +38,8 @@ public class RestOptions { public static final ConfigOption BIND_ADDRESS = key("rest.bind-address") .noDefaultValue() - .withDeprecatedKeys(WebOptions.ADDRESS.key(), ConfigConstants.DEFAULT_JOB_MANAGER_WEB_FRONTEND_ADDRESS.key()) + .withFallbackKeys(WebOptions.ADDRESS.key()) + .withDeprecatedKeys(ConfigConstants.DEFAULT_JOB_MANAGER_WEB_FRONTEND_ADDRESS.key()) .withDescription("The address that the server binds itself."); /** From 2e3e27b14be22083c159fea788355698a0338864 Mon Sep 17 00:00:00 2001 From: Alberto Romero Date: Mon, 7 Oct 2019 18:56:52 +0100 Subject: [PATCH 003/746] [FLINK-13524][es][docs] Fix typo --- docs/dev/connectors/elasticsearch.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/dev/connectors/elasticsearch.md b/docs/dev/connectors/elasticsearch.md index c306bdbe94fe..45b041959a73 100644 --- a/docs/dev/connectors/elasticsearch.md +++ b/docs/dev/connectors/elasticsearch.md @@ -247,7 +247,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 = { From 76763da007cec27ba43396fc2df28e0d6aaf4c37 Mon Sep 17 00:00:00 2001 From: shining-huang <2440239821@qq.com> Date: Tue, 8 Oct 2019 02:03:43 +0800 Subject: [PATCH 004/746] [hotfix][table][docs] Add missing braces --- docs/dev/table/sourceSinks.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/dev/table/sourceSinks.md b/docs/dev/table/sourceSinks.md index 12b4ca73161e..01ebecaea3e5 100644 --- a/docs/dev/table/sourceSinks.md +++ b/docs/dev/table/sourceSinks.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`: From dd365de286f02f59c213f7bcf48100a9040dccad Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 29 Sep 2019 17:07:25 +0200 Subject: [PATCH 005/746] [hotfix] Remove unnecessary generics from DispatcherResourceManagerComponent --- .../entrypoint/StandaloneJobClusterEntryPoint.java | 2 +- .../entrypoint/MesosJobClusterEntrypoint.java | 2 +- .../entrypoint/MesosSessionClusterEntrypoint.java | 2 +- .../runtime/entrypoint/ClusterEntrypoint.java | 6 +++--- .../StandaloneSessionClusterEntrypoint.java | 2 +- ...tDispatcherResourceManagerComponentFactory.java | 6 +++--- .../DispatcherResourceManagerComponent.java | 8 ++++---- .../DispatcherResourceManagerComponentFactory.java | 5 ++--- .../JobDispatcherResourceManagerComponent.java | 2 +- ...bDispatcherResourceManagerComponentFactory.java | 2 +- .../SessionDispatcherResourceManagerComponent.java | 2 +- ...nDispatcherResourceManagerComponentFactory.java | 2 +- .../flink/runtime/minicluster/MiniCluster.java | 8 ++++---- .../runtime/minicluster/TestingMiniCluster.java | 6 +++--- .../recovery/ProcessFailureCancelingITCase.java | 2 +- .../ZooKeeperLeaderElectionITCase.java | 14 +++++++------- .../yarn/entrypoint/YarnJobClusterEntrypoint.java | 2 +- .../entrypoint/YarnSessionClusterEntrypoint.java | 2 +- 18 files changed, 37 insertions(+), 38 deletions(-) diff --git a/flink-container/src/main/java/org/apache/flink/container/entrypoint/StandaloneJobClusterEntryPoint.java b/flink-container/src/main/java/org/apache/flink/container/entrypoint/StandaloneJobClusterEntryPoint.java index 6001ec6fdb71..fbaba809f5da 100644 --- a/flink-container/src/main/java/org/apache/flink/container/entrypoint/StandaloneJobClusterEntryPoint.java +++ b/flink-container/src/main/java/org/apache/flink/container/entrypoint/StandaloneJobClusterEntryPoint.java @@ -74,7 +74,7 @@ private StandaloneJobClusterEntryPoint( } @Override - protected DispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration) { + protected DispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration) { return new JobDispatcherResourceManagerComponentFactory( StandaloneResourceManagerFactory.INSTANCE, new ClassPathJobGraphRetriever(jobId, savepointRestoreSettings, programArguments, jobClassName)); diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java index 377b5b5971ab..8bbd6c6f2264 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java @@ -109,7 +109,7 @@ protected CompletableFuture stopClusterServices(boolean cleanupHaData) { } @Override - protected DispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration) { + protected DispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration) { return new JobDispatcherResourceManagerComponentFactory( new MesosResourceManagerFactory( mesosServices, diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java index 98796282cf20..afd53c816815 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java @@ -108,7 +108,7 @@ protected CompletableFuture stopClusterServices(boolean cleanupHaData) { } @Override - protected DispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration) { + protected DispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration) { return new SessionDispatcherResourceManagerComponentFactory( new MesosResourceManagerFactory( mesosServices, 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 7f48fa02a298..a50cc1be902a 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 @@ -118,7 +118,7 @@ public abstract class ClusterEntrypoint implements AutoCloseableAsync, FatalErro private final AtomicBoolean isShutDown = new AtomicBoolean(false); @GuardedBy("lock") - private DispatcherResourceManagerComponent clusterComponent; + private DispatcherResourceManagerComponent clusterComponent; @GuardedBy("lock") private MetricRegistryImpl metricRegistry; @@ -210,7 +210,7 @@ private void runCluster(Configuration configuration) throws Exception { configuration.setString(JobManagerOptions.ADDRESS, commonRpcService.getAddress()); configuration.setInteger(JobManagerOptions.PORT, commonRpcService.getPort()); - final DispatcherResourceManagerComponentFactory dispatcherResourceManagerComponentFactory = createDispatcherResourceManagerComponentFactory(configuration); + final DispatcherResourceManagerComponentFactory dispatcherResourceManagerComponentFactory = createDispatcherResourceManagerComponentFactory(configuration); clusterComponent = dispatcherResourceManagerComponentFactory.create( configuration, @@ -475,7 +475,7 @@ private void cleanupDirectories() throws IOException { // Abstract methods // -------------------------------------------------- - protected abstract DispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration); + protected abstract DispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration); protected abstract ArchivedExecutionGraphStore createSerializableExecutionGraphStore( Configuration configuration, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/StandaloneSessionClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/StandaloneSessionClusterEntrypoint.java index 127fc8b98312..1918a516d41f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/StandaloneSessionClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/StandaloneSessionClusterEntrypoint.java @@ -37,7 +37,7 @@ public StandaloneSessionClusterEntrypoint(Configuration configuration) { } @Override - protected DispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration) { + protected DispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration) { return new SessionDispatcherResourceManagerComponentFactory(StandaloneResourceManagerFactory.INSTANCE); } 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 ca3daf3a3bed..f7722c3ee692 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 @@ -75,7 +75,7 @@ * @param type of the {@link Dispatcher} * @param type of the {@link RestfulGateway} given to the {@link WebMonitorEndpoint} */ -public abstract class AbstractDispatcherResourceManagerComponentFactory implements DispatcherResourceManagerComponentFactory { +public abstract class AbstractDispatcherResourceManagerComponentFactory implements DispatcherResourceManagerComponentFactory { private final Logger log = LoggerFactory.getLogger(getClass()); @@ -98,7 +98,7 @@ public AbstractDispatcherResourceManagerComponentFactory( } @Override - public DispatcherResourceManagerComponent create( + public DispatcherResourceManagerComponent create( Configuration configuration, RpcService rpcService, HighAvailabilityServices highAvailabilityServices, @@ -266,7 +266,7 @@ public DispatcherResourceManagerComponent create( } } - protected abstract DispatcherResourceManagerComponent createDispatcherResourceManagerComponent( + protected abstract DispatcherResourceManagerComponent createDispatcherResourceManagerComponent( T dispatcher, ResourceManager resourceManager, LeaderRetrievalService dispatcherLeaderRetrievalService, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponent.java index 022e59b65fa7..581423e690e1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponent.java @@ -40,10 +40,10 @@ * Component which starts a {@link Dispatcher}, {@link ResourceManager} and {@link WebMonitorEndpoint} * in the same process. */ -public class DispatcherResourceManagerComponent implements AutoCloseableAsync { +public class DispatcherResourceManagerComponent implements AutoCloseableAsync { @Nonnull - private final T dispatcher; + private final Dispatcher dispatcher; @Nonnull private final ResourceManager resourceManager; @@ -64,7 +64,7 @@ public class DispatcherResourceManagerComponent implements private final AtomicBoolean isRunning = new AtomicBoolean(true); DispatcherResourceManagerComponent( - @Nonnull T dispatcher, + @Nonnull Dispatcher dispatcher, @Nonnull ResourceManager resourceManager, @Nonnull LeaderRetrievalService dispatcherLeaderRetrievalService, @Nonnull LeaderRetrievalService resourceManagerRetrievalService, @@ -107,7 +107,7 @@ public final CompletableFuture getShutDownFuture() { } @Nonnull - public T getDispatcher() { + public Dispatcher getDispatcher() { return dispatcher; } 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 51ddb8f6df92..744941ca3af2 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 @@ -21,7 +21,6 @@ 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.Dispatcher; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.metrics.MetricRegistry; @@ -32,9 +31,9 @@ /** * Factory for the {@link DispatcherResourceManagerComponent}. */ -public interface DispatcherResourceManagerComponentFactory { +public interface DispatcherResourceManagerComponentFactory { - DispatcherResourceManagerComponent create( + DispatcherResourceManagerComponent create( Configuration configuration, RpcService rpcService, HighAvailabilityServices highAvailabilityServices, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/JobDispatcherResourceManagerComponent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/JobDispatcherResourceManagerComponent.java index 097ed72c5370..d8be4e26173d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/JobDispatcherResourceManagerComponent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/JobDispatcherResourceManagerComponent.java @@ -30,7 +30,7 @@ * {@link DispatcherResourceManagerComponent} for a job cluster. The dispatcher component starts * a {@link MiniDispatcher}. */ -class JobDispatcherResourceManagerComponent extends DispatcherResourceManagerComponent { +class JobDispatcherResourceManagerComponent extends DispatcherResourceManagerComponent { JobDispatcherResourceManagerComponent( MiniDispatcher dispatcher, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/JobDispatcherResourceManagerComponentFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/JobDispatcherResourceManagerComponentFactory.java index 4ac8a8c662c8..f39899f866d6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/JobDispatcherResourceManagerComponentFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/JobDispatcherResourceManagerComponentFactory.java @@ -39,7 +39,7 @@ public JobDispatcherResourceManagerComponentFactory(@Nonnull ResourceManagerFact } @Override - protected DispatcherResourceManagerComponent createDispatcherResourceManagerComponent( + protected DispatcherResourceManagerComponent createDispatcherResourceManagerComponent( MiniDispatcher dispatcher, ResourceManager resourceManager, LeaderRetrievalService dispatcherLeaderRetrievalService, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/SessionDispatcherResourceManagerComponent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/SessionDispatcherResourceManagerComponent.java index 0d782b8339ff..f9eed62ebf35 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/SessionDispatcherResourceManagerComponent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/SessionDispatcherResourceManagerComponent.java @@ -26,7 +26,7 @@ /** * {@link DispatcherResourceManagerComponent} used by session clusters. */ -class SessionDispatcherResourceManagerComponent extends DispatcherResourceManagerComponent { +class SessionDispatcherResourceManagerComponent extends DispatcherResourceManagerComponent { SessionDispatcherResourceManagerComponent( Dispatcher dispatcher, ResourceManager resourceManager, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/SessionDispatcherResourceManagerComponentFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/SessionDispatcherResourceManagerComponentFactory.java index 3b363d7fdc81..166228f9be8e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/SessionDispatcherResourceManagerComponentFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/SessionDispatcherResourceManagerComponentFactory.java @@ -48,7 +48,7 @@ public SessionDispatcherResourceManagerComponentFactory( } @Override - protected DispatcherResourceManagerComponent createDispatcherResourceManagerComponent( + protected DispatcherResourceManagerComponent createDispatcherResourceManagerComponent( Dispatcher dispatcher, ResourceManager resourceManager, LeaderRetrievalService dispatcherLeaderRetrievalService, 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 561f28fb4f66..ca7264aad053 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 @@ -169,7 +169,7 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { private LeaderRetrievalService clusterRestEndpointLeaderRetrievalService; @GuardedBy("lock") - private Collection> dispatcherResourceManagerComponents; + private Collection dispatcherResourceManagerComponents; @GuardedBy("lock") private RpcGatewayRetriever dispatcherGatewayRetriever; @@ -228,7 +228,7 @@ public HighAvailabilityServices getHighAvailabilityServices() { @VisibleForTesting @Nonnull - protected Collection> getDispatcherResourceManagerComponents() { + protected Collection getDispatcherResourceManagerComponents() { synchronized (lock) { return Collections.unmodifiableCollection(dispatcherResourceManagerComponents); } @@ -374,7 +374,7 @@ public void start() throws Exception { } @VisibleForTesting - protected Collection> createDispatcherResourceManagerComponents( + protected Collection createDispatcherResourceManagerComponents( Configuration configuration, RpcServiceFactory rpcServiceFactory, HighAvailabilityServices haServices, @@ -755,7 +755,7 @@ private CompletableFuture shutDownResourceManagerComponents() { final Collection> terminationFutures = new ArrayList<>(dispatcherResourceManagerComponents.size()); - for (DispatcherResourceManagerComponent dispatcherResourceManagerComponent : dispatcherResourceManagerComponents) { + for (DispatcherResourceManagerComponent dispatcherResourceManagerComponent : dispatcherResourceManagerComponents) { terminationFutures.add(dispatcherResourceManagerComponent.closeAsync()); } 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 3b863137b038..cb83db23879c 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 @@ -67,7 +67,7 @@ public TestingMiniCluster(TestingMiniClusterConfiguration miniClusterConfigurati @Nonnull @Override - public Collection> getDispatcherResourceManagerComponents() { + public Collection getDispatcherResourceManagerComponents() { return super.getDispatcherResourceManagerComponents(); } @@ -97,7 +97,7 @@ protected HighAvailabilityServices createHighAvailabilityServices(Configuration } @Override - protected Collection> createDispatcherResourceManagerComponents( + protected Collection createDispatcherResourceManagerComponents( Configuration configuration, RpcServiceFactory rpcServiceFactory, HighAvailabilityServices haServices, @@ -108,7 +108,7 @@ protected Collection> createDisp FatalErrorHandler fatalErrorHandler) throws Exception { SessionDispatcherResourceManagerComponentFactory dispatcherResourceManagerComponentFactory = createTestingDispatcherResourceManagerComponentFactory(); - final List> result = new ArrayList<>(numberDispatcherResourceManagerComponents); + final List result = new ArrayList<>(numberDispatcherResourceManagerComponents); for (int i = 0; i < numberDispatcherResourceManagerComponents; i++) { result.add( 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 46f9a09e14c0..f7d9797026ab 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 @@ -124,7 +124,7 @@ public void testCancelingOnProcessFailure() throws Exception { final SessionDispatcherResourceManagerComponentFactory resourceManagerComponentFactory = new SessionDispatcherResourceManagerComponentFactory( StandaloneResourceManagerFactory.INSTANCE); - DispatcherResourceManagerComponent dispatcherResourceManagerComponent = null; + DispatcherResourceManagerComponent dispatcherResourceManagerComponent = null; final HighAvailabilityServices haServices = HighAvailabilityServicesUtils.createHighAvailabilityServices( config, diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java index a3dfb710531b..4d4ce35fb429 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java @@ -123,7 +123,7 @@ public void testJobExecutionOnClusterWithLeaderChange() throws Exception { miniCluster.submitJob(jobGraph).get(); - Collection> dispatcherResourceManagerComponents = miniCluster.getDispatcherResourceManagerComponents(); + Collection dispatcherResourceManagerComponents = miniCluster.getDispatcherResourceManagerComponents(); final NewLeaderRetriever newLeaderRetriever = new NewLeaderRetriever(); final HighAvailabilityServices highAvailabilityServices = miniCluster.getHighAvailabilityServices(); @@ -131,7 +131,7 @@ public void testJobExecutionOnClusterWithLeaderChange() throws Exception { dispatcherLeaderRetriever.start(newLeaderRetriever); for (int i = 0; i < numDispatchers - 1; i++) { - final DispatcherResourceManagerComponent leadingDispatcherResourceManagerComponent = getLeadingDispatcherResourceManagerComponent( + final DispatcherResourceManagerComponent leadingDispatcherResourceManagerComponent = getLeadingDispatcherResourceManagerComponent( dispatcherResourceManagerComponents, newLeaderRetriever); @@ -142,7 +142,7 @@ public void testJobExecutionOnClusterWithLeaderChange() throws Exception { leadingDispatcherResourceManagerComponent.closeAsync(); } - final DispatcherResourceManagerComponent leadingDispatcherResourceManagerComponent = getLeadingDispatcherResourceManagerComponent( + final DispatcherResourceManagerComponent leadingDispatcherResourceManagerComponent = getLeadingDispatcherResourceManagerComponent( dispatcherResourceManagerComponents, newLeaderRetriever); @@ -158,8 +158,8 @@ public void testJobExecutionOnClusterWithLeaderChange() throws Exception { } @Nonnull - protected DispatcherResourceManagerComponent getLeadingDispatcherResourceManagerComponent( - Collection> dispatcherResourceManagerComponents, + protected DispatcherResourceManagerComponent getLeadingDispatcherResourceManagerComponent( + Collection dispatcherResourceManagerComponents, NewLeaderRetriever newLeaderRetriever) throws Exception { final Tuple2 leaderInformation = newLeaderRetriever.waitUntilNewLeader().get(); @@ -171,8 +171,8 @@ protected DispatcherResourceManagerComponent getLeadingDispatcherResourceMana } @Nonnull - private static Optional> findLeadingDispatcherResourceManagerComponent(Collection> dispatcherResourceManagerComponents, String address) { - for (DispatcherResourceManagerComponent dispatcherResourceManagerComponent : dispatcherResourceManagerComponents) { + private static Optional findLeadingDispatcherResourceManagerComponent(Collection dispatcherResourceManagerComponents, String address) { + for (DispatcherResourceManagerComponent dispatcherResourceManagerComponent : dispatcherResourceManagerComponents) { if (dispatcherResourceManagerComponent.getDispatcher().getAddress().equals(address)) { return Optional.of(dispatcherResourceManagerComponent); } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnJobClusterEntrypoint.java b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnJobClusterEntrypoint.java index 15de5a80d461..039722615423 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnJobClusterEntrypoint.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnJobClusterEntrypoint.java @@ -62,7 +62,7 @@ protected String getRPCPortRange(Configuration configuration) { } @Override - protected DispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration) { + protected DispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration) { return new JobDispatcherResourceManagerComponentFactory( YarnResourceManagerFactory.getInstance(), FileJobGraphRetriever.createFrom(configuration)); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnSessionClusterEntrypoint.java b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnSessionClusterEntrypoint.java index c22c548f73fe..b008665bdfb1 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnSessionClusterEntrypoint.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnSessionClusterEntrypoint.java @@ -60,7 +60,7 @@ protected String getRPCPortRange(Configuration configuration) { } @Override - protected DispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration) { + protected DispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration) { return new SessionDispatcherResourceManagerComponentFactory(YarnResourceManagerFactory.getInstance()); } From 936a317e7e51b0a03f5f12e3d4c21fedaa6d328e Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 18 Apr 2019 15:01:58 +0200 Subject: [PATCH 006/746] [FLINK-14280] Introduce DispatcherRunner In order to better separate concerns which are currently contained in the Dispatcher, this commit introduces the DispatcherRunner abstraction. The DispatcherRunner is used by the DispatcherResourceManagerComponent and encapsulates how the Dispatcher is executed. This closes #9807. --- .../dispatcher/SessionDispatcherFactory.java | 10 ++-- .../dispatcher/runner/DispatcherRunner.java | 45 ++++++++++++++ .../runner/DispatcherRunnerFactory.java | 32 ++++++++++ .../runner/DispatcherRunnerImpl.java | 59 +++++++++++++++++++ .../runner/MiniDispatcherRunner.java | 42 +++++++++++++ .../runner/MiniDispatcherRunnerFactory.java | 46 +++++++++++++++ .../runner/MiniDispatcherRunnerImpl.java | 45 ++++++++++++++ .../StandaloneDispatcherRunnerFactory.java | 43 ++++++++++++++ ...atcherResourceManagerComponentFactory.java | 31 +++++----- .../DispatcherResourceManagerComponent.java | 15 ++--- ...JobDispatcherResourceManagerComponent.java | 7 ++- ...atcherResourceManagerComponentFactory.java | 12 ++-- ...ionDispatcherResourceManagerComponent.java | 6 +- ...atcherResourceManagerComponentFactory.java | 15 ++--- .../SessionDispatcherWithUUIDFactory.java | 5 +- .../minicluster/TestingMiniCluster.java | 3 +- .../ZooKeeperLeaderElectionITCase.java | 19 +++++- 17 files changed, 380 insertions(+), 55 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunner.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerFactory.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunner.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerFactory.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerImpl.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StandaloneDispatcherRunnerFactory.java 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 f9b373c02904..0d29a0c2a580 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 @@ -20,18 +20,16 @@ import org.apache.flink.runtime.rpc.RpcService; -import javax.annotation.Nonnull; - /** * {@link DispatcherFactory} which creates a {@link StandaloneDispatcher}. */ -public enum SessionDispatcherFactory implements DispatcherFactory { +public enum SessionDispatcherFactory implements DispatcherFactory { INSTANCE; @Override - public Dispatcher createDispatcher( - @Nonnull RpcService rpcService, - @Nonnull PartialDispatcherServices partialDispatcherServices) throws Exception { + public StandaloneDispatcher createDispatcher( + RpcService rpcService, + PartialDispatcherServices partialDispatcherServices) throws Exception { // create the default dispatcher return new StandaloneDispatcher( rpcService, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunner.java new file mode 100644 index 000000000000..da490fa64df3 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunner.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.runtime.dispatcher.runner; + +import org.apache.flink.runtime.dispatcher.Dispatcher; +import org.apache.flink.util.AutoCloseableAsync; + +import java.util.concurrent.CompletableFuture; + +/** + * The {@link DispatcherRunner} encapsulates how a {@link Dispatcher} is being executed. + */ +public interface DispatcherRunner extends AutoCloseableAsync { + + /** + * Get the currently running {@link Dispatcher}. + * + * @return the currently running dispatcher + */ + Dispatcher getDispatcher(); + + /** + * Return the termination future of this runner. The termination future + * is being completed, once the runner has been completely terminated. + * + * @return termination future of this runner + */ + CompletableFuture getTerminationFuture(); +} 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 new file mode 100644 index 000000000000..67787f04cfe7 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerFactory.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.dispatcher.runner; + +import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; +import org.apache.flink.runtime.rpc.RpcService; + +/** + * Factory interface for the {@link DispatcherRunner}. + * + * @param type of the dispatcher runner being created + */ +public interface DispatcherRunnerFactory { + + T createDispatcherRunner(RpcService rpcService, PartialDispatcherServices partialDispatcherServices) throws Exception; +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java new file mode 100644 index 000000000000..123ca3550ca8 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.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.runtime.dispatcher.runner; + +import org.apache.flink.runtime.dispatcher.Dispatcher; +import org.apache.flink.runtime.dispatcher.DispatcherFactory; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; +import org.apache.flink.runtime.rpc.RpcService; + +import java.util.concurrent.CompletableFuture; + +/** + * Runner responsible for executing a {@link Dispatcher} or a subclass thereof. + */ +public class DispatcherRunnerImpl implements DispatcherRunner { + + private final T dispatcher; + + DispatcherRunnerImpl( + DispatcherFactory dispatcherFactory, + RpcService rpcService, + PartialDispatcherServices partialDispatcherServices) throws Exception { + this.dispatcher = dispatcherFactory.createDispatcher( + rpcService, + partialDispatcherServices); + dispatcher.start(); + } + + @Override + public T getDispatcher() { + return dispatcher; + } + + @Override + public CompletableFuture closeAsync() { + return dispatcher.closeAsync(); + } + + @Override + public CompletableFuture getTerminationFuture() { + return dispatcher.getTerminationFuture(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunner.java new file mode 100644 index 000000000000..6aa8531f744a --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunner.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.runtime.dispatcher.runner; + +import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.dispatcher.MiniDispatcher; + +import java.util.concurrent.CompletableFuture; + +/** + * Interface for a {@link DispatcherRunner} which runs a {@link MiniDispatcher}. + */ +public interface MiniDispatcherRunner extends DispatcherRunner { + + @Override + MiniDispatcher getDispatcher(); + + /** + * Return shut down future of this runner. The shut down future is being + * completed with the final {@link ApplicationStatus} once the runner wants + * to shut down. + * + * @return future with the final application status + */ + CompletableFuture getShutDownFuture(); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerFactory.java new file mode 100644 index 000000000000..58a1c76fbfd7 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerFactory.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.runner; + +import org.apache.flink.runtime.dispatcher.JobDispatcherFactory; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; +import org.apache.flink.runtime.entrypoint.component.JobGraphRetriever; +import org.apache.flink.runtime.rpc.RpcService; + +/** + * Factory for the {@link MiniDispatcherRunnerImpl}. + */ +public class MiniDispatcherRunnerFactory implements DispatcherRunnerFactory { + + private final JobGraphRetriever jobGraphRetriever; + + public MiniDispatcherRunnerFactory(JobGraphRetriever jobGraphRetriever) { + this.jobGraphRetriever = jobGraphRetriever; + } + + @Override + public MiniDispatcherRunnerImpl createDispatcherRunner( + RpcService rpcService, + PartialDispatcherServices partialDispatcherServices) throws Exception { + return new MiniDispatcherRunnerImpl( + new JobDispatcherFactory(jobGraphRetriever), + rpcService, + partialDispatcherServices); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerImpl.java new file mode 100644 index 000000000000..ff28b3f769b4 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerImpl.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.runtime.dispatcher.runner; + +import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.dispatcher.DispatcherFactory; +import org.apache.flink.runtime.dispatcher.MiniDispatcher; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; +import org.apache.flink.runtime.rpc.RpcService; + +import java.util.concurrent.CompletableFuture; + +/** + * Runner which runs a {@link MiniDispatcher} implementation. + */ +public class MiniDispatcherRunnerImpl extends DispatcherRunnerImpl implements MiniDispatcherRunner { + + MiniDispatcherRunnerImpl( + DispatcherFactory dispatcherFactory, + RpcService rpcService, + PartialDispatcherServices partialDispatcherServices) throws Exception { + super(dispatcherFactory, rpcService, partialDispatcherServices); + } + + @Override + public CompletableFuture getShutDownFuture() { + return getDispatcher().getJobTerminationFuture(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StandaloneDispatcherRunnerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StandaloneDispatcherRunnerFactory.java new file mode 100644 index 000000000000..d7099f58314d --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StandaloneDispatcherRunnerFactory.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.runner; + +import org.apache.flink.runtime.dispatcher.DispatcherFactory; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; +import org.apache.flink.runtime.dispatcher.StandaloneDispatcher; +import org.apache.flink.runtime.rpc.RpcService; + +/** + * Factory which creates a {@link DispatcherRunnerImpl} which runs a {@link StandaloneDispatcher}. + */ +public class StandaloneDispatcherRunnerFactory implements DispatcherRunnerFactory> { + + private final DispatcherFactory dispatcherFactory; + + public StandaloneDispatcherRunnerFactory(DispatcherFactory dispatcherFactory) { + this.dispatcherFactory = dispatcherFactory; + } + + @Override + public DispatcherRunnerImpl createDispatcherRunner( + RpcService rpcService, + PartialDispatcherServices dispatcherFactoryServices) throws Exception { + return new DispatcherRunnerImpl<>(dispatcherFactory, rpcService, dispatcherFactoryServices); + } +} 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 f7722c3ee692..af47c0cc80c8 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 @@ -26,12 +26,12 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.dispatcher.ArchivedExecutionGraphStore; -import org.apache.flink.runtime.dispatcher.Dispatcher; -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.dispatcher.runner.DispatcherRunner; +import org.apache.flink.runtime.dispatcher.runner.DispatcherRunnerFactory; import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; @@ -72,15 +72,15 @@ /** * Abstract class which implements the creation of the {@link DispatcherResourceManagerComponent} components. * - * @param type of the {@link Dispatcher} + * @param type of the {@link DispatcherRunner} * @param type of the {@link RestfulGateway} given to the {@link WebMonitorEndpoint} */ -public abstract class AbstractDispatcherResourceManagerComponentFactory implements DispatcherResourceManagerComponentFactory { +public abstract class AbstractDispatcherResourceManagerComponentFactory implements DispatcherResourceManagerComponentFactory { private final Logger log = LoggerFactory.getLogger(getClass()); @Nonnull - private final DispatcherFactory dispatcherFactory; + private final DispatcherRunnerFactory dispatcherRunnerFactory; @Nonnull private final ResourceManagerFactory resourceManagerFactory; @@ -89,10 +89,10 @@ public abstract class AbstractDispatcherResourceManagerComponentFactory restEndpointFactory; public AbstractDispatcherResourceManagerComponentFactory( - @Nonnull DispatcherFactory dispatcherFactory, + @Nonnull DispatcherRunnerFactory dispatcherRunnerFactory, @Nonnull ResourceManagerFactory resourceManagerFactory, @Nonnull RestEndpointFactory restEndpointFactory) { - this.dispatcherFactory = dispatcherFactory; + this.dispatcherRunnerFactory = dispatcherRunnerFactory; this.resourceManagerFactory = resourceManagerFactory; this.restEndpointFactory = restEndpointFactory; } @@ -115,7 +115,7 @@ public DispatcherResourceManagerComponent create( ResourceManager resourceManager = null; JobManagerMetricGroup jobManagerMetricGroup = null; ResourceManagerMetricGroup resourceManagerMetricGroup = null; - T dispatcher = null; + T dispatcherRunner = null; try { dispatcherLeaderRetrievalService = highAvailabilityServices.getDispatcherLeaderRetriever(); @@ -195,20 +195,19 @@ public DispatcherResourceManagerComponent create( historyServerArchivist, metricRegistry.getMetricQueryServiceGatewayRpcAddress()); - dispatcher = dispatcherFactory.createDispatcher( + log.debug("Starting Dispatcher."); + dispatcherRunner = dispatcherRunnerFactory.createDispatcherRunner( rpcService, partialDispatcherServices); log.debug("Starting ResourceManager."); resourceManager.start(); - resourceManagerRetrievalService.start(resourceManagerGatewayRetriever); - log.debug("Starting Dispatcher."); - dispatcher.start(); + resourceManagerRetrievalService.start(resourceManagerGatewayRetriever); dispatcherLeaderRetrievalService.start(dispatcherGatewayRetriever); return createDispatcherResourceManagerComponent( - dispatcher, + dispatcherRunner, resourceManager, dispatcherLeaderRetrievalService, resourceManagerRetrievalService, @@ -242,8 +241,8 @@ public DispatcherResourceManagerComponent create( terminationFutures.add(resourceManager.closeAsync()); } - if (dispatcher != null) { - terminationFutures.add(dispatcher.closeAsync()); + if (dispatcherRunner != null) { + terminationFutures.add(dispatcherRunner.closeAsync()); } final FutureUtils.ConjunctFuture terminationFuture = FutureUtils.completeAll(terminationFutures); @@ -267,7 +266,7 @@ public DispatcherResourceManagerComponent create( } protected abstract DispatcherResourceManagerComponent createDispatcherResourceManagerComponent( - T dispatcher, + T dispatcherRunner, ResourceManager resourceManager, LeaderRetrievalService dispatcherLeaderRetrievalService, LeaderRetrievalService resourceManagerRetrievalService, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponent.java index 581423e690e1..db9cdc5de16f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponent.java @@ -21,6 +21,7 @@ import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.dispatcher.Dispatcher; +import org.apache.flink.runtime.dispatcher.runner.DispatcherRunner; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.resourcemanager.ResourceManager; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; @@ -43,7 +44,7 @@ public class DispatcherResourceManagerComponent implements AutoCloseableAsync { @Nonnull - private final Dispatcher dispatcher; + private final DispatcherRunner dispatcherRunner; @Nonnull private final ResourceManager resourceManager; @@ -64,13 +65,13 @@ public class DispatcherResourceManagerComponent implements AutoCloseableAsync { private final AtomicBoolean isRunning = new AtomicBoolean(true); DispatcherResourceManagerComponent( - @Nonnull Dispatcher dispatcher, + @Nonnull DispatcherRunner dispatcherRunner, @Nonnull ResourceManager resourceManager, @Nonnull LeaderRetrievalService dispatcherLeaderRetrievalService, @Nonnull LeaderRetrievalService resourceManagerRetrievalService, @Nonnull WebMonitorEndpoint webMonitorEndpoint) { + this.dispatcherRunner = dispatcherRunner; this.resourceManager = resourceManager; - this.dispatcher = dispatcher; this.dispatcherLeaderRetrievalService = dispatcherLeaderRetrievalService; this.resourceManagerRetrievalService = resourceManagerRetrievalService; this.webMonitorEndpoint = webMonitorEndpoint; @@ -90,7 +91,7 @@ private void registerShutDownFuture() { } }); - dispatcher + dispatcherRunner .getTerminationFuture() .whenComplete( (aVoid, throwable) -> { @@ -107,8 +108,8 @@ public final CompletableFuture getShutDownFuture() { } @Nonnull - public Dispatcher getDispatcher() { - return dispatcher; + public DispatcherRunner getDispatcherRunner() { + return dispatcherRunner; } @Nonnull @@ -163,7 +164,7 @@ private CompletableFuture closeAsyncInternal() { exception = ExceptionUtils.firstOrSuppressed(e, exception); } - terminationFutures.add(dispatcher.closeAsync()); + terminationFutures.add(dispatcherRunner.closeAsync()); terminationFutures.add(resourceManager.closeAsync()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/JobDispatcherResourceManagerComponent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/JobDispatcherResourceManagerComponent.java index d8be4e26173d..27e22ea883a0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/JobDispatcherResourceManagerComponent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/JobDispatcherResourceManagerComponent.java @@ -20,6 +20,7 @@ import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.dispatcher.MiniDispatcher; +import org.apache.flink.runtime.dispatcher.runner.MiniDispatcherRunner; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.resourcemanager.ResourceManager; import org.apache.flink.runtime.webmonitor.WebMonitorEndpoint; @@ -33,16 +34,16 @@ class JobDispatcherResourceManagerComponent extends DispatcherResourceManagerComponent { JobDispatcherResourceManagerComponent( - MiniDispatcher dispatcher, + MiniDispatcherRunner dispatcherRunner, ResourceManager resourceManager, LeaderRetrievalService dispatcherLeaderRetrievalService, LeaderRetrievalService resourceManagerRetrievalService, WebMonitorEndpoint webMonitorEndpoint) { - super(dispatcher, resourceManager, dispatcherLeaderRetrievalService, resourceManagerRetrievalService, webMonitorEndpoint); + super(dispatcherRunner, resourceManager, dispatcherLeaderRetrievalService, resourceManagerRetrievalService, webMonitorEndpoint); final CompletableFuture shutDownFuture = getShutDownFuture(); - dispatcher.getJobTerminationFuture().whenComplete((applicationStatus, throwable) -> { + dispatcherRunner.getShutDownFuture().whenComplete((applicationStatus, throwable) -> { if (throwable != null) { shutDownFuture.completeExceptionally(throwable); } else { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/JobDispatcherResourceManagerComponentFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/JobDispatcherResourceManagerComponentFactory.java index f39899f866d6..d7baa2932641 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/JobDispatcherResourceManagerComponentFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/JobDispatcherResourceManagerComponentFactory.java @@ -18,8 +18,8 @@ package org.apache.flink.runtime.entrypoint.component; -import org.apache.flink.runtime.dispatcher.JobDispatcherFactory; -import org.apache.flink.runtime.dispatcher.MiniDispatcher; +import org.apache.flink.runtime.dispatcher.runner.MiniDispatcherRunner; +import org.apache.flink.runtime.dispatcher.runner.MiniDispatcherRunnerFactory; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.resourcemanager.ResourceManager; import org.apache.flink.runtime.resourcemanager.ResourceManagerFactory; @@ -32,21 +32,21 @@ /** * {@link DispatcherResourceManagerComponentFactory} for a {@link JobDispatcherResourceManagerComponent}. */ -public class JobDispatcherResourceManagerComponentFactory extends AbstractDispatcherResourceManagerComponentFactory { +public class JobDispatcherResourceManagerComponentFactory extends AbstractDispatcherResourceManagerComponentFactory { public JobDispatcherResourceManagerComponentFactory(@Nonnull ResourceManagerFactory resourceManagerFactory, @Nonnull JobGraphRetriever jobGraphRetriever) { - super(new JobDispatcherFactory(jobGraphRetriever), resourceManagerFactory, JobRestEndpointFactory.INSTANCE); + super(new MiniDispatcherRunnerFactory(jobGraphRetriever), resourceManagerFactory, JobRestEndpointFactory.INSTANCE); } @Override protected DispatcherResourceManagerComponent createDispatcherResourceManagerComponent( - MiniDispatcher dispatcher, + MiniDispatcherRunner dispatcherRunner, ResourceManager resourceManager, LeaderRetrievalService dispatcherLeaderRetrievalService, LeaderRetrievalService resourceManagerRetrievalService, WebMonitorEndpoint webMonitorEndpoint) { return new JobDispatcherResourceManagerComponent( - dispatcher, + dispatcherRunner, resourceManager, dispatcherLeaderRetrievalService, resourceManagerRetrievalService, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/SessionDispatcherResourceManagerComponent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/SessionDispatcherResourceManagerComponent.java index f9eed62ebf35..442f05621a00 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/SessionDispatcherResourceManagerComponent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/SessionDispatcherResourceManagerComponent.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.entrypoint.component; -import org.apache.flink.runtime.dispatcher.Dispatcher; +import org.apache.flink.runtime.dispatcher.runner.DispatcherRunner; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.resourcemanager.ResourceManager; import org.apache.flink.runtime.webmonitor.WebMonitorEndpoint; @@ -28,11 +28,11 @@ */ class SessionDispatcherResourceManagerComponent extends DispatcherResourceManagerComponent { SessionDispatcherResourceManagerComponent( - Dispatcher dispatcher, + DispatcherRunner dispatcherRunner, ResourceManager resourceManager, LeaderRetrievalService dispatcherLeaderRetrievalService, LeaderRetrievalService resourceManagerRetrievalService, WebMonitorEndpoint webMonitorEndpoint) { - super(dispatcher, resourceManager, dispatcherLeaderRetrievalService, resourceManagerRetrievalService, webMonitorEndpoint); + super(dispatcherRunner, resourceManager, dispatcherLeaderRetrievalService, resourceManagerRetrievalService, webMonitorEndpoint); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/SessionDispatcherResourceManagerComponentFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/SessionDispatcherResourceManagerComponentFactory.java index 166228f9be8e..40c8e7d0a3d6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/SessionDispatcherResourceManagerComponentFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/SessionDispatcherResourceManagerComponentFactory.java @@ -19,10 +19,11 @@ package org.apache.flink.runtime.entrypoint.component; import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.runtime.dispatcher.Dispatcher; -import org.apache.flink.runtime.dispatcher.DispatcherFactory; import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.dispatcher.SessionDispatcherFactory; +import org.apache.flink.runtime.dispatcher.runner.DispatcherRunner; +import org.apache.flink.runtime.dispatcher.runner.DispatcherRunnerFactory; +import org.apache.flink.runtime.dispatcher.runner.StandaloneDispatcherRunnerFactory; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.resourcemanager.ResourceManager; import org.apache.flink.runtime.resourcemanager.ResourceManagerFactory; @@ -34,28 +35,28 @@ /** * {@link DispatcherResourceManagerComponentFactory} for a {@link SessionDispatcherResourceManagerComponent}. */ -public class SessionDispatcherResourceManagerComponentFactory extends AbstractDispatcherResourceManagerComponentFactory { +public class SessionDispatcherResourceManagerComponentFactory extends AbstractDispatcherResourceManagerComponentFactory { public SessionDispatcherResourceManagerComponentFactory(@Nonnull ResourceManagerFactory resourceManagerFactory) { - this(SessionDispatcherFactory.INSTANCE, resourceManagerFactory); + this(new StandaloneDispatcherRunnerFactory(SessionDispatcherFactory.INSTANCE), resourceManagerFactory); } @VisibleForTesting public SessionDispatcherResourceManagerComponentFactory( - @Nonnull DispatcherFactory dispatcherFactory, + @Nonnull DispatcherRunnerFactory dispatcherFactory, @Nonnull ResourceManagerFactory resourceManagerFactory) { super(dispatcherFactory, resourceManagerFactory, SessionRestEndpointFactory.INSTANCE); } @Override protected DispatcherResourceManagerComponent createDispatcherResourceManagerComponent( - Dispatcher dispatcher, + DispatcherRunner dispatcherRunner, ResourceManager resourceManager, LeaderRetrievalService dispatcherLeaderRetrievalService, LeaderRetrievalService resourceManagerRetrievalService, WebMonitorEndpoint webMonitorEndpoint) { return new SessionDispatcherResourceManagerComponent( - dispatcher, + dispatcherRunner, resourceManager, dispatcherLeaderRetrievalService, resourceManagerRetrievalService, 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 a85ec92790ed..ac6a70e05e4e 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 @@ -19,7 +19,6 @@ package org.apache.flink.runtime.minicluster; 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.DispatcherServices; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; @@ -32,11 +31,11 @@ * {@link DispatcherFactory} which creates a {@link StandaloneDispatcher} which has an * endpoint id with a random UUID suffix. */ -public enum SessionDispatcherWithUUIDFactory implements DispatcherFactory { +public enum SessionDispatcherWithUUIDFactory implements DispatcherFactory { INSTANCE; @Override - public Dispatcher createDispatcher( + public StandaloneDispatcher createDispatcher( @Nonnull RpcService rpcService, @Nonnull PartialDispatcherServices partialDispatcherServices) throws Exception { // create the default dispatcher 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 cb83db23879c..691a2a7063fc 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 @@ -22,6 +22,7 @@ import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.dispatcher.MemoryArchivedExecutionGraphStore; +import org.apache.flink.runtime.dispatcher.runner.StandaloneDispatcherRunnerFactory; import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponent; import org.apache.flink.runtime.entrypoint.component.SessionDispatcherResourceManagerComponentFactory; import org.apache.flink.runtime.heartbeat.HeartbeatServices; @@ -134,7 +135,7 @@ public CompletableFuture getDispatcherGatewayFuture() { private SessionDispatcherResourceManagerComponentFactory createTestingDispatcherResourceManagerComponentFactory() { return new SessionDispatcherResourceManagerComponentFactory( - SessionDispatcherWithUUIDFactory.INSTANCE, + new StandaloneDispatcherRunnerFactory(SessionDispatcherWithUUIDFactory.INSTANCE), StandaloneResourceManagerWithUUIDFactory.INSTANCE); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java index 4d4ce35fb429..168fa1f91f2e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java @@ -24,6 +24,7 @@ import org.apache.flink.configuration.ClusterOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.dispatcher.Dispatcher; +import org.apache.flink.runtime.dispatcher.runner.DispatcherRunner; import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponent; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; @@ -57,6 +58,7 @@ import java.util.concurrent.CompletableFuture; import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; import static org.junit.Assert.assertThat; /** @@ -135,7 +137,10 @@ public void testJobExecutionOnClusterWithLeaderChange() throws Exception { dispatcherResourceManagerComponents, newLeaderRetriever); - final Dispatcher dispatcher = leadingDispatcherResourceManagerComponent.getDispatcher(); + final DispatcherRunner dispatcherRunner = leadingDispatcherResourceManagerComponent.getDispatcherRunner(); + final Dispatcher dispatcher = dispatcherRunner.getDispatcher(); + + assertThat(dispatcher, is(notNullValue())); CommonTestUtils.waitUntilCondition(() -> dispatcher.requestJobStatus(jobGraph.getJobID(), RPC_TIMEOUT).get() == JobStatus.RUNNING, timeout, 50L); @@ -146,7 +151,12 @@ public void testJobExecutionOnClusterWithLeaderChange() throws Exception { dispatcherResourceManagerComponents, newLeaderRetriever); - CompletableFuture jobResultFuture = leadingDispatcherResourceManagerComponent.getDispatcher().requestJobResult(jobGraph.getJobID(), RPC_TIMEOUT); + final DispatcherRunner dispatcherRunner = leadingDispatcherResourceManagerComponent.getDispatcherRunner(); + final Dispatcher dispatcher = dispatcherRunner.getDispatcher(); + + assertThat(dispatcher, is(notNullValue())); + + CompletableFuture jobResultFuture = dispatcher.requestJobResult(jobGraph.getJobID(), RPC_TIMEOUT); BlockingOperator.unblock(); assertThat(jobResultFuture.get().isSuccess(), is(true)); @@ -173,7 +183,10 @@ protected DispatcherResourceManagerComponent getLeadingDispatcherResourceManager @Nonnull private static Optional findLeadingDispatcherResourceManagerComponent(Collection dispatcherResourceManagerComponents, String address) { for (DispatcherResourceManagerComponent dispatcherResourceManagerComponent : dispatcherResourceManagerComponents) { - if (dispatcherResourceManagerComponent.getDispatcher().getAddress().equals(address)) { + final DispatcherRunner dispatcherRunner = dispatcherResourceManagerComponent.getDispatcherRunner(); + final Dispatcher dispatcher = dispatcherRunner.getDispatcher(); + + if (dispatcher != null && dispatcher.getAddress().equals(address)) { return Optional.of(dispatcherResourceManagerComponent); } } From 01ad0f6fe23615484e36b4710cd0fdc3a7276bed Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 29 Sep 2019 17:57:32 +0200 Subject: [PATCH 007/746] [FLINK-14281] Add DispatcherRunner#getShutDownFuture The DispatcherRunner#getShutDownFuture is completed if the DispatcherRunner implementation intends to be shut down by its owner. This closes #9808. --- .../dispatcher/runner/DispatcherRunner.java | 10 +++++---- .../runner/DispatcherRunnerImpl.java | 22 ++++++++++++++----- .../DispatcherResourceManagerComponent.java | 20 +---------------- 3 files changed, 23 insertions(+), 29 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunner.java index da490fa64df3..2157564cbcc0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunner.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.dispatcher.runner; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.dispatcher.Dispatcher; import org.apache.flink.util.AutoCloseableAsync; @@ -36,10 +37,11 @@ public interface DispatcherRunner extends AutoCloseableAsync { Dispatcher getDispatcher(); /** - * Return the termination future of this runner. The termination future - * is being completed, once the runner has been completely terminated. + * Return shut down future of this runner. The shut down future is being + * completed with the final {@link ApplicationStatus} once the runner wants + * to shut down. * - * @return termination future of this runner + * @return future with the final application status */ - CompletableFuture getTerminationFuture(); + CompletableFuture getShutDownFuture(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java index 123ca3550ca8..6a0c85228aa7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.dispatcher.runner; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.dispatcher.Dispatcher; import org.apache.flink.runtime.dispatcher.DispatcherFactory; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; @@ -28,17 +30,25 @@ /** * Runner responsible for executing a {@link Dispatcher} or a subclass thereof. */ -public class DispatcherRunnerImpl implements DispatcherRunner { +class DispatcherRunnerImpl implements DispatcherRunner { private final T dispatcher; + private final CompletableFuture shutDownFuture; + DispatcherRunnerImpl( - DispatcherFactory dispatcherFactory, - RpcService rpcService, - PartialDispatcherServices partialDispatcherServices) throws Exception { + DispatcherFactory dispatcherFactory, + RpcService rpcService, + PartialDispatcherServices partialDispatcherServices) throws Exception { this.dispatcher = dispatcherFactory.createDispatcher( rpcService, partialDispatcherServices); + shutDownFuture = new CompletableFuture<>(); + + FutureUtils.forward( + dispatcher.getTerminationFuture().thenApply((ignored) -> ApplicationStatus.UNKNOWN), + shutDownFuture); + dispatcher.start(); } @@ -53,7 +63,7 @@ public CompletableFuture closeAsync() { } @Override - public CompletableFuture getTerminationFuture() { - return dispatcher.getTerminationFuture(); + public CompletableFuture getShutDownFuture() { + return shutDownFuture; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponent.java index db9cdc5de16f..770e92aea3bc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponent.java @@ -82,25 +82,7 @@ public class DispatcherResourceManagerComponent implements AutoCloseableAsync { } private void registerShutDownFuture() { - terminationFuture.whenComplete( - (aVoid, throwable) -> { - if (throwable != null) { - shutDownFuture.completeExceptionally(throwable); - } else { - shutDownFuture.complete(ApplicationStatus.SUCCEEDED); - } - }); - - dispatcherRunner - .getTerminationFuture() - .whenComplete( - (aVoid, throwable) -> { - if (throwable != null) { - shutDownFuture.completeExceptionally(throwable); - } else { - shutDownFuture.complete(ApplicationStatus.SUCCEEDED); - } - }); + FutureUtils.forward(dispatcherRunner.getShutDownFuture(), shutDownFuture); } public final CompletableFuture getShutDownFuture() { From 870b56418cb795edba9d4e8e282fa91d675efd7a Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 29 Sep 2019 18:06:00 +0200 Subject: [PATCH 008/746] [FLINK-14282] Simplify DispatcherResourceManagerComponent hierarchy Remove unnecessary subclasses of AbstractDispatcherResourceManagerComponent and rename this class into DefaultDispatcherResourceManagerComponent. Moreover, this commit removes the unnecessary generics from the DispatcherRunnerFactory to further simplify the code base. This closes #9809. --- .../StandaloneJobClusterEntryPoint.java | 4 +- .../entrypoint/MesosJobClusterEntrypoint.java | 7 +- .../MesosSessionClusterEntrypoint.java | 7 +- .../runner/DispatcherRunnerFactory.java | 6 +- .../runner/MiniDispatcherRunner.java | 42 ------------ .../runner/MiniDispatcherRunnerFactory.java | 2 +- .../runner/MiniDispatcherRunnerImpl.java | 2 +- .../StandaloneDispatcherRunnerFactory.java | 2 +- .../StandaloneSessionClusterEntrypoint.java | 7 +- ...tcherResourceManagerComponentFactory.java} | 40 ++++++++---- ...JobDispatcherResourceManagerComponent.java | 54 --------------- ...atcherResourceManagerComponentFactory.java | 55 ---------------- ...atcherResourceManagerComponentFactory.java | 65 ------------------- .../runtime/minicluster/MiniCluster.java | 9 +-- ...tcherResourceManagerComponentFactory.java} | 28 ++++---- .../minicluster/TestingMiniCluster.java | 9 +-- .../ProcessFailureCancelingITCase.java | 5 +- .../entrypoint/YarnJobClusterEntrypoint.java | 7 +- .../YarnSessionClusterEntrypoint.java | 4 +- 19 files changed, 75 insertions(+), 280 deletions(-) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunner.java rename flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/{AbstractDispatcherResourceManagerComponentFactory.java => DefaultDispatcherResourceManagerComponentFactory.java} (86%) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/JobDispatcherResourceManagerComponent.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/JobDispatcherResourceManagerComponentFactory.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/SessionDispatcherResourceManagerComponentFactory.java rename flink-runtime/src/{main/java/org/apache/flink/runtime/entrypoint/component/SessionDispatcherResourceManagerComponent.java => test/java/org/apache/flink/runtime/entrypoint/component/TestingDefaultDispatcherResourceManagerComponentFactory.java} (51%) diff --git a/flink-container/src/main/java/org/apache/flink/container/entrypoint/StandaloneJobClusterEntryPoint.java b/flink-container/src/main/java/org/apache/flink/container/entrypoint/StandaloneJobClusterEntryPoint.java index fbaba809f5da..8935f358d523 100644 --- a/flink-container/src/main/java/org/apache/flink/container/entrypoint/StandaloneJobClusterEntryPoint.java +++ b/flink-container/src/main/java/org/apache/flink/container/entrypoint/StandaloneJobClusterEntryPoint.java @@ -23,8 +23,8 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; import org.apache.flink.runtime.entrypoint.JobClusterEntrypoint; +import org.apache.flink.runtime.entrypoint.component.DefaultDispatcherResourceManagerComponentFactory; import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponentFactory; -import org.apache.flink.runtime.entrypoint.component.JobDispatcherResourceManagerComponentFactory; import org.apache.flink.runtime.entrypoint.parser.CommandLineParser; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; @@ -75,7 +75,7 @@ private StandaloneJobClusterEntryPoint( @Override protected DispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration) { - return new JobDispatcherResourceManagerComponentFactory( + return DefaultDispatcherResourceManagerComponentFactory.createJobComponentFactory( StandaloneResourceManagerFactory.INSTANCE, new ClassPathJobGraphRetriever(jobId, savepointRestoreSettings, programArguments, jobClassName)); } diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java index 8bbd6c6f2264..6c1f486ba9b4 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java @@ -30,9 +30,8 @@ import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; import org.apache.flink.runtime.entrypoint.JobClusterEntrypoint; -import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponentFactory; +import org.apache.flink.runtime.entrypoint.component.DefaultDispatcherResourceManagerComponentFactory; import org.apache.flink.runtime.entrypoint.component.FileJobGraphRetriever; -import org.apache.flink.runtime.entrypoint.component.JobDispatcherResourceManagerComponentFactory; import org.apache.flink.runtime.util.EnvironmentInformation; import org.apache.flink.runtime.util.JvmShutdownSafeguard; import org.apache.flink.runtime.util.SignalHandler; @@ -109,8 +108,8 @@ protected CompletableFuture stopClusterServices(boolean cleanupHaData) { } @Override - protected DispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration) { - return new JobDispatcherResourceManagerComponentFactory( + protected DefaultDispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration) { + return DefaultDispatcherResourceManagerComponentFactory.createJobComponentFactory( new MesosResourceManagerFactory( mesosServices, schedulerConfiguration, diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java index afd53c816815..4033d7d83d5e 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java @@ -30,8 +30,7 @@ import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; import org.apache.flink.runtime.entrypoint.SessionClusterEntrypoint; -import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponentFactory; -import org.apache.flink.runtime.entrypoint.component.SessionDispatcherResourceManagerComponentFactory; +import org.apache.flink.runtime.entrypoint.component.DefaultDispatcherResourceManagerComponentFactory; import org.apache.flink.runtime.util.EnvironmentInformation; import org.apache.flink.runtime.util.JvmShutdownSafeguard; import org.apache.flink.runtime.util.SignalHandler; @@ -108,8 +107,8 @@ protected CompletableFuture stopClusterServices(boolean cleanupHaData) { } @Override - protected DispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration) { - return new SessionDispatcherResourceManagerComponentFactory( + protected DefaultDispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration) { + return DefaultDispatcherResourceManagerComponentFactory.createSessionComponentFactory( new MesosResourceManagerFactory( mesosServices, mesosConfig, 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 67787f04cfe7..9212ea871fed 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 @@ -23,10 +23,8 @@ /** * Factory interface for the {@link DispatcherRunner}. - * - * @param type of the dispatcher runner being created */ -public interface DispatcherRunnerFactory { +public interface DispatcherRunnerFactory { - T createDispatcherRunner(RpcService rpcService, PartialDispatcherServices partialDispatcherServices) throws Exception; + DispatcherRunner createDispatcherRunner(RpcService rpcService, PartialDispatcherServices partialDispatcherServices) throws Exception; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunner.java deleted file mode 100644 index 6aa8531f744a..000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunner.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.runtime.dispatcher.runner; - -import org.apache.flink.runtime.clusterframework.ApplicationStatus; -import org.apache.flink.runtime.dispatcher.MiniDispatcher; - -import java.util.concurrent.CompletableFuture; - -/** - * Interface for a {@link DispatcherRunner} which runs a {@link MiniDispatcher}. - */ -public interface MiniDispatcherRunner extends DispatcherRunner { - - @Override - MiniDispatcher getDispatcher(); - - /** - * Return shut down future of this runner. The shut down future is being - * completed with the final {@link ApplicationStatus} once the runner wants - * to shut down. - * - * @return future with the final application status - */ - CompletableFuture getShutDownFuture(); -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerFactory.java index 58a1c76fbfd7..48ef6744615e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerFactory.java @@ -26,7 +26,7 @@ /** * Factory for the {@link MiniDispatcherRunnerImpl}. */ -public class MiniDispatcherRunnerFactory implements DispatcherRunnerFactory { +public class MiniDispatcherRunnerFactory implements DispatcherRunnerFactory { private final JobGraphRetriever jobGraphRetriever; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerImpl.java index ff28b3f769b4..0fa25ae28037 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerImpl.java @@ -29,7 +29,7 @@ /** * Runner which runs a {@link MiniDispatcher} implementation. */ -public class MiniDispatcherRunnerImpl extends DispatcherRunnerImpl implements MiniDispatcherRunner { +class MiniDispatcherRunnerImpl extends DispatcherRunnerImpl { MiniDispatcherRunnerImpl( DispatcherFactory dispatcherFactory, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StandaloneDispatcherRunnerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StandaloneDispatcherRunnerFactory.java index d7099f58314d..4560c2335c59 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StandaloneDispatcherRunnerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StandaloneDispatcherRunnerFactory.java @@ -26,7 +26,7 @@ /** * Factory which creates a {@link DispatcherRunnerImpl} which runs a {@link StandaloneDispatcher}. */ -public class StandaloneDispatcherRunnerFactory implements DispatcherRunnerFactory> { +public class StandaloneDispatcherRunnerFactory implements DispatcherRunnerFactory { private final DispatcherFactory dispatcherFactory; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/StandaloneSessionClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/StandaloneSessionClusterEntrypoint.java index 1918a516d41f..e549ca9c5c18 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/StandaloneSessionClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/StandaloneSessionClusterEntrypoint.java @@ -19,8 +19,7 @@ package org.apache.flink.runtime.entrypoint; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponentFactory; -import org.apache.flink.runtime.entrypoint.component.SessionDispatcherResourceManagerComponentFactory; +import org.apache.flink.runtime.entrypoint.component.DefaultDispatcherResourceManagerComponentFactory; import org.apache.flink.runtime.entrypoint.parser.CommandLineParser; import org.apache.flink.runtime.resourcemanager.StandaloneResourceManagerFactory; import org.apache.flink.runtime.util.EnvironmentInformation; @@ -37,8 +36,8 @@ public StandaloneSessionClusterEntrypoint(Configuration configuration) { } @Override - protected DispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration) { - return new SessionDispatcherResourceManagerComponentFactory(StandaloneResourceManagerFactory.INSTANCE); + protected DefaultDispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration) { + return DefaultDispatcherResourceManagerComponentFactory.createSessionComponentFactory(StandaloneResourceManagerFactory.INSTANCE); } public static void main(String[] args) { 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/DefaultDispatcherResourceManagerComponentFactory.java similarity index 86% rename from flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/AbstractDispatcherResourceManagerComponentFactory.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DefaultDispatcherResourceManagerComponentFactory.java index af47c0cc80c8..6c7aa789372e 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/DefaultDispatcherResourceManagerComponentFactory.java @@ -30,8 +30,11 @@ 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.dispatcher.SessionDispatcherFactory; import org.apache.flink.runtime.dispatcher.runner.DispatcherRunner; import org.apache.flink.runtime.dispatcher.runner.DispatcherRunnerFactory; +import org.apache.flink.runtime.dispatcher.runner.MiniDispatcherRunnerFactory; +import org.apache.flink.runtime.dispatcher.runner.StandaloneDispatcherRunnerFactory; import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; @@ -44,7 +47,9 @@ import org.apache.flink.runtime.resourcemanager.ResourceManagerFactory; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.resourcemanager.ResourceManagerId; +import org.apache.flink.runtime.rest.JobRestEndpointFactory; import org.apache.flink.runtime.rest.RestEndpointFactory; +import org.apache.flink.runtime.rest.SessionRestEndpointFactory; import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcher; import org.apache.flink.runtime.rest.handler.legacy.metrics.MetricFetcherImpl; import org.apache.flink.runtime.rest.handler.legacy.metrics.VoidMetricFetcher; @@ -72,15 +77,14 @@ /** * Abstract class which implements the creation of the {@link DispatcherResourceManagerComponent} components. * - * @param type of the {@link DispatcherRunner} * @param type of the {@link RestfulGateway} given to the {@link WebMonitorEndpoint} */ -public abstract class AbstractDispatcherResourceManagerComponentFactory implements DispatcherResourceManagerComponentFactory { +public class DefaultDispatcherResourceManagerComponentFactory implements DispatcherResourceManagerComponentFactory { private final Logger log = LoggerFactory.getLogger(getClass()); @Nonnull - private final DispatcherRunnerFactory dispatcherRunnerFactory; + private final DispatcherRunnerFactory dispatcherRunnerFactory; @Nonnull private final ResourceManagerFactory resourceManagerFactory; @@ -88,8 +92,8 @@ public abstract class AbstractDispatcherResourceManagerComponentFactory restEndpointFactory; - public AbstractDispatcherResourceManagerComponentFactory( - @Nonnull DispatcherRunnerFactory dispatcherRunnerFactory, + DefaultDispatcherResourceManagerComponentFactory( + @Nonnull DispatcherRunnerFactory dispatcherRunnerFactory, @Nonnull ResourceManagerFactory resourceManagerFactory, @Nonnull RestEndpointFactory restEndpointFactory) { this.dispatcherRunnerFactory = dispatcherRunnerFactory; @@ -115,7 +119,7 @@ public DispatcherResourceManagerComponent create( ResourceManager resourceManager = null; JobManagerMetricGroup jobManagerMetricGroup = null; ResourceManagerMetricGroup resourceManagerMetricGroup = null; - T dispatcherRunner = null; + DispatcherRunner dispatcherRunner = null; try { dispatcherLeaderRetrievalService = highAvailabilityServices.getDispatcherLeaderRetriever(); @@ -206,7 +210,7 @@ public DispatcherResourceManagerComponent create( resourceManagerRetrievalService.start(resourceManagerGatewayRetriever); dispatcherLeaderRetrievalService.start(dispatcherGatewayRetriever); - return createDispatcherResourceManagerComponent( + return new DispatcherResourceManagerComponent( dispatcherRunner, resourceManager, dispatcherLeaderRetrievalService, @@ -265,10 +269,20 @@ public DispatcherResourceManagerComponent create( } } - protected abstract DispatcherResourceManagerComponent createDispatcherResourceManagerComponent( - T dispatcherRunner, - ResourceManager resourceManager, - LeaderRetrievalService dispatcherLeaderRetrievalService, - LeaderRetrievalService resourceManagerRetrievalService, - WebMonitorEndpoint webMonitorEndpoint); + public static DefaultDispatcherResourceManagerComponentFactory createSessionComponentFactory( + ResourceManagerFactory resourceManagerFactory) { + return new DefaultDispatcherResourceManagerComponentFactory<>( + new StandaloneDispatcherRunnerFactory(SessionDispatcherFactory.INSTANCE), + resourceManagerFactory, + SessionRestEndpointFactory.INSTANCE); + } + + public static DefaultDispatcherResourceManagerComponentFactory createJobComponentFactory( + ResourceManagerFactory resourceManagerFactory, + JobGraphRetriever jobGraphRetriever) { + return new DefaultDispatcherResourceManagerComponentFactory<>( + new MiniDispatcherRunnerFactory(jobGraphRetriever), + resourceManagerFactory, + JobRestEndpointFactory.INSTANCE); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/JobDispatcherResourceManagerComponent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/JobDispatcherResourceManagerComponent.java deleted file mode 100644 index 27e22ea883a0..000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/JobDispatcherResourceManagerComponent.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.entrypoint.component; - -import org.apache.flink.runtime.clusterframework.ApplicationStatus; -import org.apache.flink.runtime.dispatcher.MiniDispatcher; -import org.apache.flink.runtime.dispatcher.runner.MiniDispatcherRunner; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; -import org.apache.flink.runtime.resourcemanager.ResourceManager; -import org.apache.flink.runtime.webmonitor.WebMonitorEndpoint; - -import java.util.concurrent.CompletableFuture; - -/** - * {@link DispatcherResourceManagerComponent} for a job cluster. The dispatcher component starts - * a {@link MiniDispatcher}. - */ -class JobDispatcherResourceManagerComponent extends DispatcherResourceManagerComponent { - - JobDispatcherResourceManagerComponent( - MiniDispatcherRunner dispatcherRunner, - ResourceManager resourceManager, - LeaderRetrievalService dispatcherLeaderRetrievalService, - LeaderRetrievalService resourceManagerRetrievalService, - WebMonitorEndpoint webMonitorEndpoint) { - super(dispatcherRunner, resourceManager, dispatcherLeaderRetrievalService, resourceManagerRetrievalService, webMonitorEndpoint); - - final CompletableFuture shutDownFuture = getShutDownFuture(); - - dispatcherRunner.getShutDownFuture().whenComplete((applicationStatus, throwable) -> { - if (throwable != null) { - shutDownFuture.completeExceptionally(throwable); - } else { - shutDownFuture.complete(applicationStatus); - } - }); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/JobDispatcherResourceManagerComponentFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/JobDispatcherResourceManagerComponentFactory.java deleted file mode 100644 index d7baa2932641..000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/JobDispatcherResourceManagerComponentFactory.java +++ /dev/null @@ -1,55 +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.entrypoint.component; - -import org.apache.flink.runtime.dispatcher.runner.MiniDispatcherRunner; -import org.apache.flink.runtime.dispatcher.runner.MiniDispatcherRunnerFactory; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; -import org.apache.flink.runtime.resourcemanager.ResourceManager; -import org.apache.flink.runtime.resourcemanager.ResourceManagerFactory; -import org.apache.flink.runtime.rest.JobRestEndpointFactory; -import org.apache.flink.runtime.webmonitor.RestfulGateway; -import org.apache.flink.runtime.webmonitor.WebMonitorEndpoint; - -import javax.annotation.Nonnull; - -/** - * {@link DispatcherResourceManagerComponentFactory} for a {@link JobDispatcherResourceManagerComponent}. - */ -public class JobDispatcherResourceManagerComponentFactory extends AbstractDispatcherResourceManagerComponentFactory { - - public JobDispatcherResourceManagerComponentFactory(@Nonnull ResourceManagerFactory resourceManagerFactory, @Nonnull JobGraphRetriever jobGraphRetriever) { - super(new MiniDispatcherRunnerFactory(jobGraphRetriever), resourceManagerFactory, JobRestEndpointFactory.INSTANCE); - } - - @Override - protected DispatcherResourceManagerComponent createDispatcherResourceManagerComponent( - MiniDispatcherRunner dispatcherRunner, - ResourceManager resourceManager, - LeaderRetrievalService dispatcherLeaderRetrievalService, - LeaderRetrievalService resourceManagerRetrievalService, - WebMonitorEndpoint webMonitorEndpoint) { - return new JobDispatcherResourceManagerComponent( - dispatcherRunner, - resourceManager, - dispatcherLeaderRetrievalService, - resourceManagerRetrievalService, - webMonitorEndpoint); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/SessionDispatcherResourceManagerComponentFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/SessionDispatcherResourceManagerComponentFactory.java deleted file mode 100644 index 40c8e7d0a3d6..000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/SessionDispatcherResourceManagerComponentFactory.java +++ /dev/null @@ -1,65 +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.entrypoint.component; - -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.runtime.dispatcher.DispatcherGateway; -import org.apache.flink.runtime.dispatcher.SessionDispatcherFactory; -import org.apache.flink.runtime.dispatcher.runner.DispatcherRunner; -import org.apache.flink.runtime.dispatcher.runner.DispatcherRunnerFactory; -import org.apache.flink.runtime.dispatcher.runner.StandaloneDispatcherRunnerFactory; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; -import org.apache.flink.runtime.resourcemanager.ResourceManager; -import org.apache.flink.runtime.resourcemanager.ResourceManagerFactory; -import org.apache.flink.runtime.rest.SessionRestEndpointFactory; -import org.apache.flink.runtime.webmonitor.WebMonitorEndpoint; - -import javax.annotation.Nonnull; - -/** - * {@link DispatcherResourceManagerComponentFactory} for a {@link SessionDispatcherResourceManagerComponent}. - */ -public class SessionDispatcherResourceManagerComponentFactory extends AbstractDispatcherResourceManagerComponentFactory { - - public SessionDispatcherResourceManagerComponentFactory(@Nonnull ResourceManagerFactory resourceManagerFactory) { - this(new StandaloneDispatcherRunnerFactory(SessionDispatcherFactory.INSTANCE), resourceManagerFactory); - } - - @VisibleForTesting - public SessionDispatcherResourceManagerComponentFactory( - @Nonnull DispatcherRunnerFactory dispatcherFactory, - @Nonnull ResourceManagerFactory resourceManagerFactory) { - super(dispatcherFactory, resourceManagerFactory, SessionRestEndpointFactory.INSTANCE); - } - - @Override - protected DispatcherResourceManagerComponent createDispatcherResourceManagerComponent( - DispatcherRunner dispatcherRunner, - ResourceManager resourceManager, - LeaderRetrievalService dispatcherLeaderRetrievalService, - LeaderRetrievalService resourceManagerRetrievalService, - WebMonitorEndpoint webMonitorEndpoint) { - return new SessionDispatcherResourceManagerComponent( - dispatcherRunner, - resourceManager, - dispatcherLeaderRetrievalService, - resourceManagerRetrievalService, - webMonitorEndpoint); - } -} 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 ca7264aad053..466269c10999 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 @@ -40,8 +40,9 @@ import org.apache.flink.runtime.dispatcher.DispatcherId; import org.apache.flink.runtime.dispatcher.MemoryArchivedExecutionGraphStore; import org.apache.flink.runtime.entrypoint.ClusterInformation; +import org.apache.flink.runtime.entrypoint.component.DefaultDispatcherResourceManagerComponentFactory; import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponent; -import org.apache.flink.runtime.entrypoint.component.SessionDispatcherResourceManagerComponentFactory; +import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponentFactory; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; @@ -383,7 +384,7 @@ protected Collection createDispatc MetricRegistry metricRegistry, MetricQueryServiceRetriever metricQueryServiceRetriever, FatalErrorHandler fatalErrorHandler) throws Exception { - SessionDispatcherResourceManagerComponentFactory dispatcherResourceManagerComponentFactory = createDispatcherResourceManagerComponentFactory(); + DispatcherResourceManagerComponentFactory dispatcherResourceManagerComponentFactory = createDispatcherResourceManagerComponentFactory(); return Collections.singleton( dispatcherResourceManagerComponentFactory.create( configuration, @@ -398,8 +399,8 @@ protected Collection createDispatc } @Nonnull - private SessionDispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory() { - return new SessionDispatcherResourceManagerComponentFactory(StandaloneResourceManagerFactory.INSTANCE); + private DispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory() { + return DefaultDispatcherResourceManagerComponentFactory.createSessionComponentFactory(StandaloneResourceManagerFactory.INSTANCE); } @VisibleForTesting diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/SessionDispatcherResourceManagerComponent.java b/flink-runtime/src/test/java/org/apache/flink/runtime/entrypoint/component/TestingDefaultDispatcherResourceManagerComponentFactory.java similarity index 51% rename from flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/SessionDispatcherResourceManagerComponent.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/entrypoint/component/TestingDefaultDispatcherResourceManagerComponentFactory.java index 442f05621a00..cc53537865ed 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/SessionDispatcherResourceManagerComponent.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/entrypoint/component/TestingDefaultDispatcherResourceManagerComponentFactory.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,21 +18,21 @@ package org.apache.flink.runtime.entrypoint.component; -import org.apache.flink.runtime.dispatcher.runner.DispatcherRunner; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; -import org.apache.flink.runtime.resourcemanager.ResourceManager; -import org.apache.flink.runtime.webmonitor.WebMonitorEndpoint; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.dispatcher.runner.DispatcherRunnerFactory; +import org.apache.flink.runtime.resourcemanager.ResourceManagerFactory; +import org.apache.flink.runtime.rest.SessionRestEndpointFactory; /** - * {@link DispatcherResourceManagerComponent} used by session clusters. + * Testing extension of the {@link DefaultDispatcherResourceManagerComponentFactory}. */ -class SessionDispatcherResourceManagerComponent extends DispatcherResourceManagerComponent { - SessionDispatcherResourceManagerComponent( - DispatcherRunner dispatcherRunner, - ResourceManager resourceManager, - LeaderRetrievalService dispatcherLeaderRetrievalService, - LeaderRetrievalService resourceManagerRetrievalService, - WebMonitorEndpoint webMonitorEndpoint) { - super(dispatcherRunner, resourceManager, dispatcherLeaderRetrievalService, resourceManagerRetrievalService, webMonitorEndpoint); +public class TestingDefaultDispatcherResourceManagerComponentFactory { + public static DispatcherResourceManagerComponentFactory createSessionComponentFactory( + DispatcherRunnerFactory dispatcherRunnerFactory, + ResourceManagerFactory resourceManagerFactory) { + return new DefaultDispatcherResourceManagerComponentFactory<>( + dispatcherRunnerFactory, + resourceManagerFactory, + SessionRestEndpointFactory.INSTANCE); } } 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 691a2a7063fc..a73ef9f88eea 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 @@ -24,7 +24,8 @@ import org.apache.flink.runtime.dispatcher.MemoryArchivedExecutionGraphStore; import org.apache.flink.runtime.dispatcher.runner.StandaloneDispatcherRunnerFactory; import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponent; -import org.apache.flink.runtime.entrypoint.component.SessionDispatcherResourceManagerComponentFactory; +import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponentFactory; +import org.apache.flink.runtime.entrypoint.component.TestingDefaultDispatcherResourceManagerComponentFactory; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.metrics.MetricRegistry; @@ -107,7 +108,7 @@ protected Collection createDispatc MetricRegistry metricRegistry, MetricQueryServiceRetriever metricQueryServiceRetriever, FatalErrorHandler fatalErrorHandler) throws Exception { - SessionDispatcherResourceManagerComponentFactory dispatcherResourceManagerComponentFactory = createTestingDispatcherResourceManagerComponentFactory(); + DispatcherResourceManagerComponentFactory dispatcherResourceManagerComponentFactory = createTestingDispatcherResourceManagerComponentFactory(); final List result = new ArrayList<>(numberDispatcherResourceManagerComponents); @@ -133,8 +134,8 @@ public CompletableFuture getDispatcherGatewayFuture() { return super.getDispatcherGatewayFuture(); } - private SessionDispatcherResourceManagerComponentFactory createTestingDispatcherResourceManagerComponentFactory() { - return new SessionDispatcherResourceManagerComponentFactory( + private DispatcherResourceManagerComponentFactory createTestingDispatcherResourceManagerComponentFactory() { + return TestingDefaultDispatcherResourceManagerComponentFactory.createSessionComponentFactory( new StandaloneDispatcherRunnerFactory(SessionDispatcherWithUUIDFactory.INSTANCE), StandaloneResourceManagerWithUUIDFactory.INSTANCE); } 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 f7d9797026ab..11d0c82c7428 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 @@ -42,8 +42,9 @@ import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.dispatcher.DispatcherId; import org.apache.flink.runtime.dispatcher.MemoryArchivedExecutionGraphStore; +import org.apache.flink.runtime.entrypoint.component.DefaultDispatcherResourceManagerComponentFactory; import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponent; -import org.apache.flink.runtime.entrypoint.component.SessionDispatcherResourceManagerComponentFactory; +import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponentFactory; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; @@ -122,7 +123,7 @@ public void testCancelingOnProcessFailure() throws Exception { final int jobManagerPort = rpcService.getPort(); config.setInteger(JobManagerOptions.PORT, jobManagerPort); - final SessionDispatcherResourceManagerComponentFactory resourceManagerComponentFactory = new SessionDispatcherResourceManagerComponentFactory( + final DispatcherResourceManagerComponentFactory resourceManagerComponentFactory = DefaultDispatcherResourceManagerComponentFactory.createSessionComponentFactory( StandaloneResourceManagerFactory.INSTANCE); DispatcherResourceManagerComponent dispatcherResourceManagerComponent = null; diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnJobClusterEntrypoint.java b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnJobClusterEntrypoint.java index 039722615423..3058ddae46ca 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnJobClusterEntrypoint.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnJobClusterEntrypoint.java @@ -21,9 +21,8 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; import org.apache.flink.runtime.entrypoint.JobClusterEntrypoint; -import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponentFactory; +import org.apache.flink.runtime.entrypoint.component.DefaultDispatcherResourceManagerComponentFactory; import org.apache.flink.runtime.entrypoint.component.FileJobGraphRetriever; -import org.apache.flink.runtime.entrypoint.component.JobDispatcherResourceManagerComponentFactory; import org.apache.flink.runtime.security.SecurityContext; import org.apache.flink.runtime.util.EnvironmentInformation; import org.apache.flink.runtime.util.JvmShutdownSafeguard; @@ -62,8 +61,8 @@ protected String getRPCPortRange(Configuration configuration) { } @Override - protected DispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration) { - return new JobDispatcherResourceManagerComponentFactory( + protected DefaultDispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration) { + return DefaultDispatcherResourceManagerComponentFactory.createJobComponentFactory( YarnResourceManagerFactory.getInstance(), FileJobGraphRetriever.createFrom(configuration)); } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnSessionClusterEntrypoint.java b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnSessionClusterEntrypoint.java index b008665bdfb1..bc63e3126e49 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnSessionClusterEntrypoint.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnSessionClusterEntrypoint.java @@ -21,8 +21,8 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; import org.apache.flink.runtime.entrypoint.SessionClusterEntrypoint; +import org.apache.flink.runtime.entrypoint.component.DefaultDispatcherResourceManagerComponentFactory; import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponentFactory; -import org.apache.flink.runtime.entrypoint.component.SessionDispatcherResourceManagerComponentFactory; import org.apache.flink.runtime.security.SecurityContext; import org.apache.flink.runtime.util.EnvironmentInformation; import org.apache.flink.runtime.util.JvmShutdownSafeguard; @@ -61,7 +61,7 @@ protected String getRPCPortRange(Configuration configuration) { @Override protected DispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration) { - return new SessionDispatcherResourceManagerComponentFactory(YarnResourceManagerFactory.getInstance()); + return DefaultDispatcherResourceManagerComponentFactory.createSessionComponentFactory(YarnResourceManagerFactory.getInstance()); } public static void main(String[] args) { From 1c5f7558626bb8c301d564c0cbc15126d6cc1731 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 29 Sep 2019 18:24:25 +0200 Subject: [PATCH 009/746] [FLINK-14284] Add shut down future to Dispatcher This commit adds the shutDownFuture to all Dispatcher implementations. Moreover, forwards this signal to the DispatcherResourceManagerComponent via the DispatcherRunnerImpl. This closes #9810. --- .../flink/runtime/dispatcher/Dispatcher.java | 15 ++++++++++++++- .../flink/runtime/dispatcher/MiniDispatcher.java | 13 +++---------- .../dispatcher/runner/DispatcherRunnerImpl.java | 10 +--------- .../runner/MiniDispatcherRunnerImpl.java | 2 +- .../flink/runtime/dispatcher/DispatcherTest.java | 4 ++-- .../runtime/dispatcher/MiniDispatcherTest.java | 2 +- 6 files changed, 22 insertions(+), 24 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 742e787f72ef..bf4518cd89fa 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 @@ -29,6 +29,7 @@ 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.ApplicationStatus; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; @@ -134,6 +135,8 @@ public abstract class Dispatcher extends FencedRpcEndpoint impleme private CompletableFuture recoveryOperation = CompletableFuture.completedFuture(null); + protected final CompletableFuture shutDownFuture; + public Dispatcher( RpcService rpcService, String endpointId, @@ -169,6 +172,16 @@ public Dispatcher( this.jobManagerRunnerFactory = dispatcherServices.getJobManagerRunnerFactory(); this.jobManagerTerminationFutures = new HashMap<>(2); + + this.shutDownFuture = new CompletableFuture<>(); + } + + //------------------------------------------------------ + // Getters + //------------------------------------------------------ + + public CompletableFuture getShutDownFuture() { + return shutDownFuture; } //------------------------------------------------------ @@ -603,7 +616,7 @@ public CompletableFuture stopWithSavepoint( @Override public CompletableFuture shutDownCluster() { - closeAsync(); + shutDownFuture.complete(ApplicationStatus.SUCCEEDED); return CompletableFuture.completedFuture(Acknowledge.get()); } 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 6aefd00b0e59..2fc805c34fbc 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 @@ -46,8 +46,6 @@ public class MiniDispatcher extends Dispatcher { private final JobClusterEntrypoint.ExecutionMode executionMode; - private final CompletableFuture jobTerminationFuture; - public MiniDispatcher( RpcService rpcService, String endpointId, @@ -61,11 +59,6 @@ public MiniDispatcher( new SingleJobJobGraphStore(jobGraph)); this.executionMode = checkNotNull(executionMode); - this.jobTerminationFuture = new CompletableFuture<>(); - } - - public CompletableFuture getJobTerminationFuture() { - return jobTerminationFuture; } @Override @@ -94,7 +87,7 @@ public CompletableFuture requestJobResult(JobID jobId, Time timeout) ApplicationStatus status = result.getSerializedThrowable().isPresent() ? ApplicationStatus.FAILED : ApplicationStatus.SUCCEEDED; - jobTerminationFuture.complete(status); + shutDownFuture.complete(status); }); } @@ -107,7 +100,7 @@ protected void jobReachedGloballyTerminalState(ArchivedExecutionGraph archivedEx if (executionMode == ClusterEntrypoint.ExecutionMode.DETACHED) { // shut down since we don't have to wait for the execution result retrieval - jobTerminationFuture.complete(ApplicationStatus.fromJobStatus(archivedExecutionGraph.getState())); + shutDownFuture.complete(ApplicationStatus.fromJobStatus(archivedExecutionGraph.getState())); } } @@ -116,6 +109,6 @@ protected void jobNotFinished(JobID jobId) { super.jobNotFinished(jobId); // shut down since we have done our job - jobTerminationFuture.complete(ApplicationStatus.UNKNOWN); + shutDownFuture.complete(ApplicationStatus.UNKNOWN); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java index 6a0c85228aa7..7532091de3fd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.dispatcher.runner; import org.apache.flink.runtime.clusterframework.ApplicationStatus; -import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.dispatcher.Dispatcher; import org.apache.flink.runtime.dispatcher.DispatcherFactory; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; @@ -34,8 +33,6 @@ class DispatcherRunnerImpl implements DispatcherRunner { private final T dispatcher; - private final CompletableFuture shutDownFuture; - DispatcherRunnerImpl( DispatcherFactory dispatcherFactory, RpcService rpcService, @@ -43,11 +40,6 @@ class DispatcherRunnerImpl implements DispatcherRunner { this.dispatcher = dispatcherFactory.createDispatcher( rpcService, partialDispatcherServices); - shutDownFuture = new CompletableFuture<>(); - - FutureUtils.forward( - dispatcher.getTerminationFuture().thenApply((ignored) -> ApplicationStatus.UNKNOWN), - shutDownFuture); dispatcher.start(); } @@ -64,6 +56,6 @@ public CompletableFuture closeAsync() { @Override public CompletableFuture getShutDownFuture() { - return shutDownFuture; + return dispatcher.getShutDownFuture(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerImpl.java index 0fa25ae28037..eaac5c24ff3e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerImpl.java @@ -40,6 +40,6 @@ class MiniDispatcherRunnerImpl extends DispatcherRunnerImpl { @Override public CompletableFuture getShutDownFuture() { - return getDispatcher().getJobTerminationFuture(); + return getDispatcher().getShutDownFuture(); } } 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 44d0d04e51f5..46620142f154 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 @@ -769,14 +769,14 @@ public void testJobSuspensionWhenDispatcherLosesLeadership() throws Exception { } @Test - public void testShutDownClusterShouldTerminateDispatcher() throws Exception { + public void testShutDownClusterShouldCompleteShutDownFuture() throws Exception { dispatcher = createAndStartDispatcher(heartbeatServices, haServices, DefaultJobManagerRunnerFactory.INSTANCE); dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); final DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); dispatcherGateway.shutDownCluster().get(); - dispatcher.getTerminationFuture().get(); + dispatcher.getShutDownFuture().get(); } private final class BlockingJobManagerRunnerFactory extends TestingJobManagerRunnerFactory { 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 bc2a239da277..8f855298c3a2 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 @@ -180,7 +180,7 @@ public void testTerminationAfterJobCompletion() throws Exception { testingJobManagerRunner.completeResultFuture(archivedExecutionGraph); // wait until we terminate - miniDispatcher.getJobTerminationFuture().get(); + miniDispatcher.getShutDownFuture().get(); } finally { RpcUtils.terminateRpcEndpoint(miniDispatcher, timeout); } From 282dd840fe80bd71830d1c493cdb7e1d0ad5cc6e Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 29 Sep 2019 18:41:34 +0200 Subject: [PATCH 010/746] [FLINK-14285] Remove generics from Dispatcher factories Simplify Dispatcher factories by removing generics from them. This closes #9811. --- .../runtime/dispatcher/DispatcherFactory.java | 6 +-- .../dispatcher/JobDispatcherFactory.java | 2 +- .../dispatcher/SessionDispatcherFactory.java | 2 +- ....java => DispatcherRunnerFactoryImpl.java} | 12 ++--- .../runner/DispatcherRunnerImpl.java | 8 ++-- .../runner/MiniDispatcherRunnerFactory.java | 46 ------------------- .../runner/MiniDispatcherRunnerImpl.java | 45 ------------------ ...atcherResourceManagerComponentFactory.java | 27 +++++------ ...atcherResourceManagerComponentFactory.java | 2 +- .../SessionDispatcherWithUUIDFactory.java | 2 +- .../minicluster/TestingMiniCluster.java | 4 +- 11 files changed, 31 insertions(+), 125 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/{StandaloneDispatcherRunnerFactory.java => DispatcherRunnerFactoryImpl.java} (72%) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerFactory.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerImpl.java 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 4de454278de5..4e7ea67ebc1c 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 @@ -27,12 +27,12 @@ /** * {@link Dispatcher} factory interface. */ -public interface DispatcherFactory { +public interface DispatcherFactory { /** - * Create a {@link Dispatcher} of the given type {@link T}. + * Create a {@link Dispatcher}. */ - T createDispatcher( + Dispatcher createDispatcher( @Nonnull RpcService rpcService, @Nonnull PartialDispatcherServices partialDispatcherServices) throws Exception; 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 a74610e047cc..6ca6157351da 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 @@ -31,7 +31,7 @@ /** * {@link DispatcherFactory} which creates a {@link MiniDispatcher}. */ -public class JobDispatcherFactory implements DispatcherFactory { +public class JobDispatcherFactory implements DispatcherFactory { private final JobGraphRetriever jobGraphRetriever; 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 0d29a0c2a580..a87992bb20a0 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 @@ -23,7 +23,7 @@ /** * {@link DispatcherFactory} which creates a {@link StandaloneDispatcher}. */ -public enum SessionDispatcherFactory implements DispatcherFactory { +public enum SessionDispatcherFactory implements DispatcherFactory { INSTANCE; @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StandaloneDispatcherRunnerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerFactoryImpl.java similarity index 72% rename from flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StandaloneDispatcherRunnerFactory.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerFactoryImpl.java index 4560c2335c59..e0926217ea88 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StandaloneDispatcherRunnerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerFactoryImpl.java @@ -26,18 +26,18 @@ /** * Factory which creates a {@link DispatcherRunnerImpl} which runs a {@link StandaloneDispatcher}. */ -public class StandaloneDispatcherRunnerFactory implements DispatcherRunnerFactory { +public class DispatcherRunnerFactoryImpl implements DispatcherRunnerFactory { - private final DispatcherFactory dispatcherFactory; + private final DispatcherFactory dispatcherFactory; - public StandaloneDispatcherRunnerFactory(DispatcherFactory dispatcherFactory) { + public DispatcherRunnerFactoryImpl(DispatcherFactory dispatcherFactory) { this.dispatcherFactory = dispatcherFactory; } @Override - public DispatcherRunnerImpl createDispatcherRunner( + public DispatcherRunnerImpl createDispatcherRunner( RpcService rpcService, - PartialDispatcherServices dispatcherFactoryServices) throws Exception { - return new DispatcherRunnerImpl<>(dispatcherFactory, rpcService, dispatcherFactoryServices); + PartialDispatcherServices partialDispatcherServices) throws Exception { + return new DispatcherRunnerImpl(dispatcherFactory, rpcService, partialDispatcherServices); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java index 7532091de3fd..a5f7b6a58335 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java @@ -29,12 +29,12 @@ /** * Runner responsible for executing a {@link Dispatcher} or a subclass thereof. */ -class DispatcherRunnerImpl implements DispatcherRunner { +class DispatcherRunnerImpl implements DispatcherRunner { - private final T dispatcher; + private final Dispatcher dispatcher; DispatcherRunnerImpl( - DispatcherFactory dispatcherFactory, + DispatcherFactory dispatcherFactory, RpcService rpcService, PartialDispatcherServices partialDispatcherServices) throws Exception { this.dispatcher = dispatcherFactory.createDispatcher( @@ -45,7 +45,7 @@ class DispatcherRunnerImpl implements DispatcherRunner { } @Override - public T getDispatcher() { + public Dispatcher getDispatcher() { return dispatcher; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerFactory.java deleted file mode 100644 index 48ef6744615e..000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerFactory.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.dispatcher.runner; - -import org.apache.flink.runtime.dispatcher.JobDispatcherFactory; -import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; -import org.apache.flink.runtime.entrypoint.component.JobGraphRetriever; -import org.apache.flink.runtime.rpc.RpcService; - -/** - * Factory for the {@link MiniDispatcherRunnerImpl}. - */ -public class MiniDispatcherRunnerFactory implements DispatcherRunnerFactory { - - private final JobGraphRetriever jobGraphRetriever; - - public MiniDispatcherRunnerFactory(JobGraphRetriever jobGraphRetriever) { - this.jobGraphRetriever = jobGraphRetriever; - } - - @Override - public MiniDispatcherRunnerImpl createDispatcherRunner( - RpcService rpcService, - PartialDispatcherServices partialDispatcherServices) throws Exception { - return new MiniDispatcherRunnerImpl( - new JobDispatcherFactory(jobGraphRetriever), - rpcService, - partialDispatcherServices); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerImpl.java deleted file mode 100644 index eaac5c24ff3e..000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/MiniDispatcherRunnerImpl.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.runtime.dispatcher.runner; - -import org.apache.flink.runtime.clusterframework.ApplicationStatus; -import org.apache.flink.runtime.dispatcher.DispatcherFactory; -import org.apache.flink.runtime.dispatcher.MiniDispatcher; -import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; -import org.apache.flink.runtime.rpc.RpcService; - -import java.util.concurrent.CompletableFuture; - -/** - * Runner which runs a {@link MiniDispatcher} implementation. - */ -class MiniDispatcherRunnerImpl extends DispatcherRunnerImpl { - - MiniDispatcherRunnerImpl( - DispatcherFactory dispatcherFactory, - RpcService rpcService, - PartialDispatcherServices partialDispatcherServices) throws Exception { - super(dispatcherFactory, rpcService, partialDispatcherServices); - } - - @Override - public CompletableFuture getShutDownFuture() { - return getDispatcher().getShutDownFuture(); - } -} 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 6c7aa789372e..dd89a757a6a0 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 @@ -29,12 +29,12 @@ 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.JobDispatcherFactory; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; import org.apache.flink.runtime.dispatcher.SessionDispatcherFactory; import org.apache.flink.runtime.dispatcher.runner.DispatcherRunner; import org.apache.flink.runtime.dispatcher.runner.DispatcherRunnerFactory; -import org.apache.flink.runtime.dispatcher.runner.MiniDispatcherRunnerFactory; -import org.apache.flink.runtime.dispatcher.runner.StandaloneDispatcherRunnerFactory; +import org.apache.flink.runtime.dispatcher.runner.DispatcherRunnerFactoryImpl; import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; @@ -56,7 +56,6 @@ 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; import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever; @@ -76,10 +75,8 @@ /** * Abstract class which implements the creation of the {@link DispatcherResourceManagerComponent} components. - * - * @param type of the {@link RestfulGateway} given to the {@link WebMonitorEndpoint} */ -public class DefaultDispatcherResourceManagerComponentFactory implements DispatcherResourceManagerComponentFactory { +public class DefaultDispatcherResourceManagerComponentFactory implements DispatcherResourceManagerComponentFactory { private final Logger log = LoggerFactory.getLogger(getClass()); @@ -90,12 +87,12 @@ public class DefaultDispatcherResourceManagerComponentFactory resourceManagerFactory; @Nonnull - private final RestEndpointFactory restEndpointFactory; + private final RestEndpointFactory restEndpointFactory; DefaultDispatcherResourceManagerComponentFactory( @Nonnull DispatcherRunnerFactory dispatcherRunnerFactory, @Nonnull ResourceManagerFactory resourceManagerFactory, - @Nonnull RestEndpointFactory restEndpointFactory) { + @Nonnull RestEndpointFactory restEndpointFactory) { this.dispatcherRunnerFactory = dispatcherRunnerFactory; this.resourceManagerFactory = resourceManagerFactory; this.restEndpointFactory = restEndpointFactory; @@ -115,7 +112,7 @@ public DispatcherResourceManagerComponent create( LeaderRetrievalService dispatcherLeaderRetrievalService = null; LeaderRetrievalService resourceManagerRetrievalService = null; - WebMonitorEndpoint webMonitorEndpoint = null; + WebMonitorEndpoint webMonitorEndpoint = null; ResourceManager resourceManager = null; JobManagerMetricGroup jobManagerMetricGroup = null; ResourceManagerMetricGroup resourceManagerMetricGroup = null; @@ -269,19 +266,19 @@ public DispatcherResourceManagerComponent create( } } - public static DefaultDispatcherResourceManagerComponentFactory createSessionComponentFactory( + public static DefaultDispatcherResourceManagerComponentFactory createSessionComponentFactory( ResourceManagerFactory resourceManagerFactory) { - return new DefaultDispatcherResourceManagerComponentFactory<>( - new StandaloneDispatcherRunnerFactory(SessionDispatcherFactory.INSTANCE), + return new DefaultDispatcherResourceManagerComponentFactory( + new DispatcherRunnerFactoryImpl(SessionDispatcherFactory.INSTANCE), resourceManagerFactory, SessionRestEndpointFactory.INSTANCE); } - public static DefaultDispatcherResourceManagerComponentFactory createJobComponentFactory( + public static DefaultDispatcherResourceManagerComponentFactory createJobComponentFactory( ResourceManagerFactory resourceManagerFactory, JobGraphRetriever jobGraphRetriever) { - return new DefaultDispatcherResourceManagerComponentFactory<>( - new MiniDispatcherRunnerFactory(jobGraphRetriever), + return new DefaultDispatcherResourceManagerComponentFactory( + new DispatcherRunnerFactoryImpl(new JobDispatcherFactory(jobGraphRetriever)), resourceManagerFactory, JobRestEndpointFactory.INSTANCE); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/entrypoint/component/TestingDefaultDispatcherResourceManagerComponentFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/entrypoint/component/TestingDefaultDispatcherResourceManagerComponentFactory.java index cc53537865ed..03fe15f87b35 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/entrypoint/component/TestingDefaultDispatcherResourceManagerComponentFactory.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/entrypoint/component/TestingDefaultDispatcherResourceManagerComponentFactory.java @@ -30,7 +30,7 @@ public class TestingDefaultDispatcherResourceManagerComponentFactory { public static DispatcherResourceManagerComponentFactory createSessionComponentFactory( DispatcherRunnerFactory dispatcherRunnerFactory, ResourceManagerFactory resourceManagerFactory) { - return new DefaultDispatcherResourceManagerComponentFactory<>( + return new DefaultDispatcherResourceManagerComponentFactory( dispatcherRunnerFactory, resourceManagerFactory, SessionRestEndpointFactory.INSTANCE); 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 ac6a70e05e4e..307c3cfeb38d 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 @@ -31,7 +31,7 @@ * {@link DispatcherFactory} which creates a {@link StandaloneDispatcher} which has an * endpoint id with a random UUID suffix. */ -public enum SessionDispatcherWithUUIDFactory implements DispatcherFactory { +public enum SessionDispatcherWithUUIDFactory implements DispatcherFactory { INSTANCE; @Override 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 a73ef9f88eea..a4bb3dcd5157 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 @@ -22,7 +22,7 @@ import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.dispatcher.MemoryArchivedExecutionGraphStore; -import org.apache.flink.runtime.dispatcher.runner.StandaloneDispatcherRunnerFactory; +import org.apache.flink.runtime.dispatcher.runner.DispatcherRunnerFactoryImpl; import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponent; import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponentFactory; import org.apache.flink.runtime.entrypoint.component.TestingDefaultDispatcherResourceManagerComponentFactory; @@ -136,7 +136,7 @@ public CompletableFuture getDispatcherGatewayFuture() { private DispatcherResourceManagerComponentFactory createTestingDispatcherResourceManagerComponentFactory() { return TestingDefaultDispatcherResourceManagerComponentFactory.createSessionComponentFactory( - new StandaloneDispatcherRunnerFactory(SessionDispatcherWithUUIDFactory.INSTANCE), + new DispatcherRunnerFactoryImpl(SessionDispatcherWithUUIDFactory.INSTANCE), StandaloneResourceManagerWithUUIDFactory.INSTANCE); } } From 1082c7a470355b938ad0f7de2fa73ca5461c1bc0 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 7 Jun 2019 21:24:58 +0200 Subject: [PATCH 011/746] [FLINK-14286] Remove Akka specific parsing from LeaderConnectionInfo Remove unused Akka specific parsing from LeaderConnectionInfo. This closes #9812. --- .../runtime/util/LeaderConnectionInfo.java | 56 +++---------------- .../runtime/util/LeaderRetrievalUtils.java | 10 +--- .../dispatcher/ZooKeeperHADispatcherTest.java | 2 +- .../LeaderChangeClusterComponentsTest.java | 2 +- .../ProcessFailureCancelingITCase.java | 2 +- 5 files changed, 13 insertions(+), 59 deletions(-) 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 2c94c43826e2..3c9ed142a5c8 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 @@ -18,74 +18,34 @@ package org.apache.flink.runtime.util; -import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.util.FlinkException; - -import akka.actor.Address; - -import java.net.MalformedURLException; import java.util.UUID; /** * Wrapper class for a pair of connection address and leader session ID. */ public class LeaderConnectionInfo { + private final UUID leaderSessionId; private final String address; - private final UUID leaderSessionID; - - private final String hostname; - - private final int port; - - public LeaderConnectionInfo(String address, UUID leaderSessionID) throws FlinkException { + LeaderConnectionInfo(UUID leaderSessionId, String address) { + this.leaderSessionId = leaderSessionId; this.address = address; - this.leaderSessionID = leaderSessionID; - - final Address akkaAddress; - // this only works as long as the address is Akka based - try { - akkaAddress = AkkaUtils.getAddressFromAkkaURL(address); - } catch (MalformedURLException e) { - throw new FlinkException("Could not extract the hostname from the given address \'" + - address + "\'.", e); - } - - if (akkaAddress.host().isDefined()) { - hostname = akkaAddress.host().get(); - } else { - hostname = "localhost"; - } + } - if (akkaAddress.port().isDefined()) { - port = (int) akkaAddress.port().get(); - } else { - port = -1; - } + public UUID getLeaderSessionId() { + return leaderSessionId; } public String getAddress() { return address; } - public UUID getLeaderSessionID() { - return leaderSessionID; - } - - public String getHostname() { - return hostname; - } - - public int getPort() { - return port; - } - @Override public String toString() { return "LeaderConnectionInfo{" + - "address='" + address + '\'' + - ", leaderSessionID=" + leaderSessionID + + "leaderSessionId=" + leaderSessionId + + ", address='" + address + '\'' + '}'; } } 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 e3b2e2722801..3012bbb10229 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 @@ -23,7 +23,6 @@ import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.net.ConnectionUtils; -import org.apache.flink.util.FlinkException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -120,13 +119,8 @@ public Future getLeaderConnectionInfoFuture() { @Override public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) { if (leaderAddress != null && !leaderAddress.equals("") && !connectionInfo.isCompleted()) { - try { - final LeaderConnectionInfo leaderConnectionInfo = new LeaderConnectionInfo(leaderAddress, leaderSessionID); - connectionInfo.success(leaderConnectionInfo); - } catch (FlinkException e) { - connectionInfo.failure(e); - } - + final LeaderConnectionInfo leaderConnectionInfo = new LeaderConnectionInfo(leaderSessionID, leaderAddress); + connectionInfo.success(leaderConnectionInfo); } } 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 bb29c258b310..b40946c7431f 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 @@ -296,7 +296,7 @@ public void testStandbyDispatcherJobRecovery() throws Exception { haServices.getDispatcherLeaderRetriever(), Duration.ofMillis(TIMEOUT.toMilliseconds())); - final DispatcherGateway dispatcherGateway = rpcService.connect(leaderConnectionInfo.getAddress(), DispatcherId.fromUuid(leaderConnectionInfo.getLeaderSessionID()), DispatcherGateway.class).get(); + final DispatcherGateway dispatcherGateway = rpcService.connect(leaderConnectionInfo.getAddress(), DispatcherId.fromUuid(leaderConnectionInfo.getLeaderSessionId()), DispatcherGateway.class).get(); final JobGraph nonEmptyJobGraph = DispatcherHATest.createNonEmptyJobGraph(); dispatcherGateway.submitJob(nonEmptyJobGraph, TIMEOUT).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 cd9b04770594..0731d96fee28 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 @@ -159,7 +159,7 @@ public void testTaskExecutorsReconnectToClusterWithLeadershipChange() throws Exc assertThat( LeaderRetrievalUtils.retrieveLeaderConnectionInfo( highAvailabilityServices.getResourceManagerLeaderRetriever(), - TESTING_TIMEOUT).getLeaderSessionID(), + TESTING_TIMEOUT).getLeaderSessionId(), is(notNullValue())); waitUntilTaskExecutorsHaveConnected(NUM_TMS, deadline); 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 11d0c82c7428..3b0fa172fcd3 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 @@ -272,7 +272,7 @@ static DispatcherGateway retrieveDispatcherGateway(RpcService rpcService, HighAv return rpcService.connect( leaderConnectionInfo.getAddress(), - DispatcherId.fromUuid(leaderConnectionInfo.getLeaderSessionID()), + DispatcherId.fromUuid(leaderConnectionInfo.getLeaderSessionId()), DispatcherGateway.class).get(); } From 44416549a9f63abb9b89f0cd84fa97cd9115a128 Mon Sep 17 00:00:00 2001 From: caoyingjie Date: Tue, 8 Oct 2019 16:43:05 +0800 Subject: [PATCH 012/746] [FLINK-14118][benchmarks] Add network throughput benchmark for data skew scenario. (#9850) --- ...aSkewStreamNetworkThroughputBenchmark.java | 65 +++++++++++++++++++ ...wStreamNetworkThroughputBenchmarkTest.java | 29 +++++++++ .../StreamNetworkThroughputBenchmark.java | 10 ++- 3 files changed, 101 insertions(+), 3 deletions(-) create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/DataSkewStreamNetworkThroughputBenchmark.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/DataSkewStreamNetworkThroughputBenchmarkTest.java diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/DataSkewStreamNetworkThroughputBenchmark.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/DataSkewStreamNetworkThroughputBenchmark.java new file mode 100644 index 000000000000..d97d3f4cec51 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/DataSkewStreamNetworkThroughputBenchmark.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.streaming.runtime.io.benchmark; + +import org.apache.flink.core.io.IOReadableWritable; +import org.apache.flink.runtime.io.network.api.writer.ChannelSelector; +import org.apache.flink.runtime.io.network.api.writer.RecordWriterBuilder; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * Network throughput benchmarks for data skew scenario executed by the external + * flink-benchmarks project. + */ +public class DataSkewStreamNetworkThroughputBenchmark extends StreamNetworkThroughputBenchmark { + + @Override + protected void setChannelSelector(RecordWriterBuilder recordWriterBuilder, boolean broadcastMode) { + checkArgument(!broadcastMode, "Combining broadcasting with data skew doesn't make sense"); + recordWriterBuilder.setChannelSelector(new DataSkewChannelSelector()); + } + + /** + * A {@link ChannelSelector} which selects channel 0 for nearly all records. And all other channels + * except for channel 0 will be only selected at most once. + */ + private static class DataSkewChannelSelector implements ChannelSelector { + private int numberOfChannels; + private int channelIndex = 0; + + @Override + public void setup(int numberOfChannels) { + this.numberOfChannels = numberOfChannels; + } + + @Override + public int selectChannel(IOReadableWritable record) { + if (channelIndex >= numberOfChannels) { + return 0; + } + return channelIndex++; + } + + @Override + public boolean isBroadcast() { + return false; + } + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/DataSkewStreamNetworkThroughputBenchmarkTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/DataSkewStreamNetworkThroughputBenchmarkTest.java new file mode 100644 index 000000000000..14c3d8f1bfe3 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/DataSkewStreamNetworkThroughputBenchmarkTest.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.streaming.runtime.io.benchmark; + +/** + * Tests for various network benchmarks based on {@link DataSkewStreamNetworkThroughputBenchmark}. + */ +public class DataSkewStreamNetworkThroughputBenchmarkTest extends StreamNetworkThroughputBenchmarkTest { + @Override + protected StreamNetworkThroughputBenchmark createBenchmark() { + return new DataSkewStreamNetworkThroughputBenchmark(); + } +} 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 ed3a05f87c82..a1afc8bb141a 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 @@ -116,9 +116,7 @@ public void setUp( for (int writer = 0; writer < recordWriters; writer++) { ResultPartitionWriter resultPartitionWriter = environment.createResultPartitionWriter(writer); RecordWriterBuilder recordWriterBuilder = new RecordWriterBuilder().setTimeout(flushTimeout); - if (broadcastMode) { - recordWriterBuilder.setChannelSelector(new BroadcastPartitioner()); - } + setChannelSelector(recordWriterBuilder, broadcastMode); writerThreads[writer] = new LongRecordWriterThread( recordWriterBuilder.build(resultPartitionWriter), broadcastMode); @@ -127,6 +125,12 @@ public void setUp( receiver = environment.createReceiver(); } + protected void setChannelSelector(RecordWriterBuilder recordWriterBuilder, boolean broadcastMode) { + if (broadcastMode) { + recordWriterBuilder.setChannelSelector(new BroadcastPartitioner()); + } + } + /** * Shuts down a benchmark previously set up via {@link #setUp}. * From c5c59feec0ed8a0ac5213802d79956c815a2b812 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Sun, 6 Oct 2019 23:14:50 +0800 Subject: [PATCH 013/746] [FLINK-14271][runtime] Remove legacy RestartPipelinedRegionStrategy components --- .../configuration/JobManagerOptions.java | 10 +- .../failover/FailoverRegion.java | 264 ------- .../failover/FailoverStrategyLoader.java | 6 - .../RestartPipelinedRegionStrategy.java | 268 ------- ...overStrategyCheckpointCoordinatorTest.java | 3 +- ...entFailoverStrategyExecutionGraphTest.java | 540 -------------- .../ExecutionGraphTestUtils.java | 21 - .../executiongraph/FailoverRegionTest.java | 694 ------------------ .../RestartPipelinedRegionStrategyTest.java | 388 ---------- .../PipelinedFailoverRegionBuildingTest.java | 651 ---------------- 10 files changed, 4 insertions(+), 2841 deletions(-) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/FailoverRegion.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/RestartPipelinedRegionStrategy.java delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ConcurrentFailoverStrategyExecutionGraphTest.java delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/FailoverRegionTest.java delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/RestartPipelinedRegionStrategyTest.java delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/PipelinedFailoverRegionBuildingTest.java 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 a6d6f05daa23..f35977b09532 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 @@ -107,14 +107,10 @@ public class JobManagerOptions { /** * This option specifies the failover strategy, i.e. how the job computation recovers from task failures. * - *

The options "individual" and "region-legacy" are intentionally not included - * as they have some known limitations or issues: - *

    - *
  • "individual" strategy only works when all tasks are not connected, in which case the "region" + *

    The option "individual" is intentionally not included for its known limitations. + * It only works when all tasks are not connected, in which case the "region" * failover strategy would also restart failed tasks individually. - *

  • "region-legacy" strategy is not able to backtrack missing input result partitions. - *
- * The new "region" strategy supersedes "individual" and "region-legacy" strategies and should always work. + * The new "region" strategy supersedes "individual" strategy and should always work. */ public static final ConfigOption EXECUTION_FAILOVER_STRATEGY = key("jobmanager.execution.failover-strategy") diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/FailoverRegion.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/FailoverRegion.java deleted file mode 100644 index 9a93bb8dbde3..000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/FailoverRegion.java +++ /dev/null @@ -1,264 +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.annotation.VisibleForTesting; -import org.apache.flink.runtime.checkpoint.CheckpointException; -import org.apache.flink.runtime.checkpoint.CheckpointFailureReason; -import org.apache.flink.runtime.clusterframework.types.AllocationID; -import org.apache.flink.runtime.concurrent.FutureUtils; -import org.apache.flink.runtime.executiongraph.Execution; -import org.apache.flink.runtime.executiongraph.ExecutionGraph; -import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; -import org.apache.flink.runtime.executiongraph.ExecutionVertex; -import org.apache.flink.runtime.executiongraph.GlobalModVersionMismatch; -import org.apache.flink.runtime.jobgraph.JobStatus; -import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; -import org.apache.flink.runtime.jobmanager.scheduler.LocationPreferenceConstraint; -import org.apache.flink.util.AbstractID; -import org.apache.flink.util.FlinkException; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * FailoverRegion manages the failover of a minimal pipeline connected sub graph. - * It will change from CREATED to CANCELING and then to CANCELLED and at last to RUNNING, - */ -public class FailoverRegion { - - private static final AtomicReferenceFieldUpdater STATE_UPDATER = - AtomicReferenceFieldUpdater.newUpdater(FailoverRegion.class, JobStatus.class, "state"); - - /** The log object used for debugging. */ - private static final Logger LOG = LoggerFactory.getLogger(FailoverRegion.class); - - // ------------------------------------------------------------------------ - - /** a unique id for debugging */ - private final AbstractID id = new AbstractID(); - - private final ExecutionGraph executionGraph; - - private final List connectedExecutionVertexes; - - private final Map tasks; - - /** Current status of the job execution */ - private volatile JobStatus state = JobStatus.RUNNING; - - public FailoverRegion( - ExecutionGraph executionGraph, - List connectedExecutions, - Map tasks) { - - this.executionGraph = checkNotNull(executionGraph); - this.connectedExecutionVertexes = checkNotNull(connectedExecutions); - this.tasks = checkNotNull(tasks); - - LOG.debug("Created failover region {} with vertices: {}", id, connectedExecutions); - } - - public void onExecutionFail(Execution taskExecution, Throwable cause) { - // TODO: check if need to failover the preceding region - if (!executionGraph.getRestartStrategy().canRestart()) { - // delegate the failure to a global fail that will check the restart strategy and not restart - executionGraph.failGlobal(cause); - } - else { - cancel(taskExecution.getGlobalModVersion()); - } - } - - private void allVerticesInTerminalState(long globalModVersionOfFailover) { - while (true) { - JobStatus curStatus = this.state; - if (curStatus.equals(JobStatus.CANCELLING)) { - if (transitionState(curStatus, JobStatus.CANCELED)) { - reset(globalModVersionOfFailover); - break; - } - } - else { - LOG.info("FailoverRegion {} is {} when allVerticesInTerminalState.", id, state); - break; - } - } - } - - public JobStatus getState() { - return state; - } - - // Notice the region to failover, - private void failover(long globalModVersionOfFailover) { - if (!executionGraph.getRestartStrategy().canRestart()) { - executionGraph.failGlobal(new FlinkException("RestartStrategy validate fail")); - } - else { - JobStatus curStatus = this.state; - if (curStatus.equals(JobStatus.RUNNING)) { - cancel(globalModVersionOfFailover); - } - else if (curStatus.equals(JobStatus.CANCELED)) { - reset(globalModVersionOfFailover); - } - else { - LOG.info("FailoverRegion {} is {} when notified to failover.", id, state); - } - } - } - - // cancel all executions in this sub graph - private void cancel(final long globalModVersionOfFailover) { - executionGraph.getJobMasterMainThreadExecutor().assertRunningInMainThread(); - while (true) { - JobStatus curStatus = this.state; - if (curStatus.equals(JobStatus.RUNNING)) { - if (transitionState(curStatus, JobStatus.CANCELLING)) { - - createTerminationFutureOverAllConnectedVertexes() - .thenAccept((nullptr) -> allVerticesInTerminalState(globalModVersionOfFailover)); - break; - } - } else { - LOG.info("FailoverRegion {} is {} when cancel.", id, state); - break; - } - } - } - - @VisibleForTesting - protected CompletableFuture createTerminationFutureOverAllConnectedVertexes() { - // we build a future that is complete once all vertices have reached a terminal state - final ArrayList> futures = new ArrayList<>(connectedExecutionVertexes.size()); - - // cancel all tasks (that still need cancelling) - for (ExecutionVertex vertex : connectedExecutionVertexes) { - futures.add(vertex.cancel()); - } - - return FutureUtils.waitForAll(futures); - } - - // reset all executions in this sub graph - private void reset(long globalModVersionOfFailover) { - try { - // reset all connected ExecutionVertexes - final Collection colGroups = new HashSet<>(); - final long restartTimestamp = System.currentTimeMillis(); - - for (ExecutionVertex ev : connectedExecutionVertexes) { - CoLocationGroup cgroup = ev.getJobVertex().getCoLocationGroup(); - if (cgroup != null && !colGroups.contains(cgroup)){ - cgroup.resetConstraints(); - colGroups.add(cgroup); - } - - ev.resetForNewExecution(restartTimestamp, globalModVersionOfFailover); - } - if (transitionState(JobStatus.CANCELED, JobStatus.CREATED)) { - restart(globalModVersionOfFailover); - } - else { - LOG.info("FailoverRegion {} switched from CANCELLING to CREATED fail, will fail this region again.", id); - failover(globalModVersionOfFailover); - } - } - catch (GlobalModVersionMismatch e) { - // happens when a global recovery happens concurrently to the regional recovery - // go back to a clean state - state = JobStatus.RUNNING; - } - catch (Throwable e) { - LOG.info("FailoverRegion {} reset fail, will failover again.", id); - failover(globalModVersionOfFailover); - } - } - - // restart all executions in this sub graph - private void restart(long globalModVersionOfFailover) { - try { - if (transitionState(JobStatus.CREATED, JobStatus.RUNNING)) { - // if we have checkpointed state, reload it into the executions - if (executionGraph.getCheckpointCoordinator() != null) { - // we abort pending checkpoints for - // i) enable new checkpoint could be triggered without waiting for last checkpoint expired. - // ii) ensure the EXACTLY_ONCE semantics if needed. - executionGraph.getCheckpointCoordinator().abortPendingCheckpoints( - new CheckpointException(CheckpointFailureReason.JOB_FAILOVER_REGION)); - - executionGraph.getCheckpointCoordinator().restoreLatestCheckpointedState( - tasks, false, true); - } - - HashSet previousAllocationsInRegion = new HashSet<>(connectedExecutionVertexes.size()); - for (ExecutionVertex connectedExecutionVertex : connectedExecutionVertexes) { - AllocationID latestPriorAllocation = connectedExecutionVertex.getLatestPriorAllocation(); - if (latestPriorAllocation != null) { - previousAllocationsInRegion.add(latestPriorAllocation); - } - } - - //TODO, use restart strategy to schedule them. - //restart all connected ExecutionVertexes - for (ExecutionVertex ev : connectedExecutionVertexes) { - try { - ev.scheduleForExecution( - executionGraph.getSlotProviderStrategy(), - LocationPreferenceConstraint.ANY, - previousAllocationsInRegion); // some inputs not belonging to the failover region might have failed concurrently - } - catch (Throwable e) { - failover(globalModVersionOfFailover); - } - } - } - else { - LOG.info("FailoverRegion {} switched from CREATED to RUNNING fail, will fail this region again.", id); - failover(globalModVersionOfFailover); - } - } catch (Exception e) { - LOG.info("FailoverRegion {} restart failed, failover again.", id, e); - failover(globalModVersionOfFailover); - } - } - - private boolean transitionState(JobStatus current, JobStatus newState) { - if (STATE_UPDATER.compareAndSet(this, current, newState)) { - LOG.info("FailoverRegion {} switched from state {} to {}.", id, current, newState); - return true; - } - else { - return false; - } - } - -} 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 632f3af47900..e0818275fee0 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 @@ -41,9 +41,6 @@ public class FailoverStrategyLoader { /** Config name for the {@link AdaptedRestartPipelinedRegionStrategyNG}. */ public static final String PIPELINED_REGION_RESTART_STRATEGY_NAME = "region"; - /** 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"; @@ -75,9 +72,6 @@ public static FailoverStrategy.Factory loadFailoverStrategy(Configuration config case PIPELINED_REGION_RESTART_STRATEGY_NAME: return new AdaptedRestartPipelinedRegionStrategyNG.Factory(); - case LEGACY_PIPELINED_REGION_RESTART_STRATEGY_NAME: - return new RestartPipelinedRegionStrategy.Factory(); - case INDIVIDUAL_RESTART_STRATEGY_NAME: return new RestartIndividualStrategy.Factory(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/RestartPipelinedRegionStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/RestartPipelinedRegionStrategy.java deleted file mode 100644 index 811c0a014563..000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/RestartPipelinedRegionStrategy.java +++ /dev/null @@ -1,268 +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.annotation.VisibleForTesting; -import org.apache.flink.runtime.executiongraph.Execution; -import org.apache.flink.runtime.executiongraph.ExecutionEdge; -import org.apache.flink.runtime.executiongraph.ExecutionGraph; -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.jobgraph.JobVertexID; -import org.apache.flink.util.FlinkException; -import org.apache.flink.util.FlinkRuntimeException; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.IdentityHashMap; -import java.util.List; -import java.util.Map; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * A failover strategy that restarts regions of the ExecutionGraph with state. A region is defined - * by this strategy as the weakly connected component of tasks that communicate via pipelined - * data exchange. - */ -public class RestartPipelinedRegionStrategy extends FailoverStrategy { - - /** The log object used for debugging. */ - private static final Logger LOG = LoggerFactory.getLogger(RestartPipelinedRegionStrategy.class); - - /** The execution graph on which this FailoverStrategy works */ - private final ExecutionGraph executionGraph; - - /** Fast lookup from vertex to failover region */ - private final HashMap vertexToRegion; - - /** - * Creates a new failover strategy to restart pipelined regions that works on the given - * execution graph. - * - * @param executionGraph The execution graph on which this FailoverStrategy will work - */ - public RestartPipelinedRegionStrategy(ExecutionGraph executionGraph) { - this.executionGraph = checkNotNull(executionGraph); - this.vertexToRegion = new HashMap<>(); - } - - // ------------------------------------------------------------------------ - // failover implementation - // ------------------------------------------------------------------------ - - @Override - public void onTaskFailure(Execution taskExecution, Throwable cause) { - final ExecutionVertex ev = taskExecution.getVertex(); - final FailoverRegion failoverRegion = vertexToRegion.get(ev); - - if (failoverRegion == null) { - executionGraph.failGlobal(new FlinkException( - "Can not find a failover region for the execution " + ev.getTaskNameWithSubtaskIndex(), cause)); - } - else { - LOG.info("Recovering task failure for {} #{} ({}) via restart of failover region", - taskExecution.getVertex().getTaskNameWithSubtaskIndex(), - taskExecution.getAttemptNumber(), - taskExecution.getAttemptId()); - - failoverRegion.onExecutionFail(taskExecution, cause); - } - } - - @Override - public void notifyNewVertices(List newJobVerticesTopological) { - generateAllFailoverRegion(newJobVerticesTopological); - } - - @Override - public String getStrategyName() { - return "Pipelined Region Failover"; - } - - /** - * Generate all the FailoverRegion from the new added job vertexes - */ - private void generateAllFailoverRegion(List newJobVerticesTopological) { - final IdentityHashMap> vertexToRegion = new IdentityHashMap<>(); - - // we use the map (list -> null) to imitate an IdentityHashSet (which does not exist) - final IdentityHashMap, Object> distinctRegions = new IdentityHashMap<>(); - - // this loop will worst case iterate over every edge in the graph (complexity is O(#edges)) - - for (ExecutionJobVertex ejv : newJobVerticesTopological) { - - // currently, jobs with a co-location constraint fail as one - // we want to improve that in the future (or get rid of co-location constraints) - if (ejv.getCoLocationGroup() != null) { - makeAllOneRegion(newJobVerticesTopological); - return; - } - - // see if this JobVertex one has pipelined inputs at all - final List inputs = ejv.getInputs(); - final int numInputs = inputs.size(); - boolean hasPipelinedInputs = false; - - for (IntermediateResult input : inputs) { - if (input.getResultType().isPipelined()) { - hasPipelinedInputs = true; - break; - } - } - - if (hasPipelinedInputs) { - // build upon the predecessors - for (ExecutionVertex ev : ejv.getTaskVertices()) { - - // remember the region in which we are - ArrayList thisRegion = null; - - for (int inputNum = 0; inputNum < numInputs; inputNum++) { - if (inputs.get(inputNum).getResultType().isPipelined()) { - - for (ExecutionEdge edge : ev.getInputEdges(inputNum)) { - final ExecutionVertex predecessor = edge.getSource().getProducer(); - final ArrayList predecessorRegion = vertexToRegion.get(predecessor); - - if (thisRegion != null) { - // we already have a region. see if it is the same as the predecessor's region - if (predecessorRegion != thisRegion) { - - // we need to merge our region and the predecessor's region - predecessorRegion.addAll(thisRegion); - distinctRegions.remove(thisRegion); - thisRegion = predecessorRegion; - - // remap the vertices from that merged region - for (ExecutionVertex inPredRegion: predecessorRegion) { - vertexToRegion.put(inPredRegion, thisRegion); - } - } - } - else if (predecessor != null) { - // first case, make this our region - thisRegion = predecessorRegion; - thisRegion.add(ev); - vertexToRegion.put(ev, thisRegion); - } - else { - // throw an uncaught exception here - // this is a bug and not a recoverable situation - throw new FlinkRuntimeException( - "bug in the logic to construct the pipelined failover regions"); - } - } - } - } - } - } - else { - // no pipelined inputs, start a new region - for (ExecutionVertex ev : ejv.getTaskVertices()) { - ArrayList region = new ArrayList<>(1); - region.add(ev); - vertexToRegion.put(ev, region); - distinctRegions.put(region, null); - } - } - } - - // now that we have all regions, create the failover region objects - LOG.info("Creating {} individual failover regions for job {} ({})", - distinctRegions.size(), executionGraph.getJobName(), executionGraph.getJobID()); - - for (List region : distinctRegions.keySet()) { - final FailoverRegion failoverRegion = createFailoverRegion(executionGraph, region); - for (ExecutionVertex ev : region) { - this.vertexToRegion.put(ev, failoverRegion); - } - } - } - - private void makeAllOneRegion(List jobVertices) { - LOG.warn("Cannot decompose ExecutionGraph into individual failover regions due to use of " + - "Co-Location constraints (iterations). Job will fail over as one holistic unit."); - - final ArrayList allVertices = new ArrayList<>(); - - for (ExecutionJobVertex ejv : jobVertices) { - - // safe some incremental size growing - allVertices.ensureCapacity(allVertices.size() + ejv.getParallelism()); - - allVertices.addAll(Arrays.asList(ejv.getTaskVertices())); - } - - final FailoverRegion singleRegion = createFailoverRegion(executionGraph, allVertices); - for (ExecutionVertex ev : allVertices) { - vertexToRegion.put(ev, singleRegion); - } - } - - @VisibleForTesting - protected FailoverRegion createFailoverRegion(ExecutionGraph eg, List connectedExecutions) { - Map tasks = initTasks(connectedExecutions); - return new FailoverRegion(eg, connectedExecutions, tasks); - } - - @VisibleForTesting - protected Map initTasks(List connectedExecutions) { - Map tasks = new HashMap<>(connectedExecutions.size()); - for (ExecutionVertex executionVertex : connectedExecutions) { - JobVertexID jobvertexId = executionVertex.getJobvertexId(); - ExecutionJobVertex jobVertex = executionVertex.getJobVertex(); - tasks.putIfAbsent(jobvertexId, jobVertex); - } - return tasks; - } - - // ------------------------------------------------------------------------ - // testing - // ------------------------------------------------------------------------ - - /** - * Finds the failover region that contains the given execution vertex. - */ - @VisibleForTesting - public FailoverRegion getFailoverRegion(ExecutionVertex ev) { - return vertexToRegion.get(ev); - } - - // ------------------------------------------------------------------------ - // factory - // ------------------------------------------------------------------------ - - /** - * Factory that instantiates the RestartPipelinedRegionStrategy. - */ - public static class Factory implements FailoverStrategy.Factory { - - @Override - public FailoverStrategy create(ExecutionGraph executionGraph) { - return new RestartPipelinedRegionStrategy(executionGraph); - } - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java index 1d7771608ff7..17a5bccead2a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java @@ -28,7 +28,6 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionVertex; import org.apache.flink.runtime.executiongraph.failover.AdaptedRestartPipelinedRegionStrategyNG; -import org.apache.flink.runtime.executiongraph.failover.FailoverRegion; import org.apache.flink.runtime.executiongraph.failover.FailoverStrategy; import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; import org.apache.flink.runtime.state.SharedStateRegistry; @@ -65,7 +64,7 @@ public void setUp() { /** * Tests that {@link CheckpointCoordinator#abortPendingCheckpoints(CheckpointException)} - * called by {@link AdaptedRestartPipelinedRegionStrategyNG} or {@link FailoverRegion} could handle + * called by {@link AdaptedRestartPipelinedRegionStrategyNG} could handle * the {@code currentPeriodicTrigger} null situation well. */ @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ConcurrentFailoverStrategyExecutionGraphTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ConcurrentFailoverStrategyExecutionGraphTest.java deleted file mode 100644 index 3198dc193660..000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ConcurrentFailoverStrategyExecutionGraphTest.java +++ /dev/null @@ -1,540 +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.api.common.time.Time; -import org.apache.flink.runtime.blob.VoidBlobWriter; -import org.apache.flink.runtime.checkpoint.CheckpointCoordinator; -import org.apache.flink.runtime.checkpoint.CheckpointOptions; -import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy; -import org.apache.flink.runtime.checkpoint.CheckpointStatsTracker; -import org.apache.flink.runtime.checkpoint.PendingCheckpoint; -import org.apache.flink.runtime.checkpoint.StandaloneCheckpointIDCounter; -import org.apache.flink.runtime.checkpoint.StandaloneCompletedCheckpointStore; -import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; -import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; -import org.apache.flink.runtime.concurrent.FutureUtils; -import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; -import org.apache.flink.runtime.execution.ExecutionState; -import org.apache.flink.runtime.execution.SuppressRestartsException; -import org.apache.flink.runtime.executiongraph.failover.FailoverRegion; -import org.apache.flink.runtime.executiongraph.failover.FailoverStrategy.Factory; -import org.apache.flink.runtime.executiongraph.failover.RestartIndividualStrategy; -import org.apache.flink.runtime.executiongraph.failover.RestartPipelinedRegionStrategy; -import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; -import org.apache.flink.runtime.executiongraph.utils.SimpleSlotProvider; -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.tasks.CheckpointCoordinatorConfiguration; -import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; -import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; -import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; -import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; -import org.apache.flink.runtime.state.memory.MemoryStateBackend; -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 javax.annotation.Nonnull; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CompletableFuture; - -import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.waitUntilExecutionState; -import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.waitUntilJobStatus; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.timeout; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -/** - * These tests make sure that global failover (restart all) always takes precedence over - * local recovery strategies. - * - *

This test must be in the package it resides in, because it uses package-private methods - * from the ExecutionGraph classes. - */ -public class ConcurrentFailoverStrategyExecutionGraphTest extends TestLogger { - - private final ComponentMainThreadExecutor mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread(); - - /** - * Tests that a cancellation concurrent to a local failover leads to a properly - * cancelled state. - */ - @Test - public void testCancelWhileInLocalFailover() throws Exception { - - // the logic in this test is as follows: - // - start a job - // - cause a task failure and delay the local recovery action via the manual executor - // - cancel the job to go into cancelling - // - resume in local recovery action - // - validate that this does in fact not start a new task, because the graph as a - // whole should now be cancelled already - - final JobID jid = new JobID(); - final int parallelism = 2; - - final SimpleSlotProvider slotProvider = new SimpleSlotProvider(jid, parallelism); - - final ExecutionGraph graph = createSampleGraph( - jid, - TestRestartPipelinedRegionStrategy::new, - TestRestartStrategy.directExecuting(), - slotProvider, - parallelism); - - graph.start(mainThreadExecutor); - TestRestartPipelinedRegionStrategy strategy = (TestRestartPipelinedRegionStrategy) graph.getFailoverStrategy(); - - // This future is used to block the failover strategy execution until we complete it - final CompletableFuture blocker = new CompletableFuture<>(); - strategy.setBlockerFuture(blocker); - - final ExecutionJobVertex ejv = graph.getVerticesTopologically().iterator().next(); - final ExecutionVertex vertex1 = ejv.getTaskVertices()[0]; - final ExecutionVertex vertex2 = ejv.getTaskVertices()[1]; - - graph.scheduleForExecution(); - assertEquals(JobStatus.RUNNING, graph.getState()); - - // let one of the vertices fail - that triggers a local recovery action - vertex1.getCurrentExecutionAttempt().fail(new Exception("test failure")); - assertEquals(ExecutionState.FAILED, vertex1.getCurrentExecutionAttempt().getState()); - - // graph should still be running and the failover recovery action should be queued - assertEquals(JobStatus.RUNNING, graph.getState()); - - // now cancel the job - graph.cancel(); - - assertEquals(JobStatus.CANCELLING, graph.getState()); - assertEquals(ExecutionState.FAILED, vertex1.getCurrentExecutionAttempt().getState()); - assertEquals(ExecutionState.CANCELING, vertex2.getCurrentExecutionAttempt().getState()); - - // let the recovery action continue - blocker.complete(null); - - // now report that cancelling is complete for the other vertex - vertex2.getCurrentExecutionAttempt().completeCancelling(); - - assertEquals(JobStatus.CANCELED, graph.getTerminationFuture().get()); - assertTrue(vertex1.getCurrentExecutionAttempt().getState().isTerminal()); - assertTrue(vertex2.getCurrentExecutionAttempt().getState().isTerminal()); - - // make sure all slots are recycled - assertEquals(parallelism, slotProvider.getNumberOfAvailableSlots()); - } - - /** - * Tests that a terminal global failure concurrent to a local failover - * leads to a properly failed state. - */ - @Test - public void testGlobalFailureConcurrentToLocalFailover() throws Exception { - - // the logic in this test is as follows: - // - start a job - // - cause a task failure and delay the local recovery action via the manual executor - // - cause a global failure - // - resume in local recovery action - // - validate that this does in fact not start a new task, because the graph as a - // whole should now be terminally failed already - - final JobID jid = new JobID(); - final int parallelism = 2; - - final SimpleSlotProvider slotProvider = new SimpleSlotProvider(jid, parallelism); - - final ExecutionGraph graph = createSampleGraph( - jid, - TestRestartPipelinedRegionStrategy::new, - TestRestartStrategy.directExecuting(), - slotProvider, - parallelism); - - graph.start(mainThreadExecutor); - TestRestartPipelinedRegionStrategy strategy = (TestRestartPipelinedRegionStrategy) graph.getFailoverStrategy(); - - // This future is used to block the failover strategy execution until we complete it - final CompletableFuture blocker = new CompletableFuture<>(); - strategy.setBlockerFuture(blocker); - - final ExecutionJobVertex ejv = graph.getVerticesTopologically().iterator().next(); - final ExecutionVertex vertex1 = ejv.getTaskVertices()[0]; - final ExecutionVertex vertex2 = ejv.getTaskVertices()[1]; - - graph.scheduleForExecution(); - assertEquals(JobStatus.RUNNING, graph.getState()); - - // let one of the vertices fail - that triggers a local recovery action - vertex1.getCurrentExecutionAttempt().fail(new Exception("test failure")); - assertEquals(ExecutionState.FAILED, vertex1.getCurrentExecutionAttempt().getState()); - - // graph should still be running and the failover recovery action should be queued - assertEquals(JobStatus.RUNNING, graph.getState()); - - // now cancel the job - graph.failGlobal(new SuppressRestartsException(new Exception("test exception"))); - - assertEquals(JobStatus.FAILING, graph.getState()); - assertEquals(ExecutionState.FAILED, vertex1.getCurrentExecutionAttempt().getState()); - assertEquals(ExecutionState.CANCELING, vertex2.getCurrentExecutionAttempt().getState()); - - // let the recovery action continue - blocker.complete(null); - - // now report that cancelling is complete for the other vertex - vertex2.getCurrentExecutionAttempt().completeCancelling(); - - assertEquals(JobStatus.FAILED, graph.getState()); - assertTrue(vertex1.getCurrentExecutionAttempt().getState().isTerminal()); - assertTrue(vertex2.getCurrentExecutionAttempt().getState().isTerminal()); - - // make sure all slots are recycled - assertEquals(parallelism, slotProvider.getNumberOfAvailableSlots()); - } - - /** - * Tests that a local failover does not try to trump a global failover. - */ - @Test - public void testGlobalRecoveryConcurrentToLocalRecovery() throws Exception { - - // the logic in this test is as follows: - // - start a job - // - cause a task failure and delay the local recovery action via the manual executor - // - cause a global failure that is recovering immediately - // - resume in local recovery action - // - validate that this does in fact not cause another task restart, because the global - // recovery should already have restarted the task graph - - final JobID jid = new JobID(); - final int parallelism = 2; - - final SimpleSlotProvider slotProvider = new SimpleSlotProvider(jid, parallelism); - - final ExecutionGraph graph = createSampleGraph( - jid, - TestRestartPipelinedRegionStrategy::new, - new TestRestartStrategy(2, false), // twice restart, no delay - slotProvider, - parallelism); - - TestRestartPipelinedRegionStrategy strategy = (TestRestartPipelinedRegionStrategy) graph.getFailoverStrategy(); - - // This future is used to block the failover strategy execution until we complete it - CompletableFuture blocker = new CompletableFuture<>(); - strategy.setBlockerFuture(blocker); - - graph.start(mainThreadExecutor); - - final ExecutionJobVertex ejv = graph.getVerticesTopologically().iterator().next(); - final ExecutionVertex vertex1 = ejv.getTaskVertices()[0]; - final ExecutionVertex vertex2 = ejv.getTaskVertices()[1]; - - graph.scheduleForExecution(); - assertEquals(JobStatus.RUNNING, graph.getState()); - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(vertex1).getState()); - - // let one of the vertices fail - that triggers a local recovery action - vertex2.getCurrentExecutionAttempt().fail(new Exception("test failure")); - assertEquals(ExecutionState.FAILED, vertex2.getCurrentExecutionAttempt().getState()); - assertEquals(JobStatus.CANCELLING, strategy.getFailoverRegion(vertex2).getState()); - - // graph should still be running and the failover recovery action should be queued - assertEquals(JobStatus.RUNNING, graph.getState()); - - // now cancel the job - graph.failGlobal(new Exception("test exception")); - - assertEquals(JobStatus.FAILING, graph.getState()); - assertEquals(ExecutionState.FAILED, vertex2.getCurrentExecutionAttempt().getState()); - assertEquals(ExecutionState.CANCELING, vertex1.getCurrentExecutionAttempt().getState()); - - // now report that cancelling is complete for the other vertex - vertex1.getCurrentExecutionAttempt().completeCancelling(); - - waitUntilJobStatus(graph, JobStatus.RUNNING, 1000); - assertEquals(JobStatus.RUNNING, graph.getState()); - - waitUntilExecutionState(vertex1.getCurrentExecutionAttempt(), ExecutionState.DEPLOYING, 1000); - waitUntilExecutionState(vertex2.getCurrentExecutionAttempt(), ExecutionState.DEPLOYING, 1000); - vertex1.getCurrentExecutionAttempt().switchToRunning(); - vertex2.getCurrentExecutionAttempt().switchToRunning(); - assertEquals(ExecutionState.RUNNING, vertex1.getCurrentExecutionAttempt().getState()); - assertEquals(ExecutionState.RUNNING, vertex2.getCurrentExecutionAttempt().getState()); - - // let the recovery action continue - this should do nothing any more - blocker.complete(null); - - // validate that the graph is still peachy - assertEquals(JobStatus.RUNNING, graph.getState()); - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(vertex1).getState()); - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(vertex2).getState()); - assertEquals(ExecutionState.RUNNING, vertex1.getCurrentExecutionAttempt().getState()); - assertEquals(ExecutionState.RUNNING, vertex2.getCurrentExecutionAttempt().getState()); - assertEquals(1, vertex1.getCurrentExecutionAttempt().getAttemptNumber()); - assertEquals(1, vertex2.getCurrentExecutionAttempt().getAttemptNumber()); - assertEquals(1, vertex1.getCopyOfPriorExecutionsList().size()); - assertEquals(1, vertex2.getCopyOfPriorExecutionsList().size()); - - // make sure all slots are in use - assertEquals(0, slotProvider.getNumberOfAvailableSlots()); - - blocker = new CompletableFuture<>(); - strategy.setBlockerFuture(blocker); - - // validate that a task failure then can be handled by the local recovery - vertex2.getCurrentExecutionAttempt().fail(new Exception("test failure")); - - // let the local recovery action continue - this should recover the vertex2 - blocker.complete(null); - - waitUntilExecutionState(vertex2.getCurrentExecutionAttempt(), ExecutionState.DEPLOYING, 1000); - vertex2.getCurrentExecutionAttempt().switchToRunning(); - - // validate that the local recovery result - assertEquals(JobStatus.RUNNING, graph.getState()); - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(vertex1).getState()); - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(vertex2).getState()); - assertEquals(ExecutionState.RUNNING, vertex1.getCurrentExecutionAttempt().getState()); - assertEquals(ExecutionState.RUNNING, vertex2.getCurrentExecutionAttempt().getState()); - assertEquals(1, vertex1.getCurrentExecutionAttempt().getAttemptNumber()); - assertEquals(2, vertex2.getCurrentExecutionAttempt().getAttemptNumber()); - assertEquals(1, vertex1.getCopyOfPriorExecutionsList().size()); - assertEquals(2, vertex2.getCopyOfPriorExecutionsList().size()); - - // make sure all slots are in use - assertEquals(0, slotProvider.getNumberOfAvailableSlots()); - } - - /** - * Tests that a local failure fails all pending checkpoints which have not been acknowledged by the failing - * task. - */ - @Test - public void testLocalFailureFailsPendingCheckpoints() throws Exception { - final JobID jid = new JobID(); - final int parallelism = 2; - final long verifyTimeout = 5000L; - - final TaskManagerGateway taskManagerGateway = mock(TaskManagerGateway.class); - when(taskManagerGateway.submitTask(any(TaskDeploymentDescriptor.class), any(Time.class))).thenReturn(CompletableFuture.completedFuture(Acknowledge.get())); - when(taskManagerGateway.cancelTask(any(ExecutionAttemptID.class), any(Time.class))).thenReturn(CompletableFuture.completedFuture(Acknowledge.get())); - - final SimpleSlotProvider slotProvider = new SimpleSlotProvider(jid, parallelism, taskManagerGateway); - - final CheckpointCoordinatorConfiguration checkpointCoordinatorConfiguration = new CheckpointCoordinatorConfiguration( - 10L, - 100000L, - 1L, - 3, - CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION, - true, - false, - 0); - - final ExecutionGraph graph = createSampleGraph( - jid, - (eg) -> new RestartIndividualStrategy(eg) { - @Override - protected void performExecutionVertexRestart( - ExecutionVertex vertexToRecover, - long globalModVersion) { - } - } - , - new TestRestartStrategy(2, false), // twice restart, no delay - slotProvider, - parallelism); - - graph.start(mainThreadExecutor); - - final List allVertices = new ArrayList<>(graph.getAllVertices().values()); - - final StandaloneCheckpointIDCounter standaloneCheckpointIDCounter = new StandaloneCheckpointIDCounter(); - - graph.enableCheckpointing( - checkpointCoordinatorConfiguration, - allVertices, - allVertices, - allVertices, - Collections.emptyList(), - standaloneCheckpointIDCounter, - new StandaloneCompletedCheckpointStore(1), - new MemoryStateBackend(), - new CheckpointStatsTracker( - 1, - allVertices, - checkpointCoordinatorConfiguration, - UnregisteredMetricGroups.createUnregisteredTaskMetricGroup())); - - final CheckpointCoordinator checkpointCoordinator = graph.getCheckpointCoordinator(); - - final ExecutionJobVertex ejv = graph.getVerticesTopologically().iterator().next(); - final ExecutionVertex vertex1 = ejv.getTaskVertices()[0]; - final ExecutionVertex vertex2 = ejv.getTaskVertices()[1]; - - graph.scheduleForExecution(); - assertEquals(JobStatus.RUNNING, graph.getState()); - - verify(taskManagerGateway, timeout(verifyTimeout).times(parallelism)).submitTask(any(TaskDeploymentDescriptor.class), any(Time.class)); - - // switch all executions to running - for (ExecutionVertex executionVertex : graph.getAllExecutionVertices()) { - executionVertex.getCurrentExecutionAttempt().switchToRunning(); - } - - // wait for a first checkpoint to be triggered - verify(taskManagerGateway, timeout(verifyTimeout).times(3)).triggerCheckpoint( - eq(vertex1.getCurrentExecutionAttempt().getAttemptId()), - any(JobID.class), - anyLong(), - anyLong(), - any(CheckpointOptions.class), - any(Boolean.class)); - - verify(taskManagerGateway, timeout(verifyTimeout).times(3)).triggerCheckpoint( - eq(vertex2.getCurrentExecutionAttempt().getAttemptId()), - any(JobID.class), - anyLong(), - anyLong(), - any(CheckpointOptions.class), - any(Boolean.class)); - - assertEquals(3, checkpointCoordinator.getNumberOfPendingCheckpoints()); - - long checkpointToAcknowledge = standaloneCheckpointIDCounter.getLast(); - - checkpointCoordinator.receiveAcknowledgeMessage( - new AcknowledgeCheckpoint( - graph.getJobID(), - vertex1.getCurrentExecutionAttempt().getAttemptId(), - checkpointToAcknowledge), - "Unknown location"); - - Map oldPendingCheckpoints = new HashMap<>(3); - - for (PendingCheckpoint pendingCheckpoint : checkpointCoordinator.getPendingCheckpoints().values()) { - assertFalse(pendingCheckpoint.isDiscarded()); - oldPendingCheckpoints.put(pendingCheckpoint.getCheckpointId(), pendingCheckpoint); - } - - // let one of the vertices fail - this should trigger the failing of not acknowledged pending checkpoints - vertex1.getCurrentExecutionAttempt().fail(new Exception("test failure")); - - for (PendingCheckpoint pendingCheckpoint : oldPendingCheckpoints.values()) { - if (pendingCheckpoint.getCheckpointId() == checkpointToAcknowledge) { - assertFalse(pendingCheckpoint.isDiscarded()); - } else { - assertTrue(pendingCheckpoint.isDiscarded()); - } - } - } - - // ------------------------------------------------------------------------ - // utilities - // ------------------------------------------------------------------------ - - private ExecutionGraph createSampleGraph( - JobID jid, - Factory failoverStrategy, - RestartStrategy restartStrategy, - SlotProvider slotProvider, - int parallelism) throws Exception { - - final JobInformation jobInformation = new DummyJobInformation( - jid, - "test job"); - - // build a simple execution graph with on job vertex, parallelism 2 - final Time timeout = Time.seconds(10L); - final ExecutionGraph graph = new ExecutionGraph( - jobInformation, - TestingUtils.defaultExecutor(), - TestingUtils.defaultExecutor(), - timeout, - restartStrategy, - failoverStrategy, - slotProvider, - getClass().getClassLoader(), - VoidBlobWriter.getInstance(), - timeout); - - JobVertex jv = new JobVertex("test vertex"); - jv.setInvokableClass(NoOpInvokable.class); - jv.setParallelism(parallelism); - - JobGraph jg = new JobGraph(jid, "testjob", jv); - graph.attachJobGraph(jg.getVerticesSortedTopologicallyFromSources()); - - return graph; - } - - /** - * Test implementation of the {@link RestartPipelinedRegionStrategy} that makes it possible to control when the - * failover action is performed via {@link CompletableFuture}. - */ - static class TestRestartPipelinedRegionStrategy extends RestartPipelinedRegionStrategy { - - @Nonnull - CompletableFuture blockerFuture; - - public TestRestartPipelinedRegionStrategy(ExecutionGraph executionGraph) { - super(executionGraph); - this.blockerFuture = CompletableFuture.completedFuture(null); - } - - public void setBlockerFuture(@Nonnull CompletableFuture blockerFuture) { - this.blockerFuture = blockerFuture; - } - - @Override - protected FailoverRegion createFailoverRegion(ExecutionGraph eg, List connectedExecutions) { - Map tasks = initTasks(connectedExecutions); - return new FailoverRegion(eg, connectedExecutions, tasks) { - @Override - protected CompletableFuture createTerminationFutureOverAllConnectedVertexes() { - ArrayList> terminationAndBlocker = new ArrayList<>(2); - terminationAndBlocker.add(super.createTerminationFutureOverAllConnectedVertexes()); - terminationAndBlocker.add(blockerFuture); - return FutureUtils.waitForAll(terminationAndBlocker); - } - }; - } - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java index 25d169c8fd2d..dbe2a0f29c09 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java @@ -33,7 +33,6 @@ import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; import org.apache.flink.runtime.execution.ExecutionState; -import org.apache.flink.runtime.executiongraph.failover.FailoverRegion; import org.apache.flink.runtime.executiongraph.failover.FailoverStrategy; import org.apache.flink.runtime.executiongraph.failover.RestartAllStrategy; import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; @@ -233,26 +232,6 @@ public static Predicate isInExecutionState(ExecutionState execu return (AccessExecution execution) -> execution.getState() == executionState; } - public static void waitUntilFailoverRegionState(FailoverRegion region, JobStatus status, long maxWaitMillis) - throws TimeoutException { - checkNotNull(region); - checkNotNull(status); - checkArgument(maxWaitMillis >= 0); - - // this is a poor implementation - we may want to improve it eventually - final long deadline = maxWaitMillis == 0 ? Long.MAX_VALUE : System.nanoTime() + (maxWaitMillis * 1_000_000); - - while (region.getState() != status && System.nanoTime() < deadline) { - try { - Thread.sleep(2); - } catch (InterruptedException ignored) {} - } - - if (System.nanoTime() >= deadline) { - throw new TimeoutException(); - } - } - /** * Takes all vertices in the given ExecutionGraph and switches their current * execution to RUNNING. diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/FailoverRegionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/FailoverRegionTest.java deleted file mode 100644 index ba4226b562c8..000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/FailoverRegionTest.java +++ /dev/null @@ -1,694 +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.api.common.time.Time; -import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; -import org.apache.flink.mock.Whitebox; -import org.apache.flink.runtime.JobException; -import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.runtime.checkpoint.CheckpointCoordinator; -import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTest; -import org.apache.flink.runtime.checkpoint.CheckpointException; -import org.apache.flink.runtime.checkpoint.CheckpointMetrics; -import org.apache.flink.runtime.checkpoint.CheckpointProperties; -import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy; -import org.apache.flink.runtime.checkpoint.CheckpointStatsTracker; -import org.apache.flink.runtime.checkpoint.JobManagerTaskRestore; -import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; -import org.apache.flink.runtime.checkpoint.PendingCheckpoint; -import org.apache.flink.runtime.checkpoint.StandaloneCheckpointIDCounter; -import org.apache.flink.runtime.checkpoint.StandaloneCompletedCheckpointStore; -import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; -import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; -import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; -import org.apache.flink.runtime.execution.ExecutionState; -import org.apache.flink.runtime.executiongraph.failover.FailoverStrategy; -import org.apache.flink.runtime.executiongraph.failover.FailoverStrategy.Factory; -import org.apache.flink.runtime.executiongraph.failover.RestartPipelinedRegionStrategy; -import org.apache.flink.runtime.executiongraph.restart.InfiniteDelayRestartStrategy; -import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; -import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; -import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; -import org.apache.flink.runtime.executiongraph.utils.SimpleSlotProvider; -import org.apache.flink.runtime.io.network.partition.ResultPartitionType; -import org.apache.flink.runtime.jobgraph.DistributionPattern; -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.OperatorID; -import org.apache.flink.runtime.jobgraph.ScheduleMode; -import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; -import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; -import org.apache.flink.runtime.jobmanager.scheduler.LocationPreferenceConstraint; -import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; -import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; -import org.apache.flink.runtime.state.CheckpointStorageCoordinatorView; -import org.apache.flink.runtime.state.OperatorStateHandle; -import org.apache.flink.runtime.state.memory.MemoryStateBackend; -import org.apache.flink.runtime.testingUtils.TestingUtils; -import org.apache.flink.util.FlinkException; -import org.apache.flink.util.TestLogger; - -import org.junit.Ignore; -import org.junit.Test; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CompletableFuture; - -import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.waitUntilExecutionState; -import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.waitUntilFailoverRegionState; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; -import static org.mockito.Mockito.mock; - -public class FailoverRegionTest extends TestLogger { - - private static final long checkpointId = 42L; - - /** - * Tests that a job only has one failover region and can recover from task failure successfully with state. - * @throws Exception if fail to create the single region execution graph or fail to acknowledge all checkpoints. - */ - @Test - public void testSingleRegionFailover() throws Exception { - RestartStrategy restartStrategy = new InfiniteDelayRestartStrategy(10); - ExecutionGraph eg = createSingleRegionExecutionGraph(restartStrategy); - RestartPipelinedRegionStrategy strategy = (RestartPipelinedRegionStrategy) eg.getFailoverStrategy(); - - ExecutionVertex ev = eg.getAllExecutionVertices().iterator().next(); - - assertNotNull(eg.getCheckpointCoordinator()); - assertFalse(eg.getCheckpointCoordinator().getPendingCheckpoints().isEmpty()); - - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(ev).getState()); - - acknowledgeAllCheckpoints(eg.getCheckpointCoordinator(), eg.getAllExecutionVertices().iterator()); - - // verify checkpoint has been completed successfully. - assertEquals(1, eg.getCheckpointCoordinator().getCheckpointStore().getNumberOfRetainedCheckpoints()); - assertEquals(checkpointId, eg.getCheckpointCoordinator().getCheckpointStore().getLatestCheckpoint(false).getCheckpointID()); - - ev.getCurrentExecutionAttempt().fail(new Exception("Test Exception")); - assertEquals(JobStatus.CANCELLING, strategy.getFailoverRegion(ev).getState()); - - for (ExecutionVertex evs : eg.getAllExecutionVertices()) { - evs.getCurrentExecutionAttempt().completeCancelling(); - } - - verifyCheckpointRestoredAsExpected(eg); - - assertEquals(JobStatus.RUNNING, strategy.getFailoverRegion(ev).getState()); - } - - /** - * Tests that a job has server failover regions and one region failover does not influence others - * - *

-	 *     (a1) ---> (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 7b00e4f09f15..000000000000 --- 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 f17dddcfb061..000000000000 --- 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); - } -} From 4064b5b67d6d220e1d5518bca96688f51cbbb891 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 4 Oct 2019 13:56:44 +0200 Subject: [PATCH 014/746] [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 000000000000..965a50b3f353 --- /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 6b6b85cf8d1e..665c4aa479d0 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.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 JobMaster( 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 @@ private Acknowledge suspendExecution(final Exception cause) { } 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 543a5c3137be..8698e842ac54 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.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 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 @@ private void startHeartbeatServices() { } 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 b1238ec8e8da..f0db4cd82b2d 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 TaskExecutor( 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 TaskExecutor( 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 void onStart() throws Exception { private void startTaskExecutorServices() throws Exception { try { - startHeartbeatServices(); - // start by connecting to the ResourceManager resourceManagerLeaderRetriever.start(new ResourceManagerLeaderListener()); @@ -412,38 +427,9 @@ private void stopTaskExecutorServices() throws Exception { // 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 // ====================================================================== From be8fdcaea45136b15e8783831ac3d2bdb2608c3b Mon Sep 17 00:00:00 2001 From: ouyangwulin Date: Tue, 8 Oct 2019 22:50:40 +0800 Subject: [PATCH 015/746] [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 628184d2560f..70fdcc5cf304 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 6b0dbf1b6fd6..fa8e14e2f8ea 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 28d9eee184a7..19404ffbe0cb 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.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.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 void open(MetricConfig config) { 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 fe21fb083cb7..cc88eb5f3312 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()); } From 71dfb4cbc337261f4a799954128df46d685542d9 Mon Sep 17 00:00:00 2001 From: Yangze Guo Date: Wed, 9 Oct 2019 16:26:51 +0800 Subject: [PATCH 016/746] [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 8992d50950e8..2bd0096bbf77 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 01c780895c3e..2264ec40cdd6 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 From c910d71c358ab55763e77fdd2be2811dcdec6c41 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 9 Oct 2019 15:25:13 +0200 Subject: [PATCH 017/746] [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 c67526cbc67c..9733b3df7590 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. From 0d112f5bc61e6f8400e000e13add08abae1067a1 Mon Sep 17 00:00:00 2001 From: Marcos Klein Date: Wed, 9 Oct 2019 06:56:46 -0700 Subject: [PATCH 018/746] [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 1617ac7fe275..951fbf96785d 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 final void invoke() throws Exception { synchronized (lock) { operatorChain.releaseOutputs(); } + } else { + // failed to allocate operatorChain, clean up record writers + for (RecordWriter>> writer: recordWriters) { + writer.close(); + } } mailboxProcessor.close(); From df44cce7f1e1bc42715e6cd0e7268c1a46b07af8 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Tue, 8 Oct 2019 20:20:53 +0800 Subject: [PATCH 019/746] [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 4e3a5ae2f2e5..e89f6224eea5 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 db4915038144..76df93dd7761 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 ffeb4864c5e5..ed16daa09e3a 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 000000000000..b473673460a9 --- /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 000000000000..544e4f3a705c --- /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 ce4ddd4010a4..de54d5fb05e0 100644 --- a/flink-python/pyflink/gen_protos.py +++ b/flink-python/pyflink/gen_protos.py @@ -34,25 +34,21 @@ # 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 190a362d928e..db6582d39d78 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 21d3f09e629a..888824ab7cae 100644 --- a/flink-python/pyflink/testing/test_case_utils.py +++ b/flink-python/pyflink/testing/test_case_utils.py @@ -27,7 +27,6 @@ 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 f5b26da98e7e..11c1c1909e19 100644 --- a/flink-python/setup.py +++ b/flink-python/setup.py @@ -23,12 +23,6 @@ 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 @@ 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 @@ def run(self): 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 @@ def run(self): '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 aafec9a3fa12..08978f97463c 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 From 592fba617655d26fbbafdde3f03d267022cc3a94 Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Sun, 29 Sep 2019 14:43:16 +0800 Subject: [PATCH 020/746] [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 5378ae971a2d..860e7e2aef7a 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 ------------- From d1d7853c45773ed3ec8b7a577993b080a45f1d77 Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Thu, 10 Oct 2019 12:19:33 +0800 Subject: [PATCH 021/746] [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 860e7e2aef7a..f4f73e0685a9 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 ------------- From 17fae0e0273c031118bdedbcdb8a03c401257690 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 9 Oct 2019 15:10:09 +0200 Subject: [PATCH 022/746] [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 c8fc42f6536a..bb92960053fa 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 + + + + From d231032595cfb292d2ed1c42725493e765210fc5 Mon Sep 17 00:00:00 2001 From: caoyingjie Date: Thu, 10 Oct 2019 16:11:45 +0800 Subject: [PATCH 023/746] [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 c06c2337469e..ecebc6734495 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 @@ int getCurrentReaderPosition() { 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 int getCached() { 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 0c38ae2c2cb4..ecf695696aab 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 @@ public void flush() { } // 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(); From a5198b891da02434ceb1af17a12abf46b1e7e377 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Mon, 7 Oct 2019 10:40:58 +0200 Subject: [PATCH 024/746] [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 45b041959a73..597b89b921f3 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 aed915f7d549..1fba302c3057 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 478ba47129e2..8bcb0ca5ffdb 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 @@ * * @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 260f80e5860b..1514aba61ab5 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 @@ * * @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; From 11c26109dd647baf59c1c9da9934c46c8d9f1ff7 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 7 Jun 2019 21:31:12 +0200 Subject: [PATCH 025/746] [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 3012bbb10229..ad746a52b564 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 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 static LeaderConnectionInfo retrieveLeaderConnectionInfo( 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 static InetAddress findConnectingAddress( * 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); } } From 7e7ee2812f459bcfdab8862db8f6db2cab0e1368 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 7 Jun 2019 16:07:21 +0200 Subject: [PATCH 026/746] [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 bf4518cd89fa..1c7fb6a41414 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 void grantLeadership(final UUID newLeaderSessionID) { 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 e3e98e109045..0cf37ddf9fe8 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 @@ private void removeContender(EmbeddedLeaderElectionService service) { /** * 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 @@ private void confirmLeader(final EmbeddedLeaderElectionService service, final UU 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 void stop() throws Exception { } @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 89b297722e0d..879b52356c31 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 void stop() { } @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 void confirmLeaderSessionID(UUID leaderSessionID) { 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 7b2609a65d86..52740a64f688 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 @@ private CompletionStage startJobMaster(UUID leaderSessionId) { final CompletableFuture currentLeaderGatewayFuture = leaderGatewayFuture; return startFuture.thenAcceptAsync( - (Acknowledge ack) -> confirmLeaderSessionIdIfStillLeader(leaderSessionId, currentLeaderGatewayFuture), + (Acknowledge ack) -> confirmLeaderSessionIdIfStillLeader( + leaderSessionId, + jobMasterService.getAddress(), + currentLeaderGatewayFuture), executor); } @@ -358,10 +361,14 @@ private CompletableFuture getJobSchedulingStatus() { } } - 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 10f2f68efa8b..c7a5588009db 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 @@ * 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 ec997a332b37..d0bb07289853 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 void stop() { } @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 87684c871d7f..91227dca6312 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 ZooKeeperLeaderElectionService(CuratorFramework client, String latchPath, issuedLeaderSessionID = null; confirmedLeaderSessionID = null; + confirmedLeaderAddress = null; leaderContender = null; running = false; @@ -177,12 +180,12 @@ public void stop() throws Exception{ } @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 void confirmLeaderSessionID(UUID leaderSessionID) { 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 void confirmLeaderSessionID(UUID leaderSessionID) { } } + 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 void isLeader() { synchronized (lock) { if (running) { issuedLeaderSessionID = UUID.randomUUID(); - confirmedLeaderSessionID = null; + clearConfirmedLeaderInformation(); if (LOG.isDebugEnabled()) { LOG.debug( @@ -233,16 +241,23 @@ public void isLeader() { } } + 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 void nodeChanged() throws Exception { "Writing leader information into empty node by {}.", leaderContender.getAddress()); } - writeLeaderInformation(confirmedLeaderSessionID); + writeLeaderInformation(); } else { byte[] data = childData.getData(); @@ -286,7 +301,7 @@ public void nodeChanged() throws Exception { "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 void nodeChanged() throws Exception { 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 void nodeChanged() throws Exception { "Correcting leader information by {}.", leaderContender.getAddress()); } - writeLeaderInformation(confirmedLeaderSessionID); + writeLeaderInformation(); } } } @@ -320,24 +335,22 @@ public void nodeChanged() throws Exception { /** * 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 @@ protected void writeLeaderInformation(UUID leaderSessionID) { 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 8698e842ac54..56450ff1c85a 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 void grantLeadership(final UUID newLeaderSessionID) { (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 1c31edb1efd7..f474ad50df84 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 @@ protected CompletableFuture shutDownInternal() { @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 7f06d60c949b..3b9ce6513b99 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 void testJobManagerLeaderRetrieval() throws Exception { final UUID leaderId = leaderIdArgumentCaptor.getValue(); - leaderElectionService.confirmLeaderSessionID(leaderId); + leaderElectionService.confirmLeadership(leaderId, address); verify(leaderRetrievalListener).notifyLeaderAddress(eq(address), eq(leaderId)); } @@ -163,7 +163,7 @@ public void testResourceManagerLeaderRetrieval() throws Exception { final UUID leaderId = leaderIdArgumentCaptor.getValue(); - leaderElectionService.confirmLeaderSessionID(leaderId); + leaderElectionService.confirmLeadership(leaderId, address); verify(leaderRetrievalListener).notifyLeaderAddress(eq(address), eq(leaderId)); } @@ -191,8 +191,8 @@ public void testConcurrentLeadershipOperations() throws Exception { 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 3875c4f171c5..535679ea65a4 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 @@ private static LeaderContender mockContender(final LeaderElectionService service @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 44016f82cc54..9090cae6c2e0 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 void testHasLeadership() throws Exception { 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 b03c1654c7ce..34408ff2ea4f 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 void grantLeadership(UUID leaderSessionID) { 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 e60f2f776ab2..f282ebdc4d69 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 synchronized void stop() throws Exception { } @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 08f8a0f045f5..ff2f1ebc30d3 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 void testGrantAndRevokeLeadership() throws Exception { 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 96e17e95e7d9..db45ee02cb25 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 void stop() throws Exception { } @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 bdc7863c79a6..0d09f8790270 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 @@ private static LeaderContender mockContender(final LeaderElectionService service @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)); From d963eabdbc9ee59f4606c4f9bb6465ee1e88f0e5 Mon Sep 17 00:00:00 2001 From: "Wind (Jiayi Liao)" Date: Fri, 11 Oct 2019 04:07:22 +0800 Subject: [PATCH 027/746] [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 9616e9138b66..a971386c44f0 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 @@ protected void testOneToOneAtLeastOnce(boolean regularSink) throws Exception { // 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); From 2750b2ea29c440c6e699f756e5cd3431653fc49f Mon Sep 17 00:00:00 2001 From: Liupengcheng Date: Tue, 8 Oct 2019 14:09:58 +0800 Subject: [PATCH 028/746] [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 76df93dd7761..29dc3c419fbd 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 de54d5fb05e0..b5908029a013 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 From 71e440d5f62fbd43df18f999b7818de48a279eb0 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Mon, 30 Sep 2019 11:15:53 +0800 Subject: [PATCH 029/746] [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 7544af20177c..b4999835d7fc 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.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 @@ 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 @@ void randomEmit(T record, int targetChannelIndex) throws IOException, Interrupte 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 void flushTargetPartition(int targetChannel) { @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 BufferBuilder getBufferBuilder(int targetChannel) throws IOException, Int */ @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 BufferBuilder requestNewBufferBuilder(int targetChannel) throws IOExcepti } } - 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 void clearBuffers() { } 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 eaeb5091f650..5f5e5964b988 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.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 void broadcastEmit(T record) throws IOException, InterruptedException { @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 BufferBuilder getBufferBuilder(int targetChannel) throws IOException, Int @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 b648eef3450e..0d58a8a099b7 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.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 @@ protected void finishBufferBuilder(BufferBuilder bufferBuilder) { 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 From 64557da16b4a1b9910194a20a0aa63a7dbb95600 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Thu, 10 Oct 2019 18:43:05 +0800 Subject: [PATCH 030/746] [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 365ca2096e93..b6c99b682e87 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 2ee76fcf5a53..70681fd145d9 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 @@ protected void initOutputs() throws Exception { 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 faf2dfe92429..5f3e7081aeb5 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 RoundRobinSubtaskIndexSender(Environment environment) { @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 58af0e20d610..20ce3b3919be 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 void invoke() throws Exception { // 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 e97dec778ad5..23e5dcee8939 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 Sender(Environment environment) { @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 TestInvokableRecordCancel(Environment environment) { @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 76969760d9ab..0d5588e78b39 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 @@ private static class ProducerThread extends Thread { 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 ea11f919dbef..90ee2822c378 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 951fbf96785d..61b1bce3801a 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 @@ private static RecordWriter>> crea } } - 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 49d9ef562b78..91d4cf43fa83 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 TestSourceInvokable(Environment environment) { @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 146cad9157b1..3357cb69ad04 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 SpeedTestProducer(Environment environment) { @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 void invoke() throws Exception { SpeedTestRecord.class, getEnvironment().getTaskManagerInfo().getTmpDirectories()); - RecordWriter writer = new RecordWriterBuilder().build(getEnvironment().getWriter(0)); + RecordWriter writer = new RecordWriterBuilder().build(getEnvironment().getWriter(0)); try { SpeedTestRecord record; From eea05611cea64c13a1c5d95a3b5e109ad7a9fb32 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 7 Jun 2019 16:09:55 +0200 Subject: [PATCH 031/746] [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 c7a5588009db..52b6986dbbc2 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 @@ /** * 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. From 4813a2396cc7a1a054b2c46c673ea96ced982523 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 7 Jun 2019 21:05:44 +0200 Subject: [PATCH 032/746] [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 0cf37ddf9fe8..ad84cc47a41a 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 @@ private CompletableFuture updateLeader() { 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 52740a64f688..9b3966400f65 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 @@ private CompletableFuture verifyJobSchedulingStatusAndStartJobManager(UUID 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 @@ private void confirmLeaderSessionIdIfStillLeader( 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 void revokeLeadership() { } 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 @@ private void setNewLeaderGatewayFuture() { } @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 dcf0a4e51291..8b28618e9318 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 91227dca6312..382b33e2fbeb 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 void isLeader() { if (LOG.isDebugEnabled()) { LOG.debug( "Grant leadership to contender {} with session ID {}.", - leaderContender.getAddress(), + leaderContender.getDescription(), issuedLeaderSessionID); } @@ -252,7 +252,7 @@ public void notLeader() { if (running) { LOG.debug( "Revoke leadership of {} ({}@{}).", - leaderContender, + leaderContender.getDescription(), confirmedLeaderSessionID, confirmedLeaderAddress); @@ -277,7 +277,7 @@ public void nodeChanged() throws Exception { 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 void nodeChanged() throws Exception { if (LOG.isDebugEnabled()) { LOG.debug( "Writing leader information into empty node by {}.", - leaderContender.getAddress()); + leaderContender.getDescription()); } writeLeaderInformation(); } else { @@ -299,7 +299,7 @@ public void nodeChanged() throws Exception { 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 void nodeChanged() throws Exception { if (LOG.isDebugEnabled()) { LOG.debug( "Correcting leader information by {}.", - leaderContender.getAddress()); + leaderContender.getDescription()); } writeLeaderInformation(); } @@ -410,7 +410,7 @@ protected void handleStateChange(ConnectionState newState) { 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 @@ protected void handleStateChange(ConnectionState newState) { 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 3c9ed142a5c8..c58691989d0a 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 f474ad50df84..6b1ff00e8c00 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 void revokeLeadership() { } @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 3b9ce6513b99..840d96cf51bb 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.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 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 void testResourceManagerLeaderElection() throws Exception { */ @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 void testJobManagerLeaderRetrieval() throws Exception { */ @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 void testConcurrentLeadershipOperations() throws Exception { 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 0b286716769c..9bad8128758c 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 @@ public void revokeLeadership() { } @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 535679ea65a4..910db15999cf 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.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.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 void testImmediateShutdown() throws Exception { service.shutdown(); final LeaderContender contender = mock(LeaderContender.class); - + // should not be possible to start try { service.start(contender); @@ -210,15 +214,10 @@ private static LeaderContender mockContender(final LeaderElectionService service 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 9090cae6c2e0..a1d99cdb4f34 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 void testHasLeadership() throws Exception { 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 void revokeLeadership() { } @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 34408ff2ea4f..3775d996357a 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 void grantLeadership(UUID leaderSessionID) { this.leaderSessionID = leaderSessionID; - leaderElectionService.confirmLeadership(leaderSessionID, getAddress()); + leaderElectionService.confirmLeadership(leaderSessionID, address); leader = true; @@ -139,7 +139,7 @@ public void revokeLeadership() { } @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 f282ebdc4d69..8d25f609b050 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 { * *

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 synchronized void stop() throws Exception { @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 synchronized CompletableFuture isLeader(UUID leaderSessionID) { contender.grantLeadership(leaderSessionID); } - return confirmationFuture; + return confirmationFuture.thenApply(LeaderConnectionInfo::getLeaderSessionId); } public synchronized void notLeader() { @@ -102,8 +104,8 @@ public synchronized void notLeader() { } 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 b476896b5d40..bf5b2c8d3d6f 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.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 void testZooKeeperReelection() throws Exception { 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 void testZooKeeperReelection() throws Exception { 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 void testZooKeeperReelection() throws Exception { } } + @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 void testMultipleLeaders() throws Exception { } 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 0d09f8790270..ab4f70a482d8 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.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 @@ private static LeaderContender mockContender(final LeaderElectionService service 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 { From 394d93036b7206aa049fd69c9d2daa938db642a2 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 13 Sep 2019 16:37:48 +0200 Subject: [PATCH 033/746] [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 73f3ecb6637f..b60ee3cff18f 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 @@ */ 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 // ------------------------------------------------------------------------ From c2f686637eed86ff02aec08a16c3fcca09f352fe Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 14 Sep 2019 17:15:16 +0200 Subject: [PATCH 034/746] [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 a99daf77301c..67acfe4112a3 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 void testLeaderRetrievalGateway() throws Exception { 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 fced9010ee58..edbeca4bbe35 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 eecf7b0d089f..8f51a41740c2 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 void testConnectFailure() throws Exception { 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 becbba7659a1..f07ee375879e 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 void testFileCleanup() throws Exception { 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 d9f15601d363..a626d340acc5 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 void setUp() throws Exception { @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 void testCompletedResult() throws Exception { .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 void testCompletedResult() throws Exception { @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 ec8be0fa5aa1..db0715a2b393 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 545b34a335b3..0ccfaaa58efb 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 void testCacheEntryCleanup() throws Exception { 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 24c79f3c4e94..2ce36957a402 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 CompletableFuture requestJob(JobID jobId, @RpcTim /** * 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 Builder setRequestArchivedJobFunction(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 1a3c6d78a186..b309940025eb 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 String getHostname() { 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 static class Builder { 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 Builder() { 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, From eb1bf7dcaf2307e0c040d12de9892f6fd304b23b Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 13 Sep 2019 23:55:17 +0200 Subject: [PATCH 035/746] [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 7c7a4c188cb6..bc0da64cf63d 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 CompletableFuture getTerminationFuture() { // Asynchronous executions // ------------------------------------------------------------------------ - /** * Execute the runnable in the main thread of the underlying RPC endpoint. * From aa09f11c8d0f3782e2a3c6c15b01bb7935d6fa5b Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 14 Sep 2019 17:56:08 +0200 Subject: [PATCH 036/746] [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 000000000000..27110025b811 --- /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(); +} From 707975e518a563ba201a34f6a9da891c47327112 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 15 Sep 2019 16:32:32 +0200 Subject: [PATCH 037/746] [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 000000000000..d845fb7b8330 --- /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; + } + }; + } +} From 467736ab5d84ee168e6700a779bacb4351e506c5 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 17 Sep 2019 18:20:00 +0200 Subject: [PATCH 038/746] [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 e31c3792bee0..790e85f6e269 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.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 @@ /** * 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. From cda6dc0c44239aa7a36105988328de5744aea125 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 14 Sep 2019 22:24:55 +0200 Subject: [PATCH 039/746] [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 634ec3790ea0..61cf8c6f7415 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 @@ /** * {@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. - * - *

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 000000000000..7a987952a94a --- /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; +} From 2c2095bdad3d47f27973a585112ed820f457de6f Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 11 Oct 2019 11:28:54 +0200 Subject: [PATCH 040/746] [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 73fa7682ea8b..834c8638503b 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 193ad0b13a83..e4a36a8f969f 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" From 1c7472547555bf7b71344516c45fc7063d2f859a Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Thu, 10 Oct 2019 14:44:13 +0200 Subject: [PATCH 041/746] [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 06e1b99ead1f..eaa86c79b8ee 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 @@ private void head(StreamOperator headOperator, OperatorID headOperatorID) { } 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 StreamConfigChainer chain( } 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); } From db6fdea7e68e95e309c3cb0999ab9a5cc58f141a Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Thu, 10 Oct 2019 14:30:33 +0200 Subject: [PATCH 042/746] [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 eaa86c79b8ee..0efb278946e9 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.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 @@ 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 StreamConfigChainer chain( 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 StreamConfigChainer chain( 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 StreamConfigChainer chain( tailConfig.setStateKeySerializer(inputSerializer); } tailConfig.setChainIndex(chainIndex); + tailConfig.setBufferTimeout(bufferTimeout); chainedConfigs.put(chainIndex, tailConfig); @@ -116,7 +140,6 @@ public StreamConfigChainer chain( } public void finish() { - List outEdgesInOrder = new LinkedList(); outEdgesInOrder.add( new StreamEdge( @@ -127,7 +150,6 @@ public void finish() { 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 aa5f25e0a4c8..c5db3ffff77b 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 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 void setupOutputForSingletonOperatorChain() { 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); From 0bd0ec17040fa9deaa16754dc74693307a89cb3e Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Thu, 10 Oct 2019 14:31:53 +0200 Subject: [PATCH 043/746] [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 0d58a8a099b7..93fa82630546 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 @@ */ 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 a4f83c1af8c9..5384541030d3 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 297c0640dc4a..222e45b70fef 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 @@ 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.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.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 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 void testOperatorNotSupportedByNonNetworkCreditMode() throws Exception { } } + /** + * 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 @@ Throwable getLastDeclinedCheckpointCause() { 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(); + } + } } From fbfa8beb6847ed14641399afbbfd9a378d91e6f5 Mon Sep 17 00:00:00 2001 From: tison Date: Fri, 11 Oct 2019 10:10:43 +0800 Subject: [PATCH 044/746] [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 3381f10544c1..6481420d2017 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", From 80b27a150026b7b5cb707bd9fa3e17f565bb8112 Mon Sep 17 00:00:00 2001 From: tison Date: Tue, 8 Oct 2019 16:09:04 +0800 Subject: [PATCH 045/746] [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 1273a4ee9d22..000000000000 --- 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 d29dcec47a84..000000000000 --- 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 5caf1b296baa..000000000000 --- 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 24667e42527d..000000000000 --- 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 879b52356c31..000000000000 --- 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 b0c7778a2bc2..000000000000 --- 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 910db15999cf..000000000000 --- 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 cf0b2f945f80..000000000000 --- 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 0cf9dc483c2e..000000000000 --- 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 466e198dda3a..000000000000 --- 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 f80c6b17c73b..000000000000 --- 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 ab4f70a482d8..000000000000 --- 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 67186e985b32..000000000000 --- 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) {} - } -} From cee6e4d95af8a8cd4806bd1d8647f1b5674b61be Mon Sep 17 00:00:00 2001 From: whlwanghailong Date: Tue, 1 Oct 2019 06:03:45 +0800 Subject: [PATCH 046/746] [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 c6ca9a38f6d3..7effbba7e501 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 4859c2a3d289..de71825ddac8 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 aeb226eb35a8..4ba2d5227f55 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)") From 00ee80313c9051a9c2dbe08d4d7528031b247a36 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Sat, 12 Oct 2019 14:09:37 +0800 Subject: [PATCH 047/746] [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 000000000000..953f0f311565 --- /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") + } + +} + From cee9bf0ab6c1d6b1678070ff2e05f0aa34f26dc0 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Thu, 10 Oct 2019 17:35:52 +0800 Subject: [PATCH 048/746] [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 000000000000..8b73d966520a --- /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 000000000000..7ef94e8cad91 --- /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 000000000000..46878248e9c1 --- /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 34e566f62ac7..a18defe8eda4 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 @@ private int getInputId(int inputIndex) { * 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 @@ private StreamTaskNetworkOutput( 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 458cd06b2e77..2610661d6c76 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.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.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 @@ private StreamTaskInput createTaskInput(CheckpointedInputGate inputGate, Dat * The network data output implementation used for processing stream elements * from {@link StreamTaskNetworkInput} in one input processor. */ - private static class StreamTaskNetworkOutput 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 @@ private StreamTaskNetworkOutput( Object lock, WatermarkGauge watermarkGauge, Counter numRecordsIn) { + super(streamStatusMaintainer, lock); this.operator = checkNotNull(operator); - this.streamStatusMaintainer = checkNotNull(streamStatusMaintainer); - this.lock = checkNotNull(lock); this.watermarkGauge = checkNotNull(watermarkGauge); this.numRecordsIn = checkNotNull(numRecordsIn); } @@ -194,12 +188,5 @@ public void emitLatencyMarker(LatencyMarker latencyMarker) throws Exception { operator.processLatencyMarker(latencyMarker); } } - - @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/tasks/SourceReaderStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceReaderStreamTask.java new file mode 100644 index 000000000000..57434d37d9fc --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceReaderStreamTask.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.runtime.tasks; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.operators.SourceReaderOperator; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.io.AbstractDataOutput; +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.StreamTaskSourceInput; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A subclass of {@link StreamTask} for executing the {@link SourceReaderOperator}. + */ +@Internal +public class SourceReaderStreamTask 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 000000000000..61934c6dcd57 --- /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() { + } + } +} From 345bfefd91c5a1a309ff1a4b397eb9882788fa2a Mon Sep 17 00:00:00 2001 From: Wei Zhong Date: Wed, 9 Oct 2019 16:33:56 +0800 Subject: [PATCH 049/746] [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 bf46ea9a5d3f..d7864be7495c 100644 --- a/flink-python/pyflink/table/tests/test_udf.py +++ b/flink-python/pyflink/table/tests/test_udf.py @@ -204,6 +204,22 @@ def eval(self, col): 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 bb5ff90cbbff..d12701193a4a 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 159d4f1dc0dc..8e5609cacabc 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 000000000000..141d1dc3cb44 --- /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 9e301578e5f5..0433fbee4819 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 79cc13b7d355..54541ba3fad1 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 dfed70be1018..e3e44ae8a1d4 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 cd4efd9c770c..2e0073cc12ff 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 f3cc8caf4f55..ce29bed319a3 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 +} From ddfed72813281255f119fd6838c197433eb6eaf3 Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Wed, 9 Oct 2019 11:38:35 +0800 Subject: [PATCH 050/746] [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 b473673460a9..f6106a0fb1c5 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 @@ 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 @@ ], 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 @@ 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 @@ 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 @@ oneofs=[ ], serialized_start=63, - serialized_end=289, + serialized_end=315, ) @@ -236,8 +243,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=291, - serialized_end=382, + serialized_start=317, + serialized_end=408, ) @@ -274,8 +281,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=460, - serialized_end=611, + serialized_start=486, + serialized_end=637, ) _SCHEMA_FIELDTYPE = _descriptor.Descriptor( @@ -335,8 +342,8 @@ 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 @@ extension_ranges=[], oneofs=[ ], - serialized_start=949, - serialized_end=1057, + serialized_start=975, + serialized_end=1083, ) _SCHEMA = _descriptor.Descriptor( @@ -410,8 +417,8 @@ 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.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 13e179bb53c3..4bd955cc71e3 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.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 @@ def get(self, value): 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 @@ def __init__(self, scalar_function, 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: + 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 db6582d39d78..4ce9ed3ceb4c 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 d7864be7495c..321cd784f287 100644 --- a/flink-python/pyflink/table/tests/test_udf.py +++ b/flink-python/pyflink/table/tests/test_udf.py @@ -112,6 +112,112 @@ def test_udf_in_join_condition_2(self): 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 44a568b2f53e..31c18f22432a 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 @@ 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 static List readPythonObjects(String fileName, boolean batched) 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 26791563787c..69c178b0ad5d 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 @@ private FlinkFnApi.UserDefinedFunction getUserDefinedFunctionProto(PythonFunctio 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 04f4c20a75a9..aca06762ed1d 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 { From aaebe3b75f0ea7a28c5449168d514c381f12c6b5 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Mon, 7 Oct 2019 10:45:53 +0200 Subject: [PATCH 051/746] [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 4051ed4f3296..0bddad0fcd43 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 @@ protected SchedulingTopology getSchedulingTopology() { return schedulingTopology; } - protected InputsLocationsRetriever getInputsLocationsRetriever() { + protected final InputsLocationsRetriever getInputsLocationsRetriever() { return inputsLocationsRetriever; } From 390428dca4cf218f57c453adb70aafcbd63b3e16 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Mon, 7 Oct 2019 10:47:00 +0200 Subject: [PATCH 052/746] [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 0bddad0fcd43..b4fd6470159a 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 @@ protected void failJob(Throwable cause) { executionGraph.failJob(cause); } - protected FailoverTopology getFailoverTopology() { + protected final FailoverTopology getFailoverTopology() { return failoverTopology; } From e39a556be6c766b77305859f8cf6fda073af8017 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Mon, 7 Oct 2019 10:47:34 +0200 Subject: [PATCH 053/746] [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 b4fd6470159a..1b4f62edcf1c 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 @@ protected final FailoverTopology getFailoverTopology() { return failoverTopology; } - protected SchedulingTopology getSchedulingTopology() { + protected final SchedulingTopology getSchedulingTopology() { return schedulingTopology; } From afd05d3fd7e10445c3d9363bf541538d003b0c35 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Wed, 2 Oct 2019 17:53:18 +0200 Subject: [PATCH 054/746] [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 b5ad29f96bb8..76db6d6636f8 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 void deploy(final ExecutionVertex executionVertex) throws JobException { @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 void disableFailDeploy() { failDeploy = false; } - public void enableFailCancel() { - failCancel = true; - } - - public void disableFailCancel() { - failCancel = false; - } } From dbe1bfa31db4a561b6faa9c1235f02dc130825ca Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Fri, 4 Oct 2019 12:21:13 +0200 Subject: [PATCH 055/746] [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 000000000000..839dc078c951 --- /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 c99949870c28..868cdab610ba 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 DefaultScheduler( 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 DefaultScheduler( 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 603e14d6e7e9..0860c3c7019f 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 SchedulerNG createInstance( 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 000000000000..cd763572e81f --- /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 a3e6e0e21c27..a6bc09579711 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.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.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.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 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 void setUp() throws Exception { 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 void startScheduling() { 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 void restartAfterDeploymentFails() { 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 void scheduleWithLazyStrategy() { 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 void restartFailedTask() { 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 void failJobIfCannotRestart() throws Exception { 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 void failJobIfNotEnoughResources() throws Exception { 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 @@ private static JobGraph singleNonParallelJobVertexJobGraph() { 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 @@ private DefaultScheduler createScheduler(final JobGraph jobGraph) throws Excepti VoidBackPressureStatsTracker.INSTANCE, executor, configuration, - slotProvider, + new SimpleSlotProvider(TEST_JOB_ID, 0), scheduledExecutorService, taskRestartExecutor, ClassLoader.getSystemClassLoader(), @@ -319,7 +349,8 @@ private DefaultScheduler createScheduler(final JobGraph jobGraph) throws Excepti 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 ae1c01ba50c4..000000000000 --- 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 000000000000..5960d7092cfb --- /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 000000000000..e5d451ada815 --- /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 76db6d6636f8..504575194044 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 @@ 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 void disableFailDeploy() { failDeploy = false; } + public List getDeployedVertices() { + return Collections.unmodifiableList(deployedVertices); + } } From 2f1200623f465b8e35daa86613eee676cac65b2b Mon Sep 17 00:00:00 2001 From: xuyang1706 Date: Wed, 31 Jul 2019 20:46:45 +0800 Subject: [PATCH 056/746] [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 391b2acd86b1..5cd4793ccdfb 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 000000000000..f9deceaec6c5 --- /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 000000000000..c1b298632d2f --- /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 000000000000..43c9a89cadfb --- /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 000000000000..d655fe8f2a87 --- /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 000000000000..7f8d6af5df9c --- /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 000000000000..870623afae3d --- /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 000000000000..8ea2f02c1544 --- /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 000000000000..ae7c7fe19809 --- /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 000000000000..3b6bcc143d71 --- /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 000000000000..98f007fd4e85 --- /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 000000000000..cb932569c732 --- /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 000000000000..ed3c374cbd90 --- /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 000000000000..60b18fe59559 --- /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 000000000000..17b36e25a181 --- /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 000000000000..292bffcda7cf --- /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 000000000000..2797685a760b --- /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); + } +} 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 057/746] [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 9c51b2b000e4..8bace82b6421 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;
 	 *

From 3346ab459ebf64f33b1d0775719a70ebf91eef95 Mon Sep 17 00:00:00 2001
From: Zhu Zhu 
Date: Tue, 8 Oct 2019 14:53:08 +0800
Subject: [PATCH 058/746] [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 55a77c8e6321..bc4ebb65c54a 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 void onExecutionStateChange(ExecutionVertexID executionVertexId, Executio
 		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());

From b025b397704c0d523bf642e0af3ee53c82bf1698 Mon Sep 17 00:00:00 2001
From: Zhu Zhu 
Date: Wed, 9 Oct 2019 15:41:20 +0800
Subject: [PATCH 059/746] [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 000000000000..eebec4fec184
--- /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;
+		}
+	}
+}

From a09bfc417eddf3c2a65685b950f0f4120793eb19 Mon Sep 17 00:00:00 2001
From: Zhu Zhu 
Date: Fri, 11 Oct 2019 22:43:41 +0800
Subject: [PATCH 060/746] [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 868cdab610ba..16b6847b5626 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 @@ private CompletableFuture cancelTasksAsync(final Set verti
 	}
 
 	private CompletableFuture cancelExecutionVertex(final ExecutionVertexID executionVertexId) {
+		executionSlotAllocator.cancel(executionVertexId);
 		return executionVertexOperations.cancel(getExecutionVertex(executionVertexId));
 	}
 
@@ -254,15 +255,10 @@ private static Map groupDepl
 	}
 
 	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()

From 03f58f4f0f072ba0799eace33112a1902989bc27 Mon Sep 17 00:00:00 2001
From: Zhu Zhu 
Date: Fri, 11 Oct 2019 22:46:48 +0800
Subject: [PATCH 061/746] [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 16b6847b5626..ce57b8bd93b1 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 @@ private void restartTasksWithDelay(final FailureHandlingResult failureHandlingRe
 	private Runnable restartTasks(final Set executionVertexVersions) {
 		return () -> {
 			final Set verticesToRestart = executionVertexVersioner.getUnmodifiedExecutionVertices(executionVertexVersions);
+
+			resetForNewExecutionIfInTerminalState(verticesToRestart);
+
 			schedulingStrategy.restartTasks(verticesToRestart);
 		};
 	}
@@ -231,7 +234,7 @@ public void allocateSlotsAndDeploy(final Collection verticesToDeploy = deploymentOptionsByVertex.keySet();
 		final Map requiredVersionByVertex = executionVertexVersioner.recordVertexModifications(verticesToDeploy);
 
-		prepareToDeployVertices(verticesToDeploy);
+		transitionToScheduled(verticesToDeploy);
 
 		final Collection slotExecutionVertexAssignments = allocateSlots(executionVertexDeploymentOptions);
 
@@ -254,11 +257,6 @@ private static Map groupDepl
 				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 a6bc09579711..3d7e5d21fa7e 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.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.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.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 void skipDeploymentIfVertexVersionOutdated() {
 		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 @@ private static JobVertex getOnlyJobVertex(final JobGraph jobGraph) {
 	}
 
 	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 @@ private DefaultScheduler createSchedulerAndStartScheduling(final JobGraph jobGra
 		}
 	}
 
-	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 @@ private DefaultScheduler createScheduler(final JobGraph jobGraph) throws Excepti
 			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,

From 42102f2b3bec5c5ccee5debdafbe288a795c564c Mon Sep 17 00:00:00 2001
From: Zhu Zhu 
Date: Mon, 14 Oct 2019 16:12:14 +0800
Subject: [PATCH 062/746] [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 ce57b8bd93b1..638a633c6337 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 @@ protected void scheduleOrUpdateConsumersInternal(final ExecutionVertexID produce
 
 	@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 void allocateSlotsAndDeploy(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 50d3f873232f..3256a171e4eb 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 3d7e5d21fa7e..5c0b60c92cc8 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.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 void vertexIsResetBeforeRestarted() throws Exception {
 		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);
 	}

From 463fe96e43e04d38c677fe8f13228dc791ab3163 Mon Sep 17 00:00:00 2001
From: Zhu Zhu 
Date: Fri, 11 Oct 2019 22:33:15 +0800
Subject: [PATCH 063/746] [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 79484082ed88..b86ff890a715 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 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 DefaultExecutionSlotAllocator(
 	public Collection allocateSlotsFor(
 			Collection executionVertexSchedulingRequirements) {
 
+		validateSchedulingRequirements(executionVertexSchedulingRequirements);
+
 		List slotExecutionVertexAssignments =
 				new ArrayList<>(executionVertexSchedulingRequirements.size());
 
@@ -136,6 +139,14 @@ public Collection allocateSlotsFor(
 		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 4088f337cfcd..1437b7cc3f44 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 void testComputeAllPriorAllocationIds() {
 		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());
 	}

From 51bd2f68b3d1dbf5e553e26730695915e13a34f4 Mon Sep 17 00:00:00 2001
From: Dian Fu 
Date: Mon, 14 Oct 2019 15:54:48 +0800
Subject: [PATCH 064/746] [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 4aeb110f6dd8..663c7d2a5913 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

From 08964911de86d17761ae155d5cba2ba07cec9fba Mon Sep 17 00:00:00 2001
From: openinx 
Date: Mon, 14 Oct 2019 23:46:08 +0800
Subject: [PATCH 065/746] [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 a6108afa9800..bb8e67665bf9 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 0b82b2d25587..57d76170d380 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
 

From e2c4213a8f0664285658f25f2169487eb952909e Mon Sep 17 00:00:00 2001
From: yanghua 
Date: Wed, 18 Sep 2019 20:05:05 +0800
Subject: [PATCH 066/746] [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 597b89b921f3..23cdb8e26945 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 1fba302c3057..e816da873e32 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 c2c5181d4d40..eba7ead105e1 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 interface RequestFactory extends Serializable { /** * 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 @@ UpdateRequest createUpdateRequest( /** * 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 @@ IndexRequest createIndexRequest( /** * 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 9d1a926efdd6..ec48557c1cbf 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 @@ 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 @@ private void validateVersion(DescriptorProperties properties) { 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 cc0bbf2574a6..5bdf889e1b13 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 000000000000..3c68eac4fe47 --- /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 000000000000..c569f3b8109b --- /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 000000000000..0b7a43741db5 --- /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 000000000000..5b874e29491e --- /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 000000000000..6001f43c8bf4 --- /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 000000000000..6f051a383bbc --- /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 000000000000..c8de4473a705 --- /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 000000000000..fcd865466684 --- /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 000000000000..d1d8186576d8 --- /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 000000000000..b6cf75c9ed80 --- /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 000000000000..ca2f54499169 --- /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 000000000000..2c7e1c039b67 --- /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 000000000000..fe10b08d2bbc --- /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 000000000000..a047b02f9bdb --- /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 000000000000..fcd865466684 --- /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 41a5eaa912f6..2c2b053b5a95 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 000000000000..e7036b5d390d --- /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 b7ac5519081b..bdf04b3139c3 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 6676b46e8ef7..b0f4456d324c 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,\ From aa62fc289106b891c38c4b72ba3f252e63e217c4 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 14 Oct 2019 13:05:45 +0200 Subject: [PATCH 067/746] [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 d1d8186576d8..63aa212dcc0a 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 2c7e1c039b67..80866c7c2e0b 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 From 255f047f31bc5a89e2f2224bb6fd61ae4a4e44d8 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Mon, 23 Sep 2019 11:15:03 +0800 Subject: [PATCH 068/746] [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 e6fd77af6406..ea940e0fc634 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 5639c8270bb4..d46e21c2b018 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 static void shutDownServices() throws Exception { 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 033ead6c4cc7..890ac3053509 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 static void main(String[] args) throws Exception { } 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 26a422f7f261..8f6880a517c8 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 static void main(String[] args) throws Exception { } 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 9a4a9e077890..fa68073d277c 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 d85a21d7942c..acab022b4382 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 b965355fb5bf..0b5f5be2ae0c 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 void compareNetworkBufShellScriptWithJava() throws Exception { 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 void compareHeapSizeShellScriptWithJava() throws Exception { 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 @@ private static Configuration getConfig( 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 @@ private static Configuration getRandomConfig(final Random ran) { 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 @@ private void compareHeapSizeJavaVsScript(final Configuration config, float toler 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 43fdf13a4bb2..de76d2bd0593 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 @@ private static Configuration getConfig() throws Exception { 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 8a622436977f..e170f0928bdb 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 @@ private static Configuration getConfig() throws Exception { 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 183ad9e59b5c..89797d7418ea 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 static void tearDown() { 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 34713f38f020..3880051b2e2b 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 static void tearDown() { 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 ffee6f1ddd41..21d4a859232b 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 @@ private static Configuration getClusterConfiguration() { } // !!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 void getConfiguration() throws Exception { 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 483f77c25e38..d292e5a6af9a 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 static Configuration createActiveResourceManagerConfiguration(Configurati final long managedMemoryBytes = TaskManagerServices.getManagedMemoryFromProcessMemory(originalConfiguration, processMemoryBytes); final Configuration resourceManagerConfig = new Configuration(originalConfiguration); - resourceManagerConfig.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, managedMemoryBytes + "b"); + resourceManagerConfig.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, managedMemoryBytes + "b"); return resourceManagerConfig; } 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 56450ff1c85a..161aef8ebd1a 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 @@ -1204,7 +1204,7 @@ protected int getNumberRequiredTaskManagerSlots() { 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 3db847dd34c3..752deabed2c4 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 static long calculateHeapSizeMB(long totalJavaMemorySizeMB, Configuration 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 static long getManagedMemoryFromProcessMemory(Configuration config, long * 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 64f5a30bcb71..ea6032f389bd 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 @@ long getMaxJvmHeapMemory() { * * @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 6d4ca693bdd9..a6eb90f6f653 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 static NettyShuffleEnvironmentConfiguration fromConfiguration( * *

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 948d4c83c15e..0d3788c09c7c 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 static long getManagedMemorySize(Configuration configuration) { * @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 9a9468e5998a..ff61e65afdae 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 @@ protected ResourceManager createActiveResourceManager( 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 cb020f690d9d..979860a1d64e 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 @@ private TestingResourceManager createAndStartResourceManager(HeartbeatServices h @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 void testCreateWorkerSlotProfiles() { 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 aee468985886..a3e295f98e68 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 void calculateHeapSizeMB() throws Exception { 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 ca83609a569c..355784d3786a 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 void calculateNetworkBufFromHeapSize() { 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 void calculateNetworkBufFromHeapSize() { /** * 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 @@ private static Configuration getConfig( 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 58ef6a8f0898..d2ace9562e7a 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 void testMemoryConfigWrong() throws Exception { 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 void testMemoryConfigWrong() throws Exception { // 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 eb63ceb44add..b648c22f67a2 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 @@ private void startMiniCluster() throws Exception { 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 b71006b31c41..f655af62f0ad 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 JobExecutionResult execute(StreamGraph streamGraph) throws Exception { 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 c102acf8a483..24799e047838 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 static void setup() { 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 490b5dc17ddc..209e9da8f03a 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 8050dba4917d..d656184e785b 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 490926fa6375..c09943ff67db 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 9eacee8afaa6..de681255c483 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 @@ protected Configuration createClusterConfig() throws IOException { 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 0bc50239618d..9ec23d3433e0 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 37f0c337651a..2f4a8572dd05 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 Integer map(Integer value) throws Exception { 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 8cce8009c929..c722311cff2d 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 WindowCheckpointingITCase(TimeCharacteristic timeCharacteristic) { 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 7ecf4887e848..a09142e7ee14 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 static void setUp() throws Exception { 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 3cd7674037c9..032a3da85ceb 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 38902c7619da..2398c14f8d9a 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 static void main(String[] args) throws Exception { 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 a1e4add6a004..0177e4a9b183 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 c61657dc9baf..e9523f90eca0 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 1677a88a3ad0..c8f44e8a275f 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 void testTaskManagerProcessFailure() throws Exception { 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 b7c91cbb7501..130d38ef21cc 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 void testDispatcherProcessFailure() throws Exception { 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 3b0fa172fcd3..bd76b0942aec 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 void testCancelingOnProcessFailure() throws Exception { 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 c312b0441009..8fe2aa351828 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 @@ private Configuration getConfiguration() { 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 24effd746caf..f0938a245831 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 563d646b7ee2..7459bac03047 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 @@ private boolean hasTaskManagerConnectedAndReportedSlots(Collection Date: Mon, 23 Sep 2019 11:19:14 +0800 Subject: [PATCH 069/746] [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 1c6685b2f4c6..8d7a23244618 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 7787ed0d9311..5a4f3a84621e 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 16da6d52cd0b..ca649b526ae7 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 c8433ed9e6e6..e05115d4b467 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 733085ebce82..4e2e956225e6 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 acab022b4382..57e4d0363e9b 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 // ------------------------------------------------------------------------ From 9b14f93eeb856c98c980fb40337ee74c488d3973 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Mon, 23 Sep 2019 16:20:39 +0800 Subject: [PATCH 070/746] [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 092c0128afbc..5d9e22d2a854 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 @@ 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 String toString() { 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 // ------------------------------------------------------------------------ From cf337043236c852eaa16e4e51d6c4e95d9a6d056 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Mon, 23 Sep 2019 16:28:12 +0800 Subject: [PATCH 071/746] [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 000000000000..0813aead4cb1 --- /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); + } +} From 59eca61bff3965a71cdb16865050c3daa0c8014b Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Tue, 24 Sep 2019 11:32:51 +0800 Subject: [PATCH 072/746] [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 000000000000..ba521bb10c57 --- /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 000000000000..a77370d38e44 --- /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; + } +} From d571b2bcab8188ae12e47a63ea4cc5d4583fb7de Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Sat, 5 Oct 2019 15:28:50 +0800 Subject: [PATCH 073/746] [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 ba521bb10c57..ee13943a78a1 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.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 a77370d38e44..8577d8f1c834 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.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(); From 4966b1fce1c22fe96c740fe979e96e1b78c434f0 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Mon, 14 Oct 2019 12:16:10 +0200 Subject: [PATCH 074/746] [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 70fdcc5cf304..34336ff5c919 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 fa8e14e2f8ea..aac82cc395d5 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 From 73ec89be5c7cb07c0d4f6d2352c7182341ff71d5 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Mon, 14 Oct 2019 12:19:22 +0200 Subject: [PATCH 075/746] [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 34336ff5c919..696b508ae430 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 aac82cc395d5..389c65cabf6c 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 From 34379182d63991baf9672c48842f06316d541170 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Mon, 30 Sep 2019 13:41:47 +0800 Subject: [PATCH 076/746] [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 3744f1161841..628781540938 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.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 @@ /** * Integration test cases for the {@link MiniCluster}. */ +@Category(AlsoRunWithSchedulerNG.class) public class MiniClusterITCase extends TestLogger { @Test @@ -110,7 +113,12 @@ public void testHandleStreamingJobsWhenNotEnoughSlot() throws Exception { } 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 void testHandleBatchJobsWhenNotEnoughSlot() throws Exception { } 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()); } } From eee2988b3b1be37c4c465fb75da06dea7e46328d Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Mon, 30 Sep 2019 14:19:10 +0800 Subject: [PATCH 077/746] [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 71c2775374e1..630bb8ef5101 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.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 @@ /** * 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 @@ private SupplierWithException jobIsRunning(Supplier Date: Mon, 30 Sep 2019 14:21:08 +0800 Subject: [PATCH 078/746] [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 12ab07754f09..fa8373544a46 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.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 bb2aede627ff..bbc57333b897 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.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.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 @@ * 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 5f3e7081aeb5..8e6a22f0f9b2 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.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 @@ * 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 20ce3b3919be..cbd2fae49faf 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.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.junit.ClassRule; import org.junit.Test; +import org.junit.experimental.categories.Category; import java.util.List; @@ -48,6 +50,7 @@ * 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 e19fe2020dda..03f36b80295c 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.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 @@ /** * 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 d003912eae35..d9f38d370dad 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.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 @@ /** * 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 0731d96fee28..fbb1bad9f379 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.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.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 @@ /** * 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 0d5588e78b39..e3fda33f5077 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.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.assertNotNull; import static org.junit.Assert.assertTrue; +@Category(AlsoRunWithSchedulerNG.class) public class TaskCancelAsyncProducerConsumerITCase extends TestLogger { // The Exceptions thrown by the producer/consumer Threads From ca5e51822e5139702dae3e85f9237cdb41d4ff70 Mon Sep 17 00:00:00 2001 From: hpeter Date: Sat, 12 Oct 2019 16:45:34 -0700 Subject: [PATCH 079/746] [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 a680a0a23064..5a36efb0cc39 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 %} From 60cc18bb6ec20a66ca79b835dc67b5cdbfb3c881 Mon Sep 17 00:00:00 2001 From: liyafan82 Date: Mon, 1 Jul 2019 18:13:38 +0800 Subject: [PATCH 080/746] [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 503b59e2886c..d9cdf5a9e41b 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.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 @@ static CompletableFuture> 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; } From 2894caf4a2a054a4838af5b631d40d15c43369ff Mon Sep 17 00:00:00 2001 From: "kevin.cyj" Date: Tue, 24 Sep 2019 18:23:56 +0800 Subject: [PATCH 081/746] [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 9e5720d15595..02bfa7cf090c 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 void testServerInitializationFailure() throws Throwable { @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 void testPortRangeSuccess() throws Throwable { 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 void testPortRangeSuccess() throws Throwable { 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 void testPortRangeSuccess() throws Throwable { 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 CompletableFuture shutdown() { @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()); } From 3dcc72b29834aae859cd053af7bd01e5162d5727 Mon Sep 17 00:00:00 2001 From: huzheng Date: Wed, 9 Oct 2019 17:36:56 +0800 Subject: [PATCH 082/746] [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 10bffabf36ec..d5237fc5041b 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 000000000000..66584f9b840e --- /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 000000000000..b9a2f5dc2a4e --- /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 57224c21a801..896dbbe1c73c 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"; From 95aec8604e7d950dbb3a2cd1f02a854428d4c61c Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Tue, 15 Oct 2019 21:01:30 +0800 Subject: [PATCH 083/746] [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 f4f73e0685a9..975a0d01d86b 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 From 4fe51c1922dd005e508e74128140d5a7b0abab3c Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Tue, 15 Oct 2019 21:01:59 +0800 Subject: [PATCH 084/746] [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 fa07821e31cf..15614c948a34 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 @@ /** * 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 125ece05bcc8..82f3e5fbf916 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 @@ /** * Connector descriptor for the Apache Kafka message queue. */ +@PublicEvolving public class Kafka extends ConnectorDescriptor { private String version; From b24e4bcb4cca7e4a9d14ae3b3fff2d2678517863 Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Mon, 30 Sep 2019 14:44:49 -0700 Subject: [PATCH 085/746] [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 cf2ace7c1686..06b20f529adc 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 @@ private static Executor lookupExecutor( public void registerFunction(String name, TableFunction tableFunction) { TypeInformation typeInfo = UserFunctionsTypeHelper.getReturnTypeOfTableFunction(tableFunction); - functionCatalog.registerTableFunction( + functionCatalog.registerTempSystemTableFunction( name, tableFunction, typeInfo @@ -160,7 +160,7 @@ public void registerFunction(String name, AggregateFunction agg TypeInformation accTypeInfo = UserFunctionsTypeHelper .getAccumulatorTypeOfAggregateFunction(aggregateFunction); - functionCatalog.registerAggregateFunction( + functionCatalog.registerTempSystemAggregateFunction( name, aggregateFunction, typeInfo, @@ -175,7 +175,7 @@ public void registerFunction(String name, TableAggregateFunction 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 88aa167aa40d..12e874ece3b8 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 Optional getCatalog(String catalogName) { @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 423b5f33001d..43280b47029c 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 void setPlannerTypeInferenceUtil(PlannerTypeInferenceUtil plannerTypeInfe 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 void registerTableFunction( // check if class could be instantiated UserFunctionsTypeHelper.validateInstantiation(function.getClass()); - registerFunction( + registerTempSystemFunction( name, new TableFunctionDefinition( name, @@ -101,7 +100,7 @@ public void registerTableFunction( ); } - public void registerAggregateFunction( + public void registerTempSystemAggregateFunction( String name, UserDefinedAggregateFunction function, TypeInformation resultType, @@ -128,12 +127,71 @@ public void registerAggregateFunction( 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 @@ private Set getUserDefinedFunctionNames() { // Get functions registered in memory result.addAll( - userFunctions.values().stream() + tempSystemFunctions.values().stream() .map(FunctionDefinition::toString) .collect(Collectors.toSet())); @@ -204,7 +262,7 @@ public Optional lookupFunction(String name) { } // 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 PlannerTypeInferenceUtil getPlannerTypeInferenceUtil() { 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 c45d324133d2..1384be09e5a4 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 a49f06fa6f35..f0289e4b1801 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 0201d772b668..a452717e0f64 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 0e002688791a..a00ea3526b22 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, From 2c411686d23f456cdc502abf1c6b97a61070a17d Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Wed, 16 Oct 2019 08:08:58 +0800 Subject: [PATCH 086/746] [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 66584f9b840e..9dcdff4d4345 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 HBase writeBufferFlushMaxRows(int writeBufferFlushMaxRows) { } /** - * 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 HBase writeBufferFlushMaxRows(int writeBufferFlushMaxRows) { * {@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; } From b1f468a82b4445a757d78184ccf4519cd0419312 Mon Sep 17 00:00:00 2001 From: Jiayi Liao Date: Wed, 16 Oct 2019 17:36:50 +0800 Subject: [PATCH 087/746] [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 c84a13bf73fb..acde2957205a 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 c84a13bf73fb..acde2957205a 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); } From 2c50380d43e1bd0ea9989851c9a5e2ccd8e48604 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Wed, 25 Sep 2019 16:36:10 +0200 Subject: [PATCH 088/746] [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 435b0a9475b2..830b0522201b 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 @@ /** * Strategy to switch between different {@link SlotProvider} allocation strategies. */ -abstract class SlotProviderStrategy { +public abstract class SlotProviderStrategy { protected final SlotProvider slotProvider; @@ -58,7 +58,7 @@ boolean isQueuedSchedulingAllowed() { * @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 CompletableFuture allocateSlot( * @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, From 6dbcddd01e86f8971b9fa87a457f36bba0d88daa Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Wed, 25 Sep 2019 16:45:59 +0200 Subject: [PATCH 089/746] [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 b86ff890a715..5642ab657a59 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 Collection allocateSlotsFor( 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 Collection allocateSlotsFor( (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 839dc078c951..bd086930e27c 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 @@ */ 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 0860c3c7019f..003035fad78f 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.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 SchedulerNG createInstance( jobGraph.isCheckpointingEnabled()) .create(); + final SlotProviderStrategy slotProviderStrategy = SlotProviderStrategy.from( + jobGraph.getScheduleMode(), + slotProvider, + slotRequestTimeout, + true); + return new DefaultScheduler( log, jobGraph, @@ -99,7 +106,7 @@ public SchedulerNG createInstance( 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 1437b7cc3f44..ce3f83711adc 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.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 @@ private DefaultExecutionSlotAllocator createExecutionSlotAllocator() { } 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) { From 88b48619e2734505a6c2ba0d53168528bc0dc143 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Mon, 14 Oct 2019 11:55:25 +0200 Subject: [PATCH 090/746] [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 5642ab657a59..22599a632dfa 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.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 Collection allocateSlotsFor( new SlotProfile( schedulingRequirements.getResourceProfile(), preferredLocations, - Arrays.asList(schedulingRequirements.getPreviousAllocationId()), + Collections.singletonList(schedulingRequirements.getPreviousAllocationId()), allPreviousAllocationIds))); SlotExecutionVertexAssignment slotExecutionVertexAssignment = From 46f8b592c84b8406aab5307f20e779e55d45de80 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Tue, 15 Oct 2019 10:55:07 +0800 Subject: [PATCH 091/746] [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 3b54b8efd513..b31f95cc93bd 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.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 @@ /** * 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 21d4a859232b..1202499309c8 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.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.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 @@ /** * 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 b4182c735c9b..1b306300c6c2 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.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 @@ /** * 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 fa932dc266ce..75113000379f 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.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.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 @@ * 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 8fd94e5d692d..3c158a818f74 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() From d632018ed915a8ca7c5f7b180e023a87cfae9a21 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 11 Sep 2019 17:36:23 +0200 Subject: [PATCH 092/746] [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 0937842fea04..0e71bcda97c3 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.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 @@ * 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 000000000000..ab75e03671a6 --- /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 {} From 8092dafa05f1dea3cbb63888649362ca62573f24 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 11 Sep 2019 17:37:33 +0200 Subject: [PATCH 093/746] [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 000000000000..61bda14a6d37 --- /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 000000000000..c224422599c9 --- /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 000000000000..ae18b6b24a53 --- /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; + } +} From fa872ebfe47bc46c0afba19091caf84f2c29a7b9 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 11 Sep 2019 17:38:31 +0200 Subject: [PATCH 094/746] [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 ee03705787ec..9fb4ce5b9f72 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 @@ 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 d9444d5e8afa..290cff9d1c18 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 @@ 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 LocalExecutor(Configuration conf) { 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 @@ private JobExecutorService createJobExecutorService(Configuration configuration) * 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 4d0dfd9c7dea..d78a04fd3ca7 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 int getDefaultParallelism() { // ------------------------------------------------------------------------ @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 4d465955127f..0ceb850681af 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 @@ 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 ContextEnvironment(ClusterClient remoteConnection, List jarFiles, 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 2095c63c45f1..83a8e1711311 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 eddf66c6d373..5b46c540936c 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 @@ static class TestPlanExecutor extends PlanExecutor { private List globalClasspaths; @Override - public JobExecutionResult executePlan(Plan plan) throws Exception { + public JobExecutionResult executePlan(Pipeline plan) throws Exception { return null; } From 55eb4b254eb531287ec84c63874b479d42f8e315 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 12 Sep 2019 13:37:18 +0200 Subject: [PATCH 095/746] [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 61bda14a6d37..33c802770809 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 static JobGraph getJobGraph( 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 f4741fb315a7..522c6fc9ffd8 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.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 @@ * */ @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 000000000000..ebd554fc9cc5 --- /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; + } +} From b47c13c00be47d3a9132761134261d80295920b4 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 12 Sep 2019 13:37:39 +0200 Subject: [PATCH 096/746] [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 f655af62f0ad..54f435459126 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 @@ 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 @@ @Public public class LocalStreamEnvironment extends StreamExecutionEnvironment { - private static final Logger LOG = LoggerFactory.getLogger(LocalStreamEnvironment.class); - private final Configuration configuration; /** @@ -83,42 +74,11 @@ protected Configuration getConfiguration() { */ @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(); } } } From 14b19dd4ac9845616604bfc72de6e84506e56caa Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 19 Sep 2019 16:37:33 +0200 Subject: [PATCH 097/746] [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 522c6fc9ffd8..ec6992fd7af4 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.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 CheckpointConfig getCheckpointConfig() { 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 bd6a4b54b328..9a4eafd967e0 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.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 StreamGraphGenerator setBlockingConnectionsBetweenChains(boolean blocking 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 58b4989dd7bd..83a81031526d 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 @@ private JobGraph createJobGraph() { 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 e09334aa8cf8..fc057e1027a4 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.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 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 @@ @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 1334d2830da3..18b25f793694 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.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.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.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 void invoke(Tuple2 value) {} @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 void testDisabledCheckpointing() throws Exception { 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. */ From 85e5a77de130f4b064d3c70671f1fd7f3fc046e2 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 19 Sep 2019 16:42:19 +0200 Subject: [PATCH 098/746] [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 13bde9662ef6..b393e5ee3df9 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.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 static JobExecutionResult executeRemotely(StreamExecutionEnvironment stre ) throws ProgramInvocationException { StreamGraph streamGraph = streamExecutionEnvironment.getStreamGraph(jobName); return executeRemotely(streamGraph, - streamExecutionEnvironment.getClass().getClassLoader(), streamExecutionEnvironment.getConfig(), jarFiles, host, @@ -242,7 +242,6 @@ public static JobExecutionResult executeRemotely(StreamExecutionEnvironment stre * @throws ProgramInvocationException */ private static JobExecutionResult executeRemotely(StreamGraph streamGraph, - ClassLoader envClassLoader, ExecutionConfig executionConfig, List jarFiles, String host, @@ -255,8 +254,6 @@ private static JobExecutionResult executeRemotely(StreamGraph streamGraph, 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 @@ private static JobExecutionResult executeRemotely(StreamGraph streamGraph, 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 JobExecutionResult execute(StreamGraph streamGraph) throws Exception { @Deprecated protected JobExecutionResult executeRemotely(StreamGraph streamGraph, List jarFiles) throws ProgramInvocationException { return executeRemotely(streamGraph, - this.getClass().getClassLoader(), getConfig(), jarFiles, host, From 3cdf06b25b7ea9794768bf718d253bf222aa15b0 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 19 Sep 2019 16:42:43 +0200 Subject: [PATCH 099/746] [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 6ee45411f63b..b2db19b41481 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 @@ 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 @@ protected StreamContextEnvironment(ContextEnvironment ctx) { 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 b30c33844a39..acfa6050be4c 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 @@ * 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 void testPortForwarding() throws Exception { 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 void testRemoteExecutionWithSavepoint() throws Exception { 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); From ee8f9309d64e1438aef74e3a4376a0576f104412 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Tue, 24 Sep 2019 10:23:22 +0200 Subject: [PATCH 100/746] [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 83a8e1711311..7c90383b21d6 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 @@ /** * 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); } } } From eafdb64e40b30ca120e318a62be21107657b5573 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Tue, 24 Sep 2019 13:34:38 +0200 Subject: [PATCH 101/746] [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 290cff9d1c18..d866a4f10109 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.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 @@ private JobExecutorService createJobExecutorService( 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 d78a04fd3ca7..91a5d3a930f4 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 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 @@ */ 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 RemoteExecutor( // ------------------------------------------------------------------------ /** - * 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 void setDefaultParallelism(int defaultParallelism) { } /** - * 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 int getDefaultParallelism() { // ------------------------------------------------------------------------ @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 JobExecutionResult executePlan(Pipeline plan) throws Exception { ClassLoader userCodeClassLoader = ClientUtils.buildUserCodeClassLoader( jarFiles, - this.globalClasspaths, + globalClasspaths, getClass().getClassLoader()); return executePlanWithJars(jobGraph, userCodeClassLoader); @@ -137,7 +123,9 @@ private JobExecutionResult executePlanWithJars(JobGraph jobGraph, ClassLoader cl 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 73e99e58b798..187c345a3fb4 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 java.net.InetAddress; import java.net.InetSocketAddress; -import java.net.URL; import java.net.UnknownHostException; import java.util.Collections; @@ -54,7 +53,7 @@ public void testUnresolvableHostname1() throws Exception { 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 void testUnresolvableHostname1() throws Exception { 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 7c90383b21d6..2af3874c547f 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.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 static PlanExecutor createLocalExecutor(Configuration configuration) { * @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 static PlanExecutor createRemoteExecutor( 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 9ce09467b1cb..dc886507a6cc 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.PlanExecutor; import org.apache.flink.configuration.Configuration; +import java.util.Collections; + import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -73,7 +75,10 @@ public JobExecutionResult execute(String jobName) throws Exception { 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 786add1400e2..9b6add766639 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 RemoteEnvironment(String host, int port, Configuration clientConfig, 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 6898a99a2e9e..e5451927422e 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 JobExecutionResult execute(String jobName) throws Exception { 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 5b46c540936c..19bc2a0aa289 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.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; +import java.net.URL; import java.util.List; import scala.Option; @@ -68,18 +69,13 @@ public void testConfigurationForwarding() throws Exception { 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 @@ static class TestPlanExecutor extends PlanExecutor { 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 54f435459126..a896cf2ed53a 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 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 @@ protected Configuration getConfiguration() { 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 b393e5ee3df9..1de834aaa70c 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 @@ private static JobExecutionResult executeRemotely(StreamGraph streamGraph, 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 af76ec9ceca1..331f6a98cb39 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 java.io.File; import java.io.FileWriter; +import java.util.Collections; /** * Integration tests for {@link LocalExecutor}. @@ -62,7 +63,7 @@ public void testLocalExecutorWithWordCount() { 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()); From 98b54e0d9d2bd3ade8ca10d2a70fb5cb7a20a4f7 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 10 Oct 2019 17:18:20 +0200 Subject: [PATCH 102/746] [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 33c802770809..e54a4280d5be 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 static JobGraph getJobGraph( 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 static JobGraph getJobGraph( throw new RuntimeException("Translator " + streamGraphTranslator + " cannot translate " + "the given pipeline " + pipeline + "."); } - - return streamGraphTranslator.translate(pipeline, - optimizerConfiguration, - defaultParallelism); + return streamGraphTranslator; } private static FlinkPipelineTranslator reflectStreamGraphTranslator() { From 70a8e0c407a21174f608adf8ff1d4254d9024490 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 11 Oct 2019 14:08:10 +0200 Subject: [PATCH 103/746] [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 377f870ca0ad..3c6f93575371 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 JobID getJobID() { 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. * From 7084e07eaba0d967cb2474bb4ea9e80da5cca46e Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 11 Oct 2019 14:17:43 +0200 Subject: [PATCH 104/746] [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 e54a4280d5be..f88c69dd52a0 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 static JobGraph getJobGraph( 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 c224422599c9..fa3bc398f4a1 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 ae18b6b24a53..43c6bb7299b3 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.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 @@ private void setDefaultParallelism(Plan plan, int defaultParallelism) { 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 ebd554fc9cc5..4a43adbb2ed0 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 JobGraph translate( "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 From 835e776a9038448333a05b6e08e7f036ae2c74be Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 11 Oct 2019 14:25:58 +0200 Subject: [PATCH 105/746] [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 1de834aaa70c..2574b8e343f8 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 @@ private static JobExecutionResult executeRemotely(StreamGraph streamGraph, 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 @@ private static JobExecutionResult executeRemotely(StreamGraph streamGraph, } 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 { From 59dd855628052c369b64c71edc1018ed378e8eec Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 11 Oct 2019 14:20:39 +0200 Subject: [PATCH 106/746] [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 c4d5fa9b3e50..552eccc7f96a 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.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.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 @@ protected void info(String[] args) throws CliArgsException, FileNotFoundExceptio 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 81794032129f..c0ea516c9301 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 @@ 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 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 void close() throws Exception { } - // ------------------------------------------------------------------------ - // 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 JobSubmissionResult run(PackagedProgram prog, int parallelism) } } - 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 Map> getAccumulators(JobID jobID) throws */ 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 d2801ac060e8..edf7d36595f7 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 OptimizerPlanEnvironment(Optimizer compiler) { @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 FlinkPlan getOptimizedPlan(PackagedProgram prog) throws ProgramInvocation } 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 @@ private void unsetAsContext() { // ------------------------------------------------------------------------ - 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 d9a10e647bdb..fa9f8b0486d1 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 static JobGraph createJobGraph( 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 static JobGraph createJobGraph( 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 48c889120343..873ba0041779 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 loadClass(String name) throws ClassNotFoundException { 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 b31f95cc93bd..634ebf0c284f 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.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.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 void testDetachedMode() throws Exception{ 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 void testGetExecutionPlan() throws ProgramInvocationException { 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 d26b0d001071..1b52f377c749 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 void testGetExecutionPlan() { 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 d146c83e35da..000000000000 --- 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 311c2861e747..3e8f2f02f666 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 @@ * 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 f3fe632207f1..000000000000 --- 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 bb1a4cfe0bca..54ef3e292367 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 JobExecutionResult execute(StreamGraph streamGraph) throws Exception { 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 ec6992fd7af4..a8b322f81efd 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.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 @@ * */ @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 @@ private void removeVertex(StreamNode toRemove) { } /** - * 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 27c3ee261a7e..73c9ccebd9f4 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 @@ 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 ExecutionConfig getExecutionConfig() { } 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 @@ private FlinkPlan createPlan(String name, Configuration flinkConfig) { 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; } } From 41492c409aebd821069f19c721ae3cb64a91758e Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 8 Oct 2019 11:26:55 +0200 Subject: [PATCH 107/746] [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 fed220fe9528..4cd8b89059d7 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 @@ public void run() { if (jobArchives == null) { continue; } - boolean updateOverview = false; int numFetchedArchives = 0; for (FileStatus jobArchive : jobArchives) { Path jobArchivePath = jobArchive.getPath(); @@ -200,7 +199,6 @@ public void run() { fw.flush(); } } - updateOverview = true; numFetchedArchives++; } catch (IOException e) { LOG.error("Failure while fetching/processing job archive for job {}.", jobID, e); @@ -223,7 +221,7 @@ public void run() { } } } - if (updateOverview) { + if (numFetchedArchives > 0) { updateJobOverview(webOverviewDir, webDir); for (int x = 0; x < numFetchedArchives; x++) { numArchivedJobs.countDown(); From f22f1eba8f7695857a2015ed178365191849dac4 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 8 Oct 2019 11:32:59 +0200 Subject: [PATCH 108/746] [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 4cd8b89059d7..6f52cef1b175 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 @@ public void run() { 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); } From 26bc3c8c65c757285c58b2cfcb0ba81111395ea4 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 8 Oct 2019 10:48:39 +0200 Subject: [PATCH 109/746] [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 d0fbc5eb8ce2..ab1e34d74078 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 static Collection getArchivedJsons(Path file) throws IOExce 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; } } } From 88ae9f8e3a5749e262fc2a9217ced4f2dc997b44 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 8 Oct 2019 11:27:34 +0200 Subject: [PATCH 110/746] [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 6f52cef1b175..79add41fba51 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 @@ public void run() { 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 @@ public void run() { 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()); 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 111/746] [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 1a45ff2a8312..2fa6c883c6b7 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 @@ protected MemorySegment nextSegment(MemorySegment current, int positionInCurrent this.segmentNum++; return next; } else { - throw new EOFException(); + throw new EOFException("Can't collect further: memorySource depleted"); } } From 36e9b2d8e4b7490e0de5cf35e15471981c4c43b8 Mon Sep 17 00:00:00 2001 From: wangpeibin Date: Thu, 27 Jun 2019 11:31:01 +0800 Subject: [PATCH 112/746] [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 b771c5e3b931..f2b506fa6d6f 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 From ff249bce3dd4ef61518fb1e647e31e8f85640b61 Mon Sep 17 00:00:00 2001 From: cyq89051127 Date: Sun, 25 Aug 2019 15:01:43 +0800 Subject: [PATCH 113/746] [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 975a0d01d86b..520662026e2a 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 6deb6af55441..9040cf1cba0f 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 a65554f8cab2..b3c2de4934d9 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 Builder setFieldDelimiter(char c) { 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 c467f1f00239..5fb32050bb8f 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 Csv fieldDelimiter(char delimiter) { } /** - * 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 bc5397168138..2f7ebabc198f 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 ece2e45e1871..02b3e2782f36 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 void testSerializationProperties() throws Exception { 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"))); From 8e81fc265d95f634401269b2acf9a4e80c0c1044 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Wed, 16 Oct 2019 18:24:01 +0800 Subject: [PATCH 114/746] [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 4d7726584e14..cd0ab7181455 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 String asSerializableString() { @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 54aa7e40100c..cd4b7ee6c229 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 String asSerializableString() { @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 eb80d25bc032..c9d265b58ff0 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 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 void testValueLiteralString() { 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 6bbbc80c11b8..5ce9937f6d07 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 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 void testMultisetType() { 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 void testMultisetType() { 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 void testMapType() { 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()) From 94b444e3ec128e7a4daaaf4a49ceeeca9d886f1d Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 16 Oct 2019 17:33:39 +0200 Subject: [PATCH 115/746] [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 13139ff9a121..8a913bcfd6ad 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` 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 116/746] [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 5d0c2c365ea8..99de21704290 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 fec3b4fa1bab..315ffaf6336e 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 @@ private 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 a6784bbfd6fb..e7f1422a94eb 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"; From 456a17685374aaa2f17479c013fde56fc528e7fb Mon Sep 17 00:00:00 2001 From: Jiayi Liao Date: Sun, 6 Oct 2019 14:30:57 +0800 Subject: [PATCH 117/746] [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 06f137b47425..6c86e06dba77 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 4fdd43370fa7..ba0763938d62 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.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 SqlCreateTable( 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 List getUniqueKeysList() { return uniqueKeysList; } - public SqlCharStringLiteral getComment() { - return comment; + public Optional getComment() { + return Optional.ofNullable(comment); } public boolean isIfNotExists() { @@ -129,60 +133,52 @@ public boolean isIfNotExists() { 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 void unparse( 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 void unparse( 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 void unparse( 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 @@ private void printIndent(SqlWriter writer) { */ 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 0bb193b31c88..868592a7de6b 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.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 SqlCreateView( 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 SqlNode getQuery() { 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 8a63683e27fa..83e7376f2061 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.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 void setType(SqlDataTypeSpec type) { 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 36c742a2ad5f..c4d1dc496078 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 void testCastAsRowType() { "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 d41ad70d5acc..3a1296536794 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.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 static Operation convert(FlinkPlannerImpl flinkPlanner, SqlNode sqlNode) */ 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 f44365f5f129..c7566602f737 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.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 static Operation convert(FlinkPlannerImpl flinkPlanner, SqlNode sqlNode) */ 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, 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 118/746] [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 e601e7f129a5..6bdb9df74a2a 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) 都可以访问的位置,例如分布式文件系统上的位置。 From 9e7f229be0ace6d7cd5196367d67162e041fdcb0 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Thu, 17 Oct 2019 14:39:20 +0800 Subject: [PATCH 119/746] [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 e816da873e32..915860fbd251 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 4d7ceea6d8a0..48849c912cac 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 9040cf1cba0f..130f55ebf94f 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 e37e7be6137e..7fd872832d22 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 4aeb110f6dd8..663c7d2a5913 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 13139ff9a121..8a913bcfd6ad 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 dd497847ceb2..cfbd9cc7c141 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 %} From 60b5698aa2b9fe9ae233957688bd67a63325b7db Mon Sep 17 00:00:00 2001 From: lining Date: Thu, 17 Oct 2019 17:03:11 +0800 Subject: [PATCH 120/746] [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 822e8373a602..91f7e98615ab 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 a3fc5bbc86b0..2345c85fd656 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 }} From 3f75754e2e506f7db54719ac2d605e365f1b5514 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 17 Oct 2019 13:04:29 +0200 Subject: [PATCH 121/746] [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 5a278083195c..c112bc6c5248 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 08978f97463c..7eab0bdb76da 100644 --- a/pom.xml +++ b/pom.xml @@ -1589,6 +1589,7 @@ under the License. Apache Flink + UTF-8 From 62aa48795936f9700c38266a8a57b24ac46eaef6 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 9 Oct 2019 10:18:48 +0200 Subject: [PATCH 122/746] [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 60a3cca85d86..8a1aa6d50e6a 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} From 8046f046e307164e62b662a30ef617a1eb99e259 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 9 Oct 2019 10:20:57 +0200 Subject: [PATCH 123/746] [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 5aa7585410f7..754b5e804327 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 8a1aa6d50e6a..6630106638a9 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} From a37a547a8f7f07c036d6b86ad93254b7d67280f7 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 9 Oct 2019 10:21:28 +0200 Subject: [PATCH 124/746] [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 afea5236f2c7..000000000000 --- 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 8c549fbce8cb..9c1d29ba8a15 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 09c38ea51608..000000000000 --- 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 21645bec9d5d..000000000000 --- 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 b620ae68fe33..000000000000 --- 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 0ed636169965..000000000000 --- 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 a1256bdf6f8f..000000000000 --- 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 828d3633f735..000000000000 --- 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 91f064493681..000000000000 --- 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 1787fc65befe..000000000000 --- 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 fe9db79b9113..000000000000 --- 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 6413b092d70f..000000000000 --- 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 dfe5c2183e82..000000000000 --- 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 62ffbccb6837..000000000000 --- 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 31ebc840539c..000000000000 --- 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 6ca0ceecc5d7..000000000000 --- 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 d112c4806aa9..000000000000 --- 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 8cc76f6c185d..000000000000 --- 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 1d9d875edb46..000000000000 --- 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 7d7dd9424812..000000000000 --- 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 1d64ed68ce64..000000000000 --- 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 3e30b1aeff26..000000000000 --- 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 b9fb2c6e6ef0..000000000000 --- 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 12edf23c6711..000000000000 --- 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 e3c8f95557db..000000000000 --- 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 6601c0131eea..000000000000 --- 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 3e30b1aeff26..000000000000 --- 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 f21351ced0ec..000000000000 --- 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 ef871e242621..000000000000 --- 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 e34763968c26..000000000000 --- 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 e3dbacb999ce..000000000000 --- 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 b1c292b54cb2..000000000000 --- 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 29ad562af0a5..000000000000 --- 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 e1cd88478edf..000000000000 --- 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 e3a30b4eafba..000000000000 --- 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 e1cd88478edf..000000000000 --- 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 8618b7333d6f..000000000000 --- 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 06ec6fbe0736..000000000000 --- 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 862172d90e8f..000000000000 --- 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 2dae87549ebc..000000000000 --- 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 19b305b00060..000000000000 --- 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 0f45e3e464c1..000000000000 --- 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 ad923a6ea4c9..000000000000 --- 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 0cfb790dc91d..000000000000 --- 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 65afc3111686..000000000000 --- 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 6d8bdabbbe0f..000000000000 --- 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 6d6a875878f4..000000000000 --- 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 93119e70ed5e..000000000000 --- 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 93119e70ed5e..000000000000 --- 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 19b305b00060..000000000000 --- 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 38debb194c6d..000000000000 --- 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 dfcd5509eddc..000000000000 --- 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 ec5f348998bd..000000000000 --- 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 c5de4726c6fb..000000000000 --- 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 91392f658d40..06d42ff66774 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" From cc519d431305531a1e28754deda468c224435d15 Mon Sep 17 00:00:00 2001 From: Nick Pavlakis Date: Wed, 20 Mar 2019 15:04:23 -0700 Subject: [PATCH 125/746] [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 000000000000..f512fe8ad8e8 --- /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."); + } + } +} From 87f84e0c67f620b6b06cf83fcd775fc0ed2ede48 Mon Sep 17 00:00:00 2001 From: Yun Gao Date: Thu, 17 Oct 2019 21:43:04 +0800 Subject: [PATCH 126/746] [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 825289664f02..fc4a40d80acf 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 @@ * 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: * From cbf7b76923feec5ce8c9cb45a3082339be5ff17e Mon Sep 17 00:00:00 2001 From: Rong Rong Date: Sun, 29 Sep 2019 12:14:02 -0700 Subject: [PATCH 127/746] [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 01ebecaea3e5..8da8046ee42f 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 7fd872832d22..82398a44d873 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 b1e716110e98..98dfc62b34f5 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.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 HBaseTableSource projectFields(int[] fields) { @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 6e3ada4c493d..011e93da028f 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 boolean isFilterPushedDown() { @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 0b5d168bd280..91447279730b 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 javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; /** @@ -223,7 +224,8 @@ public TableSchema getTableSchema() { @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 a4363e2650dc..a70c3ca8e7b7 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 1a9f8cd0e81b..c752647bbf32 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 6aad23c40eec..e4973886ef50 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 190f1c6151e3..ad8a1002e148 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 5a490fc2edcb..5ecf1310493a 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 3d4f11f98c76..837fa0829a39 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 24fab4251f90..0416218eb837 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 9c16135065be..de1d0e00dd04 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 3ea97abdce40..9864cd092982 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 5ae51ab5a31b..5b7736442e26 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 000000000000..11bdf6ebb8eb --- /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 4f767f69949d..79f5f3297860 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 6f6615875918..6a6d691bd849 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], From 4491165c6542ce1dd68562cb193a71f3a15bb96e Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 14 Sep 2019 18:18:40 +0200 Subject: [PATCH 128/746] [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 9212ea871fed..9cf8232563a4 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 @@ 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 e0926217ea88..dc2654714ad9 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.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 DispatcherRunnerFactoryImpl(DispatcherFactory dispatcherFactory) { @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 a50cc1be902a..68bb95bb529e 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 @@ private void runCluster(Configuration configuration) throws Exception { 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 dd89a757a6a0..9b72997cdc4c 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 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 DispatcherResourceManagerComponent create( 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 744941ca3af2..752291d7307c 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.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 466269c10999..7da4c6ddf243 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 HighAvailabilityServices getHighAvailabilityServices() { } } + protected Executor getIOExecutor() { + return ioExecutor; + } + @VisibleForTesting @Nonnull protected Collection getDispatcherResourceManagerComponents() { @@ -388,6 +392,7 @@ protected Collection createDispatc 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 a4bb3dcd5157..44c2d947794a 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 @@ protected Collection createDispatc 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 bd76b0942aec..db0b1be177d5 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.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 void testCancelingOnProcessFailure() throws Exception { 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 void testCancelingOnProcessFailure() throws Exception { dispatcherResourceManagerComponent = resourceManagerComponentFactory.create( config, + ioExecutor, rpcService, haServices, blobServerResource.getBlobServer(), From 34a1f9412ad126c3b60e851769022906da5e5a00 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 17 Oct 2019 19:22:32 +0200 Subject: [PATCH 129/746] [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 7eab0bdb76da..4b49170c0ff9 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 From b022a26b34d7ada92fa459e62f7ec84ce85cfe6f Mon Sep 17 00:00:00 2001 From: tison Date: Wed, 9 Oct 2019 16:00:45 +0800 Subject: [PATCH 130/746] [FLINK-14237][yarn] No need to rename shipped Flink jar This closes #9861 . --- flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java | 5 ++++- .../java/org/apache/flink/yarn/YarnClusterDescriptor.java | 4 ++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java index 3f0bcdcd264f..f67dcb44e776 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java @@ -482,10 +482,12 @@ static ContainerLaunchContext createTaskExecutorContext( } // register Flink Jar with remote HDFS + final String flinkJarPath; final LocalResource flinkJar; { Path remoteJarPath = new Path(remoteFlinkJarPath); FileSystem fs = remoteJarPath.getFileSystem(yarnConfig); + flinkJarPath = remoteJarPath.getName(); flinkJar = registerLocalResource(fs, remoteJarPath); } @@ -521,7 +523,8 @@ static ContainerLaunchContext createTaskExecutorContext( } Map taskManagerLocalResources = new HashMap<>(); - taskManagerLocalResources.put("flink.jar", flinkJar); + + taskManagerLocalResources.put(flinkJarPath, flinkJar); taskManagerLocalResources.put("flink-conf.yaml", flinkConf); //To support Yarn Secure Integration Test Scenario 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 6b7e3d910626..85d420fa2de8 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 @@ -813,7 +813,7 @@ private ApplicationReport startAppMaster( // Setup jar for ApplicationMaster Path remotePathJar = setupSingleLocalResource( - "flink.jar", + flinkJarPath.getName(), fs, appId, flinkJarPath, @@ -846,7 +846,7 @@ private ApplicationReport startAppMaster( ""); paths.add(remotePathJar); - classPathBuilder.append("flink.jar").append(File.pathSeparator); + classPathBuilder.append(flinkJarPath.getName()).append(File.pathSeparator); paths.add(remotePathConf); classPathBuilder.append("flink-conf.yaml").append(File.pathSeparator); From a65e230503ed345d850bf0ddba5da19abb880479 Mon Sep 17 00:00:00 2001 From: tison Date: Thu, 17 Oct 2019 17:18:08 +0800 Subject: [PATCH 131/746] [FLINK-14130][client] Remove ClusterClient.run() methods --- .../org/apache/flink/client/ClientUtils.java | 52 ++++++++++++++++ .../apache/flink/client/cli/CliFrontend.java | 8 +-- .../flink/client/program/ClusterClient.java | 60 +++---------------- .../client/program/ContextEnvironment.java | 4 +- .../flink/client/program/ClientTest.java | 12 ++-- 5 files changed, 71 insertions(+), 65 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 9fb4ce5b9f72..bd4b92a86953 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,10 +18,21 @@ package org.apache.flink.client; +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.JobSubmissionResult; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.client.program.ContextEnvironment; +import org.apache.flink.client.program.ContextEnvironmentFactory; +import org.apache.flink.client.program.PackagedProgram; +import org.apache.flink.client.program.ProgramInvocationException; +import org.apache.flink.client.program.ProgramMissingJobException; import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.File; import java.io.IOException; import java.net.URISyntaxException; @@ -35,6 +46,8 @@ public enum ClientUtils { ; + private static final Logger LOG = LoggerFactory.getLogger(ClientUtils.class); + /** * 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. @@ -80,4 +93,43 @@ public static ClassLoader buildUserCodeClassLoader(List jars, List cla } return FlinkUserCodeClassLoaders.parentFirst(urls, parent); } + + public static JobSubmissionResult executeProgram( + ClusterClient client, + PackagedProgram program, + int parallelism) throws ProgramMissingJobException, ProgramInvocationException { + final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(program.getUserCodeClassLoader()); + + LOG.info("Starting program (detached: {})", client.isDetached()); + + final List libraries = program.getAllLibraries(); + + ContextEnvironmentFactory factory = new ContextEnvironmentFactory( + client, + libraries, + program.getClasspaths(), + program.getUserCodeClassLoader(), + parallelism, + client.isDetached(), + program.getSavepointSettings()); + ContextEnvironment.setAsContext(factory); + + try { + program.invokeInteractiveModeForExecution(); + + JobExecutionResult result = client.getLastJobExecutionResult(); + if (result == null) { + throw new ProgramMissingJobException("The program didn't contain a Flink job."); + } + return result; + } finally { + ContextEnvironment.unsetContext(); + } + } + finally { + Thread.currentThread().setContextClassLoader(contextClassLoader); + } + } } 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 552eccc7f96a..85892279c3fe 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 @@ -25,6 +25,7 @@ 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.ClientUtils; import org.apache.flink.client.FlinkPipelineTranslationUtil; import org.apache.flink.client.deployment.ClusterDescriptor; import org.apache.flink.client.deployment.ClusterSpecification; @@ -729,12 +730,7 @@ private void disposeSavepoint(ClusterClient clusterClient, String savepointPa protected void executeProgram(PackagedProgram program, ClusterClient client, int parallelism) throws ProgramMissingJobException, ProgramInvocationException { logAndSysout("Starting execution of program"); - final JobSubmissionResult result = client.run(program, parallelism); - - if (null == result) { - throw new ProgramMissingJobException("No JobSubmissionResult returned, please make sure you called " + - "ExecutionEnvironment.execute()"); - } + JobSubmissionResult result = ClientUtils.executeProgram(client, program, parallelism); if (result.isJobExecutionResult()) { logAndSysout("Program execution finished"); 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 c0ea516c9301..dc9a98e1c6df 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 @@ -37,9 +37,7 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; -import java.net.URL; import java.util.Collection; -import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; @@ -55,11 +53,6 @@ public abstract class ClusterClient implements AutoCloseable { /** Configuration of the client. */ private final Configuration flinkConfig; - /** - * 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 - * which lets us access the execution result here. - */ protected JobExecutionResult lastJobExecutionResult; /** Switch for blocking/detached job submission of the client. */ @@ -93,50 +86,6 @@ public void close() throws Exception { } - // ------------------------------------------------------------------------ - // Program submission / execution - // ------------------------------------------------------------------------ - - /** - * General purpose method to run a user jar from the CliFrontend in either blocking or detached mode, depending - * on whether {@code setDetached(true)} or {@code setDetached(false)}. - * @param prog the packaged program - * @param parallelism the parallelism to execute the contained Flink job - * @return The result of the execution - * @throws ProgramMissingJobException - * @throws ProgramInvocationException - */ - public JobSubmissionResult run(PackagedProgram prog, int parallelism) - throws ProgramInvocationException, ProgramMissingJobException { - final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader(); - try { - Thread.currentThread().setContextClassLoader(prog.getUserCodeClassLoader()); - - log.info("Starting program (detached: {})", isDetached()); - - final List libraries = prog.getAllLibraries(); - - ContextEnvironmentFactory factory = new ContextEnvironmentFactory(this, libraries, - prog.getClasspaths(), prog.getUserCodeClassLoader(), parallelism, isDetached(), - prog.getSavepointSettings()); - ContextEnvironment.setAsContext(factory); - - try { - // invoke main method - prog.invokeInteractiveModeForExecution(); - if (lastJobExecutionResult == null) { - throw new ProgramMissingJobException("The program didn't contain a Flink job."); - } - return this.lastJobExecutionResult; - } finally { - ContextEnvironment.unsetContext(); - } - } - finally { - Thread.currentThread().setContextClassLoader(contextClassLoader); - } - } - /** * Requests the {@link JobStatus} of the job with the given {@link JobID}. */ @@ -282,4 +231,13 @@ public Configuration getFlinkConfiguration() { public void shutDownCluster() { throw new UnsupportedOperationException("The " + getClass().getSimpleName() + " does not support shutDownCluster."); } + + /** + * 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 + * which lets us access the execution result here. + */ + public JobExecutionResult getLastJobExecutionResult() { + return lastJobExecutionResult; + } } 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 0ceb850681af..55ffa3b2618f 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 @@ -117,11 +117,11 @@ public SavepointRestoreSettings getSavepointRestoreSettings() { // -------------------------------------------------------------------------------------------- - static void setAsContext(ContextEnvironmentFactory factory) { + public static void setAsContext(ContextEnvironmentFactory factory) { initializeContextEnvironment(factory); } - static void unsetContext() { + public static void unsetContext() { resetContextEnvironment(); } } 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 634ebf0c284f..c4ded34728ce 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 @@ -102,7 +102,7 @@ public void testDetachedMode() throws Exception{ try { PackagedProgram prg = new PackagedProgram(TestExecuteTwice.class); - clusterClient.run(prg, 1); + ClientUtils.executeProgram(clusterClient, prg, 1); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { assertEquals( @@ -112,7 +112,7 @@ public void testDetachedMode() throws Exception{ try { PackagedProgram prg = new PackagedProgram(TestEager.class); - clusterClient.run(prg, 1); + ClientUtils.executeProgram(clusterClient, prg, 1); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { assertEquals( @@ -122,7 +122,7 @@ public void testDetachedMode() throws Exception{ try { PackagedProgram prg = new PackagedProgram(TestGetRuntime.class); - clusterClient.run(prg, 1); + ClientUtils.executeProgram(clusterClient, prg, 1); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { assertEquals( @@ -132,7 +132,7 @@ public void testDetachedMode() throws Exception{ try { PackagedProgram prg = new PackagedProgram(TestGetAccumulator.class); - clusterClient.run(prg, 1); + ClientUtils.executeProgram(clusterClient, prg, 1); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { assertEquals( @@ -142,7 +142,7 @@ public void testDetachedMode() throws Exception{ try { PackagedProgram prg = new PackagedProgram(TestGetAllAccumulator.class); - clusterClient.run(prg, 1); + ClientUtils.executeProgram(clusterClient, prg, 1); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { assertEquals( @@ -190,7 +190,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { try { final ClusterClient client = new MiniClusterClient(new Configuration(), MINI_CLUSTER_RESOURCE.getMiniCluster()); client.setDetached(true); - client.run(packagedProgramMock, 1); + ClientUtils.executeProgram(client, packagedProgramMock, 1); fail("Creating the local execution environment should not be possible"); } catch (InvalidProgramException e) { From a612aac307752152bfa1fcca2f863473279b9aa1 Mon Sep 17 00:00:00 2001 From: tison Date: Thu, 17 Oct 2019 17:20:07 +0800 Subject: [PATCH 132/746] [FLINK-14130][client] Shift down Logger from ClusterClient to RestClusterClient --- .../flink/client/program/ClusterClient.java | 5 ---- .../program/rest/RestClusterClient.java | 25 +++++++++++-------- 2 files changed, 14 insertions(+), 16 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 dc9a98e1c6df..4bd86ee84cc9 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 @@ -31,9 +31,6 @@ import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.Preconditions; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -48,8 +45,6 @@ */ public abstract class ClusterClient implements AutoCloseable { - protected final Logger log = LoggerFactory.getLogger(getClass()); - /** Configuration of the client. */ private final Configuration 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 ecd403a07c67..d208c9fe275f 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 @@ -100,6 +100,9 @@ import org.apache.flink.shaded.netty4.io.netty.channel.ConnectTimeoutException; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -131,6 +134,8 @@ */ public class RestClusterClient extends ClusterClient { + private static final Logger LOG = LoggerFactory.getLogger(RestClusterClient.class); + private final RestClusterClientConfiguration restClusterClientConfiguration; /** Timeout for futures. */ @@ -203,25 +208,25 @@ public void close() { try { webMonitorRetrievalService.stop(); } catch (Exception e) { - log.error("An error occurred during stopping the WebMonitorRetrievalService", e); + LOG.error("An error occurred during stopping the WebMonitorRetrievalService", e); } try { clientHAServices.close(); } catch (Exception e) { - log.error("An error occurred during stopping the ClientHighAvailabilityServices", e); + LOG.error("An error occurred during stopping the ClientHighAvailabilityServices", e); } try { super.close(); } catch (Exception e) { - log.error("Error while closing the Cluster Client", e); + LOG.error("Error while closing the Cluster Client", e); } } @Override public JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException { - log.info("Submitting job {} (detached: {}).", jobGraph.getJobID(), isDetached()); + LOG.info("Submitting job {} (detached: {}).", jobGraph.getJobID(), isDetached()); final CompletableFuture jobSubmissionFuture = submitJob(jobGraph); @@ -229,7 +234,7 @@ public JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) try { final JobSubmissionResult jobSubmissionResult = jobSubmissionFuture.get(); - log.warn("Job was executed in detached mode, the results will be available on completion."); + LOG.warn("Job was executed in detached mode, the results will be available on completion."); this.lastJobExecutionResult = new DetachedJobExecutionResult(jobSubmissionResult.getJobID()); return lastJobExecutionResult; @@ -252,9 +257,7 @@ public JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) try { this.lastJobExecutionResult = jobResult.toJobExecutionResult(classLoader); return lastJobExecutionResult; - } catch (JobExecutionException e) { - throw new ProgramInvocationException("Job failed.", jobGraph.getJobID(), e); - } catch (IOException | ClassNotFoundException e) { + } catch (JobExecutionException | IOException | ClassNotFoundException e) { throw new ProgramInvocationException("Job failed.", jobGraph.getJobID(), e); } } @@ -364,7 +367,7 @@ public CompletableFuture submitJob(@Nonnull JobGraph jobGra try { Files.delete(jobGraphFile); } catch (IOException e) { - log.warn("Could not delete temporary file {}.", jobGraphFile, e); + LOG.warn("Could not delete temporary file {}.", jobGraphFile, e); } }); @@ -559,7 +562,7 @@ public void shutDownCluster() { } catch (InterruptedException e) { Thread.currentThread().interrupt(); } catch (ExecutionException e) { - log.error("Error while shutting down cluster", e); + LOG.error("Error while shutting down cluster", e); } } @@ -613,7 +616,7 @@ public String getWebInterfaceURL() { } catch (InterruptedException | ExecutionException e) { ExceptionUtils.checkInterrupted(e); - log.warn("Could not retrieve the web interface URL for the cluster.", e); + LOG.warn("Could not retrieve the web interface URL for the cluster.", e); return "Unknown address."; } } From 7ea62f7d173b21f2aa85b87d8e1e10821eab887d Mon Sep 17 00:00:00 2001 From: Yadong Xie Date: Fri, 18 Oct 2019 17:42:25 +0800 Subject: [PATCH 133/746] [FLINK-13818][web] Check whether web submission are enabled --- flink-runtime-web/web-dashboard/src/app/app.component.html | 2 +- flink-runtime-web/web-dashboard/src/app/app.component.ts | 2 ++ .../web-dashboard/src/app/interfaces/configuration.ts | 3 +++ 3 files changed, 6 insertions(+), 1 deletion(-) diff --git a/flink-runtime-web/web-dashboard/src/app/app.component.html b/flink-runtime-web/web-dashboard/src/app/app.component.html index 80769cdd18e8..d73ea508e77e 100644 --- a/flink-runtime-web/web-dashboard/src/app/app.component.html +++ b/flink-runtime-web/web-dashboard/src/app/app.component.html @@ -45,7 +45,7 @@

    Apache Flink Dashboard

  • Job Manager
  • -
  • +
  • Submit New Job
diff --git a/flink-runtime-web/web-dashboard/src/app/app.component.ts b/flink-runtime-web/web-dashboard/src/app/app.component.ts index dd83cfa0c841..5bf702660f47 100644 --- a/flink-runtime-web/web-dashboard/src/app/app.component.ts +++ b/flink-runtime-web/web-dashboard/src/app/app.component.ts @@ -36,6 +36,8 @@ export class AppComponent implements OnInit { fromEvent(window, 'online').pipe(map(() => true)) ).pipe(startWith(true)); + webSubmitEnabled = this.statusService.configuration.features['web-submit']; + showMessage() { if (this.statusService.listOfErrorMessage.length) { this.visible = true; diff --git a/flink-runtime-web/web-dashboard/src/app/interfaces/configuration.ts b/flink-runtime-web/web-dashboard/src/app/interfaces/configuration.ts index 0576c649dfed..a7eb3129ede6 100644 --- a/flink-runtime-web/web-dashboard/src/app/interfaces/configuration.ts +++ b/flink-runtime-web/web-dashboard/src/app/interfaces/configuration.ts @@ -22,4 +22,7 @@ export interface ConfigurationInterface { 'timezone-offset': number; 'flink-version': string; 'flink-revision': string; + features: { + 'web-submit': boolean; + }; } From 6fbdab71f782be9e3d7b1bedc51402d46bab1e62 Mon Sep 17 00:00:00 2001 From: Wei Zhong Date: Fri, 18 Oct 2019 15:10:22 +0800 Subject: [PATCH 134/746] [FLINK-14445][python] Fix python module build failed when making sdist. This closes #9932. --- flink-python/setup.py | 114 ++++++++++++++++++++++++++---------------- 1 file changed, 70 insertions(+), 44 deletions(-) diff --git a/flink-python/setup.py b/flink-python/setup.py index 11c1c1909e19..53175db26836 100644 --- a/flink-python/setup.py +++ b/flink-python/setup.py @@ -62,6 +62,10 @@ in_flink_source = os.path.isfile("../flink-java/src/main/java/org/apache/flink/api/java/" "ExecutionEnvironment.java") +# Due to changes in FLINK-14008, the licenses directory and NOTICE file may not exist in +# build-target folder. Just ignore them in this case. +exist_licenses = None +exist_notice = None try: if in_flink_source: @@ -97,6 +101,9 @@ NOTICE_FILE_PATH = os.path.join(FLINK_HOME, "NOTICE") README_FILE_PATH = os.path.join(FLINK_HOME, "README.txt") + exist_licenses = os.path.exists(LICENSES_PATH) + exist_notice = os.path.exists(NOTICE_FILE_PATH) + if not os.path.isdir(LIB_PATH): print(incorrect_invocation_message, file=sys.stderr) sys.exit(-1) @@ -106,22 +113,26 @@ os.symlink(OPT_PATH, OPT_TEMP_PATH) os.symlink(CONF_PATH, CONF_TEMP_PATH) os.symlink(EXAMPLES_PATH, EXAMPLES_TEMP_PATH) - os.symlink(LICENSES_PATH, LICENSES_TEMP_PATH) + if exist_licenses: + os.symlink(LICENSES_PATH, LICENSES_TEMP_PATH) os.symlink(PLUGINS_PATH, PLUGINS_TEMP_PATH) os.symlink(SCRIPTS_PATH, SCRIPTS_TEMP_PATH) os.symlink(LICENSE_FILE_PATH, LICENSE_FILE_TEMP_PATH) - os.symlink(NOTICE_FILE_PATH, NOTICE_FILE_TEMP_PATH) + if exist_notice: + os.symlink(NOTICE_FILE_PATH, NOTICE_FILE_TEMP_PATH) os.symlink(README_FILE_PATH, README_FILE_TEMP_PATH) else: copytree(LIB_PATH, LIB_TEMP_PATH) copytree(OPT_PATH, OPT_TEMP_PATH) copytree(CONF_PATH, CONF_TEMP_PATH) copytree(EXAMPLES_PATH, EXAMPLES_TEMP_PATH) - copytree(LICENSES_PATH, LICENSES_TEMP_PATH) + if exist_licenses: + copytree(LICENSES_PATH, LICENSES_TEMP_PATH) copytree(PLUGINS_PATH, PLUGINS_TEMP_PATH) copytree(SCRIPTS_PATH, SCRIPTS_TEMP_PATH) copy(LICENSE_FILE_PATH, LICENSE_FILE_TEMP_PATH) - copy(NOTICE_FILE_PATH, NOTICE_FILE_TEMP_PATH) + if exist_notice: + copy(NOTICE_FILE_PATH, NOTICE_FILE_TEMP_PATH) copy(README_FILE_PATH, README_FILE_TEMP_PATH) os.mkdir(LOG_TEMP_PATH) with open(os.path.join(LOG_TEMP_PATH, "empty.txt"), 'w') as f: @@ -134,51 +145,62 @@ "is complete, or do this in the flink-python directory of the flink source " "directory.") sys.exit(-1) + exist_licenses = os.path.exists(LICENSES_TEMP_PATH) + exist_notice = os.path.exists(NOTICE_FILE_TEMP_PATH) script_names = ["pyflink-shell.sh", "find-flink-home.sh"] scripts = [os.path.join(SCRIPTS_TEMP_PATH, script) for script in script_names] scripts.append("pyflink/find_flink_home.py") + PACKAGES = ['pyflink', + 'pyflink.table', + 'pyflink.util', + 'pyflink.datastream', + 'pyflink.dataset', + 'pyflink.common', + 'pyflink.fn_execution', + 'pyflink.lib', + 'pyflink.opt', + 'pyflink.conf', + 'pyflink.log', + 'pyflink.examples', + 'pyflink.plugins', + 'pyflink.bin'] + + PACKAGE_DIR = { + 'pyflink.lib': TEMP_PATH + '/lib', + 'pyflink.opt': TEMP_PATH + '/opt', + 'pyflink.conf': TEMP_PATH + '/conf', + 'pyflink.log': TEMP_PATH + '/log', + 'pyflink.examples': TEMP_PATH + '/examples', + 'pyflink.plugins': TEMP_PATH + '/plugins', + 'pyflink.bin': TEMP_PATH + '/bin'} + + PACKAGE_DATA = { + 'pyflink': ['LICENSE', 'README.txt'], + 'pyflink.lib': ['*.jar'], + 'pyflink.opt': ['*.*', '*/*'], + 'pyflink.conf': ['*'], + 'pyflink.log': ['*'], + 'pyflink.examples': ['*.py', '*/*.py'], + 'pyflink.plugins': ['*', '*/*'], + 'pyflink.bin': ['*']} + + if exist_licenses: + PACKAGES.append('pyflink.licenses') + PACKAGE_DIR['pyflink.licenses'] = TEMP_PATH + '/licenses' + PACKAGE_DATA['pyflink.licenses'] = ['*'] + + if exist_notice: + PACKAGE_DATA['pyflink'].append('NOTICE') + setup( name='apache-flink', version=VERSION, - packages=['pyflink', - 'pyflink.table', - 'pyflink.util', - 'pyflink.datastream', - 'pyflink.dataset', - 'pyflink.common', - 'pyflink.fn_execution', - 'pyflink.lib', - 'pyflink.opt', - 'pyflink.conf', - 'pyflink.log', - 'pyflink.examples', - 'pyflink.licenses', - 'pyflink.plugins', - 'pyflink.bin'], + packages=PACKAGES, include_package_data=True, - package_dir={ - 'pyflink.lib': TEMP_PATH + '/lib', - 'pyflink.opt': TEMP_PATH + '/opt', - 'pyflink.conf': TEMP_PATH + '/conf', - 'pyflink.log': TEMP_PATH + '/log', - 'pyflink.examples': TEMP_PATH + '/examples', - 'pyflink.licenses': TEMP_PATH + '/licenses', - 'pyflink.plugins': TEMP_PATH + '/plugins', - 'pyflink.bin': TEMP_PATH + '/bin' - }, - package_data={ - 'pyflink': ['LICENSE', 'NOTICE', 'README.txt'], - 'pyflink.lib': ['*.jar'], - 'pyflink.opt': ['*.*', '*/*'], - 'pyflink.conf': ['*'], - 'pyflink.log': ['*'], - 'pyflink.examples': ['*.py', '*/*.py'], - 'pyflink.licenses': ['*'], - 'pyflink.plugins': ['*', '*/*'], - 'pyflink.bin': ['*'] - }, + package_dir=PACKAGE_DIR, + package_data=PACKAGE_DATA, scripts=scripts, url='https://flink.apache.org', license='https://www.apache.org/licenses/LICENSE-2.0', @@ -205,22 +227,26 @@ os.remove(OPT_TEMP_PATH) os.remove(CONF_TEMP_PATH) os.remove(EXAMPLES_TEMP_PATH) - os.remove(LICENSES_TEMP_PATH) + if exist_licenses: + os.remove(LICENSES_TEMP_PATH) os.remove(PLUGINS_TEMP_PATH) os.remove(SCRIPTS_TEMP_PATH) os.remove(LICENSE_FILE_TEMP_PATH) - os.remove(NOTICE_FILE_TEMP_PATH) + if exist_notice: + os.remove(NOTICE_FILE_TEMP_PATH) os.remove(README_FILE_TEMP_PATH) else: rmtree(LIB_TEMP_PATH) rmtree(OPT_TEMP_PATH) rmtree(CONF_TEMP_PATH) rmtree(EXAMPLES_TEMP_PATH) - rmtree(LICENSES_TEMP_PATH) + if exist_licenses: + rmtree(LICENSES_TEMP_PATH) rmtree(PLUGINS_TEMP_PATH) rmtree(SCRIPTS_TEMP_PATH) os.remove(LICENSE_FILE_TEMP_PATH) - os.remove(NOTICE_FILE_TEMP_PATH) + if exist_notice: + os.remove(NOTICE_FILE_TEMP_PATH) os.remove(README_FILE_TEMP_PATH) rmtree(LOG_TEMP_PATH) os.rmdir(TEMP_PATH) From 3671b740552dda0d3f152f78af56df37302fec34 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Fri, 18 Oct 2019 11:03:56 +0200 Subject: [PATCH 135/746] [hotfix][test] Drop unused timeout parameter --- .../runtime/tasks/OneInputStreamTaskTest.java | 2 +- .../runtime/tasks/StreamTaskTestHarness.java | 11 ----------- 2 files changed, 1 insertion(+), 12 deletions(-) diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java index bbb4ed26a53c..8d34f89e2b3b 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java @@ -521,7 +521,7 @@ public void testSnapshottingAndRestoring() throws Exception { TestingStreamOperator.numberRestoreCalls = 0; testHarness.invoke(); - testHarness.waitForTaskRunning(deadline.timeLeft().toMillis()); + testHarness.waitForTaskRunning(); final OneInputStreamTask streamTask = testHarness.getTask(); 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 c5db3ffff77b..bc26c70e041a 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 @@ -282,17 +282,6 @@ public void waitForTaskCompletion(long timeout) throws Exception { * @throws Exception */ public void waitForTaskRunning() throws Exception { - waitForTaskRunning(Long.MAX_VALUE); - } - - /** - * Waits fro the task to be running. If this does not happen within the timeout, then a - * TimeoutException is thrown. - * - * @param timeout Timeout for the task to be running. - * @throws Exception - */ - public void waitForTaskRunning(long timeout) throws Exception { Preconditions.checkState(taskThread != null, "Task thread was not started."); StreamTask streamTask = taskThread.task; while (!streamTask.isRunning()) { From bf473a4a2e431d06297f0442df04a12c1286c5d4 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Thu, 17 Oct 2019 14:07:01 +0200 Subject: [PATCH 136/746] [FLINK-14004][runtime,test] Add test coverage for stateful SourceReaderOperator implementation --- .../tasks/SourceReaderStreamTaskTest.java | 153 +++++++++++++++--- 1 file changed, 135 insertions(+), 18 deletions(-) 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 index 61934c6dcd57..e740e80c33bb 100644 --- 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 @@ -18,71 +18,188 @@ package org.apache.flink.streaming.runtime.tasks; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.checkpoint.CheckpointMetaData; +import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.execution.CancelTaskException; +import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.runtime.state.TestTaskStateManager; 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.apache.flink.util.ExceptionUtils; import org.junit.Test; +import java.util.Iterator; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.LinkedBlockingQueue; +import java.util.stream.IntStream; + +import static org.apache.flink.streaming.util.TestHarnessUtil.assertOutputEquals; +import static org.apache.flink.util.Preconditions.checkState; +import static org.junit.Assert.assertEquals; /** * 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 { + private static final int TIMEOUT = 30_000; + /** + * Tests that the stream operator can snapshot and restore the operator state of chained + * operators. + */ @Test - public void testSourceOutputCorrectness() throws Exception { + public void testSnapshotAndRestore() throws Exception { final int numRecords = 10; + + TaskStateSnapshot taskStateSnapshot = executeAndWaitForCheckpoint( + numRecords, + 1, + IntStream.range(0, numRecords), + Optional.empty()); + + executeAndWaitForCheckpoint( + numRecords, + 2, + IntStream.range(numRecords, 2 * numRecords), + Optional.of(taskStateSnapshot)); + } + + private TaskStateSnapshot executeAndWaitForCheckpoint( + int numRecords, + long checkpointId, + IntStream expectedOutputStream, + Optional initialSnapshot) throws Exception { + + final LinkedBlockingQueue expectedOutput = new LinkedBlockingQueue<>(); + expectedOutputStream.forEach(record -> expectedOutput.add(new StreamRecord<>(record))); + CheckpointOptions checkpointOptions = CheckpointOptions.forCheckpointWithDefaultLocation(); + expectedOutput.add(new CheckpointBarrier(checkpointId, checkpointId, checkpointOptions)); + + final StreamTaskTestHarness testHarness = createTestHarness(numRecords); + if (initialSnapshot.isPresent()) { + testHarness.setTaskStateSnapshot(checkpointId, initialSnapshot.get()); + } + + TestTaskStateManager taskStateManager = testHarness.taskStateManager; + OneShotLatch waitForAcknowledgeLatch = new OneShotLatch(); + + taskStateManager.setWaitForReportLatch(waitForAcknowledgeLatch); + + testHarness.invoke(); + testHarness.waitForTaskRunning(); + + StreamTask streamTask = testHarness.getTask(); + CheckpointMetaData checkpointMetaData = new CheckpointMetaData(checkpointId, checkpointId); + + // wait with triggering the checkpoint until we emit all of the data + while (testHarness.getTask().inputProcessor.isAvailable().isDone()) { + Thread.sleep(1); + } + + streamTask.triggerCheckpointAsync(checkpointMetaData, checkpointOptions, false).get(); + + waitForAcknowledgeLatch.await(); + + assertEquals(checkpointId, taskStateManager.getReportedCheckpointId()); + + testHarness.getTask().cancel(); + try { + testHarness.waitForTaskCompletion(TIMEOUT); + } + catch (Exception ex) { + if (!ExceptionUtils.findThrowable(ex, CancelTaskException.class).isPresent()) { + throw ex; + } + } + + assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + return taskStateManager.getLastJobManagerTaskStateSnapshot(); + } + + private static StreamTaskTestHarness createTestHarness(int numRecords) { 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()); + streamConfig.setStreamOperator(new TestingIntegerSourceReaderOperator(numRecords)); + streamConfig.setOperatorID(new OperatorID(42, 44)); - 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()); + return testHarness; } /** * A simple {@link SourceReaderOperator} implementation for emitting limited int type records. */ - private static class TestingFiniteSourceReaderOperator extends SourceReaderOperator { + private static class TestingIntegerSourceReaderOperator extends SourceReaderOperator { private static final long serialVersionUID = 1L; private final int numRecords; + + private int lastRecord; private int counter; - TestingFiniteSourceReaderOperator(int numRecords) { + private ListState counterState; + + TestingIntegerSourceReaderOperator(int numRecords) { this.numRecords = numRecords; } @Override public InputStatus emitNext(DataOutput output) throws Exception { - output.emitRecord(new StreamRecord<>(++counter)); + output.emitRecord(new StreamRecord<>(counter++)); + + return hasEmittedEverything() ? InputStatus.NOTHING_AVAILABLE : InputStatus.MORE_AVAILABLE; + } + + private boolean hasEmittedEverything() { + return counter >= lastRecord; + } + + @Override + public void initializeState(StateInitializationContext context) throws Exception { + super.initializeState(context); + + counterState = context.getOperatorStateStore().getListState( + new ListStateDescriptor<>("counter", IntSerializer.INSTANCE)); + + Iterator counters = counterState.get().iterator(); + if (counters.hasNext()) { + counter = counters.next(); + } + lastRecord = counter + numRecords; + checkState(!counters.hasNext()); + } + + @Override + public void snapshotState(StateSnapshotContext context) throws Exception { + super.snapshotState(context); - return counter < numRecords ? InputStatus.MORE_AVAILABLE : InputStatus.END_OF_INPUT; + counterState.clear(); + counterState.add(counter); } @Override public CompletableFuture isAvailable() { - throw new UnsupportedOperationException(); + if (hasEmittedEverything()) { + return new CompletableFuture<>(); + } + return AVAILABLE; } @Override From db436d93797ef1422444ff4d92f966865c264cc3 Mon Sep 17 00:00:00 2001 From: Yun Tang Date: Fri, 18 Oct 2019 20:45:50 +0800 Subject: [PATCH 137/746] [FLINK-13601][tests] Harden RegionFailoverITCase Use CompletedCheckpointStore to record completed checkpoints, since it's reliable than notifications returned to the task. --- .../checkpointing/RegionFailoverITCase.java | 73 ++++++++++++++++--- 1 file changed, 62 insertions(+), 11 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RegionFailoverITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RegionFailoverITCase.java index 1e72ee37768c..181743a25d37 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RegionFailoverITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RegionFailoverITCase.java @@ -29,11 +29,19 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.RestartStrategyOptions; +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; +import org.apache.flink.runtime.checkpoint.StandaloneCheckpointIDCounter; +import org.apache.flink.runtime.checkpoint.StandaloneCompletedCheckpointStore; +import org.apache.flink.runtime.checkpoint.TestingCheckpointRecoveryFactory; import org.apache.flink.runtime.executiongraph.restart.FailingRestartStrategy; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.highavailability.HighAvailabilityServicesFactory; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServices; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.state.CheckpointListener; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.runtime.state.KeyGroupRangeAssignment; @@ -63,6 +71,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; @@ -104,6 +113,8 @@ public class RegionFailoverITCase extends TestLogger { public void setup() throws Exception { Configuration configuration = new Configuration(); configuration.setString(JobManagerOptions.EXECUTION_FAILOVER_STRATEGY, "region"); + configuration.setString(HighAvailabilityOptions.HA_MODE, TestingHAFactory.class.getName()); + // global failover times: 3, region failover times: NUM_OF_RESTARTS configuration.setInteger(FailingRestartStrategy.NUM_FAILURES_CONFIG_OPTION, 3); configuration.setString(RestartStrategyOptions.RESTART_STRATEGY, FailingRestartStrategy.class.getName()); @@ -189,7 +200,8 @@ private JobGraph createJobGraph() { } private static class StringGeneratingSourceFunction extends RichParallelSourceFunction> - implements CheckpointListener, CheckpointedFunction { + implements CheckpointedFunction { + private static final long serialVersionUID = 1L; private final long numElements; @@ -259,15 +271,6 @@ public void cancel() { isRunning = false; } - @Override - public void notifyCheckpointComplete(long checkpointId) { - if (getRuntimeContext().getIndexOfThisSubtask() == NUM_OF_REGIONS - 1) { - lastCompletedCheckpointId.set(checkpointId); - snapshotIndicesOfSubTask.put(checkpointId, lastRegionIndex); - numCompletedCheckpoints.incrementAndGet(); - } - } - @Override public void snapshotState(FunctionSnapshotContext context) throws Exception { int indexOfThisSubtask = getRuntimeContext().getIndexOfThisSubtask(); @@ -276,6 +279,7 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { listState.add(index); if (indexOfThisSubtask == NUM_OF_REGIONS - 1) { lastRegionIndex = index; + snapshotIndicesOfSubTask.put(context.getCheckpointId(), lastRegionIndex); } } unionListState.clear(); @@ -403,4 +407,51 @@ public void restoreState(List> state) throws Exception private static class TestException extends IOException{ private static final long serialVersionUID = 1L; } + + private static class TestingHaServices extends EmbeddedHaServices { + private final CheckpointRecoveryFactory checkpointRecoveryFactory; + + TestingHaServices(CheckpointRecoveryFactory checkpointRecoveryFactory, Executor executor) { + super(executor); + this.checkpointRecoveryFactory = checkpointRecoveryFactory; + } + + @Override + public CheckpointRecoveryFactory getCheckpointRecoveryFactory() { + return checkpointRecoveryFactory; + } + } + + /** + * An extension of {@link StandaloneCompletedCheckpointStore} which would record information + * of last completed checkpoint id and the number of completed checkpoints. + */ + private static class TestingCompletedCheckpointStore extends StandaloneCompletedCheckpointStore { + + TestingCompletedCheckpointStore() { + super(1); + } + + @Override + public void addCheckpoint(CompletedCheckpoint checkpoint) throws Exception { + super.addCheckpoint(checkpoint); + // we record the information when adding completed checkpoint instead of 'notifyCheckpointComplete' invoked + // on task side to avoid race condition. See FLINK-13601. + lastCompletedCheckpointId.set(checkpoint.getCheckpointID()); + numCompletedCheckpoints.incrementAndGet(); + } + } + + /** + * Testing HA factory which needs to be public in order to be instantiatable. + */ + public static class TestingHAFactory implements HighAvailabilityServicesFactory { + + @Override + public HighAvailabilityServices createHAServices(Configuration configuration, Executor executor) { + return new TestingHaServices( + new TestingCheckpointRecoveryFactory(new TestingCompletedCheckpointStore(), new StandaloneCheckpointIDCounter()), + executor); + } + } } From 035ed7de3e078204492030dd08bb07be52b70b58 Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sun, 17 Sep 2017 20:55:13 +0200 Subject: [PATCH 138/746] [FLINK-7629] [scala] Fix RecursiveProductFieldAccessor.fieldType --- .../util/typeutils/FieldAccessor.java | 2 +- .../util/typeutils/FieldAccessorTest.java | 18 ++++++++++++++++++ .../api/scala/CaseClassFieldAccessorTest.scala | 12 ++++++++++++ 3 files changed, 31 insertions(+), 1 deletion(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/typeutils/FieldAccessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/typeutils/FieldAccessor.java index 411e4adaefcf..5b18303a4b6c 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/typeutils/FieldAccessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/typeutils/FieldAccessor.java @@ -352,11 +352,11 @@ static final class RecursiveProductFieldAccessor extends FieldAccessor< checkNotNull(innerAccessor, "innerAccessor must not be null."); this.pos = pos; - this.fieldType = ((TupleTypeInfoBase) typeInfo).getTypeAt(pos); this.serializer = (TupleSerializerBase) typeInfo.createSerializer(config); this.length = this.serializer.getArity(); this.fields = new Object[this.length]; this.innerAccessor = innerAccessor; + this.fieldType = innerAccessor.getFieldType(); } @SuppressWarnings("unchecked") diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/typeutils/FieldAccessorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/typeutils/FieldAccessorTest.java index 2fb7964c2cd5..548ccf5e5590 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/typeutils/FieldAccessorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/typeutils/FieldAccessorTest.java @@ -48,6 +48,7 @@ public void testFlatTuple() { (TupleTypeInfo>) TypeExtractor.getForObject(t); FieldAccessor, String> f0 = FieldAccessorFactory.getAccessor(tpeInfo, "f0", null); + assertEquals(String.class, f0.getFieldType().getTypeClass()); assertEquals("aa", f0.get(t)); assertEquals("aa", t.f0); t = f0.set(t, "b"); @@ -55,6 +56,7 @@ public void testFlatTuple() { assertEquals("b", t.f0); FieldAccessor, Integer> f1 = FieldAccessorFactory.getAccessor(tpeInfo, "f1", null); + assertEquals(Integer.class, f1.getFieldType().getTypeClass()); assertEquals(5, (int) f1.get(t)); assertEquals(5, (int) t.f1); t = f1.set(t, 7); @@ -64,6 +66,7 @@ public void testFlatTuple() { assertEquals("b", t.f0); FieldAccessor, Integer> f1n = FieldAccessorFactory.getAccessor(tpeInfo, 1, null); + assertEquals(Integer.class, f1n.getFieldType().getTypeClass()); assertEquals(7, (int) f1n.get(t)); assertEquals(7, (int) t.f1); t = f1n.set(t, 10); @@ -74,6 +77,7 @@ public void testFlatTuple() { assertEquals("b", t.f0); FieldAccessor, Integer> f1ns = FieldAccessorFactory.getAccessor(tpeInfo, "1", null); + assertEquals(Integer.class, f1ns.getFieldType().getTypeClass()); assertEquals(10, (int) f1ns.get(t)); assertEquals(10, (int) t.f1); t = f1ns.set(t, 11); @@ -85,6 +89,7 @@ public void testFlatTuple() { // This is technically valid (the ".0" is selecting the 0th field of a basic type). FieldAccessor, String> f0f0 = FieldAccessorFactory.getAccessor(tpeInfo, "f0.0", null); + assertEquals(String.class, f0f0.getFieldType().getTypeClass()); assertEquals("b", f0f0.get(t)); assertEquals("b", t.f0); t = f0f0.set(t, "cc"); @@ -110,11 +115,13 @@ public void testTupleInTuple() { FieldAccessor>, String> f0 = FieldAccessorFactory .getAccessor(tpeInfo, "f0", null); + assertEquals(String.class, f0.getFieldType().getTypeClass()); assertEquals("aa", f0.get(t)); assertEquals("aa", t.f0); FieldAccessor>, Double> f1f2 = FieldAccessorFactory .getAccessor(tpeInfo, "f1.f2", null); + assertEquals(Double.class, f1f2.getFieldType().getTypeClass()); assertEquals(2.0, f1f2.get(t), 0); assertEquals(2.0, t.f1.f2, 0); t = f1f2.set(t, 3.0); @@ -125,6 +132,7 @@ public void testTupleInTuple() { FieldAccessor>, Tuple3> f1 = FieldAccessorFactory.getAccessor(tpeInfo, "f1", null); + assertEquals(Tuple3.class, f1.getFieldType().getTypeClass()); assertEquals(Tuple3.of(5, 9L, 3.0), f1.get(t)); assertEquals(Tuple3.of(5, 9L, 3.0), t.f1); t = f1.set(t, Tuple3.of(8, 12L, 4.0)); @@ -135,6 +143,7 @@ public void testTupleInTuple() { FieldAccessor>, Tuple3> f1n = FieldAccessorFactory.getAccessor(tpeInfo, 1, null); + assertEquals(Tuple3.class, f1n.getFieldType().getTypeClass()); assertEquals(Tuple3.of(8, 12L, 4.0), f1n.get(t)); assertEquals(Tuple3.of(8, 12L, 4.0), t.f1); t = f1n.set(t, Tuple3.of(10, 13L, 5.0)); @@ -175,6 +184,7 @@ public void testTupleInPojoInTuple() { (TupleTypeInfo>) TypeExtractor.getForObject(t); FieldAccessor, Long> f1tf1 = FieldAccessorFactory.getAccessor(tpeInfo, "f1.t.f1", null); + assertEquals(Long.class, f1tf1.getFieldType().getTypeClass()); assertEquals(9L, (long) f1tf1.get(t)); assertEquals(9L, (long) t.f1.t.f1); t = f1tf1.set(t, 12L); @@ -182,6 +192,7 @@ public void testTupleInPojoInTuple() { assertEquals(12L, (long) t.f1.t.f1); FieldAccessor, String> f1tf0 = FieldAccessorFactory.getAccessor(tpeInfo, "f1.t.f0", null); + assertEquals(String.class, f1tf0.getFieldType().getTypeClass()); assertEquals("ddd", f1tf0.get(t)); assertEquals("ddd", t.f1.t.f0); t = f1tf0.set(t, "alma"); @@ -190,6 +201,8 @@ public void testTupleInPojoInTuple() { FieldAccessor, Foo> f1 = FieldAccessorFactory.getAccessor(tpeInfo, "f1", null); FieldAccessor, Foo> f1n = FieldAccessorFactory.getAccessor(tpeInfo, 1, null); + assertEquals(Foo.class, f1.getFieldType().getTypeClass()); + assertEquals(Foo.class, f1n.getFieldType().getTypeClass()); assertEquals(Tuple2.of("alma", 12L), f1.get(t).t); assertEquals(Tuple2.of("alma", 12L), f1n.get(t).t); assertEquals(Tuple2.of("alma", 12L), t.f1.t); @@ -261,6 +274,7 @@ public void testPojoInPojo() { PojoTypeInfo tpeInfo = (PojoTypeInfo) TypeInformation.of(Outer.class); FieldAccessor fix = FieldAccessorFactory.getAccessor(tpeInfo, "i.x", null); + assertEquals(Long.class, fix.getFieldType().getTypeClass()); assertEquals(4L, (long) fix.get(o)); assertEquals(4L, o.i.x); o = fix.set(o, 22L); @@ -268,6 +282,7 @@ public void testPojoInPojo() { assertEquals(22L, o.i.x); FieldAccessor fi = FieldAccessorFactory.getAccessor(tpeInfo, "i", null); + assertEquals(Inner.class, fi.getFieldType().getTypeClass()); assertEquals(22L, fi.get(o).x); assertEquals(22L, (long) fix.get(o)); assertEquals(22L, o.i.x); @@ -328,6 +343,7 @@ public void testArrayInPojo() { PojoTypeInfo tpeInfo = (PojoTypeInfo) TypeInformation.of(ArrayInPojo.class); FieldAccessor fix = FieldAccessorFactory.getAccessor(tpeInfo, "arr.1", null); + assertEquals(Integer.class, fix.getFieldType().getTypeClass()); assertEquals(4, (int) fix.get(o)); assertEquals(4L, o.arr[1]); o = fix.set(o, 8); @@ -341,12 +357,14 @@ public void testBasicType() { TypeInformation tpeInfo = BasicTypeInfo.LONG_TYPE_INFO; FieldAccessor f = FieldAccessorFactory.getAccessor(tpeInfo, 0, null); + assertEquals(Long.class, f.getFieldType().getTypeClass()); assertEquals(7L, (long) f.get(x)); x = f.set(x, 12L); assertEquals(12L, (long) f.get(x)); assertEquals(12L, (long) x); FieldAccessor f2 = FieldAccessorFactory.getAccessor(tpeInfo, "*", null); + assertEquals(Long.class, f2.getFieldType().getTypeClass()); assertEquals(12L, (long) f2.get(x)); x = f2.set(x, 14L); assertEquals(14L, (long) f2.get(x)); diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CaseClassFieldAccessorTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CaseClassFieldAccessorTest.scala index 9a7b4952cc7d..028249b881d9 100644 --- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CaseClassFieldAccessorTest.scala +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CaseClassFieldAccessorTest.scala @@ -40,6 +40,9 @@ class CaseClassFieldAccessorTest extends TestLogger with JUnitSuiteLike { val accessor1 = FieldAccessorFactory.getAccessor[IntBoolean, Int](tpeInfo, "foo", null) val accessor2 = FieldAccessorFactory.getAccessor[IntBoolean, Boolean](tpeInfo, "bar", null) + assert(accessor1.getFieldType.getTypeClass.getSimpleName == "Integer") + assert(accessor2.getFieldType.getTypeClass.getSimpleName == "Boolean") + val x1 = IntBoolean(5, false) assert(accessor1.get(x1) == 5) assert(accessor2.get(x1) == false) @@ -61,6 +64,9 @@ class CaseClassFieldAccessorTest extends TestLogger with JUnitSuiteLike { val accessor1 = FieldAccessorFactory.getAccessor[IntBoolean, Int](tpeInfo, 0, null) val accessor2 = FieldAccessorFactory.getAccessor[IntBoolean, Boolean](tpeInfo, 1, null) + assert(accessor1.getFieldType.getTypeClass.getSimpleName == "Integer") + assert(accessor2.getFieldType.getTypeClass.getSimpleName == "Boolean") + val x1 = IntBoolean(5, false) assert(accessor1.get(x1) == 5) assert(accessor2.get(x1) == false) @@ -85,6 +91,7 @@ class CaseClassFieldAccessorTest extends TestLogger with JUnitSuiteLike { val cfg = new ExecutionConfig val fib = FieldAccessorFactory.getAccessor[Outer, Boolean](tpeInfo, "i.b", cfg) + assert(fib.getFieldType.getTypeClass.getSimpleName == "Boolean") assert(fib.get(x) == true) assert(x.i.b == true) x = fib.set(x, false) @@ -92,6 +99,7 @@ class CaseClassFieldAccessorTest extends TestLogger with JUnitSuiteLike { assert(x.i.b == false) val fi = FieldAccessorFactory.getAccessor[Outer, FieldAccessorTest.Inner](tpeInfo, "i", cfg) + assert(fi.getFieldType.getTypeClass.getSimpleName == "Inner") assert(fi.get(x).x == 3L) assert(x.i.x == 3L) x = fi.set(x, new FieldAccessorTest.Inner(4L, true)) @@ -99,6 +107,7 @@ class CaseClassFieldAccessorTest extends TestLogger with JUnitSuiteLike { assert(x.i.x == 4L) val fin = FieldAccessorFactory.getAccessor[Outer, FieldAccessorTest.Inner](tpeInfo, 1, cfg) + assert(fin.getFieldType.getTypeClass.getSimpleName == "Inner") assert(fin.get(x).x == 4L) assert(x.i.x == 4L) x = fin.set(x, new FieldAccessorTest.Inner(5L, true)) @@ -111,6 +120,7 @@ class CaseClassFieldAccessorTest extends TestLogger with JUnitSuiteLike { val tpeInfo = createTypeInformation[(Int, Long)] var x = (5, 6L) val f0 = FieldAccessorFactory.getAccessor[(Int, Long), Int](tpeInfo, 0, null) + assert(f0.getFieldType.getTypeClass.getSimpleName == "Integer") assert(f0.get(x) == 5) x = f0.set(x, 8) assert(f0.get(x) == 8) @@ -125,6 +135,7 @@ class CaseClassFieldAccessorTest extends TestLogger with JUnitSuiteLike { val fib = FieldAccessorFactory .getAccessor[OuterCaseClassWithInner, String](tpeInfo, "i.b", null) + assert(fib.getFieldType.getTypeClass.getSimpleName == "String") assert(fib.get(x) == "alma") assert(x.i.b == "alma") x = fib.set(x, "korte") @@ -133,6 +144,7 @@ class CaseClassFieldAccessorTest extends TestLogger with JUnitSuiteLike { val fi = FieldAccessorFactory .getAccessor[OuterCaseClassWithInner, InnerCaseClass](tpeInfo, "i", null) + assert(fi.getFieldType.getTypeClass == classOf[InnerCaseClass]) assert(fi.get(x) == InnerCaseClass(2, "korte")) x = fi.set(x, InnerCaseClass(3, "aaa")) assert(x.i == InnerCaseClass(3, "aaa")) From cfb9b452bc94c00d30e492674e566e1b350a4957 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 18 Oct 2019 14:46:07 +0200 Subject: [PATCH 139/746] [hotfix] Only use types defined in CaseClassFieldAccessorTest in the test --- .../api/scala/CaseClassFieldAccessorTest.scala | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CaseClassFieldAccessorTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CaseClassFieldAccessorTest.scala index 028249b881d9..ec849043c6e8 100644 --- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CaseClassFieldAccessorTest.scala +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/CaseClassFieldAccessorTest.scala @@ -24,7 +24,8 @@ import org.apache.flink.util.TestLogger import org.junit.Test import org.scalatest.junit.JUnitSuiteLike -case class Outer(a: Int, i: FieldAccessorTest.Inner, b: Boolean) +case class Outer(a: Int, i: Inner, b: Boolean) +case class Inner(x: Long, b: Boolean) case class IntBoolean(foo: Int, bar: Boolean) case class InnerCaseClass(a: Short, b: String) case class OuterCaseClassWithInner(a: Int, i: InnerCaseClass, b: Boolean) @@ -86,7 +87,7 @@ class CaseClassFieldAccessorTest extends TestLogger with JUnitSuiteLike { @Test def testFieldAccessorPojoInCaseClass(): Unit = { - var x = Outer(1, new FieldAccessorTest.Inner(3L, true), false) + var x = Outer(1, Inner(3L, true), false) val tpeInfo = createTypeInformation[Outer] val cfg = new ExecutionConfig @@ -98,19 +99,19 @@ class CaseClassFieldAccessorTest extends TestLogger with JUnitSuiteLike { assert(fib.get(x) == false) assert(x.i.b == false) - val fi = FieldAccessorFactory.getAccessor[Outer, FieldAccessorTest.Inner](tpeInfo, "i", cfg) + val fi = FieldAccessorFactory.getAccessor[Outer, Inner](tpeInfo, "i", cfg) assert(fi.getFieldType.getTypeClass.getSimpleName == "Inner") assert(fi.get(x).x == 3L) assert(x.i.x == 3L) - x = fi.set(x, new FieldAccessorTest.Inner(4L, true)) + x = fi.set(x, Inner(4L, true)) assert(fi.get(x).x == 4L) assert(x.i.x == 4L) - val fin = FieldAccessorFactory.getAccessor[Outer, FieldAccessorTest.Inner](tpeInfo, 1, cfg) + val fin = FieldAccessorFactory.getAccessor[Outer, Inner](tpeInfo, 1, cfg) assert(fin.getFieldType.getTypeClass.getSimpleName == "Inner") assert(fin.get(x).x == 4L) assert(x.i.x == 4L) - x = fin.set(x, new FieldAccessorTest.Inner(5L, true)) + x = fin.set(x, Inner(5L, true)) assert(fin.get(x).x == 5L) assert(x.i.x == 5L) } From 13603a9bad90997e25302a4b47028e9686301d8a Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Sun, 17 Sep 2017 21:03:46 +0200 Subject: [PATCH 140/746] [FLINK-7629] [scala] Fix KeyedStream.aggregate for nested field expressions --- .../streaming/api/scala/KeyedStream.scala | 28 ++++++++----- .../api/scala/StreamingOperatorsITCase.scala | 39 +++++++++++++++++++ 2 files changed, 58 insertions(+), 9 deletions(-) diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala index f85e1b538e2c..4c50bd01cff4 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala @@ -25,8 +25,8 @@ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.common.typeutils.TypeSerializer import org.apache.flink.streaming.api.datastream.{QueryableStateStream, DataStream => JavaStream, KeyedStream => KeyedJavaStream, WindowedStream => WindowedJavaStream} import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction.AggregationType -import org.apache.flink.streaming.api.functions.aggregation.{ComparableAggregator, SumAggregator} import org.apache.flink.streaming.api.functions.co.ProcessJoinFunction +import org.apache.flink.streaming.api.functions.aggregation.{AggregationFunction, ComparableAggregator, SumAggregator} import org.apache.flink.streaming.api.functions.query.{QueryableAppendingStateOperator, QueryableValueStateOperator} import org.apache.flink.streaming.api.functions.{KeyedProcessFunction, ProcessFunction} import org.apache.flink.streaming.api.operators.StreamGroupedReduce @@ -488,13 +488,19 @@ class KeyedStream[T, K](javaStream: KeyedJavaStream[T, K]) extends DataStream[T] aggregate(AggregationType.MAXBY, field) private def aggregate(aggregationType: AggregationType, field: String): DataStream[T] = { - val position = fieldNames2Indices(javaStream.getType(), Array(field))(0) - aggregate(aggregationType, position) + val aggregationFunc = aggregationType match { + case AggregationType.SUM => + new SumAggregator(field, javaStream.getType, javaStream.getExecutionConfig) + case _ => + new ComparableAggregator(field, javaStream.getType, aggregationType, true, + javaStream.getExecutionConfig) + } + + aggregate(aggregationFunc) } private def aggregate(aggregationType: AggregationType, position: Int): DataStream[T] = { - - val reducer = aggregationType match { + val aggregationFunc = aggregationType match { case AggregationType.SUM => new SumAggregator(position, javaStream.getType, javaStream.getExecutionConfig) case _ => @@ -502,10 +508,14 @@ class KeyedStream[T, K](javaStream: KeyedJavaStream[T, K]) extends DataStream[T] javaStream.getExecutionConfig) } - val invokable = new StreamGroupedReduce[T](reducer, - getType().createSerializer(getExecutionConfig)) - - new DataStream[T](javaStream.transform("aggregation", javaStream.getType(),invokable)) + aggregate(aggregationFunc) + } + + private def aggregate(aggregationFunc: AggregationFunction[T]): DataStream[T] = { + val invokable = + new StreamGroupedReduce[T](aggregationFunc, dataType.createSerializer(executionConfig)) + + new DataStream[T](javaStream.transform("aggregation", javaStream.getType(), invokable)) .asInstanceOf[DataStream[T]] } diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingOperatorsITCase.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingOperatorsITCase.scala index 334633ab613d..0219e0714af6 100644 --- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingOperatorsITCase.scala +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StreamingOperatorsITCase.scala @@ -31,8 +31,10 @@ class StreamingOperatorsITCase extends AbstractTestBase { var resultPath1: String = _ var resultPath2: String = _ + var resultPath3: String = _ var expected1: String = _ var expected2: String = _ + var expected3: String = _ val _tempFolder = new TemporaryFolder() @@ -44,14 +46,17 @@ class StreamingOperatorsITCase extends AbstractTestBase { val temp = tempFolder resultPath1 = temp.newFile.toURI.toString resultPath2 = temp.newFile.toURI.toString + resultPath3 = temp.newFile.toURI.toString expected1 = "" expected2 = "" + expected3 = "" } @After def after(): Unit = { TestBaseUtils.compareResultsByLinesInMemory(expected1, resultPath1) TestBaseUtils.compareResultsByLinesInMemory(expected2, resultPath2) + TestBaseUtils.compareResultsByLinesInMemory(expected3, resultPath3) } /** Tests the streaming fold operation. For this purpose a stream of Tuple[Int, Int] is created. @@ -122,4 +127,38 @@ class StreamingOperatorsITCase extends AbstractTestBase { env.execute() } + + @Test + def testKeyedAggregation(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + + env.setParallelism(1) + env.getConfig.setMaxParallelism(1) + + val inp = env.fromElements( + StreamingOperatorsITCase.Outer(1, StreamingOperatorsITCase.Inner(3, "alma"), true), + StreamingOperatorsITCase.Outer(1, StreamingOperatorsITCase.Inner(6, "alma"), true), + StreamingOperatorsITCase.Outer(2, StreamingOperatorsITCase.Inner(7, "alma"), true), + StreamingOperatorsITCase.Outer(2, StreamingOperatorsITCase.Inner(8, "alma"), true) + ) + + inp + .keyBy("a") + .sum("i.c") + .writeAsText(resultPath3, FileSystem.WriteMode.OVERWRITE) + + expected3 = + "Outer(1,Inner(3,alma),true)\n" + + "Outer(1,Inner(9,alma),true)\n" + + "Outer(2,Inner(15,alma),true)\n" + + "Outer(2,Inner(7,alma),true)" + + env.execute() + } } + +object StreamingOperatorsITCase { + case class Inner(c: Short, d: String) + case class Outer(a: Int, i: StreamingOperatorsITCase.Inner, b: Boolean) +} + From c6a3a0eeebffa88aed144c13a2eb2914e13c862c Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Thu, 17 Oct 2019 17:16:57 +0200 Subject: [PATCH 141/746] [FLINK-12848][table] Fix invalid row type caching This closes #9930. --- .../flink/api/java/typeutils/RowTypeInfo.java | 6 ++- .../table/calcite/FlinkTypeFactory.scala | 49 ++++++++----------- .../flink/table/api/TableSchemaTest.scala | 33 +++++++++++++ 3 files changed, 59 insertions(+), 29 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/RowTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/RowTypeInfo.java index 75c28ef8f804..aec070cb17de 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/RowTypeInfo.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/RowTypeInfo.java @@ -41,7 +41,11 @@ import static org.apache.flink.util.Preconditions.checkState; /** - * TypeInformation for {@link Row} + * {@link TypeInformation} for {@link Row}. + * + * Note: The implementations of {@link #hashCode()} and {@link #equals(Object)} do not check field + * names because those don't matter during serialization and runtime. This might change in future + * versions. See FLINK-14438 for more information. */ @PublicEvolving public class RowTypeInfo extends TupleTypeInfoBase { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala index f84a908e304f..8f43550381d3 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala @@ -53,44 +53,37 @@ class FlinkTypeFactory(typeSystem: RelDataTypeSystem) extends JavaTypeFactoryImp // NOTE: for future data types it might be necessary to // override more methods of RelDataTypeFactoryImpl - private val seenTypes = mutable.HashMap[(TypeInformation[_], Boolean), RelDataType]() - def createTypeFromTypeInfo( typeInfo: TypeInformation[_], isNullable: Boolean) : RelDataType = { - // we cannot use seenTypes for simple types, - // because time indicators and timestamps would be the same - - val relType = if (isSimple(typeInfo)) { - // simple types can be converted to SQL types and vice versa - val sqlType = typeInfoToSqlTypeName(typeInfo) - sqlType match { + val relType = if (isSimple(typeInfo)) { + // simple types can be converted to SQL types and vice versa + val sqlType = typeInfoToSqlTypeName(typeInfo) + sqlType match { - case INTERVAL_YEAR_MONTH => - createSqlIntervalType( - new SqlIntervalQualifier(TimeUnit.YEAR, TimeUnit.MONTH, SqlParserPos.ZERO)) + case INTERVAL_YEAR_MONTH => + createSqlIntervalType( + new SqlIntervalQualifier(TimeUnit.YEAR, TimeUnit.MONTH, SqlParserPos.ZERO)) - case INTERVAL_DAY_SECOND => - createSqlIntervalType( - new SqlIntervalQualifier(TimeUnit.DAY, TimeUnit.SECOND, SqlParserPos.ZERO)) + case INTERVAL_DAY_SECOND => + createSqlIntervalType( + new SqlIntervalQualifier(TimeUnit.DAY, TimeUnit.SECOND, SqlParserPos.ZERO)) - case TIMESTAMP if typeInfo.isInstanceOf[TimeIndicatorTypeInfo] => - if (typeInfo.asInstanceOf[TimeIndicatorTypeInfo].isEventTime) { - createRowtimeIndicatorType() - } else { - createProctimeIndicatorType() - } + case TIMESTAMP if typeInfo.isInstanceOf[TimeIndicatorTypeInfo] => + if (typeInfo.asInstanceOf[TimeIndicatorTypeInfo].isEventTime) { + createRowtimeIndicatorType() + } else { + createProctimeIndicatorType() + } - case _ => - createSqlType(sqlType) - } - } else { - // advanced types require specific RelDataType - // for storing the original TypeInformation - seenTypes.getOrElseUpdate((typeInfo, isNullable), createAdvancedType(typeInfo, isNullable)) + case _ => + createSqlType(sqlType) } + } else { + createAdvancedType(typeInfo, isNullable) + } createTypeWithNullability(relType, isNullable) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableSchemaTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableSchemaTest.scala index a5b23a6bce96..a4a05d768925 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableSchemaTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableSchemaTest.scala @@ -18,6 +18,7 @@ package org.apache.flink.table.api +import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala._ import org.apache.flink.table.api.scala._ import org.apache.flink.table.utils.TableTestBase @@ -69,4 +70,36 @@ class TableSchemaTest extends TableTestBase { assertTrue(!schema.getFieldType(-1).isPresent) assertTrue(!schema.getFieldType("c").isPresent) } + + @Test + def testTableSchemaWithDifferentRowTypes(): Unit = { + + def createInnerRow(innerFieldName: String): TypeInformation[_] = { + Types.ROW( + Array[String](innerFieldName), + Array[TypeInformation[_]](Types.INT())) + } + + def createRow(innerFieldName: String): TypeInformation[_] = { + Types.ROW( + Array[String]("field"), + Array[TypeInformation[_]](createInnerRow(innerFieldName)) + ) + } + + val util = streamTestUtil() + util.addTable("MyTableA", 'field)(createRow("A")) + util.addTable("MyTableB", 'field)(createRow("B")) + + val actualSchema = util.tableEnv + .sqlQuery("SELECT MyTableA.field AS a, MyTableB.field AS b FROM MyTableA, MyTableB") + .getSchema + + val expectedSchema = TableSchema.builder() + .field("a", createInnerRow("A")) + .field("b", createInnerRow("B")) + .build() + + assertEquals(expectedSchema, actualSchema) + } } From 81d1cb416a0e171c9ed9d600a3d983a4ceea68c0 Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Sun, 13 Oct 2019 20:47:50 +0800 Subject: [PATCH 142/746] [FLINK-14272][python][table-planner-blink] Support Blink planner for Python UDF(Scalar Function) Ports python User-Defined Scalar Function from flink planner to blink planner. This closes #9890. --- .../pyflink/table/table_environment.py | 40 +- flink-python/pyflink/table/tests/test_udf.py | 28 +- flink-python/pyflink/table/udf.py | 36 +- .../pyflink/testing/test_case_utils.py | 32 +- .../codegen/PythonFunctionCodeGenerator.scala | 131 +++++ .../plan/nodes/common/CommonPythonCalc.scala | 241 ++++++++++ .../nodes/physical/batch/BatchExecCalc.scala | 89 +--- .../physical/batch/BatchExecCalcBase.scala | 124 +++++ .../physical/batch/BatchExecPythonCalc.scala | 69 +++ .../physical/stream/StreamExecCalc.scala | 34 +- .../physical/stream/StreamExecCalcBase.scala | 71 +++ .../stream/StreamExecPythonCalc.scala | 72 +++ .../plan/rules/FlinkBatchRuleSets.scala | 5 +- .../plan/rules/FlinkStreamRuleSets.scala | 5 +- .../PythonScalarFunctionSplitRule.scala | 210 ++++++++ .../physical/batch/BatchExecCalcRule.scala | 13 +- .../batch/BatchExecPythonCalcRule.scala | 64 +++ .../physical/stream/StreamExecCalcRule.scala | 13 +- .../stream/StreamExecPythonCalcRule.scala | 64 +++ .../utils/python/PythonTableUtils.scala | 450 ++++++++++++++++++ .../utils/JavaUserDefinedScalarFunctions.java | 62 +++ .../plan/batch/table/PythonCalcTest.xml | 37 ++ .../logical/ExpressionReductionRulesTest.xml | 5 +- .../PythonScalarFunctionSplitRuleTest.xml | 164 +++++++ .../plan/stream/table/PythonCalcTest.xml | 37 ++ .../plan/batch/table/PythonCalcTest.scala | 41 ++ .../PythonScalarFunctionSplitRuleTest.scala | 111 +++++ .../plan/stream/table/PythonCalcTest.scala | 41 ++ .../codegen/PythonFunctionCodeGenerator.scala | 2 +- 29 files changed, 2131 insertions(+), 160 deletions(-) create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/PythonFunctionCodeGenerator.scala create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonCalc.scala create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecCalcBase.scala create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecPythonCalc.scala create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecCalcBase.scala create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecPythonCalc.scala create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PythonScalarFunctionSplitRule.scala create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecPythonCalcRule.scala create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecPythonCalcRule.scala create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/utils/python/PythonTableUtils.scala create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/table/PythonCalcTest.xml create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonScalarFunctionSplitRuleTest.xml create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/PythonCalcTest.xml create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/PythonCalcTest.scala create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonScalarFunctionSplitRuleTest.scala create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/PythonCalcTest.scala diff --git a/flink-python/pyflink/table/table_environment.py b/flink-python/pyflink/table/table_environment.py index 940aafcf62b1..e4af611d15c7 100644 --- a/flink-python/pyflink/table/table_environment.py +++ b/flink-python/pyflink/table/table_environment.py @@ -47,8 +47,9 @@ class TableEnvironment(object): __metaclass__ = ABCMeta - def __init__(self, j_tenv, serializer=PickleSerializer()): + def __init__(self, j_tenv, is_blink_planner, serializer=PickleSerializer()): self._j_tenv = j_tenv + self._is_blink_planner = is_blink_planner self._serializer = serializer def from_table_source(self, table_source): @@ -570,7 +571,8 @@ def register_function(self, name, function): :param function: The python user-defined function to register. :type function: UserDefinedFunctionWrapper """ - self._j_tenv.registerFunction(name, function._judf) + self._j_tenv.registerFunction(name, function._judf(self._is_blink_planner, + self.get_config()._j_table_config)) def execute(self, job_name): """ @@ -712,9 +714,17 @@ def _from_elements(self, elements, schema): execution_config = self._get_execution_config(temp_file.name, schema) gateway = get_gateway() j_objs = gateway.jvm.PythonBridgeUtils.readPythonObjects(temp_file.name, True) - j_input_format = gateway.jvm.PythonTableUtils.getInputFormat( + if self._is_blink_planner: + PythonTableUtils = gateway.jvm \ + .org.apache.flink.table.planner.utils.python.PythonTableUtils + PythonInputFormatTableSource = gateway.jvm \ + .org.apache.flink.table.planner.utils.python.PythonInputFormatTableSource + else: + PythonTableUtils = gateway.jvm.PythonTableUtils + PythonInputFormatTableSource = gateway.jvm.PythonInputFormatTableSource + j_input_format = PythonTableUtils.getInputFormat( j_objs, row_type_info, execution_config) - j_table_source = gateway.jvm.PythonInputFormatTableSource( + j_table_source = PythonInputFormatTableSource( j_input_format, row_type_info) return Table(self._j_tenv.fromTableSource(j_table_source)) @@ -728,9 +738,9 @@ def _get_execution_config(self, filename, schema): class StreamTableEnvironment(TableEnvironment): - def __init__(self, j_tenv): + def __init__(self, j_tenv, is_blink_planner): self._j_tenv = j_tenv - super(StreamTableEnvironment, self).__init__(j_tenv) + super(StreamTableEnvironment, self).__init__(j_tenv, is_blink_planner) def _get_execution_config(self, filename, schema): return self._j_tenv.execEnv().getConfig() @@ -832,14 +842,18 @@ def create(stream_execution_environment, table_config=None, environment_settings else: j_tenv = gateway.jvm.StreamTableEnvironment.create( stream_execution_environment._j_stream_execution_environment) - return StreamTableEnvironment(j_tenv) + j_planner_class = j_tenv.getPlanner().getClass() + j_blink_planner_class = get_java_class( + get_gateway().jvm.org.apache.flink.table.planner.delegation.PlannerBase) + is_blink_planner = j_blink_planner_class.isAssignableFrom(j_planner_class) + return StreamTableEnvironment(j_tenv, is_blink_planner) class BatchTableEnvironment(TableEnvironment): - def __init__(self, j_tenv): + def __init__(self, j_tenv, is_blink_planner): self._j_tenv = j_tenv - super(BatchTableEnvironment, self).__init__(j_tenv) + super(BatchTableEnvironment, self).__init__(j_tenv, is_blink_planner) def _get_execution_config(self, filename, schema): gateway = get_gateway() @@ -966,7 +980,7 @@ def create(execution_environment=None, table_config=None, environment_settings=N else: j_tenv = gateway.jvm.BatchTableEnvironment.create( execution_environment._j_execution_environment) - return BatchTableEnvironment(j_tenv) + return BatchTableEnvironment(j_tenv, False) elif environment_settings is not None and \ execution_environment is None and \ table_config is None: @@ -975,4 +989,8 @@ def create(execution_environment=None, table_config=None, environment_settings=N "set to batch mode.") j_tenv = gateway.jvm.TableEnvironment.create( environment_settings._j_environment_settings) - return BatchTableEnvironment(j_tenv) + j_planner_class = j_tenv.getPlanner().getClass() + j_blink_planner_class = get_java_class( + get_gateway().jvm.org.apache.flink.table.planner.delegation.PlannerBase) + is_blink_planner = j_blink_planner_class.isAssignableFrom(j_planner_class) + return BatchTableEnvironment(j_tenv, is_blink_planner) diff --git a/flink-python/pyflink/table/tests/test_udf.py b/flink-python/pyflink/table/tests/test_udf.py index 321cd784f287..d529681f2e91 100644 --- a/flink-python/pyflink/table/tests/test_udf.py +++ b/flink-python/pyflink/table/tests/test_udf.py @@ -18,10 +18,11 @@ 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 +from pyflink.testing.test_case_utils import PyFlinkStreamTableTestCase, \ + PyFlinkBlinkStreamTableTestCase, PyFlinkBlinkBatchTableTestCase -class UserDefinedFunctionTests(PyFlinkStreamTableTestCase): +class UserDefinedFunctionTests(object): def test_scalar_function(self): # test lambda function @@ -49,19 +50,19 @@ def partial_func(col, param): udf(functools.partial(partial_func, param=1), DataTypes.BIGINT(), DataTypes.BIGINT())) table_sink = source_sink_utils.TestAppendSink( - ['a', 'b', 'c', 'd', 'e'], + ['a', 'b', 'c', 'd', 'e', 'f'], [DataTypes.BIGINT(), DataTypes.BIGINT(), 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)") \ + "add_one_partial(a), 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"]) + self.assert_equals(actual, ["2,1,4,2,2,1", "4,0,12,4,4,3"]) def test_chaining_scalar_function(self): self.t_env.register_function( @@ -327,6 +328,21 @@ def test_udf_without_arguments(self): self.assert_equals(actual, ["1,2", "1,2", "1,2"]) +class PyFlinkStreamUserDefinedFunctionTests(UserDefinedFunctionTests, + PyFlinkStreamTableTestCase): + pass + + +class PyFlinkBlinkStreamUserDefinedFunctionTests(UserDefinedFunctionTests, + PyFlinkBlinkStreamTableTestCase): + pass + + +class PyFlinkBlinkBatchUserDefinedFunctionTests(UserDefinedFunctionTests, + PyFlinkBlinkBatchTableTestCase): + pass + + @udf(input_types=[DataTypes.BIGINT(), DataTypes.BIGINT()], result_type=DataTypes.BIGINT()) def add(i, j): return i + j diff --git a/flink-python/pyflink/table/udf.py b/flink-python/pyflink/table/udf.py index 97ac3910708e..ccc7b46ac7be 100644 --- a/flink-python/pyflink/table/udf.py +++ b/flink-python/pyflink/table/udf.py @@ -149,13 +149,12 @@ def __init__(self, func, input_types, result_type, deterministic=None, name=None self._deterministic = deterministic if deterministic is not None else ( func.is_deterministic() if isinstance(func, UserDefinedFunction) else True) - @property - def _judf(self): + def _judf(self, is_blink_planner, table_config): if self._judf_placeholder is None: - self._judf_placeholder = self._create_judf() + self._judf_placeholder = self._create_judf(is_blink_planner, table_config) return self._judf_placeholder - def _create_judf(self): + def _create_judf(self, is_blink_planner, table_config): func = self._func if not isinstance(self._func, UserDefinedFunction): func = DelegatingScalarFunction(self._func) @@ -167,13 +166,28 @@ def _create_judf(self): 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()) + if is_blink_planner: + PythonTableUtils = gateway.jvm\ + .org.apache.flink.table.planner.utils.python.PythonTableUtils + j_scalar_function = PythonTableUtils \ + .createPythonScalarFunction(table_config, + self._name, + bytearray(serialized_func), + j_input_types, + j_result_type, + self._deterministic, + _get_python_env()) + else: + PythonTableUtils = gateway.jvm.PythonTableUtils + j_scalar_function = PythonTableUtils \ + .createPythonScalarFunction(self._name, + bytearray(serialized_func), + j_input_types, + j_result_type, + self._deterministic, + _get_python_env()) + + return j_scalar_function # TODO: support to configure the python execution environment diff --git a/flink-python/pyflink/testing/test_case_utils.py b/flink-python/pyflink/testing/test_case_utils.py index 888824ab7cae..4d86f188e052 100644 --- a/flink-python/pyflink/testing/test_case_utils.py +++ b/flink-python/pyflink/testing/test_case_utils.py @@ -31,7 +31,7 @@ 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.table import BatchTableEnvironment, StreamTableEnvironment, EnvironmentSettings from pyflink.java_gateway import get_gateway if sys.version_info[0] >= 3: @@ -119,7 +119,7 @@ def prepare_csv_source(cls, path, data, data_types, fields): class PyFlinkStreamTableTestCase(PyFlinkTestCase): """ - Base class for stream unit tests. + Base class for stream tests. """ def setUp(self): @@ -131,7 +131,7 @@ def setUp(self): class PyFlinkBatchTableTestCase(PyFlinkTestCase): """ - Base class for batch unit tests. + Base class for batch tests. """ def setUp(self): @@ -149,6 +149,32 @@ def collect(self, table): return string_result +class PyFlinkBlinkStreamTableTestCase(PyFlinkTestCase): + """ + Base class for stream tests of blink planner. + """ + + def setUp(self): + super(PyFlinkBlinkStreamTableTestCase, self).setUp() + self.env = StreamExecutionEnvironment.get_execution_environment() + + self.t_env = StreamTableEnvironment.create( + self.env, environment_settings=EnvironmentSettings.new_instance() + .in_streaming_mode().use_blink_planner().build()) + + +class PyFlinkBlinkBatchTableTestCase(PyFlinkTestCase): + """ + Base class for batch tests of blink planner. + """ + + def setUp(self): + super(PyFlinkBlinkBatchTableTestCase, self).setUp() + self.t_env = BatchTableEnvironment.create( + environment_settings=EnvironmentSettings.new_instance() + .in_batch_mode().use_blink_planner().build()) + + class PythonAPICompletenessTestCase(object): """ Base class for Python API completeness tests, i.e., diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/PythonFunctionCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/PythonFunctionCodeGenerator.scala new file mode 100644 index 000000000000..68069bd26def --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/PythonFunctionCodeGenerator.scala @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.codegen + +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.table.functions.python.{PythonEnv, PythonFunction} +import org.apache.flink.table.functions.{FunctionLanguage, ScalarFunction, UserDefinedFunction} +import org.apache.flink.table.planner.codegen.CodeGenUtils.{newName, primitiveDefaultValue, primitiveTypeTermForType} +import org.apache.flink.table.planner.codegen.Indenter.toISC +import org.apache.flink.table.runtime.generated.GeneratedFunction +import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter + +/** + * A code generator for generating Python [[UserDefinedFunction]]s. + */ +object PythonFunctionCodeGenerator { + + private val PYTHON_SCALAR_FUNCTION_NAME = "PythonScalarFunction" + + /** + * Generates a [[ScalarFunction]] for the specified Python user-defined function. + * + * @param ctx The context of the code generator + * @param name name of the user-defined function + * @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 instance of generated ScalarFunction + */ + def generateScalarFunction( + ctx: CodeGeneratorContext, + name: String, + serializedScalarFunction: Array[Byte], + inputTypes: Array[TypeInformation[_]], + resultType: TypeInformation[_], + deterministic: Boolean, + pythonEnv: PythonEnv): ScalarFunction = { + val funcName = newName(PYTHON_SCALAR_FUNCTION_NAME) + val resultLogicType = TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType(resultType) + val resultTypeTerm = primitiveTypeTermForType(resultLogicType) + val defaultResultValue = primitiveDefaultValue(resultLogicType) + val inputParamCode = inputTypes.zipWithIndex.map { case (inputType, index) => + s"${primitiveTypeTermForType( + TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType(inputType))} in$index" + }.mkString(", ") + + val typeInfoTypeTerm = classOf[TypeInformation[_]].getCanonicalName + val pythonEnvTypeTerm = classOf[PythonEnv].getCanonicalName + + val resultTypeNameTerm = + ctx.addReusableObject(resultType, "resultType", typeInfoTypeTerm) + val serializedScalarFunctionNameTerm = + ctx.addReusableObject(serializedScalarFunction, "serializedScalarFunction", "byte[]") + val pythonEnvNameTerm = ctx.addReusableObject(pythonEnv, "pythonEnv", pythonEnvTypeTerm) + val inputTypesCode = inputTypes + .map(ctx.addReusableObject(_, "inputType", typeInfoTypeTerm)) + .mkString(", ") + + val funcCode = j""" + |public class $funcName extends ${classOf[ScalarFunction].getCanonicalName} + | implements ${classOf[PythonFunction].getCanonicalName} { + | + | private static final long serialVersionUID = 1L; + | + | ${ctx.reuseMemberCode()} + | + | public $funcName(Object[] references) throws Exception { + | ${ctx.reuseInitCode()} + | } + | + | public $resultTypeTerm eval($inputParamCode) { + | return $defaultResultValue; + | } + | + | @Override + | public $typeInfoTypeTerm[] getParameterTypes(Class[] signature) { + | return new $typeInfoTypeTerm[]{$inputTypesCode}; + | } + | + | @Override + | public $typeInfoTypeTerm getResultType(Class[] signature) { + | return $resultTypeNameTerm; + | } + | + | @Override + | public ${classOf[FunctionLanguage].getCanonicalName} getLanguage() { + | return ${classOf[FunctionLanguage].getCanonicalName}.PYTHON; + | } + | + | @Override + | public byte[] getSerializedPythonFunction() { + | return $serializedScalarFunctionNameTerm; + | } + | + | @Override + | public $pythonEnvTypeTerm getPythonEnv() { + | return $pythonEnvNameTerm; + | } + | + | @Override + | public boolean isDeterministic() { + | return $deterministic; + | } + | + | @Override + | public String toString() { + | return "$name"; + | } + |} + |""".stripMargin + new GeneratedFunction(funcName, funcCode, ctx.references.toArray) + .newInstance(Thread.currentThread().getContextClassLoader) + } +} diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonCalc.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonCalc.scala new file mode 100644 index 000000000000..4ca9d8f76419 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonCalc.scala @@ -0,0 +1,241 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.nodes.common + +import org.apache.calcite.plan.RelOptCluster +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rex._ +import org.apache.calcite.sql.`type`.SqlTypeName +import org.apache.flink.api.dag.Transformation +import org.apache.flink.streaming.api.operators.OneInputStreamOperator +import org.apache.flink.streaming.api.transformations.OneInputTransformation +import org.apache.flink.table.api.TableConfig +import org.apache.flink.table.dataformat.BaseRow +import org.apache.flink.table.functions.FunctionLanguage +import org.apache.flink.table.functions.python.{PythonFunction, PythonFunctionInfo, SimplePythonFunction} +import org.apache.flink.table.planner.calcite.FlinkTypeFactory +import org.apache.flink.table.planner.codegen.{CalcCodeGenerator, CodeGeneratorContext} +import org.apache.flink.table.planner.functions.utils.ScalarSqlFunction +import org.apache.flink.table.planner.plan.nodes.common.CommonPythonCalc.PYTHON_SCALAR_FUNCTION_OPERATOR_NAME +import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo +import org.apache.flink.table.types.logical.RowType + +import scala.collection.JavaConversions._ +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 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) + .filter(_.isInstanceOf[RexInputRef]) + .map(_.asInstanceOf[RexInputRef].getIndex) + (udfInputOffsets, pythonFunctionInfos) + } + + private def createPythonScalarFunctionInfo( + rexCall: RexCall, + inputNodes: mutable.Map[RexNode, Integer]): PythonFunctionInfo = rexCall.getOperator match { + case sfc: ScalarSqlFunction if sfc.scalarFunction.getLanguage == FunctionLanguage.PYTHON => + val inputs = new mutable.ArrayBuffer[AnyRef]() + rexCall.getOperands.foreach { + case pythonRexCall: RexCall if pythonRexCall.getOperator.asInstanceOf[ScalarSqlFunction] + .scalarFunction.getLanguage == FunctionLanguage.PYTHON => + // Continuous Python UDFs can be chained together + 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 { + 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.scalarFunction.asInstanceOf[PythonFunction].getSerializedPythonFunction, + sfc.scalarFunction.asInstanceOf[PythonFunction].getPythonEnv) + new PythonFunctionInfo(pythonFunction, inputs.toArray) + } + + private def getPythonScalarFunctionOperator( + inputRowTypeInfo: BaseRowTypeInfo, + outputRowTypeInfo: BaseRowTypeInfo, + udfInputOffsets: Array[Int], + pythonFunctionInfos: Array[PythonFunctionInfo], + forwardedFields: 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, + inputRowTypeInfo.toRowType, + outputRowTypeInfo.toRowType, + udfInputOffsets, + forwardedFields) + .asInstanceOf[OneInputStreamOperator[BaseRow, BaseRow]] + } + + private def createPythonOneInputTransformation( + inputTransform: Transformation[BaseRow], + calcProgram: RexProgram, + name: String) = { + val pythonRexCalls = calcProgram.getProjectList + .map(calcProgram.expandLocalRef) + .filter(_.isInstanceOf[RexCall]) + .map(_.asInstanceOf[RexCall]) + .toArray + + val forwardedFields: Array[Int] = calcProgram.getProjectList + .map(calcProgram.expandLocalRef) + .filter(_.isInstanceOf[RexInputRef]) + .map(_.asInstanceOf[RexInputRef].getIndex) + .toArray + + val resultProjectList = { + 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 + } + } + + val (pythonUdfInputOffsets, pythonFunctionInfos) = + extractPythonScalarFunctionInfos(pythonRexCalls) + + val inputLogicalTypes = + inputTransform.getOutputType.asInstanceOf[BaseRowTypeInfo].getLogicalTypes + val pythonOperatorInputTypeInfo = inputTransform.getOutputType.asInstanceOf[BaseRowTypeInfo] + val pythonOperatorResultTyeInfo = new BaseRowTypeInfo( + forwardedFields.map(inputLogicalTypes(_)) ++ + pythonRexCalls.map(node => FlinkTypeFactory.toLogicalType(node.getType)): _*) + + val pythonOperator = getPythonScalarFunctionOperator( + pythonOperatorInputTypeInfo, + pythonOperatorResultTyeInfo, + pythonUdfInputOffsets, + pythonFunctionInfos, + forwardedFields) + + val pythonInputTransform = new OneInputTransformation( + inputTransform, + name, + pythonOperator, + pythonOperatorResultTyeInfo, + inputTransform.getParallelism + ) + (pythonInputTransform, pythonOperatorResultTyeInfo, resultProjectList) + } + + private def createProjectionRexProgram( + inputRowType: RowType, + outputRelData: RelDataType, + projectList: mutable.Buffer[RexNode], + cluster: RelOptCluster) = { + val factory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory] + val inputRelData = factory.createFieldTypeFromLogicalType(inputRowType) + RexProgram.create(inputRelData, projectList, null, outputRelData, cluster.getRexBuilder) + } + + protected def createOneInputTransformation( + inputTransform: Transformation[BaseRow], + inputsContainSingleton: Boolean, + calcProgram: RexProgram, + name: String, + config : TableConfig, + ctx : CodeGeneratorContext, + cluster: RelOptCluster, + rowType: RelDataType, + opName: String): OneInputTransformation[BaseRow, BaseRow] = { + val (pythonInputTransform, pythonOperatorResultTyeInfo, resultProjectList) = + createPythonOneInputTransformation(inputTransform, calcProgram, name) + + if (inputsContainSingleton) { + pythonInputTransform.setParallelism(1) + pythonInputTransform.setMaxParallelism(1) + } + + val onlyFilter = resultProjectList.zipWithIndex.forall { case (rexNode, index) => + rexNode.isInstanceOf[RexInputRef] && rexNode.asInstanceOf[RexInputRef].getIndex == index + } + + if (onlyFilter) { + pythonInputTransform + } else { + // After executing python OneInputTransformation, the order of the output fields + // is Python Call after the forwarding fields, so in the case of sequential changes, + // a calc is needed to adjust the order. + val outputType = FlinkTypeFactory.toLogicalRowType(rowType) + val rexProgram = createProjectionRexProgram( + pythonOperatorResultTyeInfo.toRowType, rowType, resultProjectList, cluster) + val substituteOperator = CalcCodeGenerator.generateCalcOperator( + ctx, + cluster, + pythonInputTransform, + outputType, + config, + rexProgram, + None, + retainHeader = true, + opName + ) + + new OneInputTransformation( + pythonInputTransform, + name, + substituteOperator, + BaseRowTypeInfo.of(outputType), + pythonInputTransform.getParallelism) + } + } +} + +object CommonPythonCalc { + val PYTHON_SCALAR_FUNCTION_OPERATOR_NAME = + "org.apache.flink.table.runtime.operators.python.BaseRowPythonScalarFunctionOperator" +} diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecCalc.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecCalc.scala index aefcd2f15242..7684955ae2d8 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecCalc.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecCalc.scala @@ -19,28 +19,18 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch import org.apache.flink.api.dag.Transformation -import org.apache.flink.runtime.operators.DamBehavior import org.apache.flink.streaming.api.transformations.OneInputTransformation import org.apache.flink.table.dataformat.BaseRow import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.codegen.{CalcCodeGenerator, CodeGeneratorContext} import org.apache.flink.table.planner.delegation.BatchPlanner -import org.apache.flink.table.planner.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef, TraitUtil} -import org.apache.flink.table.planner.plan.nodes.common.CommonCalc -import org.apache.flink.table.planner.plan.nodes.exec.{BatchExecNode, ExecNode} import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo import org.apache.calcite.plan._ import org.apache.calcite.rel._ import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.Calc -import org.apache.calcite.rex.{RexCall, RexInputRef, RexProgram} -import org.apache.calcite.sql.SqlKind -import org.apache.calcite.util.mapping.{Mapping, MappingType, Mappings} - -import java.util - -import scala.collection.JavaConversions._ +import org.apache.calcite.rex.RexProgram /** * Batch physical RelNode for [[Calc]]. @@ -51,87 +41,12 @@ class BatchExecCalc( inputRel: RelNode, calcProgram: RexProgram, outputRowType: RelDataType) - extends CommonCalc(cluster, traitSet, inputRel, calcProgram) - with BatchPhysicalRel - with BatchExecNode[BaseRow] { - - override def deriveRowType(): RelDataType = outputRowType + extends BatchExecCalcBase(cluster, traitSet, inputRel, calcProgram, outputRowType) { override def copy(traitSet: RelTraitSet, child: RelNode, program: RexProgram): Calc = { new BatchExecCalc(cluster, traitSet, child, program, outputRowType) } - override def satisfyTraits(requiredTraitSet: RelTraitSet): Option[RelNode] = { - val requiredDistribution = requiredTraitSet.getTrait(FlinkRelDistributionTraitDef.INSTANCE) - // Does not push broadcast distribution trait down into Calc. - if (requiredDistribution.getType == RelDistribution.Type.BROADCAST_DISTRIBUTED) { - return None - } - val projects = calcProgram.getProjectList.map(calcProgram.expandLocalRef) - - def getProjectMapping: Mapping = { - val mapping = Mappings.create(MappingType.INVERSE_FUNCTION, - getInput.getRowType.getFieldCount, projects.size) - projects.zipWithIndex.foreach { - case (project, index) => - project match { - case inputRef: RexInputRef => mapping.set(inputRef.getIndex, index) - case call: RexCall if call.getKind == SqlKind.AS => - call.getOperands.head match { - case inputRef: RexInputRef => mapping.set(inputRef.getIndex, index) - case _ => // ignore - } - case _ => // ignore - } - } - mapping.inverse() - } - - val mapping = getProjectMapping - val appliedDistribution = requiredDistribution.apply(mapping) - // If both distribution and collation can be satisfied, satisfy both. If only distribution - // can be satisfied, only satisfy distribution. There is no possibility to only satisfy - // collation here except for there is no distribution requirement. - if ((!requiredDistribution.isTop) && (appliedDistribution eq FlinkRelDistribution.ANY)) { - return None - } - - val requiredCollation = requiredTraitSet.getTrait(RelCollationTraitDef.INSTANCE) - val appliedCollation = TraitUtil.apply(requiredCollation, mapping) - val canCollationPushedDown = !appliedCollation.getFieldCollations.isEmpty - // If required traits only contains collation requirements, but collation keys are not columns - // from input, then no need to satisfy required traits. - if ((appliedDistribution eq FlinkRelDistribution.ANY) && !canCollationPushedDown) { - return None - } - - var inputRequiredTraits = getInput.getTraitSet - var providedTraits = getTraitSet - if (!appliedDistribution.isTop) { - inputRequiredTraits = inputRequiredTraits.replace(appliedDistribution) - providedTraits = providedTraits.replace(requiredDistribution) - } - if (canCollationPushedDown) { - inputRequiredTraits = inputRequiredTraits.replace(appliedCollation) - providedTraits = providedTraits.replace(requiredCollation) - } - val newInput = RelOptRule.convert(getInput, inputRequiredTraits) - Some(copy(providedTraits, Seq(newInput))) - } - - //~ ExecNode methods ----------------------------------------------------------- - - override def getDamBehavior = DamBehavior.PIPELINED - - override def getInputNodes: util.List[ExecNode[BatchPlanner, _]] = - List(getInput.asInstanceOf[ExecNode[BatchPlanner, _]]) - - override def replaceInputNode( - ordinalInParent: Int, - newInputNode: ExecNode[BatchPlanner, _]): Unit = { - replaceInput(ordinalInParent, newInputNode.asInstanceOf[RelNode]) - } - override protected def translateToPlanInternal( planner: BatchPlanner): Transformation[BaseRow] = { val config = planner.getTableConfig diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecCalcBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecCalcBase.scala new file mode 100644 index 000000000000..47f3fa795449 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecCalcBase.scala @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.nodes.physical.batch + +import java.util + +import org.apache.flink.table.dataformat.BaseRow +import org.apache.flink.table.planner.plan.nodes.common.CommonCalc +import org.apache.flink.table.planner.plan.nodes.exec.{BatchExecNode, ExecNode} +import org.apache.calcite.plan._ +import org.apache.calcite.rel._ +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rel.core.Calc +import org.apache.calcite.rex.{RexCall, RexInputRef, RexProgram} +import org.apache.calcite.sql.SqlKind +import org.apache.calcite.util.mapping.{Mapping, MappingType, Mappings} +import org.apache.flink.runtime.operators.DamBehavior +import org.apache.flink.table.planner.delegation.BatchPlanner +import org.apache.flink.table.planner.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef, TraitUtil} + +import scala.collection.JavaConversions._ + +/** + * Base batch physical RelNode for [[Calc]]. + */ +abstract class BatchExecCalcBase( + cluster: RelOptCluster, + traitSet: RelTraitSet, + inputRel: RelNode, + calcProgram: RexProgram, + outputRowType: RelDataType) + extends CommonCalc(cluster, traitSet, inputRel, calcProgram) + with BatchPhysicalRel + with BatchExecNode[BaseRow] { + + override def deriveRowType(): RelDataType = outputRowType + + override def satisfyTraits(requiredTraitSet: RelTraitSet): Option[RelNode] = { + val requiredDistribution = requiredTraitSet.getTrait(FlinkRelDistributionTraitDef.INSTANCE) + // Does not push broadcast distribution trait down into Calc. + if (requiredDistribution.getType == RelDistribution.Type.BROADCAST_DISTRIBUTED) { + return None + } + val projects = calcProgram.getProjectList.map(calcProgram.expandLocalRef) + + def getProjectMapping: Mapping = { + val mapping = Mappings.create(MappingType.INVERSE_FUNCTION, + getInput.getRowType.getFieldCount, projects.size) + projects.zipWithIndex.foreach { + case (project, index) => + project match { + case inputRef: RexInputRef => mapping.set(inputRef.getIndex, index) + case call: RexCall if call.getKind == SqlKind.AS => + call.getOperands.head match { + case inputRef: RexInputRef => mapping.set(inputRef.getIndex, index) + case _ => // ignore + } + case _ => // ignore + } + } + mapping.inverse() + } + + val mapping = getProjectMapping + val appliedDistribution = requiredDistribution.apply(mapping) + // If both distribution and collation can be satisfied, satisfy both. If only distribution + // can be satisfied, only satisfy distribution. There is no possibility to only satisfy + // collation here except for there is no distribution requirement. + if ((!requiredDistribution.isTop) && (appliedDistribution eq FlinkRelDistribution.ANY)) { + return None + } + + val requiredCollation = requiredTraitSet.getTrait(RelCollationTraitDef.INSTANCE) + val appliedCollation = TraitUtil.apply(requiredCollation, mapping) + val canCollationPushedDown = !appliedCollation.getFieldCollations.isEmpty + // If required traits only contains collation requirements, but collation keys are not columns + // from input, then no need to satisfy required traits. + if ((appliedDistribution eq FlinkRelDistribution.ANY) && !canCollationPushedDown) { + return None + } + + var inputRequiredTraits = getInput.getTraitSet + var providedTraits = getTraitSet + if (!appliedDistribution.isTop) { + inputRequiredTraits = inputRequiredTraits.replace(appliedDistribution) + providedTraits = providedTraits.replace(requiredDistribution) + } + if (canCollationPushedDown) { + inputRequiredTraits = inputRequiredTraits.replace(appliedCollation) + providedTraits = providedTraits.replace(requiredCollation) + } + val newInput = RelOptRule.convert(getInput, inputRequiredTraits) + Some(copy(providedTraits, Seq(newInput))) + } + + //~ ExecNode methods ----------------------------------------------------------- + + override def getDamBehavior = DamBehavior.PIPELINED + + override def getInputNodes: util.List[ExecNode[BatchPlanner, _]] = + List(getInput.asInstanceOf[ExecNode[BatchPlanner, _]]) + + override def replaceInputNode( + ordinalInParent: Int, + newInputNode: ExecNode[BatchPlanner, _]): Unit = { + replaceInput(ordinalInParent, newInputNode.asInstanceOf[RelNode]) + } +} diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecPythonCalc.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecPythonCalc.scala new file mode 100644 index 000000000000..7adf0ee9cc9c --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecPythonCalc.scala @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.nodes.physical.batch + +import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} +import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rel.core.Calc +import org.apache.calcite.rex.RexProgram +import org.apache.flink.api.dag.Transformation +import org.apache.flink.table.dataformat.BaseRow +import org.apache.flink.table.planner.codegen.CodeGeneratorContext +import org.apache.flink.table.planner.delegation.BatchPlanner +import org.apache.flink.table.planner.plan.nodes.common.CommonPythonCalc + +/** + * Batch physical RelNode for Python ScalarFunctions. + */ +class BatchExecPythonCalc( + cluster: RelOptCluster, + traitSet: RelTraitSet, + inputRel: RelNode, + calcProgram: RexProgram, + outputRowType: RelDataType) + extends BatchExecCalcBase( + cluster, + traitSet, + inputRel, + calcProgram, + outputRowType) + with CommonPythonCalc { + + override def copy(traitSet: RelTraitSet, child: RelNode, program: RexProgram): Calc = { + new BatchExecPythonCalc(cluster, traitSet, child, program, outputRowType) + } + + override protected def translateToPlanInternal(planner: BatchPlanner): Transformation[BaseRow] = { + val inputTransform = getInputNodes.get(0).translateToPlan(planner) + .asInstanceOf[Transformation[BaseRow]] + val config = planner.getTableConfig + val ctx = CodeGeneratorContext(config) + createOneInputTransformation( + inputTransform, + inputsContainSingleton = false, + calcProgram, + getRelDetailedDescription, + config, + ctx, + cluster, + getRowType, + "BatchExecCalc") + } +} diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecCalc.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecCalc.scala index dc9810778523..11e534d4a1f7 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecCalc.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecCalc.scala @@ -24,9 +24,6 @@ import org.apache.flink.table.dataformat.BaseRow import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.codegen.{CalcCodeGenerator, CodeGeneratorContext} import org.apache.flink.table.planner.delegation.StreamPlanner -import org.apache.flink.table.planner.plan.nodes.common.CommonCalc -import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, StreamExecNode} -import org.apache.flink.table.planner.plan.utils.RelExplainUtil import org.apache.flink.table.runtime.operators.AbstractProcessStreamOperator import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo @@ -36,10 +33,6 @@ import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.Calc import org.apache.calcite.rex.RexProgram -import java.util - -import scala.collection.JavaConversions._ - /** * Stream physical RelNode for [[Calc]]. */ @@ -49,37 +42,12 @@ class StreamExecCalc( inputRel: RelNode, calcProgram: RexProgram, outputRowType: RelDataType) - extends CommonCalc(cluster, traitSet, inputRel, calcProgram) - with StreamPhysicalRel - with StreamExecNode[BaseRow] { - - override def producesUpdates: Boolean = false - - override def needsUpdatesAsRetraction(input: RelNode): Boolean = false - - override def consumesRetractions: Boolean = false - - override def producesRetractions: Boolean = false - - override def requireWatermark: Boolean = false - - override def deriveRowType(): RelDataType = outputRowType + extends StreamExecCalcBase(cluster, traitSet, inputRel, calcProgram, outputRowType) { override def copy(traitSet: RelTraitSet, child: RelNode, program: RexProgram): Calc = { new StreamExecCalc(cluster, traitSet, child, program, outputRowType) } - //~ ExecNode methods ----------------------------------------------------------- - - override def getInputNodes: util.List[ExecNode[StreamPlanner, _]] = - List(getInput.asInstanceOf[ExecNode[StreamPlanner, _]]) - - override def replaceInputNode( - ordinalInParent: Int, - newInputNode: ExecNode[StreamPlanner, _]): Unit = { - replaceInput(ordinalInParent, newInputNode.asInstanceOf[RelNode]) - } - override protected def translateToPlanInternal( planner: StreamPlanner): Transformation[BaseRow] = { val config = planner.getTableConfig diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecCalcBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecCalcBase.scala new file mode 100644 index 000000000000..662d32d8df3c --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecCalcBase.scala @@ -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.table.planner.plan.nodes.physical.stream + +import java.util + +import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} +import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rel.core.Calc +import org.apache.calcite.rex.RexProgram +import org.apache.flink.table.dataformat.BaseRow +import org.apache.flink.table.planner.delegation.StreamPlanner +import org.apache.flink.table.planner.plan.nodes.common.CommonCalc +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, StreamExecNode} + +import scala.collection.JavaConversions._ + +/** + * Base stream physical RelNode for [[Calc]]. + */ +abstract class StreamExecCalcBase( + cluster: RelOptCluster, + traitSet: RelTraitSet, + inputRel: RelNode, + calcProgram: RexProgram, + outputRowType: RelDataType) + extends CommonCalc(cluster, traitSet, inputRel, calcProgram) + with StreamPhysicalRel + with StreamExecNode[BaseRow] { + + override def producesUpdates: Boolean = false + + override def needsUpdatesAsRetraction(input: RelNode): Boolean = false + + override def consumesRetractions: Boolean = false + + override def producesRetractions: Boolean = false + + override def requireWatermark: Boolean = false + + override def deriveRowType(): RelDataType = outputRowType + + //~ ExecNode methods ----------------------------------------------------------- + + override def getInputNodes: util.List[ExecNode[StreamPlanner, _]] = + List(getInput.asInstanceOf[ExecNode[StreamPlanner, _]]) + + override def replaceInputNode( + ordinalInParent: Int, + newInputNode: ExecNode[StreamPlanner, _]): Unit = { + replaceInput(ordinalInParent, newInputNode.asInstanceOf[RelNode]) + } + +} diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecPythonCalc.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecPythonCalc.scala new file mode 100644 index 000000000000..0bebc5ea3690 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecPythonCalc.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.planner.plan.nodes.physical.stream + +import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} +import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rel.core.Calc +import org.apache.calcite.rex.RexProgram +import org.apache.flink.api.dag.Transformation +import org.apache.flink.table.dataformat.BaseRow +import org.apache.flink.table.planner.codegen.CodeGeneratorContext +import org.apache.flink.table.planner.delegation.StreamPlanner +import org.apache.flink.table.planner.plan.nodes.common.CommonPythonCalc +import org.apache.flink.table.runtime.operators.AbstractProcessStreamOperator + +/** + * Stream physical RelNode for Python ScalarFunctions. + */ +class StreamExecPythonCalc( + cluster: RelOptCluster, + traitSet: RelTraitSet, + inputRel: RelNode, + calcProgram: RexProgram, + outputRowType: RelDataType) + extends StreamExecCalcBase( + cluster, + traitSet, + inputRel, + calcProgram, + outputRowType) + with CommonPythonCalc { + + override def copy(traitSet: RelTraitSet, child: RelNode, program: RexProgram): Calc = { + new StreamExecPythonCalc(cluster, traitSet, child, program, outputRowType) + } + + override protected def translateToPlanInternal( + planner: StreamPlanner): Transformation[BaseRow] = { + val inputTransform = getInputNodes.get(0).translateToPlan(planner) + .asInstanceOf[Transformation[BaseRow]] + val config = planner.getTableConfig + val ctx = CodeGeneratorContext(config).setOperatorBaseClass( + classOf[AbstractProcessStreamOperator[BaseRow]]) + createOneInputTransformation( + inputTransform, + inputsContainSingleton(), + calcProgram, + getRelDetailedDescription, + config, + ctx, + cluster, + getRowType, + "StreamExecCalc") + } +} 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 10aaef49ce0c..9dd7ef45064e 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 @@ -352,7 +352,9 @@ object FlinkBatchRuleSets { // transpose calc past snapshot CalcSnapshotTransposeRule.INSTANCE, // merge calc after calc transpose - FlinkCalcMergeRule.INSTANCE + FlinkCalcMergeRule.INSTANCE, + // Rule that splits python ScalarFunctions from java/scala ScalarFunctions + PythonScalarFunctionSplitRule.INSTANCE ) /** @@ -367,6 +369,7 @@ object FlinkBatchRuleSets { BatchExecValuesRule.INSTANCE, // calc BatchExecCalcRule.INSTANCE, + BatchExecPythonCalcRule.INSTANCE, // union BatchExecUnionRule.INSTANCE, // sort 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 626800394934..e4431831a028 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 @@ -337,7 +337,9 @@ object FlinkStreamRuleSets { // transpose calc past snapshot CalcSnapshotTransposeRule.INSTANCE, // merge calc after calc transpose - FlinkCalcMergeRule.INSTANCE + FlinkCalcMergeRule.INSTANCE, + // Rule that splits python ScalarFunctions from java/scala ScalarFunctions. + PythonScalarFunctionSplitRule.INSTANCE ) /** @@ -353,6 +355,7 @@ object FlinkStreamRuleSets { StreamExecValuesRule.INSTANCE, // calc StreamExecCalcRule.INSTANCE, + StreamExecPythonCalcRule.INSTANCE, // union StreamExecUnionRule.INSTANCE, // sort diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PythonScalarFunctionSplitRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PythonScalarFunctionSplitRule.scala new file mode 100644 index 000000000000..fa219ca930eb --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PythonScalarFunctionSplitRule.scala @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.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, ScalarFunction} +import org.apache.flink.table.planner.functions.utils.ScalarSqlFunction +import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCalc +import org.apache.flink.table.planner.plan.utils.PythonUtil.containsFunctionOf +import org.apache.flink.table.planner.plan.utils.{InputRefVisitor, RexDefaultVisitor} + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +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.scalarFunction.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-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecCalcRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecCalcRule.scala index 7ff972bb8057..c6c2e93fc2eb 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecCalcRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecCalcRule.scala @@ -21,10 +21,13 @@ package org.apache.flink.table.planner.plan.rules.physical.batch import org.apache.flink.table.planner.plan.nodes.FlinkConventions import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCalc import org.apache.flink.table.planner.plan.nodes.physical.batch.BatchExecCalc - -import org.apache.calcite.plan.RelOptRule +import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall} 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.planner.plan.utils.PythonUtil.containsFunctionOf + +import scala.collection.JavaConverters._ /** * Rule that converts [[FlinkLogicalCalc]] to [[BatchExecCalc]]. @@ -36,6 +39,12 @@ class BatchExecCalcRule FlinkConventions.BATCH_PHYSICAL, "BatchExecCalcRule") { + 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 = rel.asInstanceOf[FlinkLogicalCalc] val newTrait = rel.getTraitSet.replace(FlinkConventions.BATCH_PHYSICAL) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecPythonCalcRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecPythonCalcRule.scala new file mode 100644 index 000000000000..1a16626a267e --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecPythonCalcRule.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.planner.plan.rules.physical.batch + +import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall} +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.planner.plan.nodes.FlinkConventions +import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCalc +import org.apache.flink.table.planner.plan.nodes.physical.batch.BatchExecPythonCalc +import org.apache.flink.table.planner.plan.utils.PythonUtil.containsFunctionOf + +import scala.collection.JavaConverters._ + +/** + * Rule that converts [[FlinkLogicalCalc]] to [[BatchExecPythonCalc]]. + */ +class BatchExecPythonCalcRule + extends ConverterRule( + classOf[FlinkLogicalCalc], + FlinkConventions.LOGICAL, + FlinkConventions.BATCH_PHYSICAL, + "BatchExecPythonCalcRule") { + + 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 = rel.asInstanceOf[FlinkLogicalCalc] + val newTrait = rel.getTraitSet.replace(FlinkConventions.BATCH_PHYSICAL) + val newInput = RelOptRule.convert(calc.getInput, FlinkConventions.BATCH_PHYSICAL) + + new BatchExecPythonCalc( + rel.getCluster, + newTrait, + newInput, + calc.getProgram, + rel.getRowType) + } +} + +object BatchExecPythonCalcRule { + val INSTANCE: RelOptRule = new BatchExecPythonCalcRule +} diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecCalcRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecCalcRule.scala index fafc799724d3..1626e2c95581 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecCalcRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecCalcRule.scala @@ -21,10 +21,13 @@ package org.apache.flink.table.planner.plan.rules.physical.stream import org.apache.flink.table.planner.plan.nodes.FlinkConventions import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCalc import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamExecCalc - -import org.apache.calcite.plan.{RelOptRule, RelTraitSet} +import org.apache.flink.table.planner.plan.utils.PythonUtil.containsFunctionOf +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 scala.collection.JavaConverters._ /** * Rule that converts [[FlinkLogicalCalc]] to [[StreamExecCalc]]. @@ -36,6 +39,12 @@ class StreamExecCalcRule FlinkConventions.STREAM_PHYSICAL, "StreamExecCalcRule") { + 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.STREAM_PHYSICAL) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecPythonCalcRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecPythonCalcRule.scala new file mode 100644 index 000000000000..858f4d2109d0 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecPythonCalcRule.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.planner.plan.rules.physical.stream + +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.planner.plan.nodes.FlinkConventions +import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCalc +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamExecPythonCalc +import org.apache.flink.table.planner.plan.utils.PythonUtil.containsFunctionOf + +import scala.collection.JavaConverters._ + +/** + * Rule that converts [[FlinkLogicalCalc]] to [[StreamExecPythonCalc]]. + */ +class StreamExecPythonCalcRule + extends ConverterRule( + classOf[FlinkLogicalCalc], + FlinkConventions.LOGICAL, + FlinkConventions.STREAM_PHYSICAL, + "StreamExecPythonCalcRule") { + + 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.STREAM_PHYSICAL) + val newInput = RelOptRule.convert(calc.getInput, FlinkConventions.STREAM_PHYSICAL) + + new StreamExecPythonCalc( + rel.getCluster, + traitSet, + newInput, + calc.getProgram, + rel.getRowType) + } +} + +object StreamExecPythonCalcRule { + val INSTANCE: RelOptRule = new StreamExecPythonCalcRule +} diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/utils/python/PythonTableUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/utils/python/PythonTableUtils.scala new file mode 100644 index 000000000000..3601854ab991 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/utils/python/PythonTableUtils.scala @@ -0,0 +1,450 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.utils.python + +import java.nio.charset.StandardCharsets +import java.sql.{Date, Time, Timestamp} +import java.time.{LocalDate, LocalDateTime, LocalTime} +import java.util.TimeZone +import java.util.function.BiConsumer + +import org.apache.flink.api.common.ExecutionConfig +import org.apache.flink.api.common.io.InputFormat +import org.apache.flink.api.common.typeinfo.{BasicArrayTypeInfo, BasicTypeInfo, PrimitiveArrayTypeInfo, TypeInformation} +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.{TableConfig, TableSchema, Types} +import org.apache.flink.table.functions.ScalarFunction +import org.apache.flink.table.functions.python.PythonEnv +import org.apache.flink.table.planner.codegen.{CodeGeneratorContext, PythonFunctionCodeGenerator} +import org.apache.flink.table.sources.InputFormatTableSource +import org.apache.flink.types.Row + +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( + config: TableConfig, + funcName: String, + serializedScalarFunction: Array[Byte], + inputTypes: Array[TypeInformation[_]], + resultType: TypeInformation[_], + deterministic: Boolean, + pythonEnv: PythonEnv): ScalarFunction = + PythonFunctionCodeGenerator.generateScalarFunction( + CodeGeneratorContext(config), + funcName, + serializedScalarFunction, + inputTypes, + resultType, + deterministic, + pythonEnv) + + /** + * Wrap the unpickled python data with an InputFormat. It will be passed to + * PythonInputFormatTableSource later. + * + * @param data The unpickled python data. + * @param dataType The python data type. + * @param config The execution config used to create serializer. + * @return An InputFormat containing the python data. + */ + def getInputFormat( + data: java.util.List[Array[Object]], + dataType: TypeInformation[Row], + config: ExecutionConfig): InputFormat[Row, _] = { + val converter = convertTo(dataType) + new CollectionInputFormat(data.map(converter(_).asInstanceOf[Row]), + dataType.createSerializer(config)) + } + + /** + * Creates a converter that converts `obj` to the type specified by the data type, or returns + * null if the type of obj is unexpected because Python doesn't enforce the type. + */ + private def convertTo(dataType: TypeInformation[_]): Any => Any = dataType match { + case _ if dataType == Types.BOOLEAN => (obj: Any) => nullSafeConvert(obj) { + case b: Boolean => b + } + + case _ if dataType == Types.BYTE => (obj: Any) => nullSafeConvert(obj) { + case c: Byte => c + case c: Short => c.toByte + case c: Int => c.toByte + case c: Long => c.toByte + } + + case _ if dataType == Types.SHORT => (obj: Any) => nullSafeConvert(obj) { + case c: Byte => c.toShort + case c: Short => c + case c: Int => c.toShort + case c: Long => c.toShort + } + + case _ if dataType == Types.INT => (obj: Any) => nullSafeConvert(obj) { + case c: Byte => c.toInt + case c: Short => c.toInt + case c: Int => c + case c: Long => c.toInt + } + + case _ if dataType == Types.LONG => (obj: Any) => nullSafeConvert(obj) { + case c: Byte => c.toLong + case c: Short => c.toLong + case c: Int => c.toLong + case c: Long => c + } + + case _ if dataType == Types.FLOAT => (obj: Any) => nullSafeConvert(obj) { + case c: Float => c + case c: Double => c.toFloat + } + + case _ if dataType == Types.DOUBLE => (obj: Any) => nullSafeConvert(obj) { + case c: Float => c.toDouble + case c: Double => c + } + + case _ if dataType == Types.DECIMAL => (obj: Any) => nullSafeConvert(obj) { + case c: java.math.BigDecimal => c + } + + case _ if dataType == Types.SQL_DATE => (obj: Any) => nullSafeConvert(obj) { + case c: Int => + val millisLocal = c.toLong * 86400000 + val millisUtc = millisLocal - getOffsetFromLocalMillis(millisLocal) + new Date(millisUtc) + } + + case _ if dataType == Types.SQL_TIME => (obj: Any) => nullSafeConvert(obj) { + case c: Long => new Time(c / 1000) + case c: Int => new Time(c.toLong / 1000) + } + + case _ if dataType == Types.SQL_TIMESTAMP => (obj: Any) => nullSafeConvert(obj) { + case c: Long => new Timestamp(c / 1000) + case c: Int => new Timestamp(c.toLong / 1000) + } + + case _ if dataType == Types.INTERVAL_MILLIS() => (obj: Any) => nullSafeConvert(obj) { + case c: Long => c / 1000 + case c: Int => c.toLong / 1000 + } + + case _ if dataType == Types.STRING => (obj: Any) => nullSafeConvert(obj) { + case _ => obj.toString + } + + case PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO => + (obj: Any) => + nullSafeConvert(obj) { + case c: String => c.getBytes(StandardCharsets.UTF_8) + case c if c.getClass.isArray && c.getClass.getComponentType.getName == "byte" => c + } + + case _: PrimitiveArrayTypeInfo[_] | + _: BasicArrayTypeInfo[_, _] | + _: ObjectArrayTypeInfo[_, _] => + var boxed = false + val elementType = dataType match { + case p: PrimitiveArrayTypeInfo[_] => + p.getComponentType + case b: BasicArrayTypeInfo[_, _] => + boxed = true + b.getComponentInfo + case o: ObjectArrayTypeInfo[_, _] => + boxed = true + o.getComponentInfo + } + val elementFromJava = convertTo(elementType) + + (obj: Any) => nullSafeConvert(obj) { + case c: java.util.List[_] => + createArray(elementType, + c.size(), + i => elementFromJava(c.get(i)), + boxed) + case c if c.getClass.isArray => + createArray(elementType, + c.asInstanceOf[Array[_]].length, + i => elementFromJava(c.asInstanceOf[Array[_]](i)), + boxed) + } + + case m: MapTypeInfo[_, _] => + val keyFromJava = convertTo(m.getKeyTypeInfo) + val valueFromJava = convertTo(m.getValueTypeInfo) + + (obj: Any) => nullSafeConvert(obj) { + case javaMap: java.util.Map[_, _] => + val map = new java.util.HashMap[Any, Any] + javaMap.forEach(new BiConsumer[Any, Any] { + override def accept(k: Any, v: Any): Unit = + map.put(keyFromJava(k), valueFromJava(v)) + }) + map + } + + case rowType: RowTypeInfo => + val fieldsFromJava = rowType.getFieldTypes.map(f => convertTo(f)) + + (obj: Any) => nullSafeConvert(obj) { + case c if c.getClass.isArray => + val r = c.asInstanceOf[Array[_]] + if (r.length != rowType.getFieldTypes.length) { + throw new IllegalStateException( + s"Input row doesn't have expected number of values required by the schema. " + + s"${rowType.getFieldTypes.length} fields are required while ${r.length} " + + s"values are provided." + ) + } + + val row = new Row(r.length) + var i = 0 + while (i < r.length) { + row.setField(i, fieldsFromJava(i)(r(i))) + i += 1 + } + row + } + + // UserDefinedType + case _ => (obj: Any) => obj + } + + private def nullSafeConvert(input: Any)(f: PartialFunction[Any, Any]): Any = { + if (input == null) { + null + } else { + f.applyOrElse(input, { + _: Any => null + }) + } + } + + private def createArray( + elementType: TypeInformation[_], + length: Int, + getElement: Int => Any, + boxed: Boolean = false): Array[_] = { + elementType match { + case BasicTypeInfo.BOOLEAN_TYPE_INFO => + if (!boxed) { + val array = new Array[Boolean](length) + for (i <- 0 until length) { + array(i) = getElement(i).asInstanceOf[Boolean] + } + array + } else { + val array = new Array[java.lang.Boolean](length) + for (i <- 0 until length) { + if (getElement(i) != null) { + array(i) = java.lang.Boolean.valueOf(getElement(i).asInstanceOf[Boolean]) + } else { + array(i) = null + } + } + array + } + + case BasicTypeInfo.BYTE_TYPE_INFO => + if (!boxed) { + val array = new Array[Byte](length) + for (i <- 0 until length) { + array(i) = getElement(i).asInstanceOf[Byte] + } + array + } else { + val array = new Array[java.lang.Byte](length) + for (i <- 0 until length) { + if (getElement(i) != null) { + array(i) = java.lang.Byte.valueOf(getElement(i).asInstanceOf[Byte]) + } else { + array(i) = null + } + } + array + } + + case BasicTypeInfo.SHORT_TYPE_INFO => + if (!boxed) { + val array = new Array[Short](length) + for (i <- 0 until length) { + array(i) = getElement(i).asInstanceOf[Short] + } + array + } else { + val array = new Array[java.lang.Short](length) + for (i <- 0 until length) { + if (getElement(i) != null) { + array(i) = java.lang.Short.valueOf(getElement(i).asInstanceOf[Short]) + } else { + array(i) = null + } + } + array + } + + case BasicTypeInfo.INT_TYPE_INFO => + if (!boxed) { + val array = new Array[Int](length) + for (i <- 0 until length) { + array(i) = getElement(i).asInstanceOf[Int] + } + array + } else { + val array = new Array[java.lang.Integer](length) + for (i <- 0 until length) { + if (getElement(i) != null) { + array(i) = java.lang.Integer.valueOf(getElement(i).asInstanceOf[Int]) + } else { + array(i) = null + } + } + array + } + + case BasicTypeInfo.LONG_TYPE_INFO => + if (!boxed) { + val array = new Array[Long](length) + for (i <- 0 until length) { + array(i) = getElement(i).asInstanceOf[Long] + } + array + } else { + val array = new Array[java.lang.Long](length) + for (i <- 0 until length) { + if (getElement(i) != null) { + array(i) = java.lang.Long.valueOf(getElement(i).asInstanceOf[Long]) + } else { + array(i) = null + } + } + array + } + + case BasicTypeInfo.FLOAT_TYPE_INFO => + if (!boxed) { + val array = new Array[Float](length) + for (i <- 0 until length) { + array(i) = getElement(i).asInstanceOf[Float] + } + array + } else { + val array = new Array[java.lang.Float](length) + for (i <- 0 until length) { + if (getElement(i) != null) { + array(i) = java.lang.Float.valueOf(getElement(i).asInstanceOf[Float]) + } else { + array(i) = null + } + } + array + } + + case BasicTypeInfo.DOUBLE_TYPE_INFO => + if (!boxed) { + val array = new Array[Double](length) + for (i <- 0 until length) { + array(i) = getElement(i).asInstanceOf[Double] + } + array + } else { + val array = new Array[java.lang.Double](length) + for (i <- 0 until length) { + if (getElement(i) != null) { + array(i) = java.lang.Double.valueOf(getElement(i).asInstanceOf[Double]) + } else { + array(i) = null + } + } + array + } + + case BasicTypeInfo.STRING_TYPE_INFO => + val array = new Array[java.lang.String](length) + for (i <- 0 until length) { + array(i) = getElement(i).asInstanceOf[java.lang.String] + } + array + + case _ => + val array = new Array[Object](length) + for (i <- 0 until length) { + array(i) = getElement(i).asInstanceOf[Object] + } + array + } + } + + def getOffsetFromLocalMillis(millisLocal: Long): Int = { + val localZone = TimeZone.getDefault + var result = localZone.getRawOffset + // the actual offset should be calculated based on milliseconds in UTC + val offset = localZone.getOffset(millisLocal - result) + if (offset != result) { + // DayLight Saving Time + result = localZone.getOffset(millisLocal - offset) + if (result != offset) { + // fallback to do the reverse lookup using java.time.LocalDateTime + // this should only happen near the start or end of DST + val localDate = LocalDate.ofEpochDay(millisLocal / 86400000) + val localTime = LocalTime.ofNanoOfDay( + Math.floorMod(millisLocal, 86400000) * 1000 * 1000) + val localDateTime = LocalDateTime.of(localDate, localTime) + val millisEpoch = localDateTime.atZone(localZone.toZoneId).toInstant.toEpochMilli + result = (millisLocal - millisEpoch).toInt + } + } + result + } +} + +/** + * An InputFormatTableSource created by python 'from_element' method. + * + * @param inputFormat The input format which contains the python data collection, + * usually created by PythonTableUtils#getInputFormat method + * @param rowTypeInfo The row type info of the python data. + * It is generated by the python 'from_element' method. + */ +class PythonInputFormatTableSource[Row]( + inputFormat: InputFormat[Row, _ <: InputSplit], + rowTypeInfo: RowTypeInfo +) extends InputFormatTableSource[Row] { + + override def getInputFormat: InputFormat[Row, _ <: InputSplit] = inputFormat + + override def getTableSchema: TableSchema = TableSchema.fromTypeInfo(rowTypeInfo) + + override def getReturnType: TypeInformation[Row] = rowTypeInfo.asInstanceOf[TypeInformation[Row]] +} diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedScalarFunctions.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedScalarFunctions.java index f3284d014241..afb992b3e825 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedScalarFunctions.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedScalarFunctions.java @@ -18,9 +18,12 @@ package org.apache.flink.table.planner.runtime.utils; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.table.functions.AggregateFunction; import org.apache.flink.table.functions.FunctionContext; +import org.apache.flink.table.functions.FunctionLanguage; import org.apache.flink.table.functions.ScalarFunction; import java.util.Arrays; @@ -154,4 +157,63 @@ public boolean isDeterministic() { } } + /** + * Test for Python Scalar Function. + */ + public static class PythonScalarFunction extends ScalarFunction { + private final String name; + + public PythonScalarFunction(String name) { + this.name = name; + } + + public int eval(int i, int j) { + return i + j; + } + + @Override + public TypeInformation getResultType(Class[] signature) { + return BasicTypeInfo.INT_TYPE_INFO; + } + + @Override + public FunctionLanguage getLanguage() { + return FunctionLanguage.PYTHON; + } + + @Override + public String toString() { + return name; + } + } + + /** + * Test for Python Scalar Function. + */ + public static class BooleanPythonScalarFunction extends ScalarFunction { + private final String name; + + public BooleanPythonScalarFunction(String name) { + this.name = name; + } + + public boolean eval(int i, int j) { + return i + j > 1; + } + + @Override + public TypeInformation getResultType(Class[] signature) { + return BasicTypeInfo.BOOLEAN_TYPE_INFO; + } + + @Override + public FunctionLanguage getLanguage() { + return FunctionLanguage.PYTHON; + } + + @Override + public String toString() { + return name; + } + } } diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/table/PythonCalcTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/table/PythonCalcTest.xml new file mode 100644 index 000000000000..e48873096811 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/table/PythonCalcTest.xml @@ -0,0 +1,37 @@ + + + + + + + + + + + + + + + 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 0433fbee4819..ae612e2ece4f 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 @@ -62,8 +62,9 @@ LogicalProject(EXPR$0=[PyUdf()], EXPR$1=[MyUdf(1)]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonScalarFunctionSplitRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonScalarFunctionSplitRuleTest.xml new file mode 100644 index 000000000000..66a06cd9f546 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonScalarFunctionSplitRuleTest.xml @@ -0,0 +1,164 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + 0]]> + + + (pyFunc2($0, $2), 0)]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) +]]> + + + (f1, 0)]) ++- FlinkLogicalCalc(select=[c, pyFunc1(a, b) AS f0, pyFunc2(a, c) AS f1]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/PythonCalcTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/PythonCalcTest.xml new file mode 100644 index 000000000000..e48873096811 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/PythonCalcTest.xml @@ -0,0 +1,37 @@ + + + + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/PythonCalcTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/PythonCalcTest.scala new file mode 100644 index 000000000000..4c01bfd96769 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/PythonCalcTest.scala @@ -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.table.planner.plan.batch.table + +import org.apache.flink.api.scala._ +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.PythonScalarFunction +import org.apache.flink.table.planner.utils.TableTestBase +import org.junit.{Before, Test} + +class PythonCalcTest extends TableTestBase { + private val util = batchTestUtil() + + @Before + def setup(): Unit = { + util.addTableSource[(Int, Int, Int)]("MyTable", 'a, 'b, 'c) + util.addFunction("pyFunc1", new PythonScalarFunction("pyFunc1")) + } + + @Test + def testPythonFunctionMixedWithJavaFunction(): Unit = { + val sqlQuery = "SELECT pyFunc1(a, b), c + 1 FROM MyTable" + util.verifyPlan(sqlQuery) + } +} diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonScalarFunctionSplitRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonScalarFunctionSplitRuleTest.scala new file mode 100644 index 000000000000..d99d39ff48b7 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonScalarFunctionSplitRuleTest.scala @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.hep.HepMatchOrder +import org.apache.flink.api.scala._ +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.planner.plan.nodes.FlinkConventions +import org.apache.flink.table.planner.plan.optimize.program._ +import org.apache.flink.table.planner.plan.rules.{FlinkBatchRuleSets, FlinkStreamRuleSets} +import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.{BooleanPythonScalarFunction, PythonScalarFunction} +import org.apache.flink.table.planner.utils.TableTestBase +import org.junit.{Before, Test} + +/** + * Test for [[PythonScalarFunctionSplitRule]]. + */ +class PythonScalarFunctionSplitRuleTest extends TableTestBase { + + private val util = batchTestUtil() + + @Before + def setup(): Unit = { + val programs = new FlinkChainedProgram[BatchOptimizeContext]() + programs.addLast( + "logical", + FlinkVolcanoProgramBuilder.newBuilder + .add(FlinkBatchRuleSets.LOGICAL_OPT_RULES) + .setRequiredOutputTraits(Array(FlinkConventions.LOGICAL)) + .build()) + programs.addLast( + "logical_rewrite", + FlinkHepRuleSetProgramBuilder.newBuilder + .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_SEQUENCE) + .setHepMatchOrder(HepMatchOrder.BOTTOM_UP) + .add(FlinkStreamRuleSets.LOGICAL_REWRITE) + .build()) + util.replaceBatchProgram(programs) + + util.addTableSource[(Int, Int, Int)]("MyTable", 'a, 'b, 'c) + util.addFunction("pyFunc1", new PythonScalarFunction("pyFunc1")) + util.addFunction("pyFunc2", new PythonScalarFunction("pyFunc2")) + util.addFunction("pyFunc3", new PythonScalarFunction("pyFunc3")) + util.addFunction("pyFunc4", new BooleanPythonScalarFunction("pyFunc4")) + } + + @Test + def testPythonFunctionAsInputOfJavaFunction(): Unit = { + val sqlQuery = "SELECT pyFunc1(a, b) + 1 FROM MyTable" + util.verifyPlan(sqlQuery) + } + + @Test + def testPythonFunctionMixedWithJavaFunction(): Unit = { + val sqlQuery = "SELECT pyFunc1(a, b), c + 1 FROM MyTable" + util.verifyPlan(sqlQuery) + } + + @Test + def testPythonFunctionMixedWithJavaFunctionInWhereClause(): Unit = { + val sqlQuery = "SELECT pyFunc1(a, b), c + 1 FROM MyTable WHERE pyFunc2(a, c) > 0" + util.verifyPlan(sqlQuery) + } + + @Test + def testPythonFunctionInWhereClause(): Unit = { + val sqlQuery = "SELECT pyFunc1(a, b) FROM MyTable WHERE pyFunc4(a, c)" + util.verifyPlan(sqlQuery) + } + + @Test + def testChainingPythonFunction(): Unit = { + val sqlQuery = "SELECT pyFunc3(pyFunc2(a + pyFunc1(a, c), b), c) FROM MyTable" + util.verifyPlan(sqlQuery) + } + + @Test + def testOnlyOnePythonFunction(): Unit = { + val sqlQuery = "SELECT pyFunc1(a, b) FROM MyTable" + util.verifyPlan(sqlQuery) + } + + @Test + def testOnlyOnePythonFunctionInWhereClause(): Unit = { + val sqlQuery = "SELECT a, b FROM MyTable WHERE pyFunc4(a, c)" + util.verifyPlan(sqlQuery) + } + + @Test + def testFieldNameUniquify(): Unit = { + util.addTableSource[(Int, Int, Int)]("MyTable2", 'f0, 'f1, 'f2) + val sqlQuery = "SELECT pyFunc1(f1, f2), f0 + 1 FROM MyTable2" + util.verifyPlan(sqlQuery) + } +} diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/PythonCalcTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/PythonCalcTest.scala new file mode 100644 index 000000000000..d25b20fb4afd --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/PythonCalcTest.scala @@ -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.table.planner.plan.stream.table + +import org.apache.flink.api.scala._ +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.PythonScalarFunction +import org.apache.flink.table.planner.utils.TableTestBase +import org.junit.{Before, Test} + +class PythonCalcTest extends TableTestBase { + private val util = streamTestUtil() + + @Before + def setup(): Unit = { + util.addTableSource[(Int, Int, Int)]("MyTable", 'a, 'b, 'c) + util.addFunction("pyFunc1", new PythonScalarFunction("pyFunc1")) + } + + @Test + def testPythonFunctionMixedWithJavaFunction(): Unit = { + val sqlQuery = "SELECT pyFunc1(a, b), c + 1 FROM MyTable" + util.verifyPlan(sqlQuery) + } +} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/PythonFunctionCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/PythonFunctionCodeGenerator.scala index e97e7b79eeb7..332d5730b368 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/PythonFunctionCodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/PythonFunctionCodeGenerator.scala @@ -34,7 +34,7 @@ object PythonFunctionCodeGenerator extends Compiler[UserDefinedFunction] { /** * Generates a [[ScalarFunction]] for the specified Python user-defined function. * - * @param name class name of the user-defined function. Must be a valid Java class identifier + * @param name name of the user-defined function * @param serializedScalarFunction serialized Python scalar function * @param inputTypes input data types * @param resultType expected result type From fc16bcbdfcbf01ed7a7a75eea0247286332dc623 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Tue, 15 Oct 2019 11:19:41 +0800 Subject: [PATCH 143/746] [FLINK-14366][tests] Enable TaskFailureITCase to pass with NG scheduler --- .../flink/test/example/failing/TaskFailureITCase.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/failing/TaskFailureITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/failing/TaskFailureITCase.java index b830508e00f7..525bd6161d88 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/example/failing/TaskFailureITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/example/failing/TaskFailureITCase.java @@ -31,6 +31,9 @@ import java.util.List; +import static org.apache.flink.util.ExceptionUtils.findThrowableWithMessage; +import static org.junit.Assert.assertTrue; + /** * Tests that both jobs, the failing and the working one, are handled correctly. The first (failing) job must be * canceled and the client must report the failure. The second (working) job must finish successfully and compute the @@ -51,13 +54,13 @@ protected void testProgram() throws Exception { Assert.fail(); } // for collection execution, no restarts. So, exception should be appended with 0. - Assert.assertEquals(EXCEPTION_STRING + ":0", e.getMessage()); + assertTrue(findThrowableWithMessage(e, EXCEPTION_STRING + ":0").isPresent()); } catch (JobExecutionException e) { //expected for cluster execution if (isCollectionExecution()) { Assert.fail(); } // for cluster execution, one restart. So, exception should be appended with 1. - Assert.assertEquals(EXCEPTION_STRING + ":1", e.getCause().getMessage()); + assertTrue(findThrowableWithMessage(e, EXCEPTION_STRING + ":1").isPresent()); } //test correct version executeTask(new TestMapper(), 0); @@ -65,6 +68,7 @@ protected void testProgram() throws Exception { private void executeTask(MapFunction mapper, int retries) throws Exception { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); env.setRestartStrategy(RestartStrategies.fixedDelayRestart(retries, 0)); List result = env.generateSequence(1, 9) .map(mapper) From d04f1b0ac5dc683b6279b581f75cee09c2a32642 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Tue, 15 Oct 2019 11:34:53 +0800 Subject: [PATCH 144/746] [FLINK-14366][tests] Enable AccumulatorErrorITCase to pass with NG scheduler --- .../flink/test/accumulators/AccumulatorErrorITCase.java | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) 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 d656184e785b..9f78ce2ed4d3 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 @@ -30,12 +30,12 @@ import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; import org.junit.ClassRule; import org.junit.Test; +import static org.apache.flink.util.ExceptionUtils.findThrowable; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -88,10 +88,7 @@ public void testInvalidTypeAccumulator() throws Exception { env.execute(); fail("Should have failed."); } catch (JobExecutionException e) { - assertTrue("Root cause should be:", - e.getCause() instanceof Exception); - assertTrue("Root cause should be:", - e.getCause().getCause() instanceof UnsupportedOperationException); + assertTrue(findThrowable(e, UnsupportedOperationException.class).isPresent()); } } @@ -202,7 +199,7 @@ private static void assertAccumulatorsShouldFail(JobExecutionResult result) { fail("Should have failed"); } catch (Exception ex) { - assertTrue(ExceptionUtils.findThrowable(ex, CustomException.class).isPresent()); + assertTrue(findThrowable(ex, CustomException.class).isPresent()); } } } From 54952b28ab76877347fa0566a1af780156f3241b Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Tue, 15 Oct 2019 11:25:40 +0800 Subject: [PATCH 145/746] [FLINK-14366][tests] Enable CustomSerializationITCase to pass with NG scheduler --- .../test/misc/CustomSerializationITCase.java | 20 ++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) 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 0177e4a9b183..6c85218b39c0 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 @@ -36,7 +36,9 @@ import org.junit.Test; import java.io.IOException; +import java.util.Optional; +import static org.apache.flink.util.ExceptionUtils.findThrowable; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -83,9 +85,9 @@ public ConsumesTooMuch map(Long value) throws Exception { env.execute(); } catch (JobExecutionException e) { - Throwable rootCause = e.getCause(); - assertTrue(rootCause instanceof IOException); - assertTrue(rootCause.getMessage().contains("broken serialization")); + Optional rootCause = findThrowable(e, IOException.class); + assertTrue(rootCause.isPresent()); + assertTrue(rootCause.get().getMessage().contains("broken serialization")); } catch (Exception e) { e.printStackTrace(); @@ -113,9 +115,9 @@ public ConsumesTooMuchSpanning map(Long value) throws Exception { env.execute(); } catch (JobExecutionException e) { - Throwable rootCause = e.getCause(); - assertTrue(rootCause instanceof IOException); - assertTrue(rootCause.getMessage().contains("broken serialization")); + Optional rootCause = findThrowable(e, IOException.class); + assertTrue(rootCause.isPresent()); + assertTrue(rootCause.get().getMessage().contains("broken serialization")); } catch (Exception e) { e.printStackTrace(); @@ -143,9 +145,9 @@ public ConsumesTooLittle map(Long value) throws Exception { env.execute(); } catch (JobExecutionException e) { - Throwable rootCause = e.getCause(); - assertTrue(rootCause instanceof IOException); - assertTrue(rootCause.getMessage().contains("broken serialization")); + Optional rootCause = findThrowable(e, IOException.class); + assertTrue(rootCause.isPresent()); + assertTrue(rootCause.get().getMessage().contains("broken serialization")); } catch (Exception e) { e.printStackTrace(); From 5b590ddec671feab17b293d56d2e6c14d532886a Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Tue, 15 Oct 2019 11:27:19 +0800 Subject: [PATCH 146/746] [FLINK-14366][tests] Enable MiscellaneousIssuesITCase to pass with NG scheduler --- .../apache/flink/test/misc/MiscellaneousIssuesITCase.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java index 1681304f8a46..6925a4d6f617 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java @@ -36,8 +36,8 @@ import org.junit.ClassRule; import org.junit.Test; +import static org.apache.flink.util.ExceptionUtils.findThrowable; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -81,8 +81,7 @@ public String map(String value) throws Exception { fail("this should fail due to null values."); } catch (JobExecutionException e) { - assertNotNull(e.getCause()); - assertTrue(e.getCause() instanceof NullPointerException); + assertTrue(findThrowable(e, NullPointerException.class).isPresent()); } } catch (Exception e) { From 76c28d80526ee76986438f4f20603f2fd7b9842c Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Tue, 15 Oct 2019 11:28:28 +0800 Subject: [PATCH 147/746] [FLINK-14366][tests] Enable SuccessAfterNetworkBuffersFailureITCase to pass with NG scheduler --- .../test/misc/SuccessAfterNetworkBuffersFailureITCase.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 e9523f90eca0..90ddf0c44a5b 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 @@ -40,6 +40,7 @@ import org.junit.ClassRule; import org.junit.Test; +import static org.apache.flink.util.ExceptionUtils.findThrowableWithMessage; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -78,7 +79,7 @@ public void testSuccessfulProgramAfterFailure() throws Exception { fail("This program execution should have failed."); } catch (JobExecutionException e) { - assertTrue(e.getCause().getMessage().contains("Insufficient number of network buffers")); + assertTrue(findThrowableWithMessage(e, "Insufficient number of network buffers").isPresent()); } runConnectedComponents(env); From e242cbd4f156f7c2750dc9887380f00423836468 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Fri, 18 Oct 2019 16:37:52 +0800 Subject: [PATCH 148/746] [FLINK-14366][tests] Enable TextOutputFormatITCase and CsvOutputFormatITCase to pass with NG scheduler --- .../flink/streaming/scala/api/CsvOutputFormatITCase.java | 7 ++++--- .../flink/streaming/scala/api/TextOutputFormatITCase.java | 3 ++- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/CsvOutputFormatITCase.java b/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/CsvOutputFormatITCase.java index 2311092ba7e2..4e6f01d8f65a 100644 --- a/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/CsvOutputFormatITCase.java +++ b/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/CsvOutputFormatITCase.java @@ -28,6 +28,7 @@ import java.io.File; +import static org.apache.flink.util.ExceptionUtils.findThrowableWithMessage; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -76,7 +77,7 @@ public void failPathWriteMode() throws Exception { OutputFormatTestPrograms.wordCountToCsv(WordCountData.TEXT, resultPath, FileSystem.WriteMode.NO_OVERWRITE); fail("File should exist."); } catch (Exception e) { - assertTrue(e.getCause().getMessage().contains("File already exists")); + assertTrue(findThrowableWithMessage(e, "File already exists").isPresent()); } } @@ -87,7 +88,7 @@ public void failPathWriteModeMillis() throws Exception { OutputFormatTestPrograms.wordCountToCsv(WordCountData.TEXT, resultPath, FileSystem.WriteMode.NO_OVERWRITE); fail("File should exist"); } catch (Exception e) { - assertTrue(e.getCause().getMessage().contains("File already exists")); + assertTrue(findThrowableWithMessage(e, "File already exists").isPresent()); } } @@ -98,7 +99,7 @@ public void failPathWriteModeMillisDelimiter() throws Exception { OutputFormatTestPrograms.wordCountToCsv(WordCountData.TEXT, resultPath, FileSystem.WriteMode.NO_OVERWRITE, "\n", ","); fail("File should exist."); } catch (Exception e) { - assertTrue(e.getCause().getMessage().contains("File already exists")); + assertTrue(findThrowableWithMessage(e, "File already exists").isPresent()); } } diff --git a/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/TextOutputFormatITCase.java b/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/TextOutputFormatITCase.java index c2e450a5856c..e39cf0e0fcd0 100644 --- a/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/TextOutputFormatITCase.java +++ b/flink-streaming-scala/src/test/java/org/apache/flink/streaming/scala/api/TextOutputFormatITCase.java @@ -28,6 +28,7 @@ import java.io.File; +import static org.apache.flink.util.ExceptionUtils.findThrowableWithMessage; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -61,7 +62,7 @@ public void failPathWriteMode() throws Exception { OutputFormatTestPrograms.wordCountToText(WordCountData.TEXT, resultPath, FileSystem.WriteMode.NO_OVERWRITE); fail("File should exist."); } catch (Exception e) { - assertTrue(e.getCause().getMessage().contains("File already exists")); + assertTrue(findThrowableWithMessage(e, "File already exists").isPresent()); } } From 21995337c76fb1ba2b1a9913dfd1209a371e2391 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Fri, 18 Oct 2019 22:00:07 +0800 Subject: [PATCH 149/746] [FLINK-14366][tests] Annotate MiniCluster tests in flink-tests with AlsoRunWithSchedulerNG AbstractTestBase in flink-test-utils is also annotated here to enabled tests based on it. 7 failed tests are not included and will be fixed in separate PRs: * ClassLoaderITCase, EventTimeWindowCheckpointingITCase and WindowCheckpointingITCase in FLINK-14371 * KeyedStateCheckpointingITCase in FLINK-14372 * ZooKeeperHighAvailabilityITCase in FLINK-14373 * RegionFailoverITCase in FLINK-14374 * BatchFineGrainedRecoveryITCase in FLINK-14440 This closes #9900. --- .../java/org/apache/flink/test/util/AbstractTestBase.java | 3 +++ .../apache/flink/runtime/metrics/JobManagerMetricsITCase.java | 3 +++ .../flink/runtime/metrics/SystemResourcesMetricsITCase.java | 3 +++ .../flink/test/accumulators/AccumulatorErrorITCase.java | 3 +++ .../apache/flink/test/accumulators/AccumulatorLiveITCase.java | 3 +++ .../org/apache/flink/test/cancelling/CancelingTestBase.java | 3 +++ .../checkpointing/EventTimeAllWindowCheckpointingITCase.java | 3 +++ .../org/apache/flink/test/checkpointing/RescalingITCase.java | 3 +++ .../test/checkpointing/ResumeCheckpointManuallyITCase.java | 3 +++ .../org/apache/flink/test/checkpointing/SavepointITCase.java | 3 +++ .../test/checkpointing/StreamFaultToleranceTestBase.java | 3 +++ .../test/checkpointing/utils/SavepointMigrationTestBase.java | 3 +++ .../apache/flink/test/example/client/JobRetrievalITCase.java | 3 +++ .../flink/test/example/failing/JobSubmissionFailsITCase.java | 3 +++ .../flink/test/manual/StreamingScalabilityAndLatency.java | 4 ++++ .../org/apache/flink/test/misc/CustomSerializationITCase.java | 3 +++ .../org/apache/flink/test/misc/MiscellaneousIssuesITCase.java | 3 +++ .../test/misc/SuccessAfterNetworkBuffersFailureITCase.java | 3 +++ .../apache/flink/test/operators/CustomDistributionITCase.java | 3 +++ .../apache/flink/test/operators/RemoteEnvironmentITCase.java | 3 +++ .../recovery/SimpleRecoveryFailureRateStrategyITBase.java | 3 +++ .../SimpleRecoveryFixedDelayRestartStrategyITBase.java | 3 +++ .../org/apache/flink/test/runtime/FileBufferReaderITCase.java | 3 +++ .../org/apache/flink/test/runtime/IPv6HostnamesITCase.java | 3 +++ .../java/org/apache/flink/test/runtime/NettyEpollITCase.java | 3 +++ .../flink/test/runtime/NetworkStackThroughputITCase.java | 3 +++ .../java/org/apache/flink/test/runtime/SchedulingITCase.java | 3 +++ .../runtime/leaderelection/ZooKeeperLeaderElectionITCase.java | 3 +++ .../operator/restore/AbstractOperatorRestoreTestBase.java | 3 +++ .../flink/test/streaming/runtime/BackPressureITCase.java | 3 +++ .../test/streaming/runtime/BigUserProgramJobSubmitITCase.java | 3 +++ .../apache/flink/test/streaming/runtime/TimestampITCase.java | 3 +++ 32 files changed, 97 insertions(+) diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java index 3ac2104b7b2b..acc0c2af2e3d 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java @@ -19,9 +19,11 @@ package org.apache.flink.test.util; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.FileUtils; import org.junit.ClassRule; +import org.junit.experimental.categories.Category; import org.junit.rules.TemporaryFolder; import java.io.File; @@ -54,6 +56,7 @@ * * */ +@Category(AlsoRunWithSchedulerNG.class) public abstract class AbstractTestBase extends TestBaseUtils { private static final int DEFAULT_PARALLELISM = 4; diff --git a/flink-tests/src/test/java/org/apache/flink/runtime/metrics/JobManagerMetricsITCase.java b/flink-tests/src/test/java/org/apache/flink/runtime/metrics/JobManagerMetricsITCase.java index e890216e7e9a..c06dd080121a 100644 --- a/flink-tests/src/test/java/org/apache/flink/runtime/metrics/JobManagerMetricsITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/runtime/metrics/JobManagerMetricsITCase.java @@ -28,11 +28,13 @@ import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; 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 java.util.ArrayList; import java.util.Collection; @@ -47,6 +49,7 @@ /** * Integration tests for proper initialization of the job manager metrics. */ +@Category(AlsoRunWithSchedulerNG.class) public class JobManagerMetricsITCase extends TestLogger { private static final String JOB_MANAGER_METRICS_PREFIX = "localhost.jobmanager."; diff --git a/flink-tests/src/test/java/org/apache/flink/runtime/metrics/SystemResourcesMetricsITCase.java b/flink-tests/src/test/java/org/apache/flink/runtime/metrics/SystemResourcesMetricsITCase.java index 7801a28dd294..9e0694f3a367 100644 --- a/flink-tests/src/test/java/org/apache/flink/runtime/metrics/SystemResourcesMetricsITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/runtime/metrics/SystemResourcesMetricsITCase.java @@ -27,10 +27,12 @@ import org.apache.flink.runtime.concurrent.FutureUtils; 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; import java.util.ArrayList; import java.util.List; @@ -48,6 +50,7 @@ /** * Integration tests for proper initialization of the system resource metrics. */ +@Category(AlsoRunWithSchedulerNG.class) public class SystemResourcesMetricsITCase extends TestLogger { @ClassRule 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 9f78ce2ed4d3..c5cefc3dbffc 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 @@ -30,10 +30,12 @@ import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.test.util.MiniClusterWithClientResource; +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; import static org.apache.flink.util.ExceptionUtils.findThrowable; import static org.junit.Assert.assertTrue; @@ -44,6 +46,7 @@ * a) throw errors during runtime * b) are not compatible with existing accumulator. */ +@Category(AlsoRunWithSchedulerNG.class) public class AccumulatorErrorITCase extends TestLogger { private static final String FAULTY_CLONE_ACCUMULATOR = "faulty-clone"; private static final String FAULTY_MERGE_ACCUMULATOR = "faulty-merge"; diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java index d4063184b62d..dadbe7455f99 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java @@ -43,12 +43,14 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.Collector; 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.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -62,6 +64,7 @@ /** * Tests the availability of accumulator results during runtime. */ +@Category(AlsoRunWithSchedulerNG.class) public class AccumulatorLiveITCase extends TestLogger { private static final Logger LOG = LoggerFactory.getLogger(AccumulatorLiveITCase.class); diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java index ab98e926f725..8546a0bb9359 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java @@ -35,10 +35,12 @@ import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.TestLogger; import org.junit.Assert; import org.junit.ClassRule; +import org.junit.experimental.categories.Category; import java.util.concurrent.TimeUnit; @@ -48,6 +50,7 @@ /** * Base class for testing job cancellation. */ +@Category(AlsoRunWithSchedulerNG.class) public abstract class CancelingTestBase extends TestLogger { private static final int MINIMUM_HEAP_SIZE_MB = 192; 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 c09943ff67db..e456c1142426 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 @@ -36,11 +36,13 @@ import org.apache.flink.test.checkpointing.utils.IntType; import org.apache.flink.test.checkpointing.utils.ValidatingSink; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; import org.junit.ClassRule; import org.junit.Test; +import org.junit.experimental.categories.Category; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.junit.Assert.assertEquals; @@ -53,6 +55,7 @@ *

This is a version of {@link EventTimeWindowCheckpointingITCase} for All-Windows. */ @SuppressWarnings("serial") +@Category(AlsoRunWithSchedulerNG.class) public class EventTimeAllWindowCheckpointingITCase extends TestLogger { private static final int PARALLELISM = 4; diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java index 8f788cbfbcd0..ba7704ec017f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java @@ -53,6 +53,7 @@ import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.Collector; import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; @@ -61,6 +62,7 @@ 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; @@ -85,6 +87,7 @@ * Test savepoint rescaling. */ @RunWith(Parameterized.class) +@Category(AlsoRunWithSchedulerNG.class) public class RescalingITCase extends TestLogger { private static final int numTaskManagers = 2; diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java index 2d5fa9f103f6..45b51385a63e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java @@ -38,11 +38,13 @@ import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.test.state.ManualWindowSpeedITCase; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.TestLogger; import org.apache.curator.test.TestingServer; import org.junit.ClassRule; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.junit.rules.TemporaryFolder; import javax.annotation.Nullable; @@ -66,6 +68,7 @@ * *

This tests considers full and incremental checkpoints and was introduced to guard against problems like FLINK-6964. */ +@Category(AlsoRunWithSchedulerNG.class) public class ResumeCheckpointManuallyITCase extends TestLogger { private static final int PARALLELISM = 2; 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 2f4a8572dd05..027f1135e6fe 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 @@ -54,6 +54,7 @@ import org.apache.flink.streaming.api.graph.StreamGraph; import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.flink.testutils.EntropyInjectingTestFileSystem; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.Collector; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; @@ -65,6 +66,7 @@ import org.junit.Before; import org.junit.Rule; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -99,6 +101,7 @@ * Integration test for triggering and resuming from savepoints. */ @SuppressWarnings("serial") +@Category(AlsoRunWithSchedulerNG.class) public class SavepointITCase extends TestLogger { private static final Logger LOG = LoggerFactory.getLogger(SavepointITCase.class); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java index 74cdeef46df1..ad949de380b0 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java @@ -27,12 +27,14 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.flink.test.util.SuccessException; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.TestLogger; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -46,6 +48,7 @@ * Test base for fault tolerant streaming programs. */ @RunWith(Parameterized.class) +@Category(AlsoRunWithSchedulerNG.class) public abstract class StreamFaultToleranceTestBase extends TestLogger { @Parameterized.Parameters(name = "FailoverStrategy: {0}") diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java index 5d40c8add12d..bb8f31d69037 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java @@ -35,12 +35,14 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 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.OptionalFailure; import org.apache.commons.io.FileUtils; import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Rule; +import org.junit.experimental.categories.Category; import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -62,6 +64,7 @@ /** * Test savepoint migration. */ +@Category(AlsoRunWithSchedulerNG.class) public abstract class SavepointMigrationTestBase extends TestBaseUtils { @BeforeClass diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java index b9f5b8fe4c7a..bbb80ba38c73 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java @@ -31,6 +31,7 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; 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; @@ -38,6 +39,7 @@ import org.junit.Before; import org.junit.ClassRule; import org.junit.Test; +import org.junit.experimental.categories.Category; import java.util.Optional; import java.util.concurrent.Semaphore; @@ -48,6 +50,7 @@ /** * Tests retrieval of a job from a running Flink cluster. */ +@Category(AlsoRunWithSchedulerNG.class) public class JobRetrievalITCase extends TestLogger { private static final Semaphore lock = new Semaphore(1); 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 032a3da85ceb..e11320857b78 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 @@ -28,11 +28,13 @@ import org.apache.flink.runtime.testtasks.NoOpInvokable; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.test.util.MiniClusterWithClientResource; +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.runner.RunWith; import org.junit.runners.Parameterized; @@ -49,6 +51,7 @@ * Tests for failing job submissions. */ @RunWith(Parameterized.class) +@Category(AlsoRunWithSchedulerNG.class) public class JobSubmissionFailsITCase extends TestLogger { private static final int NUM_TM = 2; 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 2398c14f8d9a..7fea2f537a42 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 @@ -29,12 +29,16 @@ import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; + +import org.junit.experimental.categories.Category; import static org.junit.Assert.fail; /** * Manual test to evaluate impact of checkpointing on latency. */ +@Category(AlsoRunWithSchedulerNG.class) public class StreamingScalabilityAndLatency { public static void main(String[] args) throws Exception { 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 6c85218b39c0..a0fcd5a99c2f 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 @@ -29,11 +29,13 @@ import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.types.Value; 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.Optional; @@ -47,6 +49,7 @@ * and detected in the network stack. */ @SuppressWarnings("serial") +@Category(AlsoRunWithSchedulerNG.class) public class CustomSerializationITCase extends TestLogger { private static final int PARLLELISM = 5; diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java index 6925a4d6f617..a73b3516cbf0 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java @@ -30,11 +30,13 @@ import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; import org.junit.ClassRule; import org.junit.Test; +import org.junit.experimental.categories.Category; import static org.apache.flink.util.ExceptionUtils.findThrowable; import static org.junit.Assert.assertEquals; @@ -52,6 +54,7 @@ * test cluster. */ @SuppressWarnings("serial") +@Category(AlsoRunWithSchedulerNG.class) public class MiscellaneousIssuesITCase extends TestLogger { @ClassRule 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 90ddf0c44a5b..dbf34dbc1a16 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 @@ -35,10 +35,12 @@ import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.test.util.MiniClusterWithClientResource; +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; import static org.apache.flink.util.ExceptionUtils.findThrowableWithMessage; import static org.junit.Assert.assertTrue; @@ -49,6 +51,7 @@ * This test validates that task slots in co-location constraints are properly * freed in the presence of failures. */ +@Category(AlsoRunWithSchedulerNG.class) public class SuccessAfterNetworkBuffersFailureITCase extends TestLogger { private static final int PARALLELISM = 16; diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/CustomDistributionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/CustomDistributionITCase.java index 1683dfc369fb..62eb0c0c4a2b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/CustomDistributionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/CustomDistributionITCase.java @@ -32,11 +32,13 @@ import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.test.operators.util.CollectionDataSets; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; import org.junit.ClassRule; import org.junit.Test; +import org.junit.experimental.categories.Category; import java.io.IOException; @@ -46,6 +48,7 @@ * Integration tests for custom {@link DataDistribution}. */ @SuppressWarnings("serial") +@Category(AlsoRunWithSchedulerNG.class) public class CustomDistributionITCase extends TestLogger { @ClassRule diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/RemoteEnvironmentITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/RemoteEnvironmentITCase.java index 3394e78075fb..ea272244787e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/RemoteEnvironmentITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/RemoteEnvironmentITCase.java @@ -27,11 +27,13 @@ import org.apache.flink.core.io.GenericInputSplit; 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.Collector; 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.net.URI; @@ -43,6 +45,7 @@ * Integration tests for {@link org.apache.flink.api.java.RemoteEnvironment}. */ @SuppressWarnings("serial") +@Category(AlsoRunWithSchedulerNG.class) public class RemoteEnvironmentITCase extends TestLogger { private static final int TM_SLOTS = 4; diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFailureRateStrategyITBase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFailureRateStrategyITBase.java index ef827d62b951..7cf356ac2657 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFailureRateStrategyITBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFailureRateStrategyITBase.java @@ -22,12 +22,15 @@ import org.apache.flink.configuration.RestartStrategyOptions; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.junit.ClassRule; +import org.junit.experimental.categories.Category; /** * Test cluster configuration with failure-rate recovery. */ +@Category(AlsoRunWithSchedulerNG.class) public class SimpleRecoveryFailureRateStrategyITBase extends SimpleRecoveryITCaseBase { @ClassRule diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFixedDelayRestartStrategyITBase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFixedDelayRestartStrategyITBase.java index 9bb2ed643349..eec385906475 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFixedDelayRestartStrategyITBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFixedDelayRestartStrategyITBase.java @@ -22,12 +22,15 @@ import org.apache.flink.configuration.RestartStrategyOptions; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.junit.ClassRule; +import org.junit.experimental.categories.Category; /** * Test cluster configuration with fixed-delay recovery. */ +@Category(AlsoRunWithSchedulerNG.class) public class SimpleRecoveryFixedDelayRestartStrategyITBase extends SimpleRecoveryITCaseBase { @ClassRule 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 91d4cf43fa83..4cd2daa763d1 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 @@ -38,6 +38,7 @@ import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.minicluster.MiniClusterConfiguration; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.testutils.serialization.types.ByteArrayType; import org.apache.flink.util.TestLogger; @@ -46,6 +47,7 @@ import org.junit.BeforeClass; import org.junit.Test; +import org.junit.experimental.categories.Category; import java.util.concurrent.CompletableFuture; @@ -63,6 +65,7 @@ * the first fetched buffer from {@link org.apache.flink.runtime.io.network.partition.FileChannelBoundedData} has not * been recycled while fetching the second buffer to trigger next read ahead, which breaks the above assumption. */ +@Category(AlsoRunWithSchedulerNG.class) public class FileBufferReaderITCase extends TestLogger { private static final int parallelism = 8; 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 8fe2aa351828..9f11285fd0ad 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 @@ -30,6 +30,7 @@ import org.apache.flink.test.testdata.WordCountData; 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.Collector; import org.apache.flink.util.NetUtils; import org.apache.flink.util.TestLogger; @@ -38,6 +39,7 @@ import org.junit.AssumptionViolatedException; import org.junit.Rule; import org.junit.Test; +import org.junit.experimental.categories.Category; import java.io.IOException; import java.net.Inet6Address; @@ -56,6 +58,7 @@ * Test proper handling of IPv6 address literals in URLs. */ @SuppressWarnings("serial") +@Category(AlsoRunWithSchedulerNG.class) public class IPv6HostnamesITCase extends TestLogger { @Rule diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/NettyEpollITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/NettyEpollITCase.java index ca8366bf0d55..58f95237120f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/NettyEpollITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/NettyEpollITCase.java @@ -25,10 +25,12 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.TestLogger; import org.junit.AssumptionViolatedException; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,6 +41,7 @@ * on linux. On other platforms it's basically a NO-OP. See * https://github.com/apache/flink-shaded/issues/30 */ +@Category(AlsoRunWithSchedulerNG.class) public class NettyEpollITCase extends TestLogger { private static final Logger LOG = LoggerFactory.getLogger(NettyEpollITCase.class); diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughputITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughputITCase.java index 3357cb69ad04..e78d022ae937 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 @@ -35,9 +35,11 @@ import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.test.util.MiniClusterWithClientResource; +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 org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,6 +49,7 @@ /** * Manually test the throughput of the network stack. */ +@Category(AlsoRunWithSchedulerNG.class) public class NetworkStackThroughputITCase extends TestLogger { private static final Logger LOG = LoggerFactory.getLogger(NetworkStackThroughputITCase.class); diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/SchedulingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/SchedulingITCase.java index 85bcc7830267..113d30402922 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/SchedulingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/SchedulingITCase.java @@ -38,10 +38,12 @@ import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.minicluster.MiniClusterConfiguration; import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; import org.junit.Test; +import org.junit.experimental.categories.Category; import javax.annotation.Nonnull; @@ -56,6 +58,7 @@ /** * IT case for testing Flink's scheduling strategies. */ +@Category(AlsoRunWithSchedulerNG.class) public class SchedulingITCase extends TestLogger { /** diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java index 168fa1f91f2e..4d90b6e148fb 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java @@ -39,6 +39,7 @@ import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.ZooKeeperTestUtils; import org.apache.flink.runtime.webmonitor.retriever.LeaderRetriever; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.TestLogger; import org.apache.curator.test.TestingServer; @@ -46,6 +47,7 @@ import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.junit.rules.TemporaryFolder; import javax.annotation.Nonnull; @@ -64,6 +66,7 @@ /** * Test the election of a new JobManager leader. */ +@Category(AlsoRunWithSchedulerNG.class) public class ZooKeeperLeaderElectionITCase extends TestLogger { private static final Duration TEST_TIMEOUT = Duration.ofMinutes(5L); diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java index 154477e921f1..f598a4da76c4 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java @@ -36,12 +36,14 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.graph.StreamingJobGraphGenerator; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.junit.rules.TemporaryFolder; import java.io.File; @@ -64,6 +66,7 @@ * Step 1: Migrate the job to the newer version by submitting the same job used for the old version savepoint, and create a new savepoint. * Step 2: Modify the job topology, and restore from the savepoint created in step 1. */ +@Category(AlsoRunWithSchedulerNG.class) public abstract class AbstractOperatorRestoreTestBase extends TestLogger { private static final int NUM_TMS = 1; diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/BackPressureITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/BackPressureITCase.java index 07fb227dc0a7..e7c2072219a8 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/BackPressureITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/BackPressureITCase.java @@ -36,12 +36,14 @@ import org.apache.flink.test.util.BlockingSink; import org.apache.flink.test.util.IdentityMapFunction; import org.apache.flink.test.util.InfiniteIntegerSource; +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.time.Duration; import java.util.List; @@ -52,6 +54,7 @@ /** * Integration test for operator back pressure tracking. */ +@Category(AlsoRunWithSchedulerNG.class) public class BackPressureITCase extends TestLogger { private static final JobID TEST_JOB_ID = new JobID(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/BigUserProgramJobSubmitITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/BigUserProgramJobSubmitITCase.java index 93e2803a7189..43834fd0939d 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/BigUserProgramJobSubmitITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/BigUserProgramJobSubmitITCase.java @@ -27,10 +27,12 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.graph.StreamingJobGraphGenerator; +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; import java.util.ArrayList; import java.util.Arrays; @@ -44,6 +46,7 @@ * Integration test that verifies that a user program with a big(ger) payload is successfully * submitted and run. */ +@Category(AlsoRunWithSchedulerNG.class) public class BigUserProgramJobSubmitITCase extends TestLogger { // ------------------------------------------------------------------------ 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 f0938a245831..83a57459db18 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 @@ -48,6 +48,7 @@ import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; @@ -55,6 +56,7 @@ import org.junit.Before; import org.junit.ClassRule; import org.junit.Test; +import org.junit.experimental.categories.Category; import java.util.ArrayList; import java.util.Collection; @@ -70,6 +72,7 @@ * Tests for timestamps, watermarks, and event-time sources. */ @SuppressWarnings("serial") +@Category(AlsoRunWithSchedulerNG.class) public class TimestampITCase extends TestLogger { private static final int NUM_TASK_MANAGERS = 2; From 6c883bb228e1e56b04dd9a0e82ac4bcd7245dc24 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Fri, 18 Oct 2019 17:52:51 +0800 Subject: [PATCH 150/746] [FLINK-14330][runtime] Introduce base topology interface This closes #9934. --- .../apache/flink/runtime/topology/Result.java | 38 +++++++++++++++++ .../flink/runtime/topology/ResultID.java | 25 +++++++++++ .../flink/runtime/topology/Topology.java | 42 +++++++++++++++++++ .../apache/flink/runtime/topology/Vertex.java | 34 +++++++++++++++ .../flink/runtime/topology/VertexID.java | 25 +++++++++++ 5 files changed, 164 insertions(+) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/topology/Result.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/topology/ResultID.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/topology/Topology.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/topology/Vertex.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/topology/VertexID.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/topology/Result.java b/flink-runtime/src/main/java/org/apache/flink/runtime/topology/Result.java new file mode 100644 index 000000000000..937eeaf59a28 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/topology/Result.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.topology; + +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; + +/** + * Represents a data set produced by a {@link Vertex} + * Each result is produced by one {@link Vertex}. + * Each result can be consumed by multiple {@link Vertex}. + */ +public interface Result, R extends Result> { + + RID getId(); + + ResultPartitionType getResultType(); + + V getProducer(); + + Iterable getConsumers(); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/topology/ResultID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/topology/ResultID.java new file mode 100644 index 000000000000..606544f5708e --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/topology/ResultID.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.topology; + +/** + * ID of a {@link Result}. + */ +public interface ResultID { +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/topology/Topology.java b/flink-runtime/src/main/java/org/apache/flink/runtime/topology/Topology.java new file mode 100644 index 000000000000..82cf34ed4943 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/topology/Topology.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.runtime.topology; + +/** + * Base topology for all logical and execution topologies. + * A topology consists of {@link Vertex} and {@link Result}. + */ +public interface Topology, R extends Result> { + + /** + * Returns an iterable over all vertices, topologically sorted. + * + * @return topologically sorted iterable over all vertices + */ + Iterable getVertices(); + + /** + * Returns whether the topology contains co-location constraints. + * Co-location constraints are currently used for iterations. + * + * @return whether the topology contains co-location constraints + */ + boolean containsCoLocationConstraints(); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/topology/Vertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/topology/Vertex.java new file mode 100644 index 000000000000..5f88843edea4 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/topology/Vertex.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.runtime.topology; + +/** + * Represents a logical or execution task. + * Each vertex can consume data from multiple {@link Result}. + * Each vertex can produce multiple {@link Result}. + */ +public interface Vertex, R extends Result> { + + VID getId(); + + Iterable getConsumedResults(); + + Iterable getProducedResults(); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/topology/VertexID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/topology/VertexID.java new file mode 100644 index 000000000000..b26d658d3dbc --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/topology/VertexID.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.topology; + +/** + * ID of a {@link Vertex}. + */ +public interface VertexID { +} From d1b06ac4e8ab033c699b656cb0f1d924b6bafc56 Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Tue, 15 Oct 2019 16:21:29 -0700 Subject: [PATCH 151/746] [FLINK-14401][table][hive] create DefaultFunctionDefinitionFactory to instantiate regular java class-based udf create FunctionDefinitionUtil to instantiate regular java class-based udf and add HiveFunctionDefinitionFactory to instantiate both flink and hive udf This closes #9908. --- .../connectors/hive/HiveTableFactory.java | 103 +------------ .../flink/table/catalog/hive/HiveCatalog.java | 7 + .../HiveFunctionDefinitionFactory.java | 140 ++++++++++++++++++ .../table/tests/test_catalog_completeness.py | 2 +- .../flink/table/catalog/FunctionCatalog.java | 24 ++- .../functions/FunctionDefinitionUtil.java | 77 ++++++++++ .../functions/FunctionDefinitionUtilTest.java | 135 +++++++++++++++++ .../apache/flink/table/catalog/Catalog.java | 12 +- .../factories/FunctionDefinitionFactory.java | 3 +- 9 files changed, 383 insertions(+), 120 deletions(-) create mode 100644 flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/factories/HiveFunctionDefinitionFactory.java create mode 100644 flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/functions/FunctionDefinitionUtil.java create mode 100644 flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/functions/FunctionDefinitionUtilTest.java diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableFactory.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableFactory.java index 235919c0f4fc..89f76daeb3cf 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableFactory.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableFactory.java @@ -18,29 +18,13 @@ package org.apache.flink.connectors.hive; -import org.apache.flink.api.java.typeutils.GenericTypeInfo; -import org.apache.flink.table.api.TableException; -import org.apache.flink.table.catalog.CatalogFunction; import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.CatalogTableImpl; import org.apache.flink.table.catalog.ObjectPath; import org.apache.flink.table.catalog.config.CatalogConfig; -import org.apache.flink.table.catalog.hive.client.HiveShim; -import org.apache.flink.table.catalog.hive.client.HiveShimLoader; -import org.apache.flink.table.catalog.hive.descriptors.HiveCatalogValidator; -import org.apache.flink.table.factories.FunctionDefinitionFactory; import org.apache.flink.table.factories.TableFactoryUtil; import org.apache.flink.table.factories.TableSinkFactory; import org.apache.flink.table.factories.TableSourceFactory; -import org.apache.flink.table.functions.AggregateFunctionDefinition; -import org.apache.flink.table.functions.FunctionDefinition; -import org.apache.flink.table.functions.ScalarFunctionDefinition; -import org.apache.flink.table.functions.TableFunctionDefinition; -import org.apache.flink.table.functions.hive.HiveFunctionWrapper; -import org.apache.flink.table.functions.hive.HiveGenericUDAF; -import org.apache.flink.table.functions.hive.HiveGenericUDF; -import org.apache.flink.table.functions.hive.HiveGenericUDTF; -import org.apache.flink.table.functions.hive.HiveSimpleUDF; import org.apache.flink.table.sinks.OutputFormatTableSink; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.table.sources.InputFormatTableSource; @@ -49,12 +33,6 @@ import org.apache.flink.util.Preconditions; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.exec.UDAF; -import org.apache.hadoop.hive.ql.exec.UDF; -import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator; -import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFResolver2; -import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; -import org.apache.hadoop.hive.ql.udf.generic.GenericUDTF; import org.apache.hadoop.mapred.JobConf; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,20 +46,13 @@ * A table factory implementation for Hive catalog. */ public class HiveTableFactory - implements TableSourceFactory, TableSinkFactory, FunctionDefinitionFactory { + implements TableSourceFactory, TableSinkFactory { private static final Logger LOG = LoggerFactory.getLogger(HiveTableFactory.class); private final HiveConf hiveConf; - private final String hiveVersion; - private final HiveShim hiveShim; public HiveTableFactory(HiveConf hiveConf) { this.hiveConf = checkNotNull(hiveConf, "hiveConf cannot be null"); - - // this has to come from hiveConf, otherwise we may lose what user specifies in the yaml file - this.hiveVersion = checkNotNull(hiveConf.get(HiveCatalogValidator.CATALOG_HIVE_VERSION), - "Hive version is not defined"); - this.hiveShim = HiveShimLoader.loadHiveShim(hiveVersion); } @Override @@ -145,76 +116,4 @@ public TableSink createTableSink(ObjectPath tablePath, CatalogTable table) private OutputFormatTableSink createOutputFormatTableSink(ObjectPath tablePath, CatalogTable table) { return new HiveTableSink(new JobConf(hiveConf), tablePath, table); } - - @Override - public FunctionDefinition createFunctionDefinition(String name, CatalogFunction catalogFunction) { - String functionClassName = catalogFunction.getClassName(); - - if (Boolean.valueOf(catalogFunction.getProperties().get(CatalogConfig.IS_GENERIC))) { - throw new TableException( - String.format("HiveFunctionDefinitionFactory does not support generic functions %s yet", name)); - } - - Class clazz; - try { - clazz = Thread.currentThread().getContextClassLoader().loadClass(functionClassName); - - LOG.info("Successfully loaded Hive udf '{}' with class '{}'", name, functionClassName); - } catch (ClassNotFoundException e) { - throw new TableException( - String.format("Failed to initiate an instance of class %s.", functionClassName), e); - } - - if (UDF.class.isAssignableFrom(clazz)) { - LOG.info("Transforming Hive function '{}' into a HiveSimpleUDF", name); - - return new ScalarFunctionDefinition( - name, - new HiveSimpleUDF(new HiveFunctionWrapper<>(functionClassName), hiveShim) - ); - } else if (GenericUDF.class.isAssignableFrom(clazz)) { - LOG.info("Transforming Hive function '{}' into a HiveGenericUDF", name); - - return new ScalarFunctionDefinition( - name, - new HiveGenericUDF(new HiveFunctionWrapper<>(functionClassName), hiveShim) - ); - } else if (GenericUDTF.class.isAssignableFrom(clazz)) { - LOG.info("Transforming Hive function '{}' into a HiveGenericUDTF", name); - - HiveGenericUDTF udtf = new HiveGenericUDTF(new HiveFunctionWrapper<>(functionClassName), hiveShim); - - return new TableFunctionDefinition( - name, - udtf, - GenericTypeInfo.of(Row.class) - ); - } else if (GenericUDAFResolver2.class.isAssignableFrom(clazz) || UDAF.class.isAssignableFrom(clazz)) { - HiveGenericUDAF udaf; - - if (GenericUDAFResolver2.class.isAssignableFrom(clazz)) { - LOG.info( - "Transforming Hive function '{}' into a HiveGenericUDAF with no UDAF bridging and Hive version %s", - name, hiveVersion); - - udaf = new HiveGenericUDAF(new HiveFunctionWrapper<>(functionClassName), false, hiveVersion); - } else { - LOG.info( - "Transforming Hive function '{}' into a HiveGenericUDAF with UDAF bridging and Hive version %s", - name, hiveVersion); - - udaf = new HiveGenericUDAF(new HiveFunctionWrapper<>(functionClassName), true, hiveVersion); - } - - return new AggregateFunctionDefinition( - name, - udaf, - GenericTypeInfo.of(Object.class), - GenericTypeInfo.of(GenericUDAFEvaluator.AggregationBuffer.class) - ); - } else { - throw new IllegalArgumentException( - String.format("HiveFunctionDefinitionFactory cannot initiate FunctionDefinition for class %s", functionClassName)); - } - } } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java index 7914c8964ec5..1e66551d3a1b 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java @@ -56,11 +56,13 @@ import org.apache.flink.table.catalog.hive.client.HiveShim; import org.apache.flink.table.catalog.hive.client.HiveShimLoader; import org.apache.flink.table.catalog.hive.descriptors.HiveCatalogValidator; +import org.apache.flink.table.catalog.hive.factories.HiveFunctionDefinitionFactory; import org.apache.flink.table.catalog.hive.util.HiveReflectionUtils; import org.apache.flink.table.catalog.hive.util.HiveStatsUtil; import org.apache.flink.table.catalog.hive.util.HiveTableUtil; import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; import org.apache.flink.table.catalog.stats.CatalogTableStatistics; +import org.apache.flink.table.factories.FunctionDefinitionFactory; import org.apache.flink.table.factories.TableFactory; import org.apache.flink.util.StringUtils; @@ -202,6 +204,11 @@ public Optional getTableFactory() { return Optional.of(new HiveTableFactory(hiveConf)); } + @Override + public Optional getFunctionDefinitionFactory() { + return Optional.of(new HiveFunctionDefinitionFactory(hiveConf)); + } + // ------ databases ------ @Override diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/factories/HiveFunctionDefinitionFactory.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/factories/HiveFunctionDefinitionFactory.java new file mode 100644 index 000000000000..3e32c7fa4eac --- /dev/null +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/factories/HiveFunctionDefinitionFactory.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.catalog.hive.factories; + +import org.apache.flink.api.java.typeutils.GenericTypeInfo; +import org.apache.flink.connectors.hive.HiveTableFactory; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.catalog.CatalogFunction; +import org.apache.flink.table.catalog.config.CatalogConfig; +import org.apache.flink.table.catalog.hive.client.HiveShim; +import org.apache.flink.table.catalog.hive.client.HiveShimLoader; +import org.apache.flink.table.catalog.hive.descriptors.HiveCatalogValidator; +import org.apache.flink.table.factories.FunctionDefinitionFactory; +import org.apache.flink.table.functions.AggregateFunctionDefinition; +import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.functions.FunctionDefinitionUtil; +import org.apache.flink.table.functions.ScalarFunctionDefinition; +import org.apache.flink.table.functions.TableFunctionDefinition; +import org.apache.flink.table.functions.hive.HiveFunctionWrapper; +import org.apache.flink.table.functions.hive.HiveGenericUDAF; +import org.apache.flink.table.functions.hive.HiveGenericUDF; +import org.apache.flink.table.functions.hive.HiveGenericUDTF; +import org.apache.flink.table.functions.hive.HiveSimpleUDF; +import org.apache.flink.types.Row; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.UDAF; +import org.apache.hadoop.hive.ql.exec.UDF; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFResolver2; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDTF; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A factory to instantiate Hive UDFs as Flink UDFs. + */ +public class HiveFunctionDefinitionFactory implements FunctionDefinitionFactory { + private static final Logger LOG = LoggerFactory.getLogger(HiveTableFactory.class); + + private final String hiveVersion; + private final HiveShim hiveShim; + + public HiveFunctionDefinitionFactory(HiveConf hiveConf) { + // this has to come from hiveConf, otherwise we may lose what user specifies in the yaml file + this.hiveVersion = checkNotNull(hiveConf.get(HiveCatalogValidator.CATALOG_HIVE_VERSION), + "Hive version is not defined"); + this.hiveShim = HiveShimLoader.loadHiveShim(hiveVersion); + } + + @Override + public FunctionDefinition createFunctionDefinition(String name, CatalogFunction catalogFunction) { + if (Boolean.valueOf(catalogFunction.getProperties().get(CatalogConfig.IS_GENERIC))) { + FunctionDefinitionUtil.createFunctionDefinition(name, catalogFunction); + } + + String functionClassName = catalogFunction.getClassName(); + + Class clazz; + try { + clazz = Thread.currentThread().getContextClassLoader().loadClass(functionClassName); + + LOG.info("Successfully loaded Hive udf '{}' with class '{}'", name, functionClassName); + } catch (ClassNotFoundException e) { + throw new TableException( + String.format("Failed to initiate an instance of class %s.", functionClassName), e); + } + + if (UDF.class.isAssignableFrom(clazz)) { + LOG.info("Transforming Hive function '{}' into a HiveSimpleUDF", name); + + return new ScalarFunctionDefinition( + name, + new HiveSimpleUDF(new HiveFunctionWrapper<>(functionClassName), hiveShim) + ); + } else if (GenericUDF.class.isAssignableFrom(clazz)) { + LOG.info("Transforming Hive function '{}' into a HiveGenericUDF", name); + + return new ScalarFunctionDefinition( + name, + new HiveGenericUDF(new HiveFunctionWrapper<>(functionClassName), hiveShim) + ); + } else if (GenericUDTF.class.isAssignableFrom(clazz)) { + LOG.info("Transforming Hive function '{}' into a HiveGenericUDTF", name); + + HiveGenericUDTF udtf = new HiveGenericUDTF(new HiveFunctionWrapper<>(functionClassName), hiveShim); + + return new TableFunctionDefinition( + name, + udtf, + GenericTypeInfo.of(Row.class) + ); + } else if (GenericUDAFResolver2.class.isAssignableFrom(clazz) || UDAF.class.isAssignableFrom(clazz)) { + HiveGenericUDAF udaf; + + if (GenericUDAFResolver2.class.isAssignableFrom(clazz)) { + LOG.info( + "Transforming Hive function '{}' into a HiveGenericUDAF with no UDAF bridging and Hive version %s", + name, hiveVersion); + + udaf = new HiveGenericUDAF(new HiveFunctionWrapper<>(functionClassName), false, hiveVersion); + } else { + LOG.info( + "Transforming Hive function '{}' into a HiveGenericUDAF with UDAF bridging and Hive version %s", + name, hiveVersion); + + udaf = new HiveGenericUDAF(new HiveFunctionWrapper<>(functionClassName), true, hiveVersion); + } + + return new AggregateFunctionDefinition( + name, + udaf, + GenericTypeInfo.of(Object.class), + GenericTypeInfo.of(GenericUDAFEvaluator.AggregationBuffer.class) + ); + } else { + throw new IllegalArgumentException( + String.format("HiveFunctionDefinitionFactory cannot initiate FunctionDefinition for class %s", functionClassName)); + } + } +} diff --git a/flink-python/pyflink/table/tests/test_catalog_completeness.py b/flink-python/pyflink/table/tests/test_catalog_completeness.py index 9474c30374b6..003c1445957a 100644 --- a/flink-python/pyflink/table/tests/test_catalog_completeness.py +++ b/flink-python/pyflink/table/tests/test_catalog_completeness.py @@ -40,7 +40,7 @@ def java_class(cls): @classmethod def excluded_methods(cls): # open/close are not needed in Python API as they are used internally - return {'open', 'close', 'getTableFactory'} + return {'open', 'close', 'getTableFactory', 'getFunctionDefinitionFactory'} class CatalogDatabaseAPICompletenessTests(PythonAPICompletenessTestCase, unittest.TestCase): 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 43280b47029c..68082165309d 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 @@ -25,11 +25,11 @@ import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; import org.apache.flink.table.catalog.exceptions.FunctionNotExistException; import org.apache.flink.table.delegation.PlannerTypeInferenceUtil; -import org.apache.flink.table.factories.FunctionDefinitionFactory; import org.apache.flink.table.functions.AggregateFunction; import org.apache.flink.table.functions.AggregateFunctionDefinition; import org.apache.flink.table.functions.BuiltInFunctionDefinitions; import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.functions.FunctionDefinitionUtil; import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.functions.ScalarFunctionDefinition; import org.apache.flink.table.functions.TableAggregateFunction; @@ -242,21 +242,17 @@ public Optional lookupFunction(String name) { CatalogFunction catalogFunction = catalog.getFunction( new ObjectPath(catalogManager.getCurrentDatabase(), functionName)); - if (catalog.getTableFactory().isPresent() && - catalog.getTableFactory().get() instanceof FunctionDefinitionFactory) { - - FunctionDefinitionFactory factory = (FunctionDefinitionFactory) catalog.getTableFactory().get(); - - userCandidate = factory.createFunctionDefinition(functionName, catalogFunction); - - return Optional.of( - new FunctionLookup.Result( - ObjectIdentifier.of(catalogManager.getCurrentCatalog(), catalogManager.getCurrentDatabase(), name), - userCandidate) - ); + if (catalog.getFunctionDefinitionFactory().isPresent()) { + userCandidate = catalog.getFunctionDefinitionFactory().get().createFunctionDefinition(functionName, catalogFunction); } else { - // TODO: should go through function definition discover service + userCandidate = FunctionDefinitionUtil.createFunctionDefinition(functionName, catalogFunction); } + + return Optional.of( + new FunctionLookup.Result( + ObjectIdentifier.of(catalogManager.getCurrentCatalog(), catalogManager.getCurrentDatabase(), name), + userCandidate) + ); } catch (FunctionNotExistException e) { // Ignore } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/functions/FunctionDefinitionUtil.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/functions/FunctionDefinitionUtil.java new file mode 100644 index 000000000000..98cedaf33887 --- /dev/null +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/functions/FunctionDefinitionUtil.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.table.catalog.CatalogFunction; + +/** + * A util to instantiate {@link FunctionDefinition} in the default way. + */ +public class FunctionDefinitionUtil { + + public static FunctionDefinition createFunctionDefinition(String name, CatalogFunction catalogFunction) { + // Currently only handles Java class-based functions + Object func; + try { + func = Thread.currentThread().getContextClassLoader().loadClass(catalogFunction.getClassName()).newInstance(); + } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) { + throw new IllegalStateException( + String.format("Failed instantiating '%s'", catalogFunction.getClassName()) + ); + } + + UserDefinedFunction udf = (UserDefinedFunction) func; + + if (udf instanceof ScalarFunction) { + return new ScalarFunctionDefinition( + name, + (ScalarFunction) udf + ); + } else if (udf instanceof TableFunction) { + TableFunction t = (TableFunction) udf; + return new TableFunctionDefinition( + name, + t, + t.getResultType() + ); + } else if (udf instanceof AggregateFunction) { + AggregateFunction a = (AggregateFunction) udf; + + return new AggregateFunctionDefinition( + name, + a, + a.getAccumulatorType(), + a.getResultType() + ); + } else if (udf instanceof TableAggregateFunction) { + TableAggregateFunction a = (TableAggregateFunction) udf; + + return new TableAggregateFunctionDefinition( + name, + a, + a.getAccumulatorType(), + a.getResultType() + ); + } else { + throw new UnsupportedOperationException( + String.format("Function %s should be of ScalarFunction, TableFunction, AggregateFunction, or TableAggregateFunction", catalogFunction.getClassName()) + ); + } + } +} diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/functions/FunctionDefinitionUtilTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/functions/FunctionDefinitionUtilTest.java new file mode 100644 index 000000000000..84cb81cc84bf --- /dev/null +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/functions/FunctionDefinitionUtilTest.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.functions; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.table.catalog.CatalogFunctionImpl; + +import org.junit.Test; + +import java.util.Collections; + +import static org.junit.Assert.assertTrue; + +/** + * Test for {@link FunctionDefinitionUtil}. + */ +public class FunctionDefinitionUtilTest { + @Test + public void testScalarFunction() { + FunctionDefinition fd = FunctionDefinitionUtil.createFunctionDefinition( + "test", + new CatalogFunctionImpl(TestScalarFunction.class.getName(), Collections.emptyMap()) + ); + + assertTrue(((ScalarFunctionDefinition) fd).getScalarFunction() instanceof TestScalarFunction); + } + + @Test + public void testTableFunction() { + FunctionDefinition fd = FunctionDefinitionUtil.createFunctionDefinition( + "test", + new CatalogFunctionImpl(TestTableFunction.class.getName(), Collections.emptyMap()) + ); + + assertTrue(((TableFunctionDefinition) fd).getTableFunction() instanceof TestTableFunction); + } + + @Test + public void testAggregateFunction() { + FunctionDefinition fd = FunctionDefinitionUtil.createFunctionDefinition( + "test", + new CatalogFunctionImpl(TestAggFunction.class.getName(), Collections.emptyMap()) + ); + + assertTrue(((AggregateFunctionDefinition) fd).getAggregateFunction() instanceof TestAggFunction); + } + + @Test + public void testTableAggregateFunction() { + FunctionDefinition fd = FunctionDefinitionUtil.createFunctionDefinition( + "test", + new CatalogFunctionImpl(TestTableAggFunction.class.getName(), Collections.emptyMap()) + ); + + assertTrue(((TableAggregateFunctionDefinition) fd).getTableAggregateFunction() instanceof TestTableAggFunction); + } + + /** + * Test function. + */ + public static class TestScalarFunction extends ScalarFunction { + + } + + /** + * Test function. + */ + public static class TestTableFunction extends TableFunction { + @Override + public TypeInformation getResultType() { + return TypeInformation.of(Object.class); + } + } + + /** + * Test function. + */ + public static class TestAggFunction extends AggregateFunction { + @Override + public Object createAccumulator() { + return null; + } + + @Override + public TypeInformation getResultType() { + return TypeInformation.of(Object.class); + } + + @Override + public TypeInformation getAccumulatorType() { + return TypeInformation.of(Object.class); + } + + @Override + public Object getValue(Object accumulator) { + return null; + } + } + + /** + * Test function. + */ + public static class TestTableAggFunction extends TableAggregateFunction { + @Override + public Object createAccumulator() { + return null; + } + + @Override + public TypeInformation getResultType() { + return TypeInformation.of(Object.class); + } + + @Override + public TypeInformation getAccumulatorType() { + return TypeInformation.of(Object.class); + } + } +} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/Catalog.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/Catalog.java index 8019ab0cbf6d..8f210b9b0b99 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/Catalog.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/Catalog.java @@ -34,6 +34,7 @@ import org.apache.flink.table.catalog.exceptions.TablePartitionedException; import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; import org.apache.flink.table.catalog.stats.CatalogTableStatistics; +import org.apache.flink.table.factories.FunctionDefinitionFactory; import org.apache.flink.table.factories.TableFactory; import java.util.List; @@ -48,7 +49,7 @@ public interface Catalog { /** * Get an optional {@link TableFactory} instance that's responsible for generating table-related - * instances stored in this catalog, instances such as source/sink and function definitions. + * instances stored in this catalog, instances such as source/sink. * * @return an optional TableFactory instance */ @@ -56,6 +57,15 @@ default Optional getTableFactory() { return Optional.empty(); } + /** + * Get an optional {@link FunctionDefinitionFactory} instance that's responsible for instantiating function definitions. + * + * @return an optional FunctionDefinitionFactory instance + */ + default Optional getFunctionDefinitionFactory() { + return Optional.empty(); + } + /** * Open the catalog. Used for any required preparation in initialization phase. * diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/FunctionDefinitionFactory.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/FunctionDefinitionFactory.java index 2e8c5380ebfc..1e4d76b43ca7 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/FunctionDefinitionFactory.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/FunctionDefinitionFactory.java @@ -23,9 +23,8 @@ /** * A factory to create {@link FunctionDefinition}. - * See also {@link TableFactory} for more information. */ -public interface FunctionDefinitionFactory extends TableFactory { +public interface FunctionDefinitionFactory { /** * Creates a {@link FunctionDefinition} from given {@link CatalogFunction}. From 2b1187d299bc6fd8dcae0d4e565238d7800dd4bb Mon Sep 17 00:00:00 2001 From: hequn8128 Date: Sat, 19 Oct 2019 18:34:33 +0800 Subject: [PATCH 152/746] [FLINK-14459][python] Fix python module build hang problem The latest conda installer may hang in some circumstances. We should specify a stable version instead of the latest. This closes #9941. --- flink-python/dev/lint-python.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-python/dev/lint-python.sh b/flink-python/dev/lint-python.sh index 8aee77290a37..e1becb8177f9 100755 --- a/flink-python/dev/lint-python.sh +++ b/flink-python/dev/lint-python.sh @@ -144,8 +144,8 @@ function install_wget() { # some pakcages including checks such as tox and flake8. function install_miniconda() { - OS_TO_CONDA_URL=("https://repo.continuum.io/miniconda/Miniconda3-latest-MacOSX-x86_64.sh" \ - "https://repo.continuum.io/miniconda/Miniconda3-latest-Linux-x86_64.sh") + OS_TO_CONDA_URL=("https://repo.continuum.io/miniconda/Miniconda3-4.7.10-MacOSX-x86_64.sh" \ + "https://repo.continuum.io/miniconda/Miniconda3-4.7.10-Linux-x86_64.sh") print_function "STEP" "download miniconda..." if [ ! -f "$CONDA_INSTALL" ]; then download ${OS_TO_CONDA_URL[$1]} $CONDA_INSTALL_SH From fbb941403bc2f604734e5121a905f54f2a0d0c5f Mon Sep 17 00:00:00 2001 From: Wei Zhong Date: Sat, 12 Oct 2019 20:47:31 +0800 Subject: [PATCH 153/746] [FLINK-14027][python][doc] Add documentation for Python User-Defined Scalar function. This closes #9886. --- .../generated/python_configuration.html | 21 +++ docs/dev/table/udfs.md | 124 +++++++++++++----- docs/dev/table/udfs.zh.md | 124 +++++++++++++----- docs/ops/config.md | 4 + docs/ops/config.zh.md | 4 + flink-docs/pom.xml | 5 + .../ConfigOptionsDocGenerator.java | 3 +- .../apache/flink/python/PythonOptions.java | 11 +- 8 files changed, 223 insertions(+), 73 deletions(-) create mode 100644 docs/_includes/generated/python_configuration.html diff --git a/docs/_includes/generated/python_configuration.html b/docs/_includes/generated/python_configuration.html new file mode 100644 index 000000000000..2415fc270e15 --- /dev/null +++ b/docs/_includes/generated/python_configuration.html @@ -0,0 +1,21 @@ + + + + + + + + + + + + + + + + + + + + +
KeyDefaultDescription
python.fn-execution.bundle.size
1000The maximum number of elements to include in a bundle for Python user-defined function execution. The elements are processed asynchronously. One bundle of elements are processed before processing the next bundle of elements. A larger value can improve the throughput, but at the cost of more memory usage and higher latency.
python.fn-execution.bundle.time
1000Sets the waiting timeout(in milliseconds) before processing a bundle for Python user-defined function execution. The timeout defines how long the elements of a bundle will be buffered before being processed. Lower timeouts lead to lower tail latencies, but may affect throughput.
diff --git a/docs/dev/table/udfs.md b/docs/dev/table/udfs.md index dbb9a97fa472..d91570e920fb 100644 --- a/docs/dev/table/udfs.md +++ b/docs/dev/table/udfs.md @@ -44,12 +44,12 @@ Scalar Functions If a required scalar function is not contained in the built-in functions, it is possible to define custom, user-defined scalar functions for both the Table API and SQL. A user-defined scalar functions maps zero, one, or multiple scalar values to a new scalar value. -In order to define a scalar function one has to extend the base class `ScalarFunction` in `org.apache.flink.table.functions` and implement (one or more) evaluation methods. The behavior of a scalar function is determined by the evaluation method. An evaluation method must be declared publicly and named `eval`. The parameter types and return type of the evaluation method also determine the parameter and return types of the scalar function. Evaluation methods can also be overloaded by implementing multiple methods named `eval`. Evaluation methods can also support variable arguments, such as `eval(String... strs)`. +

+
+In order to define a scalar function, one has to extend the base class `ScalarFunction` in `org.apache.flink.table.functions` and implement (one or more) evaluation methods. The behavior of a scalar function is determined by the evaluation method. An evaluation method must be declared publicly and named `eval`. The parameter types and return type of the evaluation method also determine the parameter and return types of the scalar function. Evaluation methods can also be overloaded by implementing multiple methods named `eval`. Evaluation methods can also support variable arguments, such as `eval(String... strs)`. The following example shows how to define your own hash code function, register it in the TableEnvironment, and call it in a query. Note that you can configure your scalar function via a constructor before it is registered: -
-
{% highlight java %} public class HashCode extends ScalarFunction { private int factor = 12; @@ -74,9 +74,29 @@ myTable.select("string, string.hashCode(), hashCode(string)"); // use the function in SQL API tableEnv.sqlQuery("SELECT string, hashCode(string) FROM MyTable"); {% endhighlight %} + +By default the result type of an evaluation method is determined by Flink's type extraction facilities. This is sufficient for basic types or simple POJOs but might be wrong for more complex, custom, or composite types. In these cases `TypeInformation` of the result type can be manually defined by overriding `ScalarFunction#getResultType()`. + +The following example shows an advanced example which takes the internal timestamp representation and also returns the internal timestamp representation as a long value. By overriding `ScalarFunction#getResultType()` we define that the returned long value should be interpreted as a `Types.TIMESTAMP` by the code generation. + +{% highlight java %} +public static class TimestampModifier extends ScalarFunction { + public long eval(long t) { + return t % 1000; + } + + public TypeInformation getResultType(Class[] signature) { + return Types.SQL_TIMESTAMP; + } +} +{% endhighlight %}
+In order to define a scalar function, one has to extend the base class `ScalarFunction` in `org.apache.flink.table.functions` and implement (one or more) evaluation methods. The behavior of a scalar function is determined by the evaluation method. An evaluation method must be declared publicly and named `eval`. The parameter types and return type of the evaluation method also determine the parameter and return types of the scalar function. Evaluation methods can also be overloaded by implementing multiple methods named `eval`. Evaluation methods can also support variable arguments, such as `@varargs def eval(str: String*)`. + +The following example shows how to define your own hash code function, register it in the TableEnvironment, and call it in a query. Note that you can configure your scalar function via a constructor before it is registered: + {% highlight scala %} // must be defined in static/object context class HashCode(factor: Int) extends ScalarFunction { @@ -95,14 +115,34 @@ myTable.select('string, hashCode('string)) tableEnv.registerFunction("hashCode", new HashCode(10)) tableEnv.sqlQuery("SELECT string, hashCode(string) FROM MyTable") {% endhighlight %} + +By default the result type of an evaluation method is determined by Flink's type extraction facilities. This is sufficient for basic types or simple POJOs but might be wrong for more complex, custom, or composite types. In these cases `TypeInformation` of the result type can be manually defined by overriding `ScalarFunction#getResultType()`. + +The following example shows an advanced example which takes the internal timestamp representation and also returns the internal timestamp representation as a long value. By overriding `ScalarFunction#getResultType()` we define that the returned long value should be interpreted as a `Types.TIMESTAMP` by the code generation. + +{% highlight scala %} +object TimestampModifier extends ScalarFunction { + def eval(t: Long): Long = { + t % 1000 + } + + override def getResultType(signature: Array[Class[_]]): TypeInformation[_] = { + Types.TIMESTAMP + } +} +{% endhighlight %}
+It supports to use both Java/Scala scalar functions and Python scalar functions in Python Table API and SQL. In order to define a Python scalar function, one can extend the base class `ScalarFunction` in `pyflink.table.udf` and implement an evaluation method. The behavior of a Python scalar function is determined by the evaluation method. An evaluation method must be named `eval`. Evaluation method can also support variable arguments, such as `eval(*args)`. + +The following example shows how to define your own Java and Python hash code functions, register them in the TableEnvironment, and call them in a query. Note that you can configure your scalar function via a constructor before it is registered: + {% highlight python %} ''' Java code: -// The java class must have a public no-argument constructor and can be founded in current java classloader. +// The Java class must have a public no-argument constructor and can be founded in current Java classloader. public class HashCode extends ScalarFunction { private int factor = 12; @@ -112,50 +152,64 @@ public class HashCode extends ScalarFunction { } ''' +class PyHashCode(ScalarFunction): + def __init__(self): + self.factor = 12 + + def eval(self, s): + return hash(s) * self.factor + table_env = BatchTableEnvironment.create(env) -# register the java function +# register the Java function table_env.register_java_function("hashCode", "my.java.function.HashCode") +# register the Python function +table_env.register_function("py_hash_code", udf(PyHashCode(), DataTypes.BIGINT(), DataTypes.BIGINT())) + # use the function in Python Table API -my_table.select("string, string.hashCode(), hashCode(string)") +my_table.select("string, bigint, string.hashCode(), hashCode(string), bigint.py_hash_code(), py_hash_code(bigint)") # use the function in SQL API -table_env.sql_query("SELECT string, hashCode(string) FROM MyTable") +table_env.sql_query("SELECT string, bigint, hashCode(string), py_hash_code(bigint) FROM MyTable") {% endhighlight %} -
-
-By default the result type of an evaluation method is determined by Flink's type extraction facilities. This is sufficient for basic types or simple POJOs but might be wrong for more complex, custom, or composite types. In these cases `TypeInformation` of the result type can be manually defined by overriding `ScalarFunction#getResultType()`. +There are many ways to define a Python scalar function besides extending the base class `ScalarFunction`. The following example shows the different ways to define a Python scalar function which takes two columns of bigint as input parameters and returns the sum of them as the result. -The following example shows an advanced example which takes the internal timestamp representation and also returns the internal timestamp representation as a long value. By overriding `ScalarFunction#getResultType()` we define that the returned long value should be interpreted as a `Types.TIMESTAMP` by the code generation. +{% highlight python %} +# option 1: extending the base class `ScalarFunction` +class Add(ScalarFunction): + def eval(self, i, j): + return i + j -
-
-{% highlight java %} -public static class TimestampModifier extends ScalarFunction { - public long eval(long t) { - return t % 1000; - } +add = udf(Add(), [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT()) - public TypeInformation getResultType(Class[] signature) { - return Types.SQL_TIMESTAMP; - } -} -{% endhighlight %} -
+# option 2: Python function +@udf(input_types=[DataTypes.BIGINT(), DataTypes.BIGINT()], result_type=DataTypes.BIGINT()) +def add(i, j): + return i + j -
-{% highlight scala %} -object TimestampModifier extends ScalarFunction { - def eval(t: Long): Long = { - t % 1000 - } +# option 3: lambda function +add = udf(lambda i, j: i + j, [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT()) - override def getResultType(signature: Array[Class[_]]): TypeInformation[_] = { - Types.TIMESTAMP - } -} +# option 4: callable function +class CallableAdd(object): + def __call__(self, i, j): + return i + j + +add = udf(CallableAdd(), [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT()) + +# option 5: partial function +def partial_add(i, j, k): + return i + j + k + +add = udf(functools.partial(partial_add, j=1), [DataTypes.BIGINT(), DataTypes.BIGINT()], + DataTypes.BIGINT()) + +# register the Python function +table_env.register_function("add", add) +# use the function in Python Table API +my_table.select("add(a, b)") {% endhighlight %}
@@ -272,6 +326,8 @@ table_env.register_java_function("split", "my.java.function.Split") my_table.join_lateral("split(a) as (word, length)").select("a, word, length") my_table.left_outer_join_lateral("split(a) as (word, length)").select("a, word, length") +# Register the python function. + # Use the table function in SQL with LATERAL and TABLE keywords. # CROSS JOIN a table function (equivalent to "join" in Table API). table_env.sql_query("SELECT a, word, length FROM MyTable, LATERAL TABLE(split(a)) as T(word, length)") diff --git a/docs/dev/table/udfs.zh.md b/docs/dev/table/udfs.zh.md index 42623ce4a691..b77e5f94af68 100644 --- a/docs/dev/table/udfs.zh.md +++ b/docs/dev/table/udfs.zh.md @@ -44,12 +44,12 @@ Scalar Functions If a required scalar function is not contained in the built-in functions, it is possible to define custom, user-defined scalar functions for both the Table API and SQL. A user-defined scalar functions maps zero, one, or multiple scalar values to a new scalar value. -In order to define a scalar function one has to extend the base class `ScalarFunction` in `org.apache.flink.table.functions` and implement (one or more) evaluation methods. The behavior of a scalar function is determined by the evaluation method. An evaluation method must be declared publicly and named `eval`. The parameter types and return type of the evaluation method also determine the parameter and return types of the scalar function. Evaluation methods can also be overloaded by implementing multiple methods named `eval`. Evaluation methods can also support variable arguments, such as `eval(String... strs)`. +
+
+In order to define a scalar function, one has to extend the base class `ScalarFunction` in `org.apache.flink.table.functions` and implement (one or more) evaluation methods. The behavior of a scalar function is determined by the evaluation method. An evaluation method must be declared publicly and named `eval`. The parameter types and return type of the evaluation method also determine the parameter and return types of the scalar function. Evaluation methods can also be overloaded by implementing multiple methods named `eval`. Evaluation methods can also support variable arguments, such as `eval(String... strs)`. The following example shows how to define your own hash code function, register it in the TableEnvironment, and call it in a query. Note that you can configure your scalar function via a constructor before it is registered: -
-
{% highlight java %} public class HashCode extends ScalarFunction { private int factor = 12; @@ -74,9 +74,29 @@ myTable.select("string, string.hashCode(), hashCode(string)"); // use the function in SQL API tableEnv.sqlQuery("SELECT string, hashCode(string) FROM MyTable"); {% endhighlight %} + +By default the result type of an evaluation method is determined by Flink's type extraction facilities. This is sufficient for basic types or simple POJOs but might be wrong for more complex, custom, or composite types. In these cases `TypeInformation` of the result type can be manually defined by overriding `ScalarFunction#getResultType()`. + +The following example shows an advanced example which takes the internal timestamp representation and also returns the internal timestamp representation as a long value. By overriding `ScalarFunction#getResultType()` we define that the returned long value should be interpreted as a `Types.TIMESTAMP` by the code generation. + +{% highlight java %} +public static class TimestampModifier extends ScalarFunction { + public long eval(long t) { + return t % 1000; + } + + public TypeInformation getResultType(Class[] signature) { + return Types.SQL_TIMESTAMP; + } +} +{% endhighlight %}
+In order to define a scalar function, one has to extend the base class `ScalarFunction` in `org.apache.flink.table.functions` and implement (one or more) evaluation methods. The behavior of a scalar function is determined by the evaluation method. An evaluation method must be declared publicly and named `eval`. The parameter types and return type of the evaluation method also determine the parameter and return types of the scalar function. Evaluation methods can also be overloaded by implementing multiple methods named `eval`. Evaluation methods can also support variable arguments, such as `@varargs def eval(str: String*)`. + +The following example shows how to define your own hash code function, register it in the TableEnvironment, and call it in a query. Note that you can configure your scalar function via a constructor before it is registered: + {% highlight scala %} // must be defined in static/object context class HashCode(factor: Int) extends ScalarFunction { @@ -95,14 +115,34 @@ myTable.select('string, hashCode('string)) tableEnv.registerFunction("hashCode", new HashCode(10)) tableEnv.sqlQuery("SELECT string, hashCode(string) FROM MyTable") {% endhighlight %} + +By default the result type of an evaluation method is determined by Flink's type extraction facilities. This is sufficient for basic types or simple POJOs but might be wrong for more complex, custom, or composite types. In these cases `TypeInformation` of the result type can be manually defined by overriding `ScalarFunction#getResultType()`. + +The following example shows an advanced example which takes the internal timestamp representation and also returns the internal timestamp representation as a long value. By overriding `ScalarFunction#getResultType()` we define that the returned long value should be interpreted as a `Types.TIMESTAMP` by the code generation. + +{% highlight scala %} +object TimestampModifier extends ScalarFunction { + def eval(t: Long): Long = { + t % 1000 + } + + override def getResultType(signature: Array[Class[_]]): TypeInformation[_] = { + Types.TIMESTAMP + } +} +{% endhighlight %}
+It supports to use both Java/Scala scalar functions and Python scalar functions in Python Table API and SQL. In order to define a Python scalar function, one can extend the base class `ScalarFunction` in `pyflink.table.udf` and implement an evaluation method. The behavior of a Python scalar function is determined by the evaluation method. An evaluation method must be named `eval`. Evaluation method can also support variable arguments, such as `eval(*args)`. + +The following example shows how to define your own Java and Python hash code functions, register them in the TableEnvironment, and call them in a query. Note that you can configure your scalar function via a constructor before it is registered: + {% highlight python %} ''' Java code: -// The java class must have a public no-argument constructor and can be founded in current java classloader. +// The Java class must have a public no-argument constructor and can be founded in current Java classloader. public class HashCode extends ScalarFunction { private int factor = 12; @@ -112,50 +152,64 @@ public class HashCode extends ScalarFunction { } ''' +class PyHashCode(ScalarFunction): + def __init__(self): + self.factor = 12 + + def eval(self, s): + return hash(s) * self.factor + table_env = BatchTableEnvironment.create(env) -# register the java function +# register the Java function table_env.register_java_function("hashCode", "my.java.function.HashCode") +# register the Python function +table_env.register_function("py_hash_code", udf(PyHashCode(), DataTypes.BIGINT(), DataTypes.BIGINT())) + # use the function in Python Table API -my_table.select("string, string.hashCode(), hashCode(string)") +my_table.select("string, bigint, string.hashCode(), hashCode(string), bigint.py_hash_code(), py_hash_code(bigint)") # use the function in SQL API -table_env.sql_query("SELECT string, hashCode(string) FROM MyTable") +table_env.sql_query("SELECT string, bigint, hashCode(string), py_hash_code(bigint) FROM MyTable") {% endhighlight %} -
-
-By default the result type of an evaluation method is determined by Flink's type extraction facilities. This is sufficient for basic types or simple POJOs but might be wrong for more complex, custom, or composite types. In these cases `TypeInformation` of the result type can be manually defined by overriding `ScalarFunction#getResultType()`. +There are many ways to define a Python scalar function besides extending the base class `ScalarFunction`. The following example shows the different ways to define a Python scalar function which takes two columns of bigint as input parameters and returns the sum of them as the result. -The following example shows an advanced example which takes the internal timestamp representation and also returns the internal timestamp representation as a long value. By overriding `ScalarFunction#getResultType()` we define that the returned long value should be interpreted as a `Types.TIMESTAMP` by the code generation. +{% highlight python %} +# option 1: extending the base class `ScalarFunction` +class Add(ScalarFunction): + def eval(self, i, j): + return i + j -
-
-{% highlight java %} -public static class TimestampModifier extends ScalarFunction { - public long eval(long t) { - return t % 1000; - } +add = udf(Add(), [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT()) - public TypeInformation getResultType(Class[] signature) { - return Types.SQL_TIMESTAMP; - } -} -{% endhighlight %} -
+# option 2: Python function +@udf(input_types=[DataTypes.BIGINT(), DataTypes.BIGINT()], result_type=DataTypes.BIGINT()) +def add(i, j): + return i + j -
-{% highlight scala %} -object TimestampModifier extends ScalarFunction { - def eval(t: Long): Long = { - t % 1000 - } +# option 3: lambda function +add = udf(lambda i, j: i + j, [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT()) - override def getResultType(signature: Array[Class[_]]): TypeInformation[_] = { - Types.TIMESTAMP - } -} +# option 4: callable function +class CallableAdd(object): + def __call__(self, i, j): + return i + j + +add = udf(CallableAdd(), [DataTypes.BIGINT(), DataTypes.BIGINT()], DataTypes.BIGINT()) + +# option 5: partial function +def partial_add(i, j, k): + return i + j + k + +add = udf(functools.partial(partial_add, j=1), [DataTypes.BIGINT(), DataTypes.BIGINT()], + DataTypes.BIGINT()) + +# register the Python function +table_env.register_function("add", add) +# use the function in Python Table API +my_table.select("add(a, b)") {% endhighlight %}
@@ -272,6 +326,8 @@ table_env.register_java_function("split", "my.java.function.Split") my_table.join_lateral("split(a) as (word, length)").select("a, word, length") my_table.left_outer_join_lateral("split(a) as (word, length)").select("a, word, length") +# Register the python function. + # Use the table function in SQL with LATERAL and TABLE keywords. # CROSS JOIN a table function (equivalent to "join" in Table API). table_env.sql_query("SELECT a, word, length FROM MyTable, LATERAL TABLE(split(a)) as T(word, length)") diff --git a/docs/ops/config.md b/docs/ops/config.md index 5a36efb0cc39..d70d26094f7e 100644 --- a/docs/ops/config.md +++ b/docs/ops/config.md @@ -225,6 +225,10 @@ You have to configure `jobmanager.archive.fs.dir` in order to archive terminated {% include generated/history_server_configuration.html %} +### Python + +{% include generated/python_configuration.html %} + ## Legacy - `mode`: Execution mode of Flink. Possible values are `legacy` and `new`. In order to start the legacy components, you have to specify `legacy` (DEFAULT: `new`). diff --git a/docs/ops/config.zh.md b/docs/ops/config.zh.md index cfbd9cc7c141..91c7bb2b9570 100644 --- a/docs/ops/config.zh.md +++ b/docs/ops/config.zh.md @@ -225,6 +225,10 @@ You have to configure `jobmanager.archive.fs.dir` in order to archive terminated {% include generated/history_server_configuration.html %} +### Python + +{% include generated/python_configuration.html %} + ## Legacy - `mode`: Execution mode of Flink. Possible values are `legacy` and `new`. In order to start the legacy components, you have to specify `legacy` (DEFAULT: `new`). diff --git a/flink-docs/pom.xml b/flink-docs/pom.xml index 384b09110807..9674c8a8d56b 100644 --- a/flink-docs/pom.xml +++ b/flink-docs/pom.xml @@ -99,6 +99,11 @@ under the License. flink-statebackend-rocksdb_${scala.binary.version} ${project.version} + + org.apache.flink + flink-python_${scala.binary.version} + ${project.version} + org.slf4j diff --git a/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java b/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java index 0ee34429a257..999f85a17814 100644 --- a/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java +++ b/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java @@ -63,7 +63,8 @@ public class ConfigOptionsDocGenerator { new OptionsClassLocation("flink-mesos", "org.apache.flink.mesos.runtime.clusterframework"), new OptionsClassLocation("flink-metrics/flink-metrics-prometheus", "org.apache.flink.metrics.prometheus"), new OptionsClassLocation("flink-state-backends/flink-statebackend-rocksdb", "org.apache.flink.contrib.streaming.state"), - new OptionsClassLocation("flink-table/flink-table-api-java", "org.apache.flink.table.api.config") + new OptionsClassLocation("flink-table/flink-table-api-java", "org.apache.flink.table.api.config"), + new OptionsClassLocation("flink-python", "org.apache.flink.python") }; static final Set EXCLUSIONS = new HashSet<>(Arrays.asList( 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 index 6c0ee52d3e1e..5e88320b38aa 100644 --- a/flink-python/src/main/java/org/apache/flink/python/PythonOptions.java +++ b/flink-python/src/main/java/org/apache/flink/python/PythonOptions.java @@ -34,9 +34,10 @@ public class PythonOptions { 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"); + .withDescription("The maximum number of elements to include in a bundle for Python " + + "user-defined function execution. The elements are processed asynchronously. " + + "One bundle of elements are processed before processing the next bundle of elements. " + + "A larger value can improve the throughput, but at the cost of more memory usage and higher latency."); /** * The maximum time to wait before finalising a bundle (in milliseconds). @@ -44,5 +45,7 @@ public class PythonOptions { 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)."); + .withDescription("Sets the waiting timeout(in milliseconds) before processing a bundle for " + + "Python user-defined function execution. The timeout defines how long the elements of a bundle will be " + + "buffered before being processed. Lower timeouts lead to lower tail latencies, but may affect throughput."); } From 2e9f16edab750656f2327d6aaf84482c0d5bc916 Mon Sep 17 00:00:00 2001 From: Zhanchun Zhang Date: Sun, 20 Oct 2019 21:04:02 +0800 Subject: [PATCH 154/746] [hotfix][doc] fix typo in RuntimeContext This closes #9945. --- .../org/apache/flink/api/common/functions/RuntimeContext.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java index 4a2201144548..efebf999f074 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java @@ -117,7 +117,7 @@ public interface RuntimeContext { ExecutionConfig getExecutionConfig(); /** - * Gets the ClassLoader to load classes that were are not in system's classpath, but are part of the + * Gets the ClassLoader to load classes that are not in system's classpath, but are part of the * jar file of a user job. * * @return The ClassLoader for user code classes. From ccfb82e5cebf15ebc2a486d1ed855bbf2dfe81eb Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Sat, 12 Oct 2019 16:25:52 +0800 Subject: [PATCH 155/746] [hotfix][task] Fix the code formatting in StreamTask --- .../org/apache/flink/streaming/runtime/tasks/StreamTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 61b1bce3801a..864be69f793a 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 @@ -281,7 +281,7 @@ protected void processInput(DefaultActionContext context) throws Exception { if (status == InputStatus.END_OF_INPUT) { context.allActionsCompleted(); } - else if (status == InputStatus.NOTHING_AVAILABLE){ + else if (status == InputStatus.NOTHING_AVAILABLE) { SuspendedMailboxDefaultAction suspendedDefaultAction = context.suspendDefaultAction(); inputProcessor.isAvailable().thenRun(suspendedDefaultAction::resume); } From 09c2c986a04d4090677ab17c3ad2217269116de2 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Mon, 14 Oct 2019 17:25:06 +0800 Subject: [PATCH 156/746] [hotfix][runtime] Remove legacy NullableAsyncDataInput class --- .../runtime/io/NullableAsyncDataInput.java | 39 ------------------- 1 file changed, 39 deletions(-) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/NullableAsyncDataInput.java 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 deleted file mode 100644 index 4b0095547968..000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/NullableAsyncDataInput.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.io; - -import org.apache.flink.annotation.Internal; - -import javax.annotation.Nullable; - -/** - * 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 AvailabilityProvider { - /** - * Poll the next element. This method should be non blocking. - * - * @return {@code null} will be returned if there is no data to return or - * if {@link #isFinished()} returns true. Otherwise return {@code element}. - */ - @Nullable - T pollNextNullable() throws Exception; -} From 34e53d1c5da87a0ec99583eaf5f914372ee77e89 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Sat, 12 Oct 2019 17:01:54 +0800 Subject: [PATCH 157/746] [hotfix][network] Remove optional class field from LocalBufferPool --- .../io/network/buffer/BufferPoolFactory.java | 9 +++--- .../io/network/buffer/LocalBufferPool.java | 28 +++++++++---------- .../io/network/buffer/NetworkBufferPool.java | 15 +++++++--- .../partition/ResultPartitionFactory.java | 5 ++-- .../network/buffer/NetworkBufferPoolTest.java | 9 +++--- 5 files changed, 35 insertions(+), 31 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactory.java index 0e77638cdaa5..7d623f8c23af 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactory.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.io.network.buffer; import java.io.IOException; -import java.util.Optional; /** * A factory for buffer pools. @@ -40,7 +39,7 @@ public interface BufferPoolFactory { BufferPool createBufferPool(int numRequiredBuffers, int maxUsedBuffers) throws IOException; /** - * Tries to create a buffer pool with an optional owner, which is guaranteed to provide at least the + * Tries to create a buffer pool with an owner, which is guaranteed to provide at least the * number of required buffers. * *

The buffer pool is of dynamic size with at least numRequiredBuffers buffers. @@ -49,10 +48,10 @@ public interface BufferPoolFactory { * minimum number of network buffers in this pool * @param maxUsedBuffers * maximum number of network buffers this pool offers - * @param owner - * the optional owner of this buffer pool to release memory when needed + * @param bufferPoolOwner + * the owner of this buffer pool to release memory when needed */ - BufferPool createBufferPool(int numRequiredBuffers, int maxUsedBuffers, Optional owner) throws IOException; + BufferPool createBufferPool(int numRequiredBuffers, int maxUsedBuffers, BufferPoolOwner bufferPoolOwner) throws IOException; /** * Destroy callback for updating factory book keeping. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java index e65b6f933fd1..fb57da5499cd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java @@ -25,9 +25,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.io.IOException; import java.util.ArrayDeque; -import java.util.Optional; import static org.apache.flink.util.Preconditions.checkArgument; @@ -86,7 +87,8 @@ class LocalBufferPool implements BufferPool { private boolean isDestroyed; - private final Optional owner; + @Nullable + private final BufferPoolOwner bufferPoolOwner; /** * Local buffer pool based on the given networkBufferPool with a minimal number of @@ -98,7 +100,7 @@ class LocalBufferPool implements BufferPool { * minimum number of network buffers */ LocalBufferPool(NetworkBufferPool networkBufferPool, int numberOfRequiredMemorySegments) { - this(networkBufferPool, numberOfRequiredMemorySegments, Integer.MAX_VALUE, Optional.empty()); + this(networkBufferPool, numberOfRequiredMemorySegments, Integer.MAX_VALUE, null); } /** @@ -114,7 +116,7 @@ class LocalBufferPool implements BufferPool { */ LocalBufferPool(NetworkBufferPool networkBufferPool, int numberOfRequiredMemorySegments, int maxNumberOfMemorySegments) { - this(networkBufferPool, numberOfRequiredMemorySegments, maxNumberOfMemorySegments, Optional.empty()); + this(networkBufferPool, numberOfRequiredMemorySegments, maxNumberOfMemorySegments, null); } /** @@ -127,14 +129,14 @@ class LocalBufferPool implements BufferPool { * minimum number of network buffers * @param maxNumberOfMemorySegments * maximum number of network buffers to allocate - * @param owner - * the optional owner of this buffer pool to release memory when needed + * @param bufferPoolOwner + * the owner of this buffer pool to release memory when needed */ LocalBufferPool( NetworkBufferPool networkBufferPool, int numberOfRequiredMemorySegments, int maxNumberOfMemorySegments, - Optional owner) { + @Nullable BufferPoolOwner bufferPoolOwner) { checkArgument(maxNumberOfMemorySegments >= numberOfRequiredMemorySegments, "Maximum number of memory segments (%s) should not be smaller than minimum (%s).", maxNumberOfMemorySegments, numberOfRequiredMemorySegments); @@ -150,7 +152,7 @@ class LocalBufferPool implements BufferPool { this.numberOfRequiredMemorySegments = numberOfRequiredMemorySegments; this.currentPoolSize = numberOfRequiredMemorySegments; this.maxNumberOfMemorySegments = maxNumberOfMemorySegments; - this.owner = owner; + this.bufferPoolOwner = bufferPoolOwner; } // ------------------------------------------------------------------------ @@ -231,8 +233,6 @@ private MemorySegment requestMemorySegment(boolean isBlocking) throws Interrupte synchronized (availableMemorySegments) { returnExcessMemorySegments(); - boolean askToRecycle = owner.isPresent(); - // fill availableMemorySegments with at least one element, wait if required while (availableMemorySegments.isEmpty()) { if (isDestroyed) { @@ -248,8 +248,8 @@ private MemorySegment requestMemorySegment(boolean isBlocking) throws Interrupte } } - if (askToRecycle) { - owner.get().releaseMemory(1); + if (bufferPoolOwner != null) { + bufferPoolOwner.releaseMemory(1); } if (isBlocking) { @@ -366,8 +366,8 @@ public void setNumBuffers(int numBuffers) throws IOException { // If there is a registered owner and we have still requested more buffers than our // size, trigger a recycle via the owner. - if (owner.isPresent() && numExcessBuffers > 0) { - owner.get().releaseMemory(numExcessBuffers); + if (bufferPoolOwner != null && numExcessBuffers > 0) { + bufferPoolOwner.releaseMemory(numExcessBuffers); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java index 1362d72ccb9b..c08e056d4df5 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java @@ -39,7 +39,6 @@ import java.util.Collection; import java.util.HashSet; import java.util.List; -import java.util.Optional; import java.util.Set; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.TimeUnit; @@ -263,11 +262,19 @@ public int countBuffers() { @Override public BufferPool createBufferPool(int numRequiredBuffers, int maxUsedBuffers) throws IOException { - return createBufferPool(numRequiredBuffers, maxUsedBuffers, Optional.empty()); + return internalCreateBufferPool(numRequiredBuffers, maxUsedBuffers, null); } @Override - public BufferPool createBufferPool(int numRequiredBuffers, int maxUsedBuffers, Optional owner) throws IOException { + public BufferPool createBufferPool(int numRequiredBuffers, int maxUsedBuffers, BufferPoolOwner bufferPoolOwner) throws IOException { + return internalCreateBufferPool(numRequiredBuffers, maxUsedBuffers, bufferPoolOwner); + } + + private BufferPool internalCreateBufferPool( + int numRequiredBuffers, + int maxUsedBuffers, + @Nullable BufferPoolOwner bufferPoolOwner) throws IOException { + // It is necessary to use a separate lock from the one used for buffer // requests to ensure deadlock freedom for failure cases. synchronized (factoryLock) { @@ -290,7 +297,7 @@ public BufferPool createBufferPool(int numRequiredBuffers, int maxUsedBuffers, O // We are good to go, create a new buffer pool and redistribute // non-fixed size buffers. LocalBufferPool localBufferPool = - new LocalBufferPool(this, numRequiredBuffers, maxUsedBuffers, owner); + new LocalBufferPool(this, numRequiredBuffers, maxUsedBuffers, bufferPoolOwner); allBufferPools.add(localBufferPool); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactory.java index 4933a4ee9c40..e61653bd9f28 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactory.java @@ -35,7 +35,6 @@ import java.io.File; import java.io.IOException; -import java.util.Optional; /** * Factory for {@link ResultPartition} to use in {@link NettyShuffleEnvironment}. @@ -183,14 +182,14 @@ private static void releasePartitionsQuietly(ResultSubpartition[] partitions, in FunctionWithException createBufferPoolFactory( int numberOfSubpartitions, ResultPartitionType type) { - return p -> { + return bufferPoolOwner -> { int maxNumberOfMemorySegments = type.isBounded() ? numberOfSubpartitions * networkBuffersPerChannel + floatingNetworkBuffersPerGate : Integer.MAX_VALUE; // If the partition type is back pressure-free, we register with the buffer pool for // callbacks to release memory. return bufferPoolFactory.createBufferPool(numberOfSubpartitions, maxNumberOfMemorySegments, - type.hasBackPressure() ? Optional.empty() : Optional.of(p)); + type.hasBackPressure() ? null : bufferPoolOwner); }; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java index 42871156b66c..9659c7cc3e31 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java @@ -32,7 +32,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Optional; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.CoreMatchers.nullValue; @@ -317,9 +316,9 @@ public void testRequestMemorySegmentsExceptionDuringBufferRedistribution() throw List memorySegments = Collections.emptyList(); BufferPool bufferPool = networkBufferPool.createBufferPool(1, numBuffers, // make releaseMemory calls always fail: - Optional.of(numBuffersToRecycle -> { + numBuffersToRecycle -> { throw new TestIOException(); - })); + }); try { // take all but one buffer @@ -356,9 +355,9 @@ public void testCreateBufferPoolExceptionDuringBufferRedistribution() throws IOE final List buffers = new ArrayList<>(numBuffers); BufferPool bufferPool = networkBufferPool.createBufferPool(1, numBuffers, - Optional.of(numBuffersToRecycle -> { + numBuffersToRecycle -> { throw new TestIOException(); - })); + }); try { From 47e9a82a4d3be085bd8bc906e4de3765e99a3a7b Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Fri, 18 Oct 2019 18:27:36 +0800 Subject: [PATCH 158/746] [FLINK-14394][network] Remove unnecessary interface method BufferProvider#requestBufferBlocking Currently BufferProvider#requestBufferBlocking method is only used for unit tests, so we could refactor the related tests to use methods of BufferProvider#requestBufferBuilderBlocking or BufferProvider#requestBuffer instead. Then we could remove this legacy method completely to clean up the interface. --- .../runtime/io/network/buffer/BufferProvider.java | 8 -------- .../runtime/io/network/buffer/LocalBufferPool.java | 5 ----- .../network/buffer/LocalBufferPoolDestroyTest.java | 2 +- .../io/network/buffer/LocalBufferPoolTest.java | 8 +++++--- .../runtime/io/network/buffer/NoOpBufferPool.java | 5 ----- .../io/network/netty/CancelPartitionRequestTest.java | 12 ++++++++---- .../partition/consumer/RemoteInputChannelTest.java | 6 ++++-- .../io/network/util/TestPooledBufferProvider.java | 3 +-- 8 files changed, 19 insertions(+), 30 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferProvider.java index f652d6b13b98..1217b08b01d5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferProvider.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferProvider.java @@ -35,14 +35,6 @@ public interface BufferProvider { */ Buffer requestBuffer() throws IOException; - /** - * Returns a {@link Buffer} instance from the buffer provider. - * - *

If there is no buffer available, the call will block until one becomes available again or the - * buffer provider has been destroyed. - */ - Buffer requestBufferBlocking() throws IOException, InterruptedException; - /** * Returns a {@link BufferBuilder} instance from the buffer provider. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java index fb57da5499cd..afa76a5bcecc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java @@ -205,11 +205,6 @@ public Buffer requestBuffer() throws IOException { } } - @Override - public Buffer requestBufferBlocking() throws IOException, InterruptedException { - return toBuffer(requestMemorySegment(true)); - } - @Override public BufferBuilder requestBufferBuilderBlocking() throws IOException, InterruptedException { return toBufferBuilder(requestMemorySegment(true)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolDestroyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolDestroyTest.java index 116884c30717..d1432b43c8d3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolDestroyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolDestroyTest.java @@ -136,7 +136,7 @@ public void run() { String msg = "Test assumption violated: expected no available buffer"; assertNull(msg, bufferPool.requestBuffer()); - bufferPool.requestBufferBlocking(); + bufferPool.requestBufferBuilderBlocking(); } catch (Exception t) { asyncException.set(t); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java index 2b7fa7c149d5..9c71c89d88e4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java @@ -40,6 +40,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import static org.apache.flink.util.Preconditions.checkNotNull; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; @@ -306,7 +307,8 @@ public List call() throws Exception { // Request all available buffers for (int i = 0; i < numberOfBuffers; i++) { - requested.add(localBufferPool.requestBufferBlocking()); + final Buffer buffer = checkNotNull(localBufferPool.requestBuffer()); + requested.add(buffer); } // Notify that we've requested all buffers @@ -315,7 +317,7 @@ public List call() throws Exception { // Try to request the next buffer (but pool should be destroyed either right before // the request or more likely during the request). try { - localBufferPool.requestBufferBlocking(); + localBufferPool.requestBufferBuilderBlocking(); fail("Call should have failed with an IllegalStateException"); } catch (IllegalStateException e) { @@ -445,7 +447,7 @@ private BufferRequesterTask(BufferProvider bufferProvider, int numBuffersToReque public Boolean call() throws Exception { try { for (int i = 0; i < numBuffersToRequest; i++) { - Buffer buffer = bufferProvider.requestBufferBlocking(); + Buffer buffer = checkNotNull(bufferProvider.requestBuffer()); buffer.recycleBuffer(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NoOpBufferPool.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NoOpBufferPool.java index 04c9c042f321..b9c5c5fa0b39 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NoOpBufferPool.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NoOpBufferPool.java @@ -38,11 +38,6 @@ public Buffer requestBuffer() throws IOException { throw new UnsupportedOperationException(); } - @Override - public Buffer requestBufferBlocking() throws IOException, InterruptedException { - throw new UnsupportedOperationException(); - } - @Override public BufferBuilder requestBufferBuilderBlocking() throws IOException, InterruptedException { throw new UnsupportedOperationException(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CancelPartitionRequestTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CancelPartitionRequestTest.java index 4b958f21ccca..f63849cc40a9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CancelPartitionRequestTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CancelPartitionRequestTest.java @@ -188,10 +188,14 @@ public InfiniteSubpartitionView(BufferProvider bufferProvider, CountDownLatch sy @Nullable @Override - public BufferAndBacklog getNextBuffer() throws IOException, InterruptedException { - Buffer buffer = bufferProvider.requestBufferBlocking(); - buffer.setSize(buffer.getMaxCapacity()); // fake some data - return new BufferAndBacklog(buffer, true, 0, false); + public BufferAndBacklog getNextBuffer() throws IOException { + Buffer buffer = bufferProvider.requestBuffer(); + if (buffer != null) { + buffer.setSize(buffer.getMaxCapacity()); // fake some data + return new BufferAndBacklog(buffer, true, 0, false); + } else { + return null; + } } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java index c3b2fd16ad5f..41a57fc1ba1b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java @@ -63,7 +63,9 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; +import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.buildSingleBuffer; import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createSingleInputGate; +import static org.apache.flink.util.Preconditions.checkNotNull; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.hasProperty; import static org.hamcrest.Matchers.is; @@ -730,7 +732,7 @@ public void testFailureInNotifyBufferAvailable() throws Exception { networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers); inputGate.setBufferPool(bufferPool); - buffer = bufferPool.requestBufferBlocking(); + buffer = checkNotNull(bufferPool.requestBuffer()); // trigger subscription to buffer pool failingRemoteIC.onSenderBacklog(1); @@ -954,7 +956,7 @@ public void testConcurrentRecycleAndRelease2() throws Exception { final Callable bufferPoolInteractionsTask = () -> { for (int i = 0; i < retries; ++i) { - Buffer buffer = bufferPool.requestBufferBlocking(); + Buffer buffer = buildSingleBuffer(bufferPool.requestBufferBuilderBlocking()); buffer.recycleBuffer(); } return null; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPooledBufferProvider.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPooledBufferProvider.java index bae08300c0c9..45aea352dcc7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPooledBufferProvider.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPooledBufferProvider.java @@ -65,8 +65,7 @@ public Buffer requestBuffer() throws IOException { return bufferFactory.create(); } - @Override - public Buffer requestBufferBlocking() throws IOException, InterruptedException { + private Buffer requestBufferBlocking() throws IOException, InterruptedException { Buffer buffer = buffers.poll(); if (buffer != null) { return buffer; From 6dfcbc0bf3769a1a36ef8d19d23260485dbee4c8 Mon Sep 17 00:00:00 2001 From: joe <15692118798@163.com> Date: Fri, 27 Sep 2019 18:12:46 +0800 Subject: [PATCH 159/746] [FLINK-13485][docs-zh] Translate "Table API Example Walkthrough" page into Chinese Thisc closes #9784 --- .../walkthroughs/table_api.zh.md | 170 +++++++++--------- 1 file changed, 85 insertions(+), 85 deletions(-) diff --git a/docs/getting-started/walkthroughs/table_api.zh.md b/docs/getting-started/walkthroughs/table_api.zh.md index f0cbb6222d82..5a3f95b799b4 100644 --- a/docs/getting-started/walkthroughs/table_api.zh.md +++ b/docs/getting-started/walkthroughs/table_api.zh.md @@ -24,35 +24,34 @@ specific language governing permissions and limitations under the License. --> -Apache Flink offers a Table API as a unified, relational API for batch and stream processing, i.e., queries are executed with the same semantics on unbounded, real-time streams or bounded, batch data sets and produce the same results. -The Table API in Flink is commonly used to ease the definition of data analytics, data pipelining, and ETL applications. - +Apache Filnk 提供 Table API 作为批处理和流处理统一的关系型API, +即查询在无界实时流或有界批数据集上以相同的语义执行,并产生相同的结果。 +Flink 中的 Table API 通常用于简化数据分析,数据流水线和 ETL 应用程序的定义。 * This will be replaced by the TOC {:toc} -## What Will You Be Building? - -In this tutorial, you will learn how to build a continuous ETL pipeline for tracking financial transactions by account over time. -You will start by building your report as a nightly batch job, and then migrate to a streaming pipeline. +## 接下来你会构建什么? -## Prerequisites +在本教程中,你将学习如何构建连续的 ETL 流水线,以便按账户随时跟踪金融交易。 +首先你将报表构建为每晚执行的批处理作业,然后迁移到流式管道。 -This walkthrough assumes that you have some familiarity with Java or Scala, but you should be able to follow along even if you are coming from a different programming language. -It also assumes that you are familiar with basic relational concepts such as `SELECT` and `GROUP BY` clauses. +## 先决条件 -## Help, I’m Stuck! +本演练假设你对 Java 和 Scala 有一定的了解,但即便你使用其他编程语言,相信也可以学会。 +它还假定你熟悉基本的关系概念比如 `SELECT` 和 `GROUP BY` 子句。 -If you get stuck, check out the [community support resources](https://flink.apache.org/community.html). -In particular, Apache Flink's [user mailing list](https://flink.apache.org/community.html#mailing-lists) is consistently ranked as one of the most active of any Apache project and a great way to get help quickly. +## 救命,我被困住了! -## How To Follow Along +如果你被难题困住了,可以在[社区](https://flink.apache.org/community.html)寻求帮助。 +值得一提的是,Apache Flink 的[用户邮件列表](https://flink.apache.org/community.html#mailing-lists)一直是最活跃的 Apache 项目之一,也是一个快速获得帮助的好途径。 -If you want to follow along, you will require a computer with: +## 如何跟进 +如果想要继续,你的电脑需要安装: * Java 8 * Maven -A provided Flink Maven Archetype will create a skeleton project with all the necessary dependencies quickly: +现成的 Flink Maven Archetype 可以快速创建一个具有所有必要依赖的框架项目:

@@ -87,13 +86,13 @@ $ mvn archetype:generate \ {% unless site.is_stable %}

- Note: For Maven 3.0 or higher, it is no longer possible to specify the repository (-DarchetypeCatalog) via the commandline. If you wish to use the snapshot repository, you need to add a repository entry to your settings.xml. For details about this change, please refer to Maven's official document + 注意:对于 Maven 3.0 或更高版本, 已经不再需要在命令行指定仓库(-DarchetypeCatalog)了。 如果希望使用快照(snapshot)仓库,需要在你的 setting.xml 文件中加入仓库入口。有关此变更的详细信息,请参阅 Maven 官方文档

{% endunless %} -You can edit the `groupId`, `artifactId` and `package` if you like. With the above parameters, -Maven will create a project with all the dependencies to complete this tutorial. -After importing the project into your editor, you will see a file with the following code which you can run directly inside your IDE. +你可以根据自己的意愿修改 `groupId`、`artifactId` 和 `package` 参数。通过使用以上参数, +Maven 将创建一个拥有全部所需依赖的项目来完成本教程。 +把项目导入编辑器后,你会看到一个包含以下代码的文件,你可以在 IDE 中直接运行它。
@@ -132,14 +131,14 @@ env.execute("Spend Report")
-## Breaking Down The Code +## 代码详解 -#### The Execution Environment +#### 运行环境 -The first two lines set up your `ExecutionEnvironment`. -The execution environment is how you can set properties for your Job, specify whether you are writing a batch or a streaming application, and create your sources. -This walkthrough begins with the batch environment since you are building a periodic batch report. -It is then wrapped in a `BatchTableEnvironment` to have full access to the Table API. +前两行设置了你的 `ExecutionEnvironment`。 +运行环境用来设置作业的属性、指定应用是批处理还是流处理,以及创建数据源。 +由于你正在建立一个定时的批处理报告,本教程以批处理环境作为开始。 +然后将其包装进 `BatchTableEnvironment` 中从而能够使用所有的 Tabel API。
@@ -158,12 +157,12 @@ val tEnv = BatchTableEnvironment.create(env)
-#### Registering Tables +#### 注册表 -Next, tables are registered in the execution environment that you can use to connect to external systems for reading and writing both batch and streaming data. -A table source provides access to data stored in external systems; such as a database, a key-value store, a message queue, or a file system. -A table sink emits a table to an external storage system. -Depending on the type of source and sink, they support different formats such as CSV, JSON, Avro, or Parquet. +接下来,表将会被注册到运行环境之中,这样你就可以用它们连接外部系统以读取或写入批数据或流数据。 +source 提供对存储在外部系统中的数据的访问;例如数据库、键-值存储、消息队列或文件系统。 +sink 则将表中的数据发送到外部存储系统。 +根据 source 或 sink 的类型,它们支持不同的格式,如 CSV、JSON、Avro 或 Parquet。
@@ -181,16 +180,16 @@ tEnv.registerTableSink("spend_report", new SpendReportTableSink)
-Two tables are registered; a transaction input table, and a spend report output table. -The transactions (`transactions`) table lets us read credit card transactions, which contain account ID's (`accountId`), timestamps (`timestamp`), and US$ amounts (`amount`). -In this tutorial, the table is backed by data generated in memory to avoid any dependencies on external systems. -In practice, the `BoundedTransactionTableSource` may be backed by a filesystem, a database, or any other static source. -The spend report (`spend_report`) table logs each row with log level **INFO**, instead of writing to persistent storage, so you can easily see your results. +上例代码注册了两张表。交易表作为输入表,支出报告表作为输出表。 +我们可以从交易(`transactions`)表中读取信用卡的交易记录,其中包含了账户 ID(`accountId`)字段、时间戳(`timestamp`)字段和交易金额(`amount`)字段。 +本教程中,该表使用内存中的数据,以避免对外部系统的任何依赖。 +而在实际情况下,`BoundedTransactionTableSource` 可能来源于文件系统、数据库或任何静态数据源。 +支出报告表 `spend_report` 用 **INFO** 日志级别将表的每一行数据记录到日志,而不是写入持久化存储,所以你可以很容易地查看结果。 -#### Registering A UDF +#### 注册 UDF -Along with the tables, a [user-defined function]({{ site.baseurl }}/dev/table/udfs.html) is registered for working with timestamps. -This function takes a timestamp and rounds it down to the nearest hour. +一个用来处理时间戳的[自定义函数]({{ site.baseurl }}/zh/dev/table/udfs.html)随表一起被注册到tEnv中。 +此函数将时间戳向下舍入到最接近的小时。
@@ -206,10 +205,10 @@ val truncateDateToHour = new TruncateDateToHour
-#### The Query +#### 查询 -With the environment configured and tables registered, you are ready to build your first application. -From the `TableEnvironment` you can `scan` an input table to read its rows and then write those results into an output table using `insertInto`. +完成配置环境和注册表后,你已准备好构建第一个应用程序。 +从 `TableEnvironment` 中,你可以 `scan` 一个输入表读取其中的行,然后用 `insertInto` 把这些数据写到输出表中。
@@ -229,12 +228,13 @@ tEnv
-Initially, the Job reads all transactions and logs them out with log level **INFO**. +最初,作业读取所有的交易记录并用 **INFO** 日志级别将其记录下来。 -#### Execute +#### 运行 + +Flink 应用是延迟构建的,只有完全定义好之后才交付集群运行。 +你可以调用 `ExecutionEnvironment#execute` 来开始作业的执行并给它取一个名字。 -Flink applications are built lazily and shipped to the cluster for execution only once fully formed. -You call `ExecutionEnvironment#execute` to begin the execution of your Job by giving it a name.
@@ -250,14 +250,13 @@ env.execute("Spend Report")
-## Attempt One - -Now with the skeleton of a Job set-up, you are ready to add some business logic. -The goal is to build a report that shows the total spend for each account across each hour of the day. -Just like a SQL query, Flink can select the required fields and group by your keys. -Because the timestamp field has millisecond granularity, you can use the UDF to round it down to the nearest hour. -Finally, select all the fields, summing the total spend per account-hour pair with the built-in `sum` [aggregate function]({{ site.baseurl }}/dev/table/functions.html#aggregate-functions). +## 尝试一下 +现在有了作业设置的框架,你就可以添加一些业务逻辑了。 +目标是建立一个报表来显示每天每小时每个账户的总支出。 +就像一个 SQL 查询一样,Flink 可以选取所需的字段并且按键分组。 +由于时间戳字段具有毫秒的粒度,你可以使用自定义函数将其舍入到最近的小时。 +最后,选取所有的字段,用内建的 `sum` [聚合函数]({{ site.baseurl }}/zh/dev/table/functions.html#aggregate-functions)函数合计每一个账户每小时的支出。
{% highlight java %} @@ -282,10 +281,9 @@ tEnv
-This query consumes all records from the `transactions` table, calculates the report, and outputs the results in an efficient, scalable manner. - +这个查询处理了 `transactions` 表中的所有记录,计算报告,并以高效、可扩展的方式输出结果。 {% highlight raw %} -# Query 1 output showing account id, timestamp, and amount +# 查询 1 的输出显示了账户 id、时间戳和消费总额。 > 1, 2019-01-01 00:00:00.0, $567.87 > 2, 2019-01-01 00:00:00.0, $726.23 @@ -299,11 +297,11 @@ This query consumes all records from the `transactions` table, calculates the re > 2, 2019-01-01 04:00:00.0, $760.76 {% endhighlight %} -## Adding Windows +## 添加窗口 -Grouping data based on time is a typical operation in data processing, especially when working with infinite streams. -A grouping based on time is called a [window]({{ site.baseurl }} /dev/stream/operators/windows.html) and Flink offers flexible windowing semantics. -The most basic type of window is called a `Tumble` window, which has a fixed size and whose buckets do not overlap. +根据时间进行分组在数据处理中是一种很常见的方式,特别是在处理无限的数据流时。 +基于时间的分组称为[窗口]({{ site.baseurl }}/zh/dev/stream/operators/windows.html) ,Flink 提供了灵活的窗口语义。 +其中最基础的是 `Tumble` window (滚动窗口),它具有固定大小且窗口之间不重叠。
@@ -329,16 +327,16 @@ tEnv
-This defines your application as using one hour tumbling windows based on the timestamp column. -So a row with timestamp `2019-06-01 01:23:47` is put in the `2019-06-01 01:00:00` window. +你的应用将会使用基于时间戳字段的一小时的滚动窗口。 +因此时间戳是 `2019-06-01 01:23:47` 的行被放入 `2019-06-01 01:00:00` 这个时间窗口之中。 -Aggregations based on time are unique because time, as opposed to other attributes, generally moves forward in a continuous streaming application. -In a batch context, windows offer a convenient API for grouping records by a timestamp attribute. +在持续的流式应用中,基于时间的聚合结果是唯一的,因为相较于其他属性,时间通常会向前移动。 +在批处理环境中,窗口提供了一个方便的 API,用于按时间戳属性对记录进行分组。 -Running the updated query will produce identical results as before. +运行这个更新过的查询将会得到和之前一样的结果。 {% highlight raw %} -# Query 2 output showing account id, timestamp, and amount +# 查询 2 的输出显示了账户 id、时间戳和消费总额 > 1, 2019-01-01 00:00:00.0, $567.87 > 2, 2019-01-01 00:00:00.0, $726.23 @@ -352,13 +350,13 @@ Running the updated query will produce identical results as before. > 2, 2019-01-01 04:00:00.0, $760.76 {% endhighlight %} -## Once More, With Streaming! +## 通过流处理的方式再来一次! -Because Flink's Table API offers consistent syntax and semantics for both batch and streaming, migrating from one to the other requires just two steps. +因为 Flink 的 Table API 为批处理和流处理提供了相同的语法和语义,从一种方式迁移到另一种方式只需要两步。 -The first step is to replace the batch `ExecutionEnvironment` with its streaming counterpart, `StreamExecutionEnvironment`, which creates a continuous streaming Job. -It includes stream-specific configurations, such as the time characteristic, which when set to [event time]({{ site.baseurl }}/dev/event_time.html) guarantees consistent results even when faced with out-of-order events or a Job failure. -This is what will be used by your `Tumble` window when grouping records. +第一步是把批处理的 `ExecutionEnvironment` 替换成流处理对应的 `StreamExecutionEnvironment`,后者创建连续的流作业。 +它包含特定于流处理的配置,比如时间特性。当这个属性被设置成 [事件时间]({{ site.baseurl }}/zh/dev/event_time.html)时,它能保证即使遭遇乱序事件或者作业失败的情况也能输出一致的结果。 +滚动窗口在对数据进行分组时就运用了这个特性。
@@ -380,10 +378,10 @@ val tEnv = StreamTableEnvironment.create(env)
-The second step is to migrate from a bounded data source to an infinite data source. -The project comes with an `UnboundedTransactionTableSource` that continuously creates transaction events in real-time. -Similar to the `BoundedTransactionTableSource` this table is backed by data generated in memory to avoid any dependencies on external systems. -In practice, this table might read from a streaming source such as Apache Kafka, AWS Kinesis, or Pravega. +第二步就是把有界的数据源替换成无限的数据源。 +这个项目通过 `UnboundedTransactionTableSource` 持续不断地实时生成交易事件。 +与 `BoundedTransactionTableSource` 一样,这个表也是通过在内存中生成数据从而不依赖外部系统。 +在实践中,这个表可能从一个流式数据源中读取数据,比如 Apache Kafka、AWS Kinesis 或者 Pravega。
@@ -399,27 +397,29 @@ tEnv.registerTableSource("transactions", new UnboundedTransactionTableSource)
-And that's it, a fully functional, stateful, distributed streaming application! -The query continuously consumes the stream of transactions, computes the hourly spendings, and emits results as soon as they are ready. -Since the input is unbounded, the query keeps running until it is manually stopped. -And because the Job uses time window-based aggregations, Flink can perform specific optimizations such as state clean up when the framework knows that no more records will arrive for a particular window. +这就是一个功能齐全、有状态的分布式流式应用! +这个查询会持续处理交易流,计算每小时的消费额,然后实时输出结果。 +由于输入是无界的,因此查询将一直运行,直到手动停止为止。 +因为这个作业使用了基于时间窗口的聚合,Flink 可以使用一些特定的优化,比如当系统知道一个特定的窗口不会再有新的数据到来,它就会对状态进行清理。 {% highlight raw %} -# Query 3 output showing account id, timestamp, and amount +# 查询 3 的输出显示了账户 id、时间戳消费总额 -# These rows are calculated continuously over the hour -# and output immediately at the end of the hour > 1, 2019-01-01 00:00:00.0, $567.87 > 2, 2019-01-01 00:00:00.0, $726.23 -# Flink begins computing these rows as soon as -# as the first record for the window arrives +# 这些行是在这一小时中连续计算的 +# 并在这一小时结束时立刻输出 + > 1, 2019-01-01 01:00:00.0, $686.87 > 2, 2019-01-01 01:00:00.0, $810.06 +# 当接收到该窗口的第一条数据时 +# Flink 就开始计算了 + {% endhighlight %} -## Final Application +## 最终程序
From 711f4ce258fd14d2718b2642ef4ee27803c5ca09 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 21 Oct 2019 09:28:57 +0200 Subject: [PATCH 160/746] [FLINK-8822] In config.sh, don't use extended regex for rotation The -E parameter is a FreeBSD/MacOS feature while -r is GNU. It's hard to know which of -E/-r to use. --- flink-dist/src/main/flink-bin/bin/config.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-dist/src/main/flink-bin/bin/config.sh b/flink-dist/src/main/flink-bin/bin/config.sh index 4dc57dfe655d..7d6543dab417 100755 --- a/flink-dist/src/main/flink-bin/bin/config.sh +++ b/flink-dist/src/main/flink-bin/bin/config.sh @@ -576,7 +576,7 @@ rotateLogFilesWithPrefix() { while read -r log ; do rotateLogFile "$log" # find distinct set of log file names, ignoring the rotation number (trailing dot and digit) - done < <(find "$dir" ! -type d -path "${prefix}*" | sed -E s/\.[0-9]+$// | sort | uniq) + done < <(find "$dir" ! -type d -path "${prefix}*" | sed s/\.[0-9][0-9]*$// | sort | uniq) } rotateLogFile() { From d4615e06eeec28742e4af4afa2234acb7894a885 Mon Sep 17 00:00:00 2001 From: whlwanghailong Date: Thu, 17 Oct 2019 23:24:35 +0800 Subject: [PATCH 161/746] [FLINK-14441][table-api] Fix ValueLiteralExpression#getValueAs when value is Period When value is Period, getValueAs(Integer.class) returns a Long value. This closes #9931 --- .../table/expressions/ValueLiteralExpression.java | 2 +- .../flink/table/expressions/ExpressionTest.java | 11 +++++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/ValueLiteralExpression.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/ValueLiteralExpression.java index 04c574480e75..bef3829f3256 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/ValueLiteralExpression.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/ValueLiteralExpression.java @@ -106,7 +106,7 @@ else if (valueClass == Long.class && clazz == Duration.class) { else if (valueClass == Period.class && clazz == Integer.class) { final Period period = (Period) value; - convertedValue = period.toTotalMonths(); + convertedValue = (int) period.toTotalMonths(); } else if (valueClass == Integer.class && clazz == Period.class) { 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 c9d265b58ff0..fcc8ef1772ee 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,7 @@ import java.math.BigDecimal; import java.sql.Timestamp; import java.time.LocalDateTime; +import java.time.Period; import java.util.HashMap; import java.util.Map; @@ -160,6 +161,16 @@ public void testSymbolValueLiteralExtraction() { .orElseThrow(AssertionError::new)); } + @Test + public void testPeriodValueLiteralExtraction() { + final Period period = Period.ofMonths(10); + Integer expectedValue = 10; + assertEquals( + expectedValue, + new ValueLiteralExpression(period).getValueAs(Integer.class) + .orElseThrow(AssertionError::new)); + } + // -------------------------------------------------------------------------------------------- private static Expression createExpressionTree(Integer nestedValue) { From b0f693da1df53ee5c8475477f99d8e551edc8de2 Mon Sep 17 00:00:00 2001 From: Liupengcheng Date: Mon, 21 Oct 2019 13:47:48 +0800 Subject: [PATCH 162/746] [FLINK-14123][docs] Update release notes to contain warning about OOM This closes #9949. --- docs/release-notes/flink-1.8.md | 7 +++++++ docs/release-notes/flink-1.8.zh.md | 7 +++++++ docs/release-notes/flink-1.9.md | 10 ++++++++++ docs/release-notes/flink-1.9.zh.md | 10 ++++++++++ 4 files changed, 34 insertions(+) diff --git a/docs/release-notes/flink-1.8.md b/docs/release-notes/flink-1.8.md index c9fbdad89585..eee0360c1f7b 100644 --- a/docs/release-notes/flink-1.8.md +++ b/docs/release-notes/flink-1.8.md @@ -203,4 +203,11 @@ see [here](/dev/stream/state/custom_serialization.html#implementing-a-compositetypeserializersnapshot) for instructions on using `CompositeTypeSerializerSnapshot`. +### Memory management + +In Fink 1.8.0 and prior version, the managed memory fraction of taskmanager is controlled by `taskmanager.memory.fraction`, +and with 0.7 as the default value. However, sometimes this will cause OOMs due to the fact that the default value of JVM +parameter `NewRatio` is 2, which means the old generation occupied only 2/3 (0.66) of the heap memory. So if you run into +this case, please manually change this value to a lower value. + {% top %} diff --git a/docs/release-notes/flink-1.8.zh.md b/docs/release-notes/flink-1.8.zh.md index c9fbdad89585..eee0360c1f7b 100644 --- a/docs/release-notes/flink-1.8.zh.md +++ b/docs/release-notes/flink-1.8.zh.md @@ -203,4 +203,11 @@ see [here](/dev/stream/state/custom_serialization.html#implementing-a-compositetypeserializersnapshot) for instructions on using `CompositeTypeSerializerSnapshot`. +### Memory management + +In Fink 1.8.0 and prior version, the managed memory fraction of taskmanager is controlled by `taskmanager.memory.fraction`, +and with 0.7 as the default value. However, sometimes this will cause OOMs due to the fact that the default value of JVM +parameter `NewRatio` is 2, which means the old generation occupied only 2/3 (0.66) of the heap memory. So if you run into +this case, please manually change this value to a lower value. + {% top %} diff --git a/docs/release-notes/flink-1.9.md b/docs/release-notes/flink-1.9.md index dedfa4c83269..d0523a191d99 100644 --- a/docs/release-notes/flink-1.9.md +++ b/docs/release-notes/flink-1.9.md @@ -75,6 +75,16 @@ have skipped Java 9 support. Related issues: - [FLINK-8033: JDK 9 support](https://issues.apache.org/jira/browse/FLINK-8033) +### Memory management + +In Fink 1.9.0 and prior version, the managed memory fraction of taskmanager is controlled by `taskmanager.memory.fraction`, +and with 0.7 as the default value. However, sometimes this will cause OOMs due to the fact that the default value of JVM +parameter `NewRatio` is 2, which means the old generation occupied only 2/3 (0.66) of the heap memory. So if you run into +this case, please manually change this value to a lower value. + +Related issues: +- [FLINK-14123: Lower the default value of taskmanager.memory.fraction](https://issues.apache.org/jira/browse/FLINK-14123) + ## Deprecations and breaking changes ### Scala expression DSL for Table API moved to `flink-table-api-scala` diff --git a/docs/release-notes/flink-1.9.zh.md b/docs/release-notes/flink-1.9.zh.md index dedfa4c83269..d0523a191d99 100644 --- a/docs/release-notes/flink-1.9.zh.md +++ b/docs/release-notes/flink-1.9.zh.md @@ -75,6 +75,16 @@ have skipped Java 9 support. Related issues: - [FLINK-8033: JDK 9 support](https://issues.apache.org/jira/browse/FLINK-8033) +### Memory management + +In Fink 1.9.0 and prior version, the managed memory fraction of taskmanager is controlled by `taskmanager.memory.fraction`, +and with 0.7 as the default value. However, sometimes this will cause OOMs due to the fact that the default value of JVM +parameter `NewRatio` is 2, which means the old generation occupied only 2/3 (0.66) of the heap memory. So if you run into +this case, please manually change this value to a lower value. + +Related issues: +- [FLINK-14123: Lower the default value of taskmanager.memory.fraction](https://issues.apache.org/jira/browse/FLINK-14123) + ## Deprecations and breaking changes ### Scala expression DSL for Table API moved to `flink-table-api-scala` From f0a99979b043875dca11b03b7aba9da6b5bd29bd Mon Sep 17 00:00:00 2001 From: tison Date: Sun, 20 Oct 2019 12:59:20 +0800 Subject: [PATCH 163/746] [FLINK-14461][configuration] Remove unused sessionTimeout from JobGraph This closes #9942. --- .../org/apache/flink/api/common/Plan.java | 10 --------- .../plantranslate/JobGraphGenerator.java | 1 - .../flink/runtime/jobgraph/JobGraph.java | 22 ------------------- 3 files changed, 33 deletions(-) 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 0e71bcda97c3..54bbdebf32df 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 @@ -70,8 +70,6 @@ public class Plan implements Visitable>, Pipeline { /** The ID of the Job that this dataflow plan belongs to. */ private JobID jobId; - private long sessionTimeout; - // ------------------------------------------------------------------------ /** @@ -257,14 +255,6 @@ public void setJobId(JobID jobId) { this.jobId = jobId; } - public void setSessionTimeout(long sessionTimeout) { - this.sessionTimeout = sessionTimeout; - } - - public long getSessionTimeout() { - return sessionTimeout; - } - /** * Gets the default parallelism for this job. That degree is always used when an operator is not * explicitly given a parallelism. diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java index 471bc8da5ad0..99860b2b91f0 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java @@ -249,7 +249,6 @@ public JobGraph compileJobGraph(OptimizedPlan program, JobID jobId) { } graph.setAllowQueuedScheduling(false); - graph.setSessionTimeout(program.getOriginalPlan().getSessionTimeout()); // add vertices to the graph for (JobVertex vertex : this.vertices.values()) { 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 3c6f93575371..31564d4ff599 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 @@ -73,10 +73,6 @@ public class JobGraph implements Serializable { /** Name of this job. */ private final String jobName; - /** The number of seconds after which the corresponding ExecutionGraph is removed at the - * job manager after it has been executed. */ - private long sessionTimeout = 0; - /** flag to enable queued scheduling */ private boolean allowQueuedScheduling; @@ -224,24 +220,6 @@ public SerializedValue getSerializedExecutionConfig() { return serializedExecutionConfig; } - /** - * Gets the timeout after which the corresponding ExecutionGraph is removed at the - * job manager after it has been executed. - * @return a timeout as a long in seconds. - */ - public long getSessionTimeout() { - return sessionTimeout; - } - - /** - * Sets the timeout of the session in seconds. The timeout specifies how long a job will be kept - * in the job manager after it finishes. - * @param sessionTimeout The timeout in seconds - */ - public void setSessionTimeout(long sessionTimeout) { - this.sessionTimeout = sessionTimeout; - } - public void setAllowQueuedScheduling(boolean allowQueuedScheduling) { this.allowQueuedScheduling = allowQueuedScheduling; } From 4226eae842126c8b6fb5debed8e3959a93201f38 Mon Sep 17 00:00:00 2001 From: tison Date: Fri, 18 Oct 2019 22:18:33 +0800 Subject: [PATCH 164/746] [FLINK-14456][client] Exclude lastJobExecutionResult from ClusterClient --- .../org/apache/flink/client/ClientUtils.java | 8 ++++-- .../flink/client/program/ClusterClient.java | 12 --------- .../client/program/ContextEnvironment.java | 25 ++++++++++++++++--- .../program/ContextEnvironmentFactory.java | 25 ++++++++++++++++--- .../client/program/MiniClusterClient.java | 7 ++---- .../program/rest/RestClusterClient.java | 6 ++--- .../environment/StreamContextEnvironment.java | 11 +++++--- 7 files changed, 59 insertions(+), 35 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 bd4b92a86953..e229b94e8e1f 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 @@ -38,6 +38,7 @@ import java.net.URISyntaxException; import java.net.URL; import java.util.List; +import java.util.concurrent.atomic.AtomicReference; import java.util.jar.JarFile; /** @@ -106,6 +107,8 @@ public static JobSubmissionResult executeProgram( final List libraries = program.getAllLibraries(); + final AtomicReference jobExecutionResult = new AtomicReference<>(); + ContextEnvironmentFactory factory = new ContextEnvironmentFactory( client, libraries, @@ -113,13 +116,14 @@ public static JobSubmissionResult executeProgram( program.getUserCodeClassLoader(), parallelism, client.isDetached(), - program.getSavepointSettings()); + program.getSavepointSettings(), + jobExecutionResult); ContextEnvironment.setAsContext(factory); try { program.invokeInteractiveModeForExecution(); - JobExecutionResult result = client.getLastJobExecutionResult(); + JobExecutionResult result = jobExecutionResult.get(); if (result == null) { throw new ProgramMissingJobException("The program didn't contain a Flink job."); } 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 4bd86ee84cc9..d48667cf23fe 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 @@ -18,7 +18,6 @@ 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.configuration.Configuration; @@ -48,8 +47,6 @@ public abstract class ClusterClient implements AutoCloseable { /** Configuration of the client. */ private final Configuration flinkConfig; - protected JobExecutionResult lastJobExecutionResult; - /** Switch for blocking/detached job submission of the client. */ private boolean detachedJobSubmission = false; @@ -226,13 +223,4 @@ public Configuration getFlinkConfiguration() { public void shutDownCluster() { throw new UnsupportedOperationException("The " + getClass().getSimpleName() + " does not support shutDownCluster."); } - - /** - * 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 - * which lets us access the execution result here. - */ - public JobExecutionResult getLastJobExecutionResult() { - return lastJobExecutionResult; - } } 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 55ffa3b2618f..06bfa714732f 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 @@ -30,6 +30,7 @@ import java.net.URL; import java.util.List; +import java.util.concurrent.atomic.AtomicReference; /** * Execution Environment for remote execution with the Client. @@ -48,10 +49,18 @@ public class ContextEnvironment extends ExecutionEnvironment { private final SavepointRestoreSettings savepointSettings; + private final AtomicReference jobExecutionResult; + private boolean alreadyCalled; - public ContextEnvironment(ClusterClient remoteConnection, List jarFiles, List classpaths, - ClassLoader userCodeClassLoader, SavepointRestoreSettings savepointSettings, boolean detached) { + public ContextEnvironment( + ClusterClient remoteConnection, + List jarFiles, + List classpaths, + ClassLoader userCodeClassLoader, + SavepointRestoreSettings savepointSettings, + boolean detached, + AtomicReference jobExecutionResult) { this.client = remoteConnection; this.jarFilesToAttach = jarFiles; this.classpathsToAttach = classpaths; @@ -60,6 +69,8 @@ public ContextEnvironment(ClusterClient remoteConnection, List jarFiles, this.detached = detached; this.alreadyCalled = false; + + this.jobExecutionResult = jobExecutionResult; } @Override @@ -76,11 +87,13 @@ public JobExecutionResult execute(String jobName) throws Exception { ClientUtils.addJarFiles(jobGraph, this.jarFilesToAttach); jobGraph.setClasspaths(this.classpathsToAttach); - this.lastJobExecutionResult = client + lastJobExecutionResult = client .submitJob(jobGraph, this.userCodeClassLoader) .getJobExecutionResult(); - return this.lastJobExecutionResult; + setJobExecutionResult(lastJobExecutionResult); + + return lastJobExecutionResult; } private void verifyExecuteIsCalledOnceWhenInDetachedMode() { @@ -90,6 +103,10 @@ private void verifyExecuteIsCalledOnceWhenInDetachedMode() { alreadyCalled = true; } + public void setJobExecutionResult(JobExecutionResult jobExecutionResult) { + this.jobExecutionResult.set(jobExecutionResult); + } + @Override public String toString() { return "Context Environment (parallelism = " + (getParallelism() == ExecutionConfig.PARALLELISM_DEFAULT ? "default" : getParallelism()) + ")"; diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironmentFactory.java b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironmentFactory.java index 017891aedfce..ff7f15bc300a 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironmentFactory.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironmentFactory.java @@ -19,12 +19,14 @@ package org.apache.flink.client.program; 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.api.java.ExecutionEnvironmentFactory; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import java.net.URL; import java.util.List; +import java.util.concurrent.atomic.AtomicReference; /** * The factory that instantiates the environment to be used when running jobs that are @@ -47,11 +49,19 @@ public class ContextEnvironmentFactory implements ExecutionEnvironmentFactory { private final SavepointRestoreSettings savepointSettings; + private final AtomicReference jobExecutionResult; + private boolean alreadyCalled; - public ContextEnvironmentFactory(ClusterClient client, List jarFilesToAttach, - List classpathsToAttach, ClassLoader userCodeClassLoader, int defaultParallelism, - boolean isDetached, SavepointRestoreSettings savepointSettings) { + public ContextEnvironmentFactory( + ClusterClient client, + List jarFilesToAttach, + List classpathsToAttach, + ClassLoader userCodeClassLoader, + int defaultParallelism, + boolean isDetached, + SavepointRestoreSettings savepointSettings, + AtomicReference jobExecutionResult) { this.client = client; this.jarFilesToAttach = jarFilesToAttach; this.classpathsToAttach = classpathsToAttach; @@ -60,6 +70,7 @@ public ContextEnvironmentFactory(ClusterClient client, List jarFilesToAt this.isDetached = isDetached; this.savepointSettings = savepointSettings; this.alreadyCalled = false; + this.jobExecutionResult = jobExecutionResult; } @Override @@ -67,7 +78,13 @@ public ExecutionEnvironment createExecutionEnvironment() { verifyCreateIsCalledOnceWhenInDetachedMode(); final ContextEnvironment environment = new ContextEnvironment( - client, jarFilesToAttach, classpathsToAttach, userCodeClassLoader, savepointSettings, isDetached); + client, + jarFilesToAttach, + classpathsToAttach, + userCodeClassLoader, + savepointSettings, + isDetached, + jobExecutionResult); if (defaultParallelism > 0) { environment.setParallelism(defaultParallelism); } 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 5b0214a8f28a..35c57c002c5f 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 @@ -62,9 +62,7 @@ public JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) if (isDetached()) { try { final JobSubmissionResult jobSubmissionResult = jobSubmissionResultFuture.get(); - - lastJobExecutionResult = new DetachedJobExecutionResult(jobSubmissionResult.getJobID()); - return lastJobExecutionResult; + return new DetachedJobExecutionResult(jobSubmissionResult.getJobID()); } catch (InterruptedException | ExecutionException e) { ExceptionUtils.checkInterrupted(e); @@ -84,8 +82,7 @@ public JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) } try { - lastJobExecutionResult = jobResult.toJobExecutionResult(classLoader); - return lastJobExecutionResult; + return jobResult.toJobExecutionResult(classLoader); } catch (JobExecutionException | IOException | ClassNotFoundException e) { throw new ProgramInvocationException("Job failed", jobGraph.getJobID(), e); } 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 d208c9fe275f..f629e8f6bcc0 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 @@ -236,8 +236,7 @@ public JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) LOG.warn("Job was executed in detached mode, the results will be available on completion."); - this.lastJobExecutionResult = new DetachedJobExecutionResult(jobSubmissionResult.getJobID()); - return lastJobExecutionResult; + return new DetachedJobExecutionResult(jobSubmissionResult.getJobID()); } catch (Exception e) { throw new ProgramInvocationException("Could not submit job", jobGraph.getJobID(), ExceptionUtils.stripExecutionException(e)); @@ -255,8 +254,7 @@ public JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) } try { - this.lastJobExecutionResult = jobResult.toJobExecutionResult(classLoader); - return lastJobExecutionResult; + return jobResult.toJobExecutionResult(classLoader); } catch (JobExecutionException | IOException | ClassNotFoundException e) { throw new ProgramInvocationException("Job failed.", jobGraph.getJobID(), e); } 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 b2db19b41481..f535cb42da14 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 @@ -57,9 +57,12 @@ public JobExecutionResult execute(StreamGraph streamGraph) throws Exception { // running from the CLI will override the savepoint restore settings jobGraph.setSavepointRestoreSettings(ctx.getSavepointRestoreSettings()); - return ctx - .getClient() - .submitJob(jobGraph, ctx.getUserCodeClassLoader()) - .getJobExecutionResult(); + JobExecutionResult jobExecutionResult = ctx.getClient() + .submitJob(jobGraph, ctx.getUserCodeClassLoader()) + .getJobExecutionResult(); + + ctx.setJobExecutionResult(jobExecutionResult); + + return jobExecutionResult; } } From 79f9589d5fb4314d07eede7ebcd838304edc67c5 Mon Sep 17 00:00:00 2001 From: tison Date: Fri, 18 Oct 2019 22:28:52 +0800 Subject: [PATCH 165/746] [FLINK-14457][client] Shift down ClusterClient#configuration --- .../flink/client/program/ClusterClient.java | 23 +------------------ .../client/program/MiniClusterClient.java | 8 ++++++- .../program/rest/RestClusterClient.java | 16 +++++++++---- .../yarn/CliFrontendRunWithYarnTest.java | 2 +- .../flink/yarn/util/FakeClusterClient.java | 5 ++-- 5 files changed, 24 insertions(+), 30 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 d48667cf23fe..e81bc5d6f302 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 @@ -28,7 +28,6 @@ import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.util.FlinkException; import org.apache.flink.util.OptionalFailure; -import org.apache.flink.util.Preconditions; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -44,27 +43,9 @@ */ public abstract class ClusterClient implements AutoCloseable { - /** Configuration of the client. */ - private final Configuration flinkConfig; - /** Switch for blocking/detached job submission of the client. */ private boolean detachedJobSubmission = false; - // ------------------------------------------------------------------------ - // Construction - // ------------------------------------------------------------------------ - - /** - * Creates a instance that submits the programs to the JobManager defined in the - * configuration. This method will try to resolve the JobManager hostname and throw an exception - * if that is not possible. - * - * @param flinkConfig The config used to obtain the job-manager's address, and used to configure the optimizer. - */ - public ClusterClient(Configuration flinkConfig) { - this.flinkConfig = Preconditions.checkNotNull(flinkConfig); - } - /** * User overridable hook to close the client, possibly closes internal services. * @deprecated use the {@link #close()} instead. This method stays for backwards compatibility. @@ -192,9 +173,7 @@ public boolean isDetached() { * Return the Flink configuration object. * @return The Flink configuration object */ - public Configuration getFlinkConfiguration() { - return flinkConfig.clone(); - } + public abstract Configuration getFlinkConfiguration(); /** * Calls the subclasses' submitJob method. It may decide to simply call one of the run methods or it may perform 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 35c57c002c5f..5e52740af9ab 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 @@ -49,12 +49,18 @@ public class MiniClusterClient extends ClusterClient { private final MiniCluster miniCluster; + private final Configuration configuration; public MiniClusterClient(@Nonnull Configuration configuration, @Nonnull MiniCluster miniCluster) { - super(configuration); + this.configuration = configuration; this.miniCluster = miniCluster; } + @Override + public Configuration getFlinkConfiguration() { + return new Configuration(configuration); + } + @Override public JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException { final CompletableFuture jobSubmissionResultFuture = submitJob(jobGraph); 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 f629e8f6bcc0..8aa5b388291f 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 @@ -94,7 +94,6 @@ import org.apache.flink.util.ExecutorUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.OptionalFailure; -import org.apache.flink.util.Preconditions; import org.apache.flink.util.function.CheckedSupplier; import org.apache.flink.shaded.netty4.io.netty.channel.ConnectTimeoutException; @@ -129,6 +128,8 @@ import java.util.function.Supplier; import java.util.stream.Collectors; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * A {@link ClusterClient} implementation that communicates via HTTP REST requests. */ @@ -138,6 +139,8 @@ public class RestClusterClient extends ClusterClient { private final RestClusterClientConfiguration restClusterClientConfiguration; + private final Configuration configuration; + /** Timeout for futures. */ private final Duration timeout; @@ -172,8 +175,8 @@ public RestClusterClient(Configuration config, T clusterId) throws Exception { @Nullable RestClient restClient, T clusterId, WaitStrategy waitStrategy) throws Exception { - super(configuration); + this.configuration = checkNotNull(configuration); this.timeout = AkkaUtils.getClientTimeout(configuration); this.restClusterClientConfiguration = RestClusterClientConfiguration.fromConfiguration(configuration); @@ -184,8 +187,8 @@ public RestClusterClient(Configuration config, T clusterId) throws Exception { this.restClient = new RestClient(restClusterClientConfiguration.getRestClientConfiguration(), executorService); } - this.waitStrategy = Preconditions.checkNotNull(waitStrategy); - this.clusterId = Preconditions.checkNotNull(clusterId); + this.waitStrategy = checkNotNull(waitStrategy); + this.clusterId = checkNotNull(clusterId); this.clientHAServices = HighAvailabilityServicesUtils.createClientHAService(configuration); @@ -198,6 +201,11 @@ private void startLeaderRetrievers() throws Exception { this.webMonitorRetrievalService.start(webMonitorLeaderRetriever); } + @Override + public Configuration getFlinkConfiguration() { + return new Configuration(configuration); + } + @Override public void close() { ExecutorUtils.gracefulShutdown(restClusterClientConfiguration.getRetryDelay(), TimeUnit.MILLISECONDS, retryExecutorService); 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 48c415e5d903..28a8d5aac4ef 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 @@ -99,7 +99,7 @@ private TestingFlinkYarnSessionCli( String longPrefix) throws Exception { super(configuration, configurationDirectory, shortPrefix, longPrefix); - this.clusterClient = new FakeClusterClient(configuration); + this.clusterClient = new FakeClusterClient(); this.configurationDirectory = configurationDirectory; } 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 82a9ea7d4a3e..9d7ce7260221 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 @@ -44,8 +44,9 @@ */ public class FakeClusterClient extends ClusterClient { - public FakeClusterClient(Configuration flinkConfig) throws Exception { - super(flinkConfig); + @Override + public Configuration getFlinkConfiguration() { + throw new UnsupportedOperationException("Not needed in test."); } @Override From af50b7cc8a2f827db0f4031d3d1bced427b8cb98 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Mon, 21 Oct 2019 11:53:33 +0800 Subject: [PATCH 166/746] [FLINK-14469][python] Drop Python 2 support for PyFlink - Remove the CI for Python 2.7 - Throw an exception if Python below 3.5 is used(both for install and run on worker) - Clean up code specific for Python 2.x This closes #9947. --- docs/flinkDev/building.md | 2 ++ docs/flinkDev/building.zh.md | 2 ++ flink-python/dev/lint-python.sh | 4 +-- flink-python/pyflink/__init__.py | 6 ++++ .../pyflink/common/execution_config.py | 6 +--- .../pyflink/common/restart_strategy.py | 10 ++---- .../pyflink/datastream/state_backend.py | 7 ++-- flink-python/pyflink/find_flink_home.py | 8 ++--- .../pyflink/fn_execution/coder_impl.py | 8 ++--- flink-python/pyflink/fn_execution/coders.py | 6 +--- .../pyflink/fn_execution/operations.py | 8 ++--- flink-python/pyflink/serializers.py | 24 ++++--------- flink-python/pyflink/shell.py | 5 +-- flink-python/pyflink/table/descriptors.py | 35 ++++++++----------- flink-python/pyflink/table/table.py | 4 --- flink-python/pyflink/table/table_config.py | 6 +--- flink-python/pyflink/table/table_schema.py | 6 +--- .../table/tests/test_table_environment_api.py | 12 +++---- .../pyflink/table/tests/test_types.py | 10 +----- flink-python/pyflink/table/types.py | 32 +++++------------ flink-python/pyflink/table/udf.py | 9 +---- .../pyflink/testing/source_sink_utils.py | 8 ++--- .../pyflink/testing/test_case_utils.py | 7 +--- flink-python/pyflink/util/utils.py | 5 +-- flink-python/setup.py | 6 ++-- .../python/pickle/ArrayConstructor.java | 21 ++--------- flink-python/tox.ini | 2 +- 27 files changed, 77 insertions(+), 182 deletions(-) diff --git a/docs/flinkDev/building.md b/docs/flinkDev/building.md index eb06d2775343..dd57ea1b4e8f 100644 --- a/docs/flinkDev/building.md +++ b/docs/flinkDev/building.md @@ -65,6 +65,8 @@ Then go to the root directory of flink source code and run this command to build cd flink-python; python3 setup.py sdist bdist_wheel {% endhighlight %} +Note Python 3.5 or higher is required to build PyFlink. + The sdist and wheel package will be found under `./flink-python/dist/`. Either of them could be used for pip installation, such as: {% highlight bash %} diff --git a/docs/flinkDev/building.zh.md b/docs/flinkDev/building.zh.md index 2f000d9118db..1a0edb134b36 100644 --- a/docs/flinkDev/building.zh.md +++ b/docs/flinkDev/building.zh.md @@ -65,6 +65,8 @@ mvn clean install -DskipTests -Dfast cd flink-python; python3 setup.py sdist bdist_wheel {% endhighlight %} +注意事项 构建PyFlink需要Python3.5及以上的版本. + 构建好的源码发布包和wheel包位于`./flink-python/dist/`目录下。它们均可使用pip安装,比如: {% highlight bash %} diff --git a/flink-python/dev/lint-python.sh b/flink-python/dev/lint-python.sh index e1becb8177f9..6ff52296bef9 100755 --- a/flink-python/dev/lint-python.sh +++ b/flink-python/dev/lint-python.sh @@ -178,7 +178,7 @@ function install_miniconda() { # Install some kinds of py env. function install_py_env() { - py_env=("2.7" "3.5" "3.6" "3.7") + py_env=("3.5" "3.6" "3.7") for ((i=0;i<${#py_env[@]};i++)) do if [ -d "$CURRENT_DIR/.conda/envs/${py_env[i]}" ]; then rm -rf "$CURRENT_DIR/.conda/envs/${py_env[i]}" @@ -301,7 +301,7 @@ function install_environment() { print_function "STEP" "install miniconda... [SUCCESS]" # step-3 install python environment whcih includes - # 2.7 3.3 3.4 3.5 3.6 3.7 + # 3.5 3.6 3.7 print_function "STEP" "installing python environment..." if [ $STEP -lt 3 ]; then install_py_env diff --git a/flink-python/pyflink/__init__.py b/flink-python/pyflink/__init__.py index e154fadd32b9..fbf4fe2614ee 100644 --- a/flink-python/pyflink/__init__.py +++ b/flink-python/pyflink/__init__.py @@ -15,3 +15,9 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################# +import sys + +if sys.version_info < (3, 5): + raise RuntimeError( + 'Python versions prior to 3.5 are not supported for PyFlink [' + + str(sys.version_info) + '].') diff --git a/flink-python/pyflink/common/execution_config.py b/flink-python/pyflink/common/execution_config.py index b66faf75754c..7c2a5e2584af 100644 --- a/flink-python/pyflink/common/execution_config.py +++ b/flink-python/pyflink/common/execution_config.py @@ -15,7 +15,6 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ -import sys from pyflink.common.execution_mode import ExecutionMode from pyflink.common.input_dependency_constraint import InputDependencyConstraint @@ -23,9 +22,6 @@ from pyflink.java_gateway import get_gateway from pyflink.util.utils import load_java_class -if sys.version >= '3': - unicode = str - __all__ = ['ExecutionConfig'] @@ -554,7 +550,7 @@ def set_global_job_parameters(self, global_job_parameters_dict): Configuration = gateway.jvm.org.apache.flink.configuration.Configuration j_global_job_parameters = Configuration() for key in global_job_parameters_dict: - if not isinstance(global_job_parameters_dict[key], (str, unicode)): + if not isinstance(global_job_parameters_dict[key], str): value = str(global_job_parameters_dict[key]) else: value = global_job_parameters_dict[key] diff --git a/flink-python/pyflink/common/restart_strategy.py b/flink-python/pyflink/common/restart_strategy.py index 687fb6197e20..6fc977d6ea98 100644 --- a/flink-python/pyflink/common/restart_strategy.py +++ b/flink-python/pyflink/common/restart_strategy.py @@ -15,7 +15,6 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ -import sys from abc import ABCMeta from datetime import timedelta @@ -26,9 +25,6 @@ __all__ = ['RestartStrategies', 'RestartStrategyConfiguration'] -if sys.version >= '3': - long = int - class RestartStrategyConfiguration(object): """ @@ -89,7 +85,7 @@ class FixedDelayRestartStrategyConfiguration(RestartStrategyConfiguration): def __init__(self, restart_attempts=None, delay_between_attempts_interval=None, j_restart_strategy=None): if j_restart_strategy is None: - if not isinstance(delay_between_attempts_interval, (timedelta, int, long)): + if not isinstance(delay_between_attempts_interval, (timedelta, int)): raise TypeError("The delay_between_attempts_interval 'failure_interval' " "only supports integer and datetime.timedelta, current input " "type is %s." % type(delay_between_attempts_interval)) @@ -122,11 +118,11 @@ def __init__(self, delay_between_attempts_interval=None, j_restart_strategy=None): if j_restart_strategy is None: - if not isinstance(failure_interval, (timedelta, int, long)): + if not isinstance(failure_interval, (timedelta, int)): raise TypeError("The parameter 'failure_interval' " "only supports integer and datetime.timedelta, current input " "type is %s." % type(failure_interval)) - if not isinstance(delay_between_attempts_interval, (timedelta, int, long)): + if not isinstance(delay_between_attempts_interval, (timedelta, int)): raise TypeError("The delay_between_attempts_interval 'failure_interval' " "only supports integer and datetime.timedelta, current input " "type is %s." % type(delay_between_attempts_interval)) diff --git a/flink-python/pyflink/datastream/state_backend.py b/flink-python/pyflink/datastream/state_backend.py index 6d23850c0bce..f718636819d7 100644 --- a/flink-python/pyflink/datastream/state_backend.py +++ b/flink-python/pyflink/datastream/state_backend.py @@ -15,7 +15,7 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ -import sys + from abc import ABCMeta from py4j.java_gateway import get_java_class @@ -31,9 +31,6 @@ 'CustomStateBackend', 'PredefinedOptions'] -if sys.version > '3': - xrange = range - def _from_j_state_backend(j_state_backend): if j_state_backend is None: @@ -532,7 +529,7 @@ def set_db_storage_paths(self, *paths): else: gateway = get_gateway() j_path_array = gateway.new_array(gateway.jvm.String, len(paths)) - for i in xrange(0, len(paths)): + for i in range(0, len(paths)): j_path_array[i] = paths[i] self._j_rocks_db_state_backend.setDbStoragePaths(j_path_array) diff --git a/flink-python/pyflink/find_flink_home.py b/flink-python/pyflink/find_flink_home.py index b75cc29905a4..a8f3b2b6dd62 100644 --- a/flink-python/pyflink/find_flink_home.py +++ b/flink-python/pyflink/find_flink_home.py @@ -47,12 +47,8 @@ def _find_flink_home(): os.environ['FLINK_HOME'] = build_target return build_target - if sys.version < "3": - import imp - module_home = imp.find_module("pyflink")[1] - else: - from importlib.util import find_spec - module_home = os.path.dirname(find_spec("pyflink").origin) + from importlib.util import find_spec + module_home = os.path.dirname(find_spec("pyflink").origin) if _is_flink_home(module_home): os.environ['FLINK_HOME'] = module_home diff --git a/flink-python/pyflink/fn_execution/coder_impl.py b/flink-python/pyflink/fn_execution/coder_impl.py index 2f7c5d3ab653..6e7b21c99c83 100644 --- a/flink-python/pyflink/fn_execution/coder_impl.py +++ b/flink-python/pyflink/fn_execution/coder_impl.py @@ -15,13 +15,9 @@ # 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): @@ -30,7 +26,7 @@ def __init__(self, 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)): + for i in range(len(self._field_coders)): self._field_coders[i].encode_to_stream(value[i], out_stream, nested) def decode_from_stream(self, in_stream, nested): @@ -38,7 +34,7 @@ def decode_from_stream(self, in_stream, nested): 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))]) + in_stream, nested) for idx in range(0, len(null_mask))]) @staticmethod def write_null_mask(value, out_stream): diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index c6add86477ab..219aa1fdcbcc 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -15,7 +15,6 @@ # 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 @@ -25,9 +24,6 @@ FLINK_SCHEMA_CODER_URN = "flink:coder:schema:v1" -if sys.version > '3': - xrange = range - __all__ = ['RowCoder'] @@ -57,7 +53,7 @@ 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))]) + range(len(self._field_coders))]) def __ne__(self, other): return not self == other diff --git a/flink-python/pyflink/fn_execution/operations.py b/flink-python/pyflink/fn_execution/operations.py index 4bd955cc71e3..51c471c8d52a 100644 --- a/flink-python/pyflink/fn_execution/operations.py +++ b/flink-python/pyflink/fn_execution/operations.py @@ -95,20 +95,20 @@ def __init__(self, constant_value): # 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: + if j_type == 0: self._constant_value = pickled_data # the type is DATE - elif j_type == '\x01' or j_type == 1: + elif 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: + elif 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: + elif 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) diff --git a/flink-python/pyflink/serializers.py b/flink-python/pyflink/serializers.py index 7d8e04a3b2a5..c894c2104027 100644 --- a/flink-python/pyflink/serializers.py +++ b/flink-python/pyflink/serializers.py @@ -16,18 +16,10 @@ # limitations under the License. ################################################################################ -import sys +import pickle import struct from abc import ABCMeta, abstractmethod - -if sys.version < '3': - import cPickle as pickle - protocol = 2 - from itertools import imap as map, chain -else: - import pickle - protocol = 3 - xrange = range +from itertools import chain class SpecialLengths(object): @@ -144,14 +136,10 @@ class PickleSerializer(VarLengthDataSerializer): """ def dumps(self, obj): - return pickle.dumps(obj, protocol) + return pickle.dumps(obj, 3) - if sys.version >= '3': - def loads(self, obj, encoding="bytes"): - return pickle.loads(obj, encoding=encoding) - else: - def loads(self, obj, encoding=None): - return pickle.loads(obj) + def loads(self, obj): + return pickle.loads(obj, encoding="bytes") class BatchedSerializer(Serializer): @@ -175,7 +163,7 @@ def _batched(self, iterator): yield list(iterator) elif hasattr(iterator, "__len__") and hasattr(iterator, "__getslice__"): n = len(iterator) - for i in xrange(0, n, self.batch_size): + for i in range(0, n, self.batch_size): yield iterator[i: i + self.batch_size] else: items = [] diff --git a/flink-python/pyflink/shell.py b/flink-python/pyflink/shell.py index 8fabd496e50a..c54fd63381e9 100644 --- a/flink-python/pyflink/shell.py +++ b/flink-python/pyflink/shell.py @@ -28,10 +28,7 @@ from pyflink.table.descriptors import * from pyflink.table.window import * -if sys.version > '3': - utf8_out = open(sys.stdout.fileno(), mode='w', encoding='utf8', buffering=1) -else: - utf8_out = codecs.getwriter("utf-8")(sys.stdout) +utf8_out = open(sys.stdout.fileno(), mode='w', encoding='utf8', buffering=1) print("Using Python version %s (%s, %s)" % ( platform.python_version(), diff --git a/flink-python/pyflink/table/descriptors.py b/flink-python/pyflink/table/descriptors.py index 457fc4fc56e6..aa16b7c8c2c2 100644 --- a/flink-python/pyflink/table/descriptors.py +++ b/flink-python/pyflink/table/descriptors.py @@ -15,7 +15,6 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ -import sys from abc import ABCMeta from py4j.java_gateway import get_method @@ -23,10 +22,6 @@ from pyflink.java_gateway import get_gateway -if sys.version >= '3': - long = int - unicode = str - __all__ = [ 'Rowtime', 'Schema', @@ -215,7 +210,7 @@ def field(self, field_name, field_type): :param field_type: The data type or type string of the field. :return: This schema object. """ - if isinstance(field_type, (str, unicode)): + if isinstance(field_type, str): self._j_schema = self._j_schema.field(field_name, field_type) else: self._j_schema = self._j_schema.field(field_name, _to_java_type(field_type)) @@ -338,7 +333,7 @@ def field(self, field_name, field_type): :param field_type: The data type or type string of the field. :return: This :class:`OldCsv` object. """ - if isinstance(field_type, (str, unicode)): + if isinstance(field_type, str): self._j_csv = self._j_csv.field(field_name, field_type) else: self._j_csv = self._j_csv.field(field_name, _to_java_type(field_type)) @@ -409,7 +404,7 @@ def field_delimiter(self, delimiter): :param delimiter: The field delimiter character. :return: This :class:`Csv` object. """ - if not isinstance(delimiter, (str, unicode)) or len(delimiter) != 1: + if not isinstance(delimiter, str) or len(delimiter) != 1: raise TypeError("Only one-character string is supported!") self._j_csv = self._j_csv.fieldDelimiter(delimiter) return self @@ -431,7 +426,7 @@ def quote_character(self, quote_character): :param quote_character: The quote character. :return: This :class:`Csv` object. """ - if not isinstance(quote_character, (str, unicode)) or len(quote_character) != 1: + if not isinstance(quote_character, str) or len(quote_character) != 1: raise TypeError("Only one-character string is supported!") self._j_csv = self._j_csv.quoteCharacter(quote_character) return self @@ -473,7 +468,7 @@ def escape_character(self, escape_character): :param escape_character: Escaping character (e.g. backslash). :return: This :class:`Csv` object. """ - if not isinstance(escape_character, (str, unicode)) or len(escape_character) != 1: + if not isinstance(escape_character, str) or len(escape_character) != 1: raise TypeError("Only one-character string is supported!") self._j_csv = self._j_csv.escapeCharacter(escape_character) return self @@ -627,9 +622,9 @@ def __init__(self, type, version): :param version: Property version for backwards compatibility. """ - if not isinstance(type, (str, unicode)): + if not isinstance(type, str): raise TypeError("type must be of type str.") - if not isinstance(version, (int, long)): + if not isinstance(version, int): raise TypeError("version must be of type int.") gateway = get_gateway() super(CustomFormatDescriptor, self).__init__( @@ -644,9 +639,9 @@ def property(self, key, value): :return: This object. """ - if not isinstance(key, (str, unicode)): + if not isinstance(key, str): raise TypeError("key must be of type str.") - if not isinstance(value, (str, unicode)): + if not isinstance(value, str): raise TypeError("value must be of type str.") self._j_format_descriptor = self._j_format_descriptor.property(key, value) return self @@ -716,7 +711,7 @@ def version(self, version): :param version: Kafka version. E.g., "0.8", "0.11", etc. :return: This object. """ - if not isinstance(version, (str, unicode)): + if not isinstance(version, str): version = str(version) self._j_kafka = self._j_kafka.version(version) return self @@ -932,7 +927,7 @@ def version(self, version): :param version: Elasticsearch version. E.g., "6". :return: This object. """ - if not isinstance(version, (str, unicode)): + if not isinstance(version, str): version = str(version) self._j_elasticsearch = self._j_elasticsearch.version(version) return self @@ -1195,9 +1190,9 @@ def __init__(self, type, version, format_needed): :param format_needed: Flag for basic validation of a needed format descriptor. """ - if not isinstance(type, (str, unicode)): + if not isinstance(type, str): raise TypeError("type must be of type str.") - if not isinstance(version, (int, long)): + if not isinstance(version, int): raise TypeError("version must be of type int.") if not isinstance(format_needed, bool): raise TypeError("format_needed must be of type bool.") @@ -1214,9 +1209,9 @@ def property(self, key, value): :return: This object. """ - if not isinstance(key, (str, unicode)): + if not isinstance(key, str): raise TypeError("key must be of type str.") - if not isinstance(value, (str, unicode)): + if not isinstance(value, str): raise TypeError("value must be of type str.") self._j_connector_descriptor = self._j_connector_descriptor.property(key, value) return self diff --git a/flink-python/pyflink/table/table.py b/flink-python/pyflink/table/table.py index 920a372d5fc3..4ea3ec70b5aa 100644 --- a/flink-python/pyflink/table/table.py +++ b/flink-python/pyflink/table/table.py @@ -15,7 +15,6 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ -import sys from py4j.java_gateway import get_method from pyflink.java_gateway import get_gateway @@ -24,9 +23,6 @@ from pyflink.table.window import GroupWindow from pyflink.util.utils import to_jarray -if sys.version > '3': - xrange = range - __all__ = ['Table', 'GroupedTable', 'GroupWindowedTable', 'OverWindowedTable', 'WindowGroupedTable'] diff --git a/flink-python/pyflink/table/table_config.py b/flink-python/pyflink/table/table_config.py index 2f84c2f7e4f6..150b9b52d463 100644 --- a/flink-python/pyflink/table/table_config.py +++ b/flink-python/pyflink/table/table_config.py @@ -15,7 +15,6 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ -import sys from py4j.compat import long @@ -24,9 +23,6 @@ __all__ = ['TableConfig'] -if sys.version > '3': - unicode = str - class TableConfig(object): """ @@ -69,7 +65,7 @@ def set_local_timezone(self, timezone_id): such as "America/Los_Angeles", or a custom timezone_id such as "GMT-8:00". """ - if timezone_id is not None and isinstance(timezone_id, (str, unicode)): + if timezone_id is not None and isinstance(timezone_id, str): j_timezone = get_gateway().jvm.java.time.ZoneId.of(timezone_id) self._j_table_config.setLocalTimeZone(j_timezone) else: diff --git a/flink-python/pyflink/table/table_schema.py b/flink-python/pyflink/table/table_schema.py index 233115c8ddaa..c4f3670e6699 100644 --- a/flink-python/pyflink/table/table_schema.py +++ b/flink-python/pyflink/table/table_schema.py @@ -15,15 +15,11 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ -import sys from pyflink.java_gateway import get_gateway from pyflink.table.types import _to_java_type, _from_java_type from pyflink.util.utils import to_jarray -if sys.version >= '3': - unicode = str - __all__ = ['TableSchema'] @@ -65,7 +61,7 @@ def get_field_data_type(self, field): :param field: The index of the field or the name of the field. :return: The data type of the specified field. """ - if not isinstance(field, (int, str, unicode)): + if not isinstance(field, (int, str)): raise TypeError("Expected field index or field name, got %s" % type(field)) optional_result = self._j_table_schema.getFieldDataType(field) if optional_result.isPresent(): diff --git a/flink-python/pyflink/table/tests/test_table_environment_api.py b/flink-python/pyflink/table/tests/test_table_environment_api.py index 6a7637a5335a..ff0ec0f06467 100644 --- a/flink-python/pyflink/table/tests/test_table_environment_api.py +++ b/flink-python/pyflink/table/tests/test_table_environment_api.py @@ -17,8 +17,6 @@ ################################################################################ import os -from py4j.compat import unicode - from pyflink.dataset import ExecutionEnvironment from pyflink.datastream import StreamExecutionEnvironment from pyflink.table import DataTypes, CsvTableSink, StreamTableEnvironment, EnvironmentSettings @@ -103,7 +101,7 @@ def test_explain(self): actual = t_env.explain(result) - assert isinstance(actual, str) or isinstance(actual, unicode) + assert isinstance(actual, str) def test_explain_with_extended(self): schema = RowType() \ @@ -116,7 +114,7 @@ def test_explain_with_extended(self): actual = t_env.explain(result, True) - assert isinstance(actual, str) or isinstance(actual, unicode) + assert isinstance(actual, str) def test_explain_with_multi_sinks(self): t_env = self.t_env @@ -135,7 +133,7 @@ def test_explain_with_multi_sinks(self): actual = t_env.explain(extended=True) - assert isinstance(actual, str) or isinstance(actual, unicode) + assert isinstance(actual, str) def test_sql_query(self): t_env = self.t_env @@ -257,7 +255,7 @@ def test_explain(self): actual = t_env.explain(result) - self.assertIsInstance(actual, (str, unicode)) + self.assertIsInstance(actual, str) def test_explain_with_extended(self): schema = RowType() \ @@ -270,7 +268,7 @@ def test_explain_with_extended(self): actual = t_env.explain(result, True) - assert isinstance(actual, str) or isinstance(actual, unicode) + assert isinstance(actual, str) def test_explain_with_multi_sinks(self): t_env = self.t_env diff --git a/flink-python/pyflink/table/tests/test_types.py b/flink-python/pyflink/table/tests/test_types.py index 9b1cbf76bf48..a8490ab85302 100644 --- a/flink-python/pyflink/table/tests/test_types.py +++ b/flink-python/pyflink/table/tests/test_types.py @@ -444,10 +444,6 @@ def assert_collect_success(typecode, value, element_type): supported_string_types += ['u'] # test unicode assert_collect_success('u', u'a', 'CHAR') - if sys.version_info[0] < 3: - supported_string_types += ['c'] - # test string - assert_collect_success('c', 'a', 'CHAR') # supported float and double # @@ -511,11 +507,7 @@ def get_int_data_type(size): # # Keys in _array_type_mappings is a complete list of all supported types, # and types not in _array_type_mappings are considered unsupported. - # `array.typecodes` are not supported in python 2. - if sys.version_info[0] < 3: - all_types = {'c', 'b', 'B', 'u', 'h', 'H', 'i', 'I', 'l', 'L', 'f', 'd'} - else: - all_types = set(array.typecodes) + all_types = set(array.typecodes) unsupported_types = all_types - set(supported_types) # test unsupported types for t in unsupported_types: diff --git a/flink-python/pyflink/table/types.py b/flink-python/pyflink/table/types.py index 09186240a629..f099dd3beff1 100644 --- a/flink-python/pyflink/table/types.py +++ b/flink-python/pyflink/table/types.py @@ -32,10 +32,6 @@ from pyflink.util.utils import to_jarray from pyflink.java_gateway import get_gateway -if sys.version >= '3': - long = int - basestring = unicode = str - __all__ = ['DataTypes', 'UserDefinedType', 'Row'] @@ -1024,11 +1020,11 @@ def __init__(self, name, data_type, description=None): """ assert isinstance(data_type, DataType), \ "data_type %s should be an instance of %s" % (data_type, DataType) - assert isinstance(name, basestring), "field name %s should be string" % name + assert isinstance(name, str), "field name %s should be string" % name if not isinstance(name, str): name = name.encode('utf-8') if description is not None: - assert isinstance(description, basestring), \ + assert isinstance(description, str), \ "description %s should be string" % description if not isinstance(description, str): description = description.encode('utf-8') @@ -1313,12 +1309,6 @@ def deserialize(self, datum): datetime.time: TimeType(), } -if sys.version < "3": - _type_mappings.update({ - unicode: VarCharType(0x7fffffff), - long: BigIntType(), - }) - # Mapping Python array types to Flink SQL types # We should be careful here. The size of these types in python depends on C # implementation. We need to make sure that this conversion does not lose any @@ -1394,10 +1384,6 @@ def _int_size_to_type(size): # it can be 16 bits or 32 bits depending on the platform _array_type_mappings['u'] = CharType(ctypes.sizeof(ctypes.c_wchar)) -# Type code 'c' are only available at python 2 -if sys.version_info[0] < 3: - _array_type_mappings['c'] = CharType(ctypes.sizeof(ctypes.c_char)) - def _infer_type(obj): """ @@ -1696,7 +1682,7 @@ def _to_java_type(data_type): def _is_instance_of(java_data_type, java_class): gateway = get_gateway() - if isinstance(java_class, basestring): + if isinstance(java_class, str): param = java_class elif isinstance(java_class, JavaClass): param = get_java_class(java_class) @@ -2010,15 +1996,15 @@ def __repr__(self): _acceptable_types = { BooleanType: (bool,), - TinyIntType: (int, long), - SmallIntType: (int, long), - IntType: (int, long), - BigIntType: (int, long), + TinyIntType: (int,), + SmallIntType: (int,), + IntType: (int,), + BigIntType: (int,), FloatType: (float,), DoubleType: (float,), DecimalType: (decimal.Decimal,), - CharType: (str, unicode), - VarCharType: (str, unicode), + CharType: (str,), + VarCharType: (str,), BinaryType: (bytearray,), VarBinaryType: (bytearray,), DateType: (datetime.date, datetime.datetime), diff --git a/flink-python/pyflink/table/udf.py b/flink-python/pyflink/table/udf.py index ccc7b46ac7be..8966f1efb540 100644 --- a/flink-python/pyflink/table/udf.py +++ b/flink-python/pyflink/table/udf.py @@ -19,7 +19,6 @@ import collections import functools import inspect -import sys from pyflink.java_gateway import get_gateway from pyflink.table.types import DataType, _to_java_type @@ -28,12 +27,6 @@ __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 @@ -43,7 +36,7 @@ class FunctionContext(object): pass -class UserDefinedFunction(ABC): +class UserDefinedFunction(abc.ABC): """ Base interface for user-defined function. """ diff --git a/flink-python/pyflink/testing/source_sink_utils.py b/flink-python/pyflink/testing/source_sink_utils.py index 359199182b7e..2406d988d7c7 100644 --- a/flink-python/pyflink/testing/source_sink_utils.py +++ b/flink-python/pyflink/testing/source_sink_utils.py @@ -18,7 +18,6 @@ import glob import os -import sys import unittest from py4j.java_gateway import java_import @@ -28,9 +27,6 @@ from pyflink.table.types import _to_java_type from pyflink.util import utils -if sys.version_info[0] >= 3: - xrange = range - class TestTableSink(TableSink): """ @@ -105,7 +101,7 @@ def __init__(self, field_names, field_types, keys, is_append_only): gateway = get_gateway() j_keys = gateway.new_array(gateway.jvm.String, len(keys)) - for i in xrange(0, len(keys)): + for i in range(0, len(keys)): j_keys[i] = keys[i] super(TestUpsertSink, self).__init__( @@ -134,7 +130,7 @@ def upsert_results(keys): """ gateway = get_gateway() j_keys = gateway.new_array(gateway.jvm.int, len(keys)) - for i in xrange(0, len(keys)): + for i in range(0, len(keys)): j_keys[i] = keys[i] results = gateway.jvm.RowCollector.getAndClearValues() diff --git a/flink-python/pyflink/testing/test_case_utils.py b/flink-python/pyflink/testing/test_case_utils.py index 4d86f188e052..dc9da7735dce 100644 --- a/flink-python/pyflink/testing/test_case_utils.py +++ b/flink-python/pyflink/testing/test_case_utils.py @@ -34,11 +34,6 @@ from pyflink.table import BatchTableEnvironment, StreamTableEnvironment, EnvironmentSettings from pyflink.java_gateway import get_gateway -if sys.version_info[0] >= 3: - xrange = range -else: - unittest.TestCase.assertRaisesRegex = unittest.TestCase.assertRaisesRegexp - unittest.TestCase.assertRegex = unittest.TestCase.assertRegexpMatches if os.getenv("VERBOSE"): log_level = logging.DEBUG @@ -97,7 +92,7 @@ def assert_equals(cls, actual, expected): @classmethod def to_py_list(cls, actual): py_list = [] - for i in xrange(0, actual.length()): + for i in range(0, actual.length()): py_list.append(actual.apply(i)) return py_list diff --git a/flink-python/pyflink/util/utils.py b/flink-python/pyflink/util/utils.py index b5376c0fd574..346b096770cc 100644 --- a/flink-python/pyflink/util/utils.py +++ b/flink-python/pyflink/util/utils.py @@ -15,14 +15,11 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ -import sys + from datetime import timedelta from pyflink.java_gateway import get_gateway -if sys.version >= '3': - unicode = str - def to_jarray(j_type, arr): """ diff --git a/flink-python/setup.py b/flink-python/setup.py index 53175db26836..05b00bdac1e4 100644 --- a/flink-python/setup.py +++ b/flink-python/setup.py @@ -24,8 +24,8 @@ from setuptools import setup -if sys.version_info < (2, 7): - print("Python versions prior to 2.7 are not supported for PyFlink.", +if sys.version_info < (3, 5): + print("Python versions prior to 3.5 are not supported for PyFlink.", file=sys.stderr) sys.exit(-1) @@ -206,6 +206,7 @@ license='https://www.apache.org/licenses/LICENSE-2.0', author='Flink Developers', author_email='dev@flink.apache.org', + python_requires='>=3.5', 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'], @@ -215,7 +216,6 @@ classifiers=[ 'Development Status :: 1 - Planning', 'License :: OSI Approved :: Apache Software License', - 'Programming Language :: Python :: 2.7', 'Programming Language :: Python :: 3.5', 'Programming Language :: Python :: 3.6', 'Programming Language :: Python :: 3.7'] diff --git a/flink-python/src/main/java/org/apache/flink/api/common/python/pickle/ArrayConstructor.java b/flink-python/src/main/java/org/apache/flink/api/common/python/pickle/ArrayConstructor.java index 697ee5567c09..2898ad5ffb9c 100644 --- a/flink-python/src/main/java/org/apache/flink/api/common/python/pickle/ArrayConstructor.java +++ b/flink-python/src/main/java/org/apache/flink/api/common/python/pickle/ArrayConstructor.java @@ -17,7 +17,6 @@ package org.apache.flink.api.common.python.pickle; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; /** @@ -29,24 +28,8 @@ public final class ArrayConstructor extends net.razorvine.pickle.objects.ArrayCo @Override public Object construct(Object[] args) { - if (args.length == 2 && args[1] instanceof String) { - char typecode = ((String) args[0]).charAt(0); - // This must be ISO 8859-1 / Latin 1, not UTF-8, to interoperate correctly - byte[] data = ((String) args[1]).getBytes(StandardCharsets.ISO_8859_1); - if (typecode == 'c') { - // It seems like the pickle of pypy uses the similar protocol to Python 2.6, which uses - // a string for array data instead of list as Python 2.7, and handles an array of - // typecode 'c' as 1-byte character. - char[] result = new char[data.length]; - int i = 0; - while (i < data.length) { - result[i] = (char) data[i]; - i += 1; - } - return result; - } - } else if (args.length == 2 && args[0] == "l") { - // On Python 2, an array of typecode 'l' should be handled as long rather than int. + if (args.length == 2 && args[0] == "l") { + // an array of typecode 'l' should be handled as long rather than int. ArrayList values = (ArrayList) args[1]; long[] result = new long[values.size()]; int i = 0; diff --git a/flink-python/tox.ini b/flink-python/tox.ini index 873af2028645..381040cffa40 100644 --- a/flink-python/tox.ini +++ b/flink-python/tox.ini @@ -21,7 +21,7 @@ # in multiple virtualenvs. This configuration file will run the # test suite on all supported python versions. # new environments will be excluded by default unless explicitly added to envlist. -envlist = py27, py35, py36, py37 +envlist = py35, py36, py37 [testenv] whitelist_externals= From 6e608726396f6af0848534c9eb5ec13781328a91 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Fri, 18 Oct 2019 13:50:49 +0800 Subject: [PATCH 167/746] [hotfix][runtime] Remove duplicated check from ExecutionGraph ctor --- .../org/apache/flink/runtime/executiongraph/ExecutionGraph.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 64ae21e99838..db037ed9b52e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -455,8 +455,6 @@ public ExecutionGraph( ScheduleMode scheduleMode, boolean allowQueuedScheduling) throws IOException { - checkNotNull(futureExecutor); - this.jobInformation = Preconditions.checkNotNull(jobInformation); this.blobWriter = Preconditions.checkNotNull(blobWriter); From cd7acb2167f8e55caba66c5a7b7c6daf140a307b Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Fri, 18 Oct 2019 13:54:45 +0800 Subject: [PATCH 168/746] [FLINK-14206][runtime] Let fullRestart metric count both full restarts and fine grained restarts This closes #9778. --- .../executiongraph/ExecutionGraph.java | 22 +++++++++++++------ ...aptedRestartPipelinedRegionStrategyNG.java | 2 ++ .../metrics/NumberOfFullRestartsGauge.java | 4 ++-- ...PipelinedRegionStrategyNGFailoverTest.java | 17 ++++++++++++++ .../ExecutionGraphRestartTest.java | 12 ++++------ 5 files changed, 40 insertions(+), 17 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index db037ed9b52e..ddd48fd5bc27 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -27,6 +27,8 @@ 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.Counter; +import org.apache.flink.metrics.SimpleCounter; import org.apache.flink.runtime.JobException; import org.apache.flink.runtime.accumulators.AccumulatorSnapshot; import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult; @@ -264,6 +266,9 @@ public class ExecutionGraph implements AccessExecutionGraph { @Nullable private InternalTaskFailuresListener internalTaskFailuresListener; + /** Counts all restarts. Used by other Gauges/Meters and does not register to metric group. */ + private final Counter numberOfRestartsCounter = new SimpleCounter(); + // ------ Configuration of the Execution ------- /** Flag to indicate whether the scheduler may queue tasks for execution, or needs to be able @@ -732,15 +737,13 @@ public ErrorInfo getFailureInfo() { } /** - * Gets the number of full restarts that the execution graph went through. - * If a full restart recovery is currently pending, this recovery is included in the - * count. + * Gets the number of restarts, including full restarts and fine grained restarts. + * If a recovery is currently pending, this recovery is included in the count. * - * @return The number of full restarts so far + * @return The number of restarts so far */ - public long getNumberOfFullRestarts() { - // subtract one, because the version starts at one - return globalModVersion - 1; + public long getNumberOfRestarts() { + return numberOfRestartsCounter.getCount(); } @Override @@ -1365,9 +1368,14 @@ private boolean transitionState(JobStatus current, JobStatus newState, Throwable } private long incrementGlobalModVersion() { + incrementRestarts(); return GLOBAL_VERSION_UPDATER.incrementAndGet(this); } + public void incrementRestarts() { + numberOfRestartsCounter.inc(); + } + private void initFailureCause(Throwable t) { this.failureCause = t; this.failureInfo = new ErrorInfo(t, System.currentTimeMillis()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java index eb43a2c35499..2d211dc459da 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java @@ -107,6 +107,8 @@ protected void restartTasks(final Set verticesToRestart) { final Set vertexVersions = new HashSet<>( executionVertexVersioner.recordVertexModifications(verticesToRestart).values()); + executionGraph.incrementRestarts(); + FutureUtils.assertNoException( cancelTasks(verticesToRestart) .thenComposeAsync(resetAndRescheduleTasks(globalModVersion, vertexVersions), executionGraph.getJobMasterMainThreadExecutor()) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/NumberOfFullRestartsGauge.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/NumberOfFullRestartsGauge.java index 05a641468e38..b1dcd4e7e7f2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/NumberOfFullRestartsGauge.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/NumberOfFullRestartsGauge.java @@ -42,6 +42,6 @@ public NumberOfFullRestartsGauge(ExecutionGraph executionGraph) { @Override public Long getValue() { - return eg.getNumberOfFullRestarts(); + return eg.getNumberOfRestarts(); } -} \ No newline at end of file +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGFailoverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGFailoverTest.java index abd940be0e20..f966d1693aca 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGFailoverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGFailoverTest.java @@ -294,6 +294,23 @@ public void testFailGlobalIfErrorOnRestartTasks() throws Exception { assertNotEquals(globalModVersionBeforeFailure, globalModVersionAfterFailure); } + @Test + public void testCountingRestarts() throws Exception { + final JobGraph jobGraph = createStreamingJobGraph(); + final ExecutionGraph eg = createExecutionGraph(jobGraph); + + final Iterator vertexIterator = eg.getAllExecutionVertices().iterator(); + final ExecutionVertex ev11 = vertexIterator.next(); + + // trigger task failure for fine grained recovery + ev11.getCurrentExecutionAttempt().fail(new Exception("Test Exception")); + assertEquals(1, eg.getNumberOfRestarts()); + + // trigger global failover + eg.failGlobal(new Exception("Force failover global")); + assertEquals(2, eg.getNumberOfRestarts()); + } + // ------------------------------- Test Utils ----------------------------------------- /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java index 785ae8bbd230..17ed2ed7ceb4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java @@ -90,12 +90,12 @@ import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.finishAllVertices; import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.switchToRunning; import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; /** * Tests the restart behaviour of the {@link ExecutionGraph}. @@ -549,9 +549,7 @@ public void testGlobalFailAndRestarts() throws Exception { eg.waitUntilTerminal(); assertEquals(JobStatus.FINISHED, eg.getState()); - if (eg.getNumberOfFullRestarts() > 2) { - fail("Too many restarts: " + eg.getNumberOfFullRestarts()); - } + assertThat("Too many restarts", eg.getNumberOfRestarts(), is(lessThanOrEqualTo(2L))); } /** @@ -689,10 +687,8 @@ public void testRestartWithSlotSharingAndNotEnoughResources() throws Exception { eg.start(mainThreadExecutor); eg.scheduleForExecution(); - // wait until no more changes happen - while (eg.getNumberOfFullRestarts() < numRestarts) { - Thread.sleep(1); - } + // the last suppressed restart is also counted + assertEquals(numRestarts + 1, eg.getNumberOfRestarts()); assertEquals(JobStatus.FAILED, eg.getState()); From e870235e6c8061ecb8a4364942396fd988e31cef Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Fri, 18 Oct 2019 14:15:00 +0800 Subject: [PATCH 169/746] [hotfix][tests] Remove legacy executor assumptions in ExecutionGraphRestartTest --- .../ExecutionGraphRestartTest.java | 21 ------------------- 1 file changed, 21 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java index 17ed2ed7ceb4..f102346bd599 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java @@ -69,7 +69,6 @@ import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TestLogger; -import org.junit.After; import org.junit.Test; import javax.annotation.Nonnull; @@ -80,9 +79,6 @@ import java.util.Iterator; import java.util.List; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ThreadPoolExecutor; import java.util.function.Consumer; import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.completeCancellingForAllVertices; @@ -104,18 +100,11 @@ public class ExecutionGraphRestartTest extends TestLogger { private static final int NUM_TASKS = 31; - private final ScheduledExecutorService executor = Executors.newScheduledThreadPool(4); - private static final ComponentMainThreadExecutor mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread(); private static final JobID TEST_JOB_ID = new JobID(); - @After - public void shutdown() { - executor.shutdownNow(); - } - // ------------------------------------------------------------------------ @Test @@ -593,9 +582,6 @@ public void slotPoolExecutionGraph_ConcurrentSchedulingAndAllocationFailure_Shou @Test public void testRestartWithEagerSchedulingAndSlotSharing() throws Exception { - // this test is inconclusive if not used with a proper multi-threaded executor - assertTrue("test assumptions violated", ((ThreadPoolExecutor) executor).getCorePoolSize() > 1); - final int parallelism = 20; try (SlotPool slotPool = createSlotPoolImpl()) { @@ -618,8 +604,6 @@ public void testRestartWithEagerSchedulingAndSlotSharing() throws Exception { final ExecutionGraph eg = new ExecutionGraphTestUtils.TestingExecutionGraphBuilder(TEST_JOB_ID, source, sink) .setSlotProvider(scheduler) - .setIoExecutor(executor) - .setFutureExecutor(executor) .setRestartStrategy(restartStrategy) .setScheduleMode(ScheduleMode.EAGER) .build(); @@ -650,9 +634,6 @@ public void testRestartWithEagerSchedulingAndSlotSharing() throws Exception { @Test public void testRestartWithSlotSharingAndNotEnoughResources() throws Exception { - // this test is inconclusive if not used with a proper multi-threaded executor - assertTrue("test assumptions violated", ((ThreadPoolExecutor) executor).getCorePoolSize() > 1); - final int numRestarts = 10; final int parallelism = 20; @@ -679,8 +660,6 @@ public void testRestartWithSlotSharingAndNotEnoughResources() throws Exception { final ExecutionGraph eg = new ExecutionGraphTestUtils.TestingExecutionGraphBuilder(TEST_JOB_ID, source, sink) .setSlotProvider(scheduler) .setRestartStrategy(restartStrategy) - .setIoExecutor(executor) - .setFutureExecutor(executor) .setScheduleMode(ScheduleMode.EAGER) .build(); From e1f5eade68b54f4771fe7ec50c5812c567e97174 Mon Sep 17 00:00:00 2001 From: tison Date: Sat, 19 Oct 2019 02:03:02 +0800 Subject: [PATCH 170/746] [FLINK-14434][coordination] Dispatcher#createJobManagerRunner returns on creation succeed This closes #9940. --- .../java/org/apache/flink/runtime/dispatcher/Dispatcher.java | 5 ++--- 1 file changed, 2 insertions(+), 3 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 1c7fb6a41414..c38ef28e6f5e 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 @@ -362,6 +362,7 @@ private CompletableFuture runJob(JobGraph jobGraph) { jobManagerRunnerFutures.put(jobGraph.getJobID(), jobManagerRunnerFuture); return jobManagerRunnerFuture + .thenApply(FunctionUtils.uncheckedFunction(this::startJobManagerRunner)) .thenApply(FunctionUtils.nullFn()) .whenCompleteAsync( (ignored, throwable) -> { @@ -375,7 +376,7 @@ private CompletableFuture runJob(JobGraph jobGraph) { private CompletableFuture createJobManagerRunner(JobGraph jobGraph) { final RpcService rpcService = getRpcService(); - final CompletableFuture jobManagerRunnerFuture = CompletableFuture.supplyAsync( + return CompletableFuture.supplyAsync( CheckedSupplier.unchecked(() -> jobManagerRunnerFactory.createJobManagerRunner( jobGraph, @@ -387,8 +388,6 @@ private CompletableFuture createJobManagerRunner(JobGraph jobG new DefaultJobManagerJobMetricGroupFactory(jobManagerMetricGroup), fatalErrorHandler)), rpcService.getExecutor()); - - return jobManagerRunnerFuture.thenApply(FunctionUtils.uncheckedFunction(this::startJobManagerRunner)); } private JobManagerRunner startJobManagerRunner(JobManagerRunner jobManagerRunner) throws Exception { From 6b5a00c891c09eeb1926cb8e39f02b2661ebc88b Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Mon, 9 Sep 2019 11:41:03 +0200 Subject: [PATCH 171/746] [hotfix] Annotate MemoryManager methods used for testing with the @VisibleForTesting --- .../java/org/apache/flink/runtime/memory/MemoryManager.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java index 664e5632afb9..6e0f7eb3925e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.memory; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.core.memory.HybridMemorySegment; import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.core.memory.MemorySegmentFactory; @@ -109,6 +110,7 @@ public class MemoryManager { * @param memorySize The total size of the memory to be managed by this memory manager. * @param numberOfSlots The number of slots of the task manager. */ + @VisibleForTesting public MemoryManager(long memorySize, int numberOfSlots) { this(memorySize, numberOfSlots, DEFAULT_PAGE_SIZE, MemoryType.HEAP, true); } @@ -224,6 +226,7 @@ public void shutdown() { * * @return True, if the memory manager is shut down, false otherwise. */ + @VisibleForTesting public boolean isShutdown() { return isShutDown; } @@ -233,6 +236,7 @@ public boolean isShutdown() { * * @return True, if the memory manager is empty and valid, false if it is not empty or corrupted. */ + @VisibleForTesting public boolean verifyEmpty() { synchronized (lock) { return isPreAllocated ? From 7a62d72bc9c431fc00390097a01e6d95708be8d3 Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Wed, 11 Sep 2019 15:25:28 +0200 Subject: [PATCH 172/746] [hotfix] Introduce MemoryManagerBuilder for tests --- .../flink/runtime/memory/MemoryManager.java | 11 ---- .../runtime/io/disk/ChannelViewsTest.java | 8 ++- .../io/disk/FileChannelStreamsITCase.java | 9 ++- .../io/disk/FileChannelStreamsTest.java | 6 +- .../SeekableFileChannelInputViewTest.java | 8 ++- .../runtime/io/disk/SpillingBufferTest.java | 3 +- .../io/disk/iomanager/IOManagerITCase.java | 3 +- .../runtime/memory/MemoryManagerBuilder.java | 65 +++++++++++++++++++ ...MemoryManagerConcurrentModReleaseTest.java | 13 +++- .../MemoryManagerLazyAllocationTest.java | 7 +- .../runtime/memory/MemoryManagerTest.java | 7 +- .../memory/MemorySegmentSimpleTest.java | 7 +- .../operators/drivers/TestTaskContext.java | 7 +- .../operators/hash/HashTableITCase.java | 5 +- .../NonReusingHashJoinIteratorITCase.java | 3 +- .../hash/ReOpenableHashTableITCase.java | 8 ++- .../hash/ReOpenableHashTableTestBase.java | 8 ++- .../hash/ReusingHashJoinIteratorITCase.java | 3 +- ...ckResettableMutableObjectIteratorTest.java | 3 +- ...NonReusingBlockResettableIteratorTest.java | 3 +- .../ReusingBlockResettableIteratorTest.java | 3 +- .../SpillingResettableIteratorTest.java | 7 +- ...ngResettableMutableObjectIteratorTest.java | 3 +- ...tractSortMergeOuterJoinIteratorITCase.java | 3 +- .../CombiningUnilateralSortMergerITCase.java | 3 +- .../operators/sort/ExternalSortITCase.java | 3 +- .../sort/ExternalSortLargeRecordsITCase.java | 3 +- .../sort/FixedLengthRecordSorterTest.java | 8 ++- .../sort/LargeRecordHandlerITCase.java | 14 +++- .../sort/LargeRecordHandlerTest.java | 20 ++++-- ...usingSortMergeInnerJoinIteratorITCase.java | 3 +- .../sort/NormalizedKeySorterTest.java | 8 ++- ...usingSortMergeInnerJoinIteratorITCase.java | 3 +- .../sort/UnilateralSortMergerTest.java | 6 +- .../testutils/BinaryOperatorTestBase.java | 3 +- .../operators/testutils/DriverTestBase.java | 3 +- .../operators/testutils/MockEnvironment.java | 3 +- .../testutils/UnaryOperatorTestBase.java | 3 +- .../util/HashVsSortMiniBenchmark.java | 8 ++- .../taskexecutor/TaskExecutorTest.java | 14 ++-- .../TaskManagerServicesBuilder.java | 14 ++-- .../runtime/taskmanager/TestTaskBuilder.java | 3 +- .../runtime/util/JvmExitOnFatalErrorTest.java | 3 +- .../runtime/tasks/StreamMockEnvironment.java | 3 +- .../tasks/StreamTaskTerminationTest.java | 4 +- .../hashtable/BinaryHashTableTest.java | 34 +++++----- .../runtime/hashtable/LongHashTableTest.java | 3 +- .../operators/aggregate/BytesHashMapTest.java | 40 +++++++++--- .../operators/aggregate/HashAggTest.java | 3 +- .../join/Int2SortMergeJoinOperatorTest.java | 3 +- .../join/SortMergeJoinIteratorTest.java | 3 +- .../BufferDataOverWindowOperatorTest.java | 3 +- .../sort/BinaryExternalSorterTest.java | 5 +- .../util/ResettableExternalBufferTest.java | 3 +- .../test/manual/MassiveStringSorting.java | 5 +- .../manual/MassiveStringValueSorting.java | 5 +- .../MassiveCaseClassSortingITCase.scala | 5 +- 57 files changed, 317 insertions(+), 130 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerBuilder.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java index 6e0f7eb3925e..3eac441d35a1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java @@ -104,17 +104,6 @@ public class MemoryManager { private boolean isShutDown; - /** - * Creates a memory manager with the given capacity, using the default page size. - * - * @param memorySize The total size of the memory to be managed by this memory manager. - * @param numberOfSlots The number of slots of the task manager. - */ - @VisibleForTesting - public MemoryManager(long memorySize, int numberOfSlots) { - this(memorySize, numberOfSlots, DEFAULT_PAGE_SIZE, MemoryType.HEAP, true); - } - /** * Creates a memory manager with the given capacity and given page size. * diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java index 4e94de25fea0..252a7a23a9fd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java @@ -22,7 +22,7 @@ import java.io.EOFException; import java.util.List; -import org.apache.flink.core.memory.MemoryType; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.junit.Assert; import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.runtime.io.disk.iomanager.BlockChannelReader; @@ -77,7 +77,11 @@ public class ChannelViewsTest @Before public void beforeTest() { - this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, MEMORY_PAGE_SIZE, MemoryType.HEAP, true); + this.memoryManager = MemoryManagerBuilder + .newBuilder() + .setMemorySize(MEMORY_SIZE) + .setPageSize(MEMORY_PAGE_SIZE) + .build(); this.ioManager = new IOManagerAsync(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java index c83e942f57b6..5d4cd03cc381 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java @@ -20,7 +20,7 @@ import static org.junit.Assert.*; -import org.apache.flink.core.memory.MemoryType; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.util.TestLogger; import org.junit.After; import org.junit.Before; @@ -67,8 +67,11 @@ public class FileChannelStreamsITCase extends TestLogger { @Before public void beforeTest() { - memManager = new MemoryManager(NUM_MEMORY_SEGMENTS * MEMORY_PAGE_SIZE, 1, - MEMORY_PAGE_SIZE, MemoryType.HEAP, true); + memManager = MemoryManagerBuilder + .newBuilder() + .setMemorySize(NUM_MEMORY_SEGMENTS * MEMORY_PAGE_SIZE) + .setPageSize(MEMORY_PAGE_SIZE) + .build(); ioManager = new IOManagerAsync(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java index 6bba3a6a28b8..8263572565d2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java @@ -26,13 +26,13 @@ import java.util.List; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.core.memory.MemoryType; import org.apache.flink.runtime.io.disk.iomanager.BlockChannelReader; import org.apache.flink.runtime.io.disk.iomanager.BlockChannelWriter; import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.types.StringValue; import org.junit.Test; @@ -43,7 +43,7 @@ public class FileChannelStreamsTest { @Test public void testCloseAndDeleteOutputView() { try (IOManager ioManager = new IOManagerAsync()) { - MemoryManager memMan = new MemoryManager(4 * 16*1024, 1, 16*1024, MemoryType.HEAP, true); + MemoryManager memMan = MemoryManagerBuilder.newBuilder().build(); List memory = new ArrayList(); memMan.allocatePages(new DummyInvokable(), memory, 4); @@ -73,7 +73,7 @@ public void testCloseAndDeleteOutputView() { @Test public void testCloseAndDeleteInputView() { try (IOManager ioManager = new IOManagerAsync()) { - MemoryManager memMan = new MemoryManager(4 * 16*1024, 1, 16*1024, MemoryType.HEAP, true); + MemoryManager memMan = MemoryManagerBuilder.newBuilder().build(); List memory = new ArrayList(); memMan.allocatePages(new DummyInvokable(), memory, 4); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java index 1c5a2ad3a293..cf858222855d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java @@ -25,12 +25,12 @@ import java.util.List; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.core.memory.MemoryType; import org.apache.flink.runtime.io.disk.iomanager.BlockChannelWriter; import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.junit.Test; @@ -44,7 +44,11 @@ public void testSeek() { // integers across 7.x pages (7 pages = 114.688 bytes, 8 pages = 131.072 bytes) try (IOManager ioManager = new IOManagerAsync()) { - MemoryManager memMan = new MemoryManager(4 * PAGE_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true); + MemoryManager memMan = MemoryManagerBuilder + .newBuilder() + .setMemorySize(4 * PAGE_SIZE) + .setPageSize(PAGE_SIZE) + .build(); List memory = new ArrayList(); memMan.allocatePages(new DummyInvokable(), memory, 4); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java index d7d2f28af0fb..79607444266c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.ListMemorySegmentSource; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.runtime.operators.testutils.TestData; import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.KeyMode; @@ -66,7 +67,7 @@ public class SpillingBufferTest { @Before public void beforeTest() { - memoryManager = new MemoryManager(MEMORY_SIZE, 1); + memoryManager = MemoryManagerBuilder.newBuilder().setMemorySize(MEMORY_SIZE).build(); ioManager = new IOManagerAsync(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java index 83e15bec318c..ae2bcbd80198 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java @@ -24,6 +24,7 @@ import java.util.List; import java.util.Random; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.util.TestLogger; import org.junit.Assert; @@ -59,7 +60,7 @@ public class IOManagerITCase extends TestLogger { @Before public void beforeTest() { - memoryManager = new MemoryManager(MEMORY_SIZE, 1); + memoryManager = MemoryManagerBuilder.newBuilder().setMemorySize(MEMORY_SIZE).build(); ioManager = new IOManagerAsync(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerBuilder.java new file mode 100644 index 000000000000..4bc7336f98a2 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerBuilder.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.memory; + +import org.apache.flink.core.memory.MemoryType; + +import static org.apache.flink.runtime.memory.MemoryManager.DEFAULT_PAGE_SIZE; + +/** Builder class for {@link MemoryManager}. */ +public class MemoryManagerBuilder { + private static final int DEFAULT_MEMORY_SIZE = 32 * DEFAULT_PAGE_SIZE; + + private long memorySize = DEFAULT_MEMORY_SIZE; + private int numberOfSlots = 1; + private int pageSize = DEFAULT_PAGE_SIZE; + private boolean preAllocateMemory = true; + + private MemoryManagerBuilder() { + + } + + public MemoryManagerBuilder setMemorySize(long memorySize) { + this.memorySize = memorySize; + return this; + } + + public MemoryManagerBuilder setNumberOfSlots(int numberOfSlots) { + this.numberOfSlots = numberOfSlots; + return this; + } + + public MemoryManagerBuilder setPageSize(int pageSize) { + this.pageSize = pageSize; + return this; + } + + public MemoryManagerBuilder setPreAllocateMemory(boolean preAllocateMemory) { + this.preAllocateMemory = preAllocateMemory; + return this; + } + + public MemoryManager build() { + return new MemoryManager(memorySize, numberOfSlots, pageSize, MemoryType.HEAP, preAllocateMemory); + } + + public static MemoryManagerBuilder newBuilder() { + return new MemoryManagerBuilder(); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerConcurrentModReleaseTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerConcurrentModReleaseTest.java index dac044e7e35d..cc4345e76cb3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerConcurrentModReleaseTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerConcurrentModReleaseTest.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.memory; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.core.memory.MemoryType; import org.junit.Test; @@ -40,7 +39,11 @@ public void testConcurrentModificationOnce() { final int numSegments = 10000; final int segmentSize = 4096; - MemoryManager memMan = new MemoryManager(numSegments * segmentSize, 1, segmentSize, MemoryType.HEAP, true); + MemoryManager memMan = MemoryManagerBuilder + .newBuilder() + .setMemorySize(numSegments * segmentSize) + .setPageSize(segmentSize) + .build(); ArrayList segs = new ListWithConcModExceptionOnFirstAccess<>(); memMan.allocatePages(this, segs, numSegments); @@ -59,7 +62,11 @@ public void testConcurrentModificationWhileReleasing() { final int numSegments = 10000; final int segmentSize = 4096; - MemoryManager memMan = new MemoryManager(numSegments * segmentSize, 1, segmentSize, MemoryType.HEAP, true); + MemoryManager memMan = MemoryManagerBuilder + .newBuilder() + .setMemorySize(numSegments * segmentSize) + .setPageSize(segmentSize) + .build(); ArrayList segs = new ArrayList<>(numSegments); memMan.allocatePages(this, segs, numSegments); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerLazyAllocationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerLazyAllocationTest.java index 46d0851b70fe..0d5bba07ed54 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerLazyAllocationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerLazyAllocationTest.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.memory; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.core.memory.MemoryType; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.operators.testutils.DummyInvokable; @@ -53,7 +52,11 @@ public class MemoryManagerLazyAllocationTest { @Before public void setUp() { - this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, false); + this.memoryManager = MemoryManagerBuilder + .newBuilder() + .setMemorySize(MEMORY_SIZE) + .setPageSize(PAGE_SIZE) + .build(); this.random = new Random(RANDOM_SEED); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java index 6847456ecb0f..70505258ff74 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.memory; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.core.memory.MemoryType; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.operators.testutils.DummyInvokable; @@ -53,7 +52,11 @@ public class MemoryManagerTest { @Before public void setUp() { - this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true); + this.memoryManager = MemoryManagerBuilder + .newBuilder() + .setMemorySize(MEMORY_SIZE) + .setPageSize(PAGE_SIZE) + .build(); this.random = new Random(RANDOM_SEED); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentSimpleTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentSimpleTest.java index 3a07b440ed31..fdac71abff44 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentSimpleTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentSimpleTest.java @@ -20,7 +20,6 @@ import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.core.memory.MemorySegmentFactory; -import org.apache.flink.core.memory.MemoryType; import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.junit.After; @@ -56,7 +55,11 @@ public class MemorySegmentSimpleTest { @Before public void setUp() throws Exception{ try { - this.manager = new MemoryManager(MANAGED_MEMORY_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true); + this.manager = MemoryManagerBuilder + .newBuilder() + .setMemorySize(MANAGED_MEMORY_SIZE) + .setPageSize(PAGE_SIZE) + .build(); this.segment = manager.allocatePages(new DummyInvokable(), 1).get(0); this.random = new Random(RANDOM_SEED); } catch (Exception e) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java index a4d14c41aad9..d0e25b50a600 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java @@ -24,10 +24,10 @@ import org.apache.flink.api.common.typeutils.TypeSerializerFactory; import org.apache.flink.api.java.typeutils.runtime.RuntimeSerializerFactory; import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.memory.MemoryType; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.operators.DriverStrategy; @@ -74,7 +74,10 @@ public class TestTaskContext implements TaskContext { public TestTaskContext() {} public TestTaskContext(long memoryInBytes) { - this.memoryManager = new MemoryManager(memoryInBytes, 1, 32 * 1024, MemoryType.HEAP, true); + this.memoryManager = MemoryManagerBuilder + .newBuilder() + .setMemorySize(memoryInBytes) + .build(); this.taskManageInfo = new TestingTaskManagerRuntimeInfo(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java index 1b86cec65314..dcccf4aa821e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java @@ -30,6 +30,7 @@ import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypePairComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.testutils.recordutils.RecordComparator; import org.apache.flink.runtime.testutils.recordutils.RecordSerializer; import org.apache.flink.core.memory.MemorySegment; @@ -94,8 +95,8 @@ public void setup() this.pairBuildSideComparator = new IntPairComparator(); this.pairProbeSideComparator = new IntPairComparator(); this.pairComparator = new IntPairPairComparator(); - - this.memManager = new MemoryManager(32 * 1024 * 1024,1); + + this.memManager = MemoryManagerBuilder.newBuilder().setMemorySize(32 * 1024 * 1024).build(); this.ioManager = new IOManagerAsync(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashJoinIteratorITCase.java index 553fd24b5542..b946f9f1890e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashJoinIteratorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashJoinIteratorITCase.java @@ -27,6 +27,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector; import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.runtime.operators.testutils.TestData; @@ -98,7 +99,7 @@ public void beforeTest() { this.pairRecordPairComparator = new IntPairTuplePairComparator(); this.recordPairPairComparator = new TupleIntPairPairComparator(); - this.memoryManager = new MemoryManager(MEMORY_SIZE, 1); + this.memoryManager = MemoryManagerBuilder.newBuilder().setMemorySize(MEMORY_SIZE).build(); this.ioManager = new IOManagerAsync(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReOpenableHashTableITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReOpenableHashTableITCase.java index 95f36e4731f0..12a03e4eae71 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReOpenableHashTableITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReOpenableHashTableITCase.java @@ -25,12 +25,12 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.core.memory.MemoryType; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryAllocationException; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.runtime.operators.testutils.TestData; import org.apache.flink.runtime.operators.testutils.UniformIntTupleGenerator; @@ -76,7 +76,11 @@ public void beforeTest() { this.recordProbeSideComparator = TestData.getIntIntTupleComparator(); this.pactRecordComparator = new GenericPairComparator(this.recordBuildSideComparator, this.recordProbeSideComparator); - this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true); + this.memoryManager = MemoryManagerBuilder + .newBuilder() + .setMemorySize(MEMORY_SIZE) + .setPageSize(PAGE_SIZE) + .build(); this.ioManager = new IOManagerAsync(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReOpenableHashTableTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReOpenableHashTableTestBase.java index 9acd87ab8643..537eca67ecbf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReOpenableHashTableTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReOpenableHashTableTestBase.java @@ -24,11 +24,11 @@ import org.apache.flink.api.common.typeutils.TypePairComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.core.memory.MemoryType; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.hash.NonReusingHashJoinIteratorITCase.TupleMatch; import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.runtime.operators.testutils.TestData; @@ -87,7 +87,11 @@ public void beforeTest() { this.recordProbeSideComparator = TestData.getIntIntTupleComparator(); this.pactRecordComparator = new GenericPairComparator(this.recordBuildSideComparator, this.recordProbeSideComparator); - this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true); + this.memoryManager = MemoryManagerBuilder + .newBuilder() + .setMemorySize(MEMORY_SIZE) + .setPageSize(PAGE_SIZE) + .build(); this.ioManager = new IOManagerAsync(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashJoinIteratorITCase.java index 9a69d8985bc9..76f7c3c04b07 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashJoinIteratorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashJoinIteratorITCase.java @@ -36,6 +36,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.hash.NonReusingHashJoinIteratorITCase.TupleMatch; import org.apache.flink.runtime.operators.hash.NonReusingHashJoinIteratorITCase.TupleIntPairMatch; import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector; @@ -106,7 +107,7 @@ public void beforeTest() { this.pairRecordPairComparator = new IntPairTuplePairComparator(); this.recordPairPairComparator = new TupleIntPairPairComparator(); - this.memoryManager = new MemoryManager(MEMORY_SIZE, 1); + this.memoryManager = MemoryManagerBuilder.newBuilder().setMemorySize(MEMORY_SIZE).build(); this.ioManager = new IOManagerAsync(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIteratorTest.java index 00011ae1d8d0..a1948a020e5c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIteratorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIteratorTest.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.List; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.junit.Assert; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.testutils.recordutils.RecordSerializer; @@ -54,7 +55,7 @@ public class BlockResettableMutableObjectIteratorTest { @Before public void startup() { // set up IO and memory manager - this.memman = new MemoryManager(MEMORY_CAPACITY, 1); + this.memman = MemoryManagerBuilder.newBuilder().setMemorySize(MEMORY_CAPACITY).build(); // create test objects this.objects = new ArrayList(20000); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIteratorTest.java index 01dbe1852c5a..941a9f20fa3d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIteratorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIteratorTest.java @@ -22,6 +22,7 @@ import java.util.Iterator; import java.util.List; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.junit.Assert; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.testutils.recordutils.RecordSerializer; @@ -53,7 +54,7 @@ public class NonReusingBlockResettableIteratorTest { @Before public void startup() { // set up IO and memory manager - this.memman = new MemoryManager(MEMORY_CAPACITY, 1); + this.memman = MemoryManagerBuilder.newBuilder().setMemorySize(MEMORY_CAPACITY).build(); // create test objects this.objects = new ArrayList(20000); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIteratorTest.java index ea4b66715023..77a29a4723a4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIteratorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIteratorTest.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.operators.resettable; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.testutils.recordutils.RecordSerializer; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryManager; @@ -52,7 +53,7 @@ public class ReusingBlockResettableIteratorTest { @Before public void startup() { // set up IO and memory manager - this.memman = new MemoryManager(MEMORY_CAPACITY, 1); + this.memman = MemoryManagerBuilder.newBuilder().setMemorySize(MEMORY_CAPACITY).build(); // create test objects this.objects = new ArrayList(20000); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableIteratorTest.java index 15c32fcad2ff..69b489a517cd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableIteratorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableIteratorTest.java @@ -24,11 +24,11 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.IntValueSerializer; -import org.apache.flink.core.memory.MemoryType; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.types.IntValue; @@ -57,7 +57,10 @@ public class SpillingResettableIteratorTest { @Before public void startup() { // set up IO and memory manager - this.memman = new MemoryManager(MEMORY_CAPACITY, 1, 32 * 1024, MemoryType.HEAP, true); + this.memman = MemoryManagerBuilder + .newBuilder() + .setMemorySize(MEMORY_CAPACITY) + .build(); this.ioman = new IOManagerAsync(); // create test objects diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIteratorTest.java index c442e7d7361c..505d62485c87 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIteratorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIteratorTest.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.operators.resettable; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.testutils.recordutils.RecordSerializer; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; @@ -54,7 +55,7 @@ public class SpillingResettableMutableObjectIteratorTest { @Before public void startup() { // set up IO and memory manager - this.memman = new MemoryManager(MEMORY_CAPACITY, 1); + this.memman = MemoryManagerBuilder.newBuilder().setMemorySize(MEMORY_CAPACITY).build(); this.ioman = new IOManagerAsync(); // create test objects diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java index 364dc32322c1..77a8dc6e1f05 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java @@ -38,6 +38,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.testutils.CollectionIterator; import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector; import org.apache.flink.runtime.operators.testutils.DummyInvokable; @@ -104,7 +105,7 @@ public void beforeTest() { comparator2 = typeInfo2.createComparator(new int[]{0}, new boolean[]{true}, 0, config); pairComp = new GenericPairComparator<>(comparator1, comparator2); - this.memoryManager = new MemoryManager(MEMORY_SIZE, 1); + this.memoryManager = MemoryManagerBuilder.newBuilder().setMemorySize(MEMORY_SIZE).build(); this.ioManager = new IOManagerAsync(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java index e50798d3ec35..af7be193b96b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java @@ -30,6 +30,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.runtime.operators.testutils.TestData; import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.KeyMode; @@ -81,7 +82,7 @@ public class CombiningUnilateralSortMergerITCase extends TestLogger { @SuppressWarnings("unchecked") @Before public void beforeTest() { - this.memoryManager = new MemoryManager(MEMORY_SIZE, 1); + this.memoryManager = MemoryManagerBuilder.newBuilder().setMemorySize(MEMORY_SIZE).build(); this.ioManager = new IOManagerAsync(); this.serializerFactory1 = TestData.getIntStringTupleSerializerFactory(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortITCase.java index 691a0654b421..4218ea05199b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortITCase.java @@ -27,6 +27,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.runtime.operators.testutils.RandomIntPairGenerator; import org.apache.flink.runtime.operators.testutils.TestData; @@ -77,7 +78,7 @@ public class ExternalSortITCase extends TestLogger { @SuppressWarnings("unchecked") @Before public void beforeTest() { - this.memoryManager = new MemoryManager(MEMORY_SIZE, 1); + this.memoryManager = MemoryManagerBuilder.newBuilder().setMemorySize(MEMORY_SIZE).build(); this.ioManager = new IOManagerAsync(); this.pactRecordSerializer = TestData.getIntStringTupleSerializerFactory(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortLargeRecordsITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortLargeRecordsITCase.java index 48d6ce75a4df..e13fad3c72b0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortLargeRecordsITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortLargeRecordsITCase.java @@ -33,6 +33,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.util.MutableObjectIterator; import org.apache.flink.util.TestLogger; @@ -66,7 +67,7 @@ public class ExternalSortLargeRecordsITCase extends TestLogger { @Before public void beforeTest() { - this.memoryManager = new MemoryManager(MEMORY_SIZE, 1); + this.memoryManager = MemoryManagerBuilder.newBuilder().setMemorySize(MEMORY_SIZE).build(); this.ioManager = new IOManagerAsync(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorterTest.java index fc33ffa8fcd7..ea23d99c9a01 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorterTest.java @@ -24,7 +24,6 @@ import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.core.memory.MemoryType; import org.apache.flink.runtime.io.disk.ChannelReaderInputViewIterator; import org.apache.flink.runtime.io.disk.iomanager.BlockChannelReader; import org.apache.flink.runtime.io.disk.iomanager.BlockChannelWriter; @@ -34,6 +33,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.runtime.operators.testutils.RandomIntPairGenerator; import org.apache.flink.runtime.operators.testutils.UniformIntPairGenerator; @@ -66,7 +66,11 @@ public class FixedLengthRecordSorterTest { @Before public void beforeTest() { - this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, MEMORY_PAGE_SIZE, MemoryType.HEAP, true); + this.memoryManager = MemoryManagerBuilder + .newBuilder() + .setMemorySize(MEMORY_SIZE) + .setPageSize(MEMORY_PAGE_SIZE) + .build(); this.ioManager = new IOManagerAsync(); this.serializer = new IntPairSerializer(); this.comparator = new IntPairComparator(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerITCase.java index e73ee8cab59a..b8243adb8e79 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerITCase.java @@ -29,7 +29,6 @@ import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.core.memory.MemoryType; import org.apache.flink.runtime.io.disk.FileChannelOutputView; import org.apache.flink.runtime.io.disk.SeekableFileChannelInputView; import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel; @@ -37,6 +36,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.types.Value; import org.apache.flink.util.MutableObjectIterator; @@ -63,7 +63,11 @@ public void testRecordHandlerCompositeKey() { final int NUM_RECORDS = 10; try (final IOManager ioMan = new IOManagerAsync()) { - final MemoryManager memMan = new MemoryManager(NUM_PAGES * PAGE_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true); + final MemoryManager memMan = MemoryManagerBuilder + .newBuilder() + .setMemorySize(NUM_PAGES * PAGE_SIZE) + .setPageSize(PAGE_SIZE) + .build(); final AbstractInvokable owner = new DummyInvokable(); final List initialMemory = memMan.allocatePages(owner, 6); @@ -195,7 +199,11 @@ public void fileTest() { FileIOChannel.ID channel = null; try (final IOManager ioMan = new IOManagerAsync()) { - final MemoryManager memMan = new MemoryManager(NUM_PAGES * PAGE_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true); + final MemoryManager memMan = MemoryManagerBuilder + .newBuilder() + .setMemorySize(NUM_PAGES * PAGE_SIZE) + .setPageSize(PAGE_SIZE) + .build(); final AbstractInvokable owner = new DummyInvokable(); final List memory = memMan.allocatePages(owner, NUM_PAGES); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerTest.java index 7e9acf60bb3d..b97b2d235f40 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerTest.java @@ -32,11 +32,11 @@ import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.core.memory.MemoryType; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.util.MutableObjectIterator; @@ -50,7 +50,11 @@ public void testEmptyRecordHandler() { final int NUM_PAGES = 50; try (final IOManager ioMan = new IOManagerAsync()) { - final MemoryManager memMan = new MemoryManager(NUM_PAGES * PAGE_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true); + final MemoryManager memMan = MemoryManagerBuilder + .newBuilder() + .setMemorySize(NUM_PAGES * PAGE_SIZE) + .setPageSize(PAGE_SIZE) + .build(); final AbstractInvokable owner = new DummyInvokable(); final List memory = memMan.allocatePages(owner, NUM_PAGES); @@ -95,7 +99,11 @@ public void testRecordHandlerSingleKey() { final int NUM_RECORDS = 25000; try (final IOManager ioMan = new IOManagerAsync()) { - final MemoryManager memMan = new MemoryManager(NUM_PAGES * PAGE_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true); + final MemoryManager memMan = MemoryManagerBuilder + .newBuilder() + .setMemorySize(NUM_PAGES * PAGE_SIZE) + .setPageSize(PAGE_SIZE) + .build(); final AbstractInvokable owner = new DummyInvokable(); final List initialMemory = memMan.allocatePages(owner, 6); @@ -176,7 +184,11 @@ public void testRecordHandlerCompositeKey() { final int NUM_RECORDS = 25000; try (final IOManager ioMan = new IOManagerAsync()) { - final MemoryManager memMan = new MemoryManager(NUM_PAGES * PAGE_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true); + final MemoryManager memMan = MemoryManagerBuilder + .newBuilder() + .setMemorySize(NUM_PAGES * PAGE_SIZE) + .setPageSize(PAGE_SIZE) + .build(); final AbstractInvokable owner = new DummyInvokable(); final List initialMemory = memMan.allocatePages(owner, 6); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeInnerJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeInnerJoinIteratorITCase.java index cc2736dba836..26f180e4459d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeInnerJoinIteratorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeInnerJoinIteratorITCase.java @@ -33,6 +33,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector; import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.runtime.operators.testutils.Match; @@ -105,7 +106,7 @@ public void beforeTest() { new TypeSerializer[] { IntSerializer.INSTANCE }); pairComparator = new GenericPairComparator, Tuple2>(comparator1, comparator2); - this.memoryManager = new MemoryManager(MEMORY_SIZE, 1); + this.memoryManager = MemoryManagerBuilder.newBuilder().setMemorySize(MEMORY_SIZE).build(); this.ioManager = new IOManagerAsync(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorterTest.java index 45f20cad5f1d..44f0f05cdc8a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorterTest.java @@ -25,8 +25,8 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.core.memory.MemoryType; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.runtime.operators.testutils.TestData; import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.KeyMode; @@ -58,7 +58,11 @@ public class NormalizedKeySorterTest { @Before public void beforeTest() { - this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, MEMORY_PAGE_SIZE, MemoryType.HEAP, true); + this.memoryManager = MemoryManagerBuilder + .newBuilder() + .setMemorySize(MEMORY_SIZE) + .setPageSize(MEMORY_PAGE_SIZE) + .build(); } @After diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeInnerJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeInnerJoinIteratorITCase.java index e6e382851538..712366ab14b1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeInnerJoinIteratorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeInnerJoinIteratorITCase.java @@ -33,6 +33,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector; import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.runtime.operators.testutils.Match; @@ -105,7 +106,7 @@ public void beforeTest() { new TypeSerializer[] { IntSerializer.INSTANCE }); pairComparator = new GenericPairComparator, Tuple2>(comparator1, comparator2); - this.memoryManager = new MemoryManager(MEMORY_SIZE, 1); + this.memoryManager = MemoryManagerBuilder.newBuilder().setMemorySize(MEMORY_SIZE).build(); this.ioManager = new IOManagerAsync(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/UnilateralSortMergerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/UnilateralSortMergerTest.java index c014e236eb72..607836e5e820 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/UnilateralSortMergerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/UnilateralSortMergerTest.java @@ -23,6 +23,7 @@ import org.apache.flink.runtime.io.disk.iomanager.ChannelWriterOutputView; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.runtime.operators.testutils.TestData; import org.apache.flink.runtime.util.EmptyMutableObjectIterator; @@ -51,7 +52,10 @@ public void testInMemorySorterDisposal() throws Exception { final TestingInMemorySorterFactory> inMemorySorterFactory = new TestingInMemorySorterFactory<>(); final int numPages = 32; - final MemoryManager memoryManager = new MemoryManager(MemoryManager.DEFAULT_PAGE_SIZE * numPages, 1); + final MemoryManager memoryManager = MemoryManagerBuilder + .newBuilder() + .setMemorySize(MemoryManager.DEFAULT_PAGE_SIZE * numPages) + .build(); final DummyInvokable parentTask = new DummyInvokable(); try (final IOManagerAsync ioManager = new IOManagerAsync()) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java index 1ffac3f2da90..19c424906755 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java @@ -30,6 +30,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.operators.Driver; @@ -102,7 +103,7 @@ protected BinaryOperatorTestBase(ExecutionConfig executionConfig, long memory, i this.perSortMem = perSortMemory; this.perSortFractionMem = (double) perSortMemory / totalMem; this.ioManager = new IOManagerAsync(); - this.memManager = totalMem > 0 ? new MemoryManager(totalMem, 1) : null; + this.memManager = totalMem > 0 ? MemoryManagerBuilder.newBuilder().setMemorySize(totalMem).build() : null; this.inputs = new ArrayList<>(); this.comparators = new ArrayList<>(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java index ec106ec035bc..61aab72fe520 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java @@ -28,6 +28,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.operators.Driver; @@ -107,7 +108,7 @@ protected DriverTestBase(ExecutionConfig executionConfig, long memory, int maxNu this.perSortMem = perSortMemory; this.perSortFractionMem = (double)perSortMemory/totalMem; this.ioManager = new IOManagerAsync(); - this.memManager = totalMem > 0 ? new MemoryManager(totalMem,1) : null; + this.memManager = totalMem > 0 ? MemoryManagerBuilder.newBuilder().setMemorySize(totalMem).build() : null; this.inputs = new ArrayList>(); this.comparators = new ArrayList>(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java index 3948dee46bb4..8ad9da085bcf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java @@ -40,6 +40,7 @@ import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.query.KvStateRegistry; import org.apache.flink.runtime.query.TaskKvStateRegistry; @@ -144,7 +145,7 @@ protected MockEnvironment( this.inputs = new LinkedList(); this.outputs = new LinkedList(); - this.memManager = new MemoryManager(memorySize, 1); + this.memManager = MemoryManagerBuilder.newBuilder().setMemorySize(memorySize).build(); this.ioManager = new IOManagerAsync(); this.taskManagerRuntimeInfo = taskManagerRuntimeInfo; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java index 73ea30c28441..2413fc2860c9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java @@ -30,6 +30,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.operators.Driver; @@ -108,7 +109,7 @@ protected UnaryOperatorTestBase(ExecutionConfig executionConfig, long memory, in this.perSortMem = perSortMemory; this.perSortFractionMem = (double)perSortMemory/totalMem; this.ioManager = new IOManagerAsync(); - this.memManager = totalMem > 0 ? new MemoryManager(totalMem, 1) : null; + this.memManager = totalMem > 0 ? MemoryManagerBuilder.newBuilder().setMemorySize(totalMem).build() : null; this.owner = new DummyInvokable(); Configuration config = new Configuration(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java index a9581f500118..a7d0421d7c99 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java @@ -24,11 +24,11 @@ import org.apache.flink.api.common.typeutils.TypePairComparator; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.core.memory.MemoryType; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.hash.ReusingBuildFirstHashJoinIterator; import org.apache.flink.runtime.operators.hash.ReusingBuildSecondHashJoinIterator; import org.apache.flink.runtime.operators.sort.ReusingMergeInnerJoinIterator; @@ -92,7 +92,11 @@ public void beforeTest() { this.comparator2 = TestData.getIntStringTupleComparator(); this.pairComparator11 = new GenericPairComparator(this.comparator1, this.comparator2); - this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true); + this.memoryManager = MemoryManagerBuilder + .newBuilder() + .setMemorySize(MEMORY_SIZE) + .setPageSize(PAGE_SIZE) + .build(); this.ioManager = new IOManagerAsync(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index be9fb538444d..9f099bafab7e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -26,7 +26,6 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.configuration.TaskManagerOptions; -import org.apache.flink.core.memory.MemoryType; import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.blob.BlobCacheService; import org.apache.flink.runtime.blob.TransientBlobKey; @@ -71,6 +70,7 @@ import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.query.KvStateRegistry; @@ -273,12 +273,12 @@ public void testShouldShutDownTaskManagerServicesInPostStop() throws Exception { ioManager.getSpillingDirectories(), Executors.directExecutor()); - final MemoryManager memoryManager = new MemoryManager( - 4096, - 1, - 4096, - MemoryType.HEAP, - false); + final MemoryManager memoryManager = MemoryManagerBuilder + .newBuilder() + .setMemorySize(4096) + .setPageSize(4096) + .setPreAllocateMemory(false) + .build(); nettyShuffleEnvironment.start(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java index c763db65a851..1185139b818f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java @@ -18,9 +18,9 @@ package org.apache.flink.runtime.taskexecutor; -import org.apache.flink.core.memory.MemoryType; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.io.network.TaskEventDispatcher; import org.apache.flink.runtime.memory.MemoryManager; @@ -53,12 +53,12 @@ public class TaskManagerServicesBuilder { public TaskManagerServicesBuilder() { taskManagerLocation = new LocalTaskManagerLocation(); - memoryManager = new MemoryManager( - MemoryManager.MIN_PAGE_SIZE, - 1, - MemoryManager.MIN_PAGE_SIZE, - MemoryType.HEAP, - false); + memoryManager = MemoryManagerBuilder + .newBuilder() + .setMemorySize(MemoryManager.MIN_PAGE_SIZE) + .setPageSize(MemoryManager.MIN_PAGE_SIZE) + .setPreAllocateMemory(false) + .build(); ioManager = mock(IOManager.class); shuffleEnvironment = mock(ShuffleEnvironment.class); kvStateService = new KvStateService(new KvStateRegistry(), null, null); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TestTaskBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TestTaskBuilder.java index ec1204ff2175..f907bc070920 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TestTaskBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TestTaskBuilder.java @@ -43,6 +43,7 @@ import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; @@ -187,7 +188,7 @@ public Task build() throws Exception { resultPartitions, inputGates, 0, - new MemoryManager(1024 * 1024, 1), + MemoryManagerBuilder.newBuilder().setMemorySize(1024 * 1024).build(), mock(IOManager.class), shuffleEnvironment, kvStateService, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java index 02ca74da61ae..86d331cea32c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java @@ -43,6 +43,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.io.network.TaskEventDispatcher; import org.apache.flink.runtime.taskexecutor.NoOpPartitionProducerStateChecker; @@ -161,7 +162,7 @@ public static void main(String[] args) throws Exception { final TaskInformation taskInformation = new TaskInformation( jobVertexId, "Test Task", 1, 1, OomInvokable.class.getName(), new Configuration()); - final MemoryManager memoryManager = new MemoryManager(1024 * 1024, 1); + final MemoryManager memoryManager = MemoryManagerBuilder.newBuilder().setMemorySize(1024 * 1024).build(); final IOManager ioManager = new IOManagerAsync(); final ShuffleEnvironment shuffleEnvironment = new NettyShuffleEnvironmentBuilder().build(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java index f9fa384ce8a0..4de06102cb88 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java @@ -41,6 +41,7 @@ import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; @@ -157,7 +158,7 @@ public StreamMockEnvironment( this.taskConfiguration = taskConfig; this.inputs = new LinkedList(); this.outputs = new LinkedList(); - this.memManager = new MemoryManager(memorySize, 1); + this.memManager = MemoryManagerBuilder.newBuilder().setMemorySize(memorySize).build(); this.ioManager = new IOManagerAsync(); this.taskStateManager = Preconditions.checkNotNull(taskStateManager); this.aggregateManager = new TestGlobalAggregateManager(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java index 71ef42175be8..3919f4e61349 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java @@ -49,7 +49,7 @@ import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; -import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.query.KvStateRegistry; import org.apache.flink.runtime.query.TaskKvStateRegistry; @@ -172,7 +172,7 @@ public void testConcurrentAsyncCheckpointCannotFailFinishedStreamTask() throws E Collections.emptyList(), Collections.emptyList(), 0, - new MemoryManager(32L * 1024L, 1), + MemoryManagerBuilder.newBuilder().setMemorySize(32L * 1024L).build(), new IOManagerAsync(), shuffleEnvironment, new KvStateService(new KvStateRegistry(), null, null), diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/BinaryHashTableTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/BinaryHashTableTest.java index 5cfa84085c25..dd39c95d163a 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/BinaryHashTableTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/BinaryHashTableTest.java @@ -22,11 +22,11 @@ import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.core.memory.MemoryType; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.memory.MemoryAllocationException; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.testutils.UnionIterator; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.dataformat.BaseRow; @@ -132,7 +132,7 @@ public void testInMemoryMutableHashTable() throws IOException { // create a probe input that gives 10 million pairs with 10 values sharing a key MutableObjectIterator probeInput = new UniformBinaryRowGenerator(numKeys, probeValsPerKey, true); - MemoryManager memManager = new MemoryManager(896 * PAGE_SIZE, 1); + MemoryManager memManager = MemoryManagerBuilder.newBuilder().setMemorySize(896 * PAGE_SIZE).build(); // ---------------------------------------------------------------------------------------- final BinaryHashTable table = newBinaryHashTable( this.buildSideSerializer, this.probeSideSerializer, @@ -218,7 +218,11 @@ public void testSpillingHashJoinOneRecursionPerformance() throws IOException { MutableObjectIterator probeInput = new UniformBinaryRowGenerator(numKeys, probeValsPerKey, true); // allocate the memory for the HashTable - MemoryManager memManager = new MemoryManager(200 * PAGE_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true); + MemoryManager memManager = MemoryManagerBuilder + .newBuilder() + .setMemorySize(200 * PAGE_SIZE) + .setPageSize(PAGE_SIZE) + .build(); final BinaryHashTable table = newBinaryHashTable( this.buildSideSerializer, this.probeSideSerializer, @@ -257,7 +261,7 @@ public void testSpillingHashJoinOneRecursionValidity() throws IOException { HashMap map = new HashMap<>(numKeys); // ---------------------------------------------------------------------------------------- - MemoryManager memManager = new MemoryManager(896 * PAGE_SIZE, 1); + MemoryManager memManager = MemoryManagerBuilder.newBuilder().setMemorySize(896 * PAGE_SIZE).build(); final BinaryHashTable table = newBinaryHashTable( this.buildSideSerializer, this.probeSideSerializer, new MyProjection(), new MyProjection(), memManager, @@ -332,7 +336,7 @@ public void testSpillingHashJoinWithMassiveCollisions() throws IOException { // create the map for validating the results HashMap map = new HashMap<>(numKeys); - MemoryManager memManager = new MemoryManager(896 * PAGE_SIZE, 1); + MemoryManager memManager = MemoryManagerBuilder.newBuilder().setMemorySize(896 * PAGE_SIZE).build(); // ---------------------------------------------------------------------------------------- final BinaryHashTable table = newBinaryHashTable( @@ -450,7 +454,7 @@ public void testSpillingHashJoinWithTwoRecursions() throws IOException { HashMap map = new HashMap<>(numKeys); // ---------------------------------------------------------------------------------------- - MemoryManager memManager = new MemoryManager(896 * PAGE_SIZE, 1); + MemoryManager memManager = MemoryManagerBuilder.newBuilder().setMemorySize(896 * PAGE_SIZE).build(); final BinaryHashTable table = newBinaryHashTable( this.buildSideSerializer, this.probeSideSerializer, new MyProjection(), new MyProjection(), memManager, @@ -529,7 +533,7 @@ public void testFailingHashJoinTooManyRecursions() throws IOException { probes.add(probe3); MutableObjectIterator probeInput = new UnionIterator<>(probes); // ---------------------------------------------------------------------------------------- - MemoryManager memManager = new MemoryManager(896 * PAGE_SIZE, 1); + MemoryManager memManager = MemoryManagerBuilder.newBuilder().setMemorySize(896 * PAGE_SIZE).build(); final BinaryHashTable table = newBinaryHashTable( this.buildSideSerializer, this.probeSideSerializer, new MyProjection(), new MyProjection(), memManager, @@ -562,7 +566,7 @@ public void testSparseProbeSpilling() throws IOException, MemoryAllocationExcept MutableObjectIterator buildInput = new UniformBinaryRowGenerator( numBuildKeys, numBuildVals, false); - MemoryManager memManager = new MemoryManager(128 * PAGE_SIZE, 1); + MemoryManager memManager = MemoryManagerBuilder.newBuilder().setMemorySize(128 * PAGE_SIZE).build(); final BinaryHashTable table = newBinaryHashTable( this.buildSideSerializer, this.probeSideSerializer, new MyProjection(), new MyProjection(), memManager, @@ -593,7 +597,7 @@ public void testSparseProbeSpillingWithOuterJoin() throws IOException { MutableObjectIterator buildInput = new UniformBinaryRowGenerator( numBuildKeys, numBuildVals, false); - MemoryManager memManager = new MemoryManager(96 * PAGE_SIZE, 1); + MemoryManager memManager = MemoryManagerBuilder.newBuilder().setMemorySize(96 * PAGE_SIZE).build(); final BinaryHashTable table = new BinaryHashTable(conf, new Object(), this.buildSideSerializer, this.probeSideSerializer, new MyProjection(), new MyProjection(), memManager, 96 * PAGE_SIZE, @@ -622,7 +626,7 @@ public void validateSpillingDuringInsertion() throws IOException, MemoryAllocati final int numProbeVals = 1; MutableObjectIterator buildInput = new UniformBinaryRowGenerator(numBuildKeys, numBuildVals, false); - MemoryManager memManager = new MemoryManager(85 * PAGE_SIZE, 1); + MemoryManager memManager = MemoryManagerBuilder.newBuilder().setMemorySize(85 * PAGE_SIZE).build(); final BinaryHashTable table = newBinaryHashTable( this.buildSideSerializer, this.probeSideSerializer, new MyProjection(), new MyProjection(), memManager, @@ -653,7 +657,7 @@ public void testBucketsNotFulfillSegment() throws Exception { MutableObjectIterator probeInput = new UniformBinaryRowGenerator(numKeys, probeValsPerKey, true); // allocate the memory for the HashTable - MemoryManager memManager = new MemoryManager(35 * PAGE_SIZE, 1); + MemoryManager memManager = MemoryManagerBuilder.newBuilder().setMemorySize(35 * PAGE_SIZE).build(); // ---------------------------------------------------------------------------------------- final BinaryHashTable table = new BinaryHashTable(conf, new Object(), @@ -694,7 +698,7 @@ public void testHashWithBuildSideOuterJoin1() throws Exception { // create a probe input that gives 20000 pairs with 1 values sharing a key MutableObjectIterator probeInput = new UniformBinaryRowGenerator(numKeys, probeValsPerKey, true); - MemoryManager memManager = new MemoryManager(35 * PAGE_SIZE, 1); + MemoryManager memManager = MemoryManagerBuilder.newBuilder().setMemorySize(35 * PAGE_SIZE).build(); // allocate the memory for the HashTable final BinaryHashTable table = new BinaryHashTable(conf, new Object(), this.buildSideSerializer, this.probeSideSerializer, @@ -727,7 +731,7 @@ public void testHashWithBuildSideOuterJoin2() throws Exception { MutableObjectIterator probeInput = new UniformBinaryRowGenerator(numKeys, probeValsPerKey, true); // allocate the memory for the HashTable - MemoryManager memManager = new MemoryManager(35 * PAGE_SIZE, 1); + MemoryManager memManager = MemoryManagerBuilder.newBuilder().setMemorySize(35 * PAGE_SIZE).build(); final BinaryHashTable table = newBinaryHashTable( this.buildSideSerializer, this.probeSideSerializer, new MyProjection(), new MyProjection(), memManager, @@ -746,7 +750,7 @@ public void testHashWithBuildSideOuterJoin2() throws Exception { public void testRepeatBuildJoin() throws Exception { final int numKeys = 500; final int probeValsPerKey = 1; - MemoryManager memManager = new MemoryManager(40 * PAGE_SIZE, 1); + MemoryManager memManager = MemoryManagerBuilder.newBuilder().setMemorySize(40 * PAGE_SIZE).build(); MutableObjectIterator buildInput = new MutableObjectIterator() { int cnt = 0; @@ -815,7 +819,7 @@ public BinaryRow next() throws IOException { return row; } }; - MemoryManager memManager = new MemoryManager(35 * PAGE_SIZE, 1); + MemoryManager memManager = MemoryManagerBuilder.newBuilder().setMemorySize(35 * PAGE_SIZE).build(); MutableObjectIterator probeInput = new UniformBinaryRowGenerator(numKeys, probeValsPerKey, true); final BinaryHashTable table = new BinaryHashTable( diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/LongHashTableTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/LongHashTableTest.java index 9edc5d6a9932..c32797ba75f6 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/LongHashTableTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/LongHashTableTest.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.memory.MemoryAllocationException; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.testutils.UnionIterator; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.dataformat.BaseRow; @@ -60,7 +61,7 @@ public class LongHashTableTest { private IOManager ioManager; private BinaryRowSerializer buildSideSerializer; private BinaryRowSerializer probeSideSerializer; - private MemoryManager memManager = new MemoryManager(896 * PAGE_SIZE, 1); + private MemoryManager memManager = MemoryManagerBuilder.newBuilder().setMemorySize(896 * PAGE_SIZE).build(); private boolean useCompress; private Configuration conf; diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/aggregate/BytesHashMapTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/aggregate/BytesHashMapTest.java index 573422e67ee9..3608c6825134 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/aggregate/BytesHashMapTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/aggregate/BytesHashMapTest.java @@ -20,9 +20,9 @@ import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.core.memory.MemorySegmentFactory; -import org.apache.flink.core.memory.MemoryType; import org.apache.flink.runtime.io.disk.RandomAccessInputView; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.table.dataformat.BinaryRow; import org.apache.flink.table.dataformat.BinaryRowWriter; import org.apache.flink.table.dataformat.BinaryString; @@ -95,7 +95,11 @@ public void testHashSetMode() throws IOException { rowLength(RowType.of(keyTypes)), PAGE_SIZE); int memorySize = numMemSegments * PAGE_SIZE; - MemoryManager memoryManager = new MemoryManager(numMemSegments * PAGE_SIZE, 32); + MemoryManager memoryManager = MemoryManagerBuilder + .newBuilder() + .setMemorySize(numMemSegments * PAGE_SIZE) + .setNumberOfSlots(32) + .build(); BytesHashMap table = new BytesHashMap(this, memoryManager, memorySize, keyTypes, new LogicalType[]{}); @@ -117,7 +121,11 @@ public void testBuildAndRetrieve() throws Exception { rowLength(RowType.of(keyTypes)), PAGE_SIZE); int memorySize = numMemSegments * PAGE_SIZE; - MemoryManager memoryManager = new MemoryManager(memorySize, 32); + MemoryManager memoryManager = MemoryManagerBuilder + .newBuilder() + .setMemorySize(memorySize) + .setNumberOfSlots(32) + .build(); BytesHashMap table = new BytesHashMap(this, memoryManager, memorySize, keyTypes, valueTypes); @@ -141,7 +149,11 @@ public void testBuildAndUpdate() throws Exception { PAGE_SIZE); int memorySize = numMemSegments * PAGE_SIZE; - MemoryManager memoryManager = new MemoryManager(memorySize, 32); + MemoryManager memoryManager = MemoryManagerBuilder + .newBuilder() + .setMemorySize(memorySize) + .setNumberOfSlots(32) + .build(); BytesHashMap table = new BytesHashMap(this, memoryManager, memorySize, keyTypes, valueTypes); @@ -165,7 +177,11 @@ public void testRest() throws Exception { int memorySize = numMemSegments * PAGE_SIZE; - MemoryManager memoryManager = new MemoryManager(memorySize, 32); + MemoryManager memoryManager = MemoryManagerBuilder + .newBuilder() + .setMemorySize(memorySize) + .setNumberOfSlots(32) + .build(); BytesHashMap table = new BytesHashMap(this, memoryManager, memorySize, keyTypes, valueTypes); @@ -191,9 +207,11 @@ public void testResetAndOutput() throws Exception { int minMemorySize = reservedMemSegments * PAGE_SIZE; - MemoryManager memoryManager = new MemoryManager( - minMemorySize, 32, MemoryManager.DEFAULT_PAGE_SIZE, - MemoryType.HEAP, true); + MemoryManager memoryManager = MemoryManagerBuilder + .newBuilder() + .setMemorySize(minMemorySize) + .setNumberOfSlots(32) + .build(); BytesHashMap table = new BytesHashMap(this, memoryManager, minMemorySize, keyTypes, valueTypes, true); @@ -265,7 +283,11 @@ public void testSingleKeyMultipleOps() throws Exception { PAGE_SIZE); int memorySize = numMemSegments * PAGE_SIZE; - MemoryManager memoryManager = new MemoryManager(memorySize, 32); + MemoryManager memoryManager = MemoryManagerBuilder + .newBuilder() + .setMemorySize(memorySize) + .setNumberOfSlots(32) + .build(); BytesHashMap table = new BytesHashMap(this, memoryManager, memorySize, keyTypes, valueTypes); final Random rnd = new Random(RANDOM_SEED); diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/aggregate/HashAggTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/aggregate/HashAggTest.java index abc0cddc9309..003143baa7b0 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/aggregate/HashAggTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/aggregate/HashAggTest.java @@ -22,6 +22,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.table.dataformat.BaseRow; import org.apache.flink.table.dataformat.GenericRow; @@ -43,7 +44,7 @@ public class HashAggTest { private static final int MEMORY_SIZE = 1024 * 1024 * 32; private Map outputMap = new HashMap<>(); - private MemoryManager memoryManager = new MemoryManager(MEMORY_SIZE, 1); + private MemoryManager memoryManager = MemoryManagerBuilder.newBuilder().setMemorySize(MEMORY_SIZE).build(); private IOManager ioManager; private SumHashAggTestOperator operator; diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/Int2SortMergeJoinOperatorTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/Int2SortMergeJoinOperatorTest.java index 34f9b59e4392..be3ab652d3a9 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/Int2SortMergeJoinOperatorTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/Int2SortMergeJoinOperatorTest.java @@ -21,6 +21,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.table.dataformat.BinaryRow; import org.apache.flink.table.runtime.generated.GeneratedJoinCondition; @@ -71,7 +72,7 @@ public static Collection parameters() { @Before public void setup() { - this.memManager = new MemoryManager(36 * 1024 * 1024, 1); + this.memManager = MemoryManagerBuilder.newBuilder().setMemorySize(36 * 1024 * 1024).build(); this.ioManager = new IOManagerAsync(); } diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/SortMergeJoinIteratorTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/SortMergeJoinIteratorTest.java index cb245ebfa363..b1ed06b85521 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/SortMergeJoinIteratorTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/SortMergeJoinIteratorTest.java @@ -23,6 +23,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.memory.MemoryAllocationException; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.table.dataformat.BaseRow; import org.apache.flink.table.dataformat.BinaryRow; import org.apache.flink.table.dataformat.BinaryRowWriter; @@ -73,7 +74,7 @@ public static Collection parameters() { @Before public void before() throws MemoryAllocationException { - this.memManager = new MemoryManager(MEMORY_SIZE, 1); + this.memManager = MemoryManagerBuilder.newBuilder().setMemorySize(MEMORY_SIZE).build(); this.ioManager = new IOManagerAsync(); this.serializer = new BinaryRowSerializer(1); } diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/over/BufferDataOverWindowOperatorTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/over/BufferDataOverWindowOperatorTest.java index 8cb650cb24c1..eed94f111911 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/over/BufferDataOverWindowOperatorTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/over/BufferDataOverWindowOperatorTest.java @@ -22,6 +22,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -69,7 +70,7 @@ public class BufferDataOverWindowOperatorTest { new RowType.RowField("f0", new BigIntType()))); private List collect; - private MemoryManager memoryManager = new MemoryManager(MEMORY_SIZE, 1); + private MemoryManager memoryManager = MemoryManagerBuilder.newBuilder().setMemorySize(MEMORY_SIZE).build(); private IOManager ioManager; private BufferDataOverWindowOperator operator; private GeneratedRecordComparator boundComparator = new GeneratedRecordComparator("", "", new Object[0]) { diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/sort/BinaryExternalSorterTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/sort/BinaryExternalSorterTest.java index 23dc50da5ca5..54eec5bae84a 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/sort/BinaryExternalSorterTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/sort/BinaryExternalSorterTest.java @@ -23,6 +23,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.dataformat.BaseRow; import org.apache.flink.table.dataformat.BinaryRow; @@ -94,7 +95,7 @@ private static String getString(int count) { @SuppressWarnings("unchecked") @Before public void beforeTest() { - this.memoryManager = new MemoryManager(MEMORY_SIZE, 1); + this.memoryManager = MemoryManagerBuilder.newBuilder().setMemorySize(MEMORY_SIZE).build(); this.serializer = new BinaryRowSerializer(2); this.conf.setInteger(ExecutionConfigOptions.TABLE_EXEC_SORT_MAX_NUM_FILE_HANDLES, 128); } @@ -121,7 +122,7 @@ public void testSortTwoBufferInMemory() throws Exception { LOG.debug("initializing sortmerger"); //there are two sort buffer if sortMemory > 100 * 1024 * 1024. - MemoryManager memoryManager = new MemoryManager(1024 * 1024 * 101, 1); + MemoryManager memoryManager = MemoryManagerBuilder.newBuilder().setMemorySize(1024 * 1024 * 101).build(); long minMemorySize = memoryManager.computeNumberOfPages(1) * MemoryManager.DEFAULT_PAGE_SIZE; BinaryExternalSorter sorter = new BinaryExternalSorter( new Object(), diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/ResettableExternalBufferTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/ResettableExternalBufferTest.java index 01b8d77587ce..733946a6d1a3 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/ResettableExternalBufferTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/ResettableExternalBufferTest.java @@ -21,6 +21,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.memory.MemoryAllocationException; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.table.dataformat.BinaryRow; import org.apache.flink.table.dataformat.BinaryRowWriter; import org.apache.flink.table.dataformat.BinaryString; @@ -62,7 +63,7 @@ public class ResettableExternalBufferTest { @Before public void before() { - this.memManager = new MemoryManager(MEMORY_SIZE, 1); + this.memManager = MemoryManagerBuilder.newBuilder().setMemorySize(MEMORY_SIZE).build(); this.ioManager = new IOManagerAsync(); this.random = new Random(); this.serializer = new BinaryRowSerializer(1); diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringSorting.java b/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringSorting.java index 1d48eee2445b..0b5e79871d62 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringSorting.java +++ b/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringSorting.java @@ -30,6 +30,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.sort.UnilateralSortMerger; import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.util.MutableObjectIterator; @@ -85,7 +86,7 @@ public void testStringSorting() { MemoryManager mm = null; try (IOManager ioMan = new IOManagerAsync()) { - mm = new MemoryManager(1024 * 1024, 1); + mm = MemoryManagerBuilder.newBuilder().setMemorySize(1024 * 1024).build(); TypeSerializer serializer = StringSerializer.INSTANCE; TypeComparator comparator = new StringComparator(true); @@ -179,7 +180,7 @@ public void testStringTuplesSorting() { MemoryManager mm = null; try (IOManager ioMan = new IOManagerAsync()) { - mm = new MemoryManager(1024 * 1024, 1); + mm = MemoryManagerBuilder.newBuilder().setMemorySize(1024 * 1024).build(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) new TypeHint>(){}.getTypeInfo(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringValueSorting.java b/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringValueSorting.java index 861f1dfe84b4..12e021b1ea58 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringValueSorting.java +++ b/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringValueSorting.java @@ -30,6 +30,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.operators.sort.UnilateralSortMerger; import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.apache.flink.types.StringValue; @@ -86,7 +87,7 @@ public void testStringValueSorting() { MemoryManager mm = null; try (IOManager ioMan = new IOManagerAsync()) { - mm = new MemoryManager(1024 * 1024, 1); + mm = MemoryManagerBuilder.newBuilder().setMemorySize(1024 * 1024).build(); TypeSerializer serializer = new CopyableValueSerializer(StringValue.class); TypeComparator comparator = new CopyableValueComparator(true, StringValue.class); @@ -183,7 +184,7 @@ public void testStringValueTuplesSorting() { MemoryManager mm = null; try (IOManager ioMan = new IOManagerAsync()) { - mm = new MemoryManager(1024 * 1024, 1); + mm = MemoryManagerBuilder.newBuilder().setMemorySize(1024 * 1024).build(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) new TypeHint>(){}.getTypeInfo(); diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/manual/MassiveCaseClassSortingITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/manual/MassiveCaseClassSortingITCase.scala index a970c1de565e..4ccb4e75c7ea 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/manual/MassiveCaseClassSortingITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/manual/MassiveCaseClassSortingITCase.scala @@ -27,11 +27,10 @@ import org.apache.flink.api.common.typeutils.CompositeType import org.apache.flink.api.java.typeutils.runtime.RuntimeSerializerFactory import org.apache.flink.api.scala._ import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync -import org.apache.flink.runtime.memory.MemoryManager +import org.apache.flink.runtime.memory.MemoryManagerBuilder import org.apache.flink.runtime.operators.sort.UnilateralSortMerger import org.apache.flink.runtime.operators.testutils.DummyInvokable import org.apache.flink.util.{MutableObjectIterator, TestLogger} - import org.junit.Assert._ /** @@ -90,7 +89,7 @@ class MassiveCaseClassSortingITCase extends TestLogger { 0, new ExecutionConfig) - val mm = new MemoryManager(1024 * 1024, 1) + val mm = MemoryManagerBuilder.newBuilder.setMemorySize(1024 * 1024).build val ioMan = new IOManagerAsync() sorter = new UnilateralSortMerger[StringTuple](mm, ioMan, inputIterator, From d8bfc30d0553473c5a4c399b6c90fd335b793af0 Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Mon, 9 Sep 2019 18:14:40 +0200 Subject: [PATCH 173/746] [hotfix] Remove unsed methods and fields in MemoryManager --- .../flink/runtime/memory/MemoryManager.java | 54 ------------------- 1 file changed, 54 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java index 3eac441d35a1..8958ae368ad2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java @@ -76,12 +76,6 @@ public class MemoryManager { /** Memory segments allocated per memory owner. */ private final HashMap> allocatedSegments; - /** The type of memory governed by this memory manager. */ - private final MemoryType memoryType; - - /** Mask used to round down sizes to multiples of the page size. */ - private final long roundingMask; - /** The size of the memory segments. */ private final int pageSize; @@ -130,13 +124,11 @@ public MemoryManager(long memorySize, int numberOfSlots, int pageSize, throw new IllegalArgumentException("The given page size is not a power of two."); } - this.memoryType = memoryType; this.memorySize = memorySize; this.numberOfSlots = numberOfSlots; // assign page size and bit utilities this.pageSize = pageSize; - this.roundingMask = ~((long) (pageSize - 1)); final long numPagesLong = memorySize / pageSize; if (numPagesLong > Integer.MAX_VALUE) { @@ -503,24 +495,6 @@ public void releaseAll(Object owner) { // Properties, sizes and size conversions // ------------------------------------------------------------------------ - /** - * Gets the type of memory (heap / off-heap) managed by this memory manager. - * - * @return The type of memory managed by this memory manager. - */ - public MemoryType getMemoryType() { - return memoryType; - } - - /** - * Checks whether this memory manager pre-allocates the memory. - * - * @return True if the memory manager pre-allocates the memory, false if it allocates as needed. - */ - public boolean isPreAllocated() { - return isPreAllocated; - } - /** * Gets the size of the pages handled by the memory manager. * @@ -539,15 +513,6 @@ public long getMemorySize() { return memorySize; } - /** - * Gets the total number of memory pages managed by this memory manager. - * - * @return The total number of memory pages managed by this memory manager. - */ - public int getTotalNumPages() { - return totalNumPages; - } - /** * Computes to how many pages the given number of bytes corresponds. If the given number of bytes is not an * exact multiple of a page size, the result is rounded down, such that a portion of the memory (smaller @@ -564,25 +529,6 @@ public int computeNumberOfPages(double fraction) { return (int) (totalNumPages * fraction / numberOfSlots); } - /** - * Computes the memory size of the fraction per slot. - * - * @param fraction The fraction of the memory of the task slot. - * @return The number of pages corresponding to the memory fraction. - */ - public long computeMemorySize(double fraction) { - return pageSize * (long) computeNumberOfPages(fraction); - } - - /** - * Rounds the given value down to a multiple of the memory manager's page size. - * - * @return The given value, rounded down to a multiple of the page size. - */ - public long roundDownToPageSizeMultiple(long numBytes) { - return numBytes & roundingMask; - } - // ------------------------------------------------------------------------ // Memory Pools From 8f889a7967061c9dd3aa18af78ca50e285836860 Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Thu, 12 Sep 2019 14:25:44 +0200 Subject: [PATCH 174/746] [hotfix] Checkstyle fixes in MemoryManager --- .../org/apache/flink/runtime/memory/MemoryManager.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java index 8958ae368ad2..38156cb3a566 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java @@ -242,9 +242,9 @@ public boolean verifyEmpty() { * of memory pages any more. */ public List allocatePages(Object owner, int numPages) throws MemoryAllocationException { - final ArrayList segs = new ArrayList(numPages); - allocatePages(owner, segs, numPages); - return segs; + List segments = new ArrayList<>(numPages); + allocatePages(owner, segments, numPages); + return segments; } /** @@ -526,6 +526,7 @@ public int computeNumberOfPages(double fraction) { throw new IllegalArgumentException("The fraction of memory to allocate must within (0, 1]."); } + //noinspection NumericCastThatLosesPrecision return (int) (totalNumPages * fraction / numberOfSlots); } From ffa7bdd5718a2f6c5e746d636b1515945cfb456a Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Tue, 10 Sep 2019 16:52:00 +0200 Subject: [PATCH 175/746] [hotfix] Refactor MemoryManager constructor --- .../flink/runtime/memory/MemoryManager.java | 104 ++++++++++-------- 1 file changed, 57 insertions(+), 47 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java index 38156cb3a566..c2e2c111927c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java @@ -19,11 +19,13 @@ package org.apache.flink.runtime.memory; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.core.memory.HybridMemorySegment; import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.core.memory.MemorySegmentFactory; import org.apache.flink.core.memory.MemoryType; import org.apache.flink.util.MathUtils; +import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -108,58 +110,22 @@ public class MemoryManager { * @param preAllocateMemory True, if the memory manager should immediately allocate all memory, false * if it should allocate and release the memory as needed. */ - public MemoryManager(long memorySize, int numberOfSlots, int pageSize, - MemoryType memoryType, boolean preAllocateMemory) { - // sanity checks - if (memoryType == null) { - throw new NullPointerException(); - } - if (memorySize <= 0) { - throw new IllegalArgumentException("Size of total memory must be positive."); - } - if (pageSize < MIN_PAGE_SIZE) { - throw new IllegalArgumentException("The page size must be at least " + MIN_PAGE_SIZE + " bytes."); - } - if (!MathUtils.isPowerOf2(pageSize)) { - throw new IllegalArgumentException("The given page size is not a power of two."); - } - + public MemoryManager( + long memorySize, + int numberOfSlots, + int pageSize, + MemoryType memoryType, + boolean preAllocateMemory) { + sanityCheck(memorySize, pageSize, memoryType, preAllocateMemory); + + this.allocatedSegments = new HashMap<>(); this.memorySize = memorySize; this.numberOfSlots = numberOfSlots; - - // assign page size and bit utilities this.pageSize = pageSize; - - final long numPagesLong = memorySize / pageSize; - if (numPagesLong > Integer.MAX_VALUE) { - throw new IllegalArgumentException("The given number of memory bytes (" + memorySize - + ") corresponds to more than MAX_INT pages."); - } - this.totalNumPages = (int) numPagesLong; - if (this.totalNumPages < 1) { - throw new IllegalArgumentException("The given amount of memory amounted to less than one page."); - } - - this.allocatedSegments = new HashMap>(); this.isPreAllocated = preAllocateMemory; - + this.totalNumPages = calculateTotalNumberOfPages(memorySize, pageSize); this.numNonAllocatedPages = preAllocateMemory ? 0 : this.totalNumPages; - final int memToAllocate = preAllocateMemory ? this.totalNumPages : 0; - - switch (memoryType) { - case HEAP: - this.memoryPool = new HybridHeapMemoryPool(memToAllocate, pageSize); - break; - case OFF_HEAP: - if (!preAllocateMemory) { - LOG.warn("It is advisable to set 'taskmanager.memory.preallocate' to true when" + - " the memory type 'taskmanager.memory.off-heap' is set to true."); - } - this.memoryPool = new HybridOffHeapMemoryPool(memToAllocate, pageSize); - break; - default: - throw new IllegalArgumentException("unrecognized memory type: " + memoryType); - } + this.memoryPool = createMemoryPool(preAllocateMemory ? this.totalNumPages : 0, pageSize, memoryType); LOG.debug("Initialized MemoryManager with total memory size {}, number of slots {}, page size {}, " + "memory type {}, pre allocate memory {} and number of non allocated pages {}.", @@ -171,6 +137,50 @@ public MemoryManager(long memorySize, int numberOfSlots, int pageSize, numNonAllocatedPages); } + private static void sanityCheck(long memorySize, int pageSize, MemoryType memoryType, boolean preAllocateMemory) { + Preconditions.checkNotNull(memoryType); + Preconditions.checkArgument(memorySize > 0L, "Size of total memory must be positive."); + Preconditions.checkArgument( + pageSize >= MIN_PAGE_SIZE, + "The page size must be at least %d bytes.", MIN_PAGE_SIZE); + Preconditions.checkArgument( + MathUtils.isPowerOf2(pageSize), + "The given page size is not a power of two."); + warnAboutPreallocationForOffHeap(memoryType, preAllocateMemory); + } + + private static void warnAboutPreallocationForOffHeap(MemoryType memoryType, boolean preAllocateMemory) { + if (memoryType == MemoryType.OFF_HEAP && !preAllocateMemory) { + LOG.warn("It is advisable to set '{}' to true when the memory type '{}' is set to true.", + TaskManagerOptions.MANAGED_MEMORY_PRE_ALLOCATE.key(), + TaskManagerOptions.MEMORY_OFF_HEAP.key()); + } + } + + private static int calculateTotalNumberOfPages(long memorySize, int pageSize) { + long numPagesLong = memorySize / pageSize; + Preconditions.checkArgument( + numPagesLong <= Integer.MAX_VALUE, + "The given number of memory bytes (%s) corresponds to more than MAX_INT pages.", memorySize); + + @SuppressWarnings("NumericCastThatLosesPrecision") + int totalNumPages = (int) numPagesLong; + Preconditions.checkArgument(totalNumPages >= 1, "The given amount of memory amounted to less than one page."); + + return totalNumPages; + } + + private static MemoryPool createMemoryPool(int segmentsToAllocate, int pageSize, MemoryType memoryType) { + switch (memoryType) { + case HEAP: + return new HybridHeapMemoryPool(segmentsToAllocate, pageSize); + case OFF_HEAP: + return new HybridOffHeapMemoryPool(segmentsToAllocate, pageSize); + default: + throw new IllegalArgumentException("unrecognized memory type: " + memoryType); + } + } + // ------------------------------------------------------------------------ // Shutdown // ------------------------------------------------------------------------ From 597c9691c96b899a0effa00c95b5bd82a2666c05 Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Tue, 10 Sep 2019 17:23:50 +0200 Subject: [PATCH 176/746] [hotfix] Remove and deprecate memory preallocation in MemoryManager --- .../task_manager_memory_configuration.html | 5 - docs/ops/config.md | 2 +- docs/ops/config.zh.md | 2 +- .../flink/configuration/ConfigConstants.java | 17 -- .../configuration/TaskManagerOptions.java | 24 +- flink-dist/src/main/flink-bin/bin/config.sh | 7 - .../src/main/flink-bin/bin/taskmanager.sh | 2 +- flink-dist/src/main/resources/flink-conf.yaml | 11 - .../flink/runtime/memory/MemoryManager.java | 270 +++--------------- .../taskexecutor/TaskManagerServices.java | 53 +--- .../TaskManagerServicesConfiguration.java | 11 - .../runtime/memory/MemoryManagerBuilder.java | 8 +- .../MemoryManagerLazyAllocationTest.java | 201 ------------- .../runtime/memory/MemoryManagerTest.java | 2 +- .../taskexecutor/TaskExecutorTest.java | 1 - .../TaskManagerRunnerStartupTest.java | 20 -- .../TaskManagerServicesBuilder.java | 1 - .../YARNSessionCapacitySchedulerITCase.java | 3 +- .../flink/yarn/cli/FlinkYarnSessionCli.java | 4 +- 19 files changed, 61 insertions(+), 583 deletions(-) delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerLazyAllocationTest.java diff --git a/docs/_includes/generated/task_manager_memory_configuration.html b/docs/_includes/generated/task_manager_memory_configuration.html index e05115d4b467..09b9dc6d59fd 100644 --- a/docs/_includes/generated/task_manager_memory_configuration.html +++ b/docs/_includes/generated/task_manager_memory_configuration.html @@ -52,11 +52,6 @@ (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
- false - Whether TaskManager managed memory should be pre-allocated when the TaskManager is starting. When `taskmanager.memory.off-heap` is set to true, then it is advised that this configuration is also set to true. If this configuration is set to false cleaning up of the allocated off-heap memory happens only when the configured JVM parameter MaxDirectMemorySize is reached by triggering a full GC. For streaming setups, it is highly recommended to set this value to false as the core state backends currently do not use the managed memory. -
taskmanager.memory.segment-size
"32kb" diff --git a/docs/ops/config.md b/docs/ops/config.md index d70d26094f7e..756153f14dd8 100644 --- a/docs/ops/config.md +++ b/docs/ops/config.md @@ -86,7 +86,7 @@ Configuration options to control Flink's restart behaviour in case of job failur {% include generated/task_manager_configuration.html %} -For *batch* jobs (or if `taskmanager.memoy.preallocate` is enabled) Flink allocates a fraction of 0.7 of the free memory (total memory configured via taskmanager.heap.size minus memory used for network buffers) for its managed memory. Managed memory helps Flink to run the batch operators efficiently. It prevents OutOfMemoryExceptions because Flink knows how much memory it can use to execute operations. If Flink runs out of managed memory, it utilizes disk space. Using managed memory, some operations can be performed directly on the raw data without having to deserialize the data to convert it into Java objects. All in all, managed memory improves the robustness and speed of the system. +For *batch* jobs Flink allocates a fraction of 0.7 of the free memory (total memory configured via taskmanager.heap.size minus memory used for network buffers) for its managed memory. Managed memory helps Flink to run the batch operators efficiently. It prevents OutOfMemoryExceptions because Flink knows how much memory it can use to execute operations. If Flink runs out of managed memory, it utilizes disk space. Using managed memory, some operations can be performed directly on the raw data without having to deserialize the data to convert it into Java objects. All in all, managed memory improves the robustness and speed of the system. The default fraction for managed memory can be adjusted using the taskmanager.memory.fraction parameter. An absolute value may be set using taskmanager.memory.size (overrides the fraction parameter). If desired, the managed memory may be allocated outside the JVM heap. This may improve performance in setups with large memory sizes. diff --git a/docs/ops/config.zh.md b/docs/ops/config.zh.md index 91c7bb2b9570..d965e838adb6 100644 --- a/docs/ops/config.zh.md +++ b/docs/ops/config.zh.md @@ -86,7 +86,7 @@ Configuration options to control Flink's restart behaviour in case of job failur {% include generated/task_manager_configuration.html %} -For *batch* jobs (or if `taskmanager.memoy.preallocate` is enabled) Flink allocates a fraction of 0.7 of the free memory (total memory configured via taskmanager.heap.size minus memory used for network buffers) for its managed memory. Managed memory helps Flink to run the batch operators efficiently. It prevents OutOfMemoryExceptions because Flink knows how much memory it can use to execute operations. If Flink runs out of managed memory, it utilizes disk space. Using managed memory, some operations can be performed directly on the raw data without having to deserialize the data to convert it into Java objects. All in all, managed memory improves the robustness and speed of the system. +For *batch* jobs Flink allocates a fraction of 0.7 of the free memory (total memory configured via taskmanager.heap.size minus memory used for network buffers) for its managed memory. Managed memory helps Flink to run the batch operators efficiently. It prevents OutOfMemoryExceptions because Flink knows how much memory it can use to execute operations. If Flink runs out of managed memory, it utilizes disk space. Using managed memory, some operations can be performed directly on the raw data without having to deserialize the data to convert it into Java objects. All in all, managed memory improves the robustness and speed of the system. The default fraction for managed memory can be adjusted using the taskmanager.memory.fraction parameter. An absolute value may be set using taskmanager.memory.size (overrides the fraction parameter). If desired, the managed memory may be allocated outside the JVM heap. This may improve performance in setups with large memory sizes. 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 fa68073d277c..510c11aa585e 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 @@ -273,15 +273,6 @@ public final class ConfigConstants { @Deprecated public static final String TASK_MANAGER_MEMORY_OFF_HEAP_KEY = "taskmanager.memory.off-heap"; - /** - * The config parameter for specifying whether TaskManager managed memory should be preallocated - * when the TaskManager is starting. (default is false) - * - * @deprecated Use {@link TaskManagerOptions#MANAGED_MEMORY_PRE_ALLOCATE} instead - */ - @Deprecated - public static final String TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY = "taskmanager.memory.preallocate"; - /** * The config parameter defining the number of buffers used in the network stack. This defines the * number of possible tasks and shuffles. @@ -1502,14 +1493,6 @@ public final class ConfigConstants { @Deprecated public static final String DEFAULT_TASK_MANAGER_REFUSED_REGISTRATION_PAUSE = "10 s"; - /** - * Config key has been deprecated. Therefore, no default value required. - * - * @deprecated {@link TaskManagerOptions#MANAGED_MEMORY_PRE_ALLOCATE} provides the default value now - */ - @Deprecated - public static final boolean DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE = false; - /** @deprecated Please use {@link TaskManagerOptions#TASK_CANCELLATION_INTERVAL}. */ @Deprecated public static final long DEFAULT_TASK_CANCELLATION_INTERVAL_MILLIS = 30000; 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 57e4d0363e9b..b190ee8d1b8a 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 @@ -35,8 +35,6 @@ @ConfigGroups(groups = @ConfigGroup(name = "TaskManagerMemory", keyPrefix = "taskmanager.memory")) public class TaskManagerOptions { - private static final String MANAGED_MEMORY_PRE_ALLOCATE_KEY = "taskmanager.memory.preallocate"; - // ------------------------------------------------------------------------ // General TaskManager Options // ------------------------------------------------------------------------ @@ -226,27 +224,9 @@ public class TaskManagerOptions { public static final ConfigOption MEMORY_OFF_HEAP = key("taskmanager.memory.off-heap") .defaultValue(false) - .withDescription(Description.builder() - .text("Memory allocation method (JVM heap or off-heap), used for managed memory of the" + + .withDescription("Memory allocation method (JVM heap or off-heap), used for managed memory of the" + " TaskManager. For setups with larger quantities of memory, this can" + - " improve the efficiency of the operations performed on the memory.") - .linebreak() - .text("When set to true, then it is advised that %s is also set to true.", code(MANAGED_MEMORY_PRE_ALLOCATE_KEY)) - .build()); - - /** - * Whether TaskManager managed memory should be pre-allocated when the TaskManager is starting. - */ - public static final ConfigOption MANAGED_MEMORY_PRE_ALLOCATE = - key(MANAGED_MEMORY_PRE_ALLOCATE_KEY) - .defaultValue(false) - .withDescription(Description.builder() - .text("Whether TaskManager managed memory should be pre-allocated when the TaskManager is starting." + - " When %s is set to true, then it is advised that this configuration is also" + - " set to true. If this configuration is set to false cleaning up of the allocated off-heap memory" + - " happens only when the configured JVM parameter MaxDirectMemorySize is reached by triggering a full" + - " GC. For streaming setups, it is highly recommended to set this value to false as the core state" + - " backends currently do not use the managed memory.", code(MEMORY_OFF_HEAP.key())).build()); + " improve the efficiency of the operations performed on the memory."); /** * The config parameter for automatically defining the TaskManager's binding address, diff --git a/flink-dist/src/main/flink-bin/bin/config.sh b/flink-dist/src/main/flink-bin/bin/config.sh index 7d6543dab417..b79921472449 100755 --- a/flink-dist/src/main/flink-bin/bin/config.sh +++ b/flink-dist/src/main/flink-bin/bin/config.sh @@ -111,7 +111,6 @@ KEY_TASKM_MEM_MB="taskmanager.heap.mb" KEY_TASKM_MEM_MANAGED_SIZE="taskmanager.memory.size" KEY_TASKM_MEM_MANAGED_FRACTION="taskmanager.memory.fraction" KEY_TASKM_OFFHEAP="taskmanager.memory.off-heap" -KEY_TASKM_MEM_PRE_ALLOCATE="taskmanager.memory.preallocate" KEY_TASKM_NET_BUF_FRACTION="taskmanager.network.memory.fraction" KEY_TASKM_NET_BUF_MIN="taskmanager.network.memory.min" @@ -402,12 +401,6 @@ if [ -z "${FLINK_TM_OFFHEAP}" ]; then FLINK_TM_OFFHEAP=$(readFromConfig ${KEY_TASKM_OFFHEAP} "false" "${YAML_CONF}") fi -# Define FLINK_TM_MEM_PRE_ALLOCATE if it is not already set -if [ -z "${FLINK_TM_MEM_PRE_ALLOCATE}" ]; then - FLINK_TM_MEM_PRE_ALLOCATE=$(readFromConfig ${KEY_TASKM_MEM_PRE_ALLOCATE} "false" "${YAML_CONF}") -fi - - # Define FLINK_TM_NET_BUF_FRACTION if it is not already set if [ -z "${FLINK_TM_NET_BUF_FRACTION}" ]; then FLINK_TM_NET_BUF_FRACTION=$(readFromConfig ${KEY_TASKM_NET_BUF_FRACTION} 0.1 "${YAML_CONF}") diff --git a/flink-dist/src/main/flink-bin/bin/taskmanager.sh b/flink-dist/src/main/flink-bin/bin/taskmanager.sh index b5de445d4632..f12f9d6745cf 100755 --- a/flink-dist/src/main/flink-bin/bin/taskmanager.sh +++ b/flink-dist/src/main/flink-bin/bin/taskmanager.sh @@ -40,7 +40,7 @@ if [[ $STARTSTOP == "start" ]] || [[ $STARTSTOP == "start-foreground" ]]; then # if memory allocation mode is lazy and no other JVM options are set, # set the 'Concurrent Mark Sweep GC' - if [[ $FLINK_TM_MEM_PRE_ALLOCATE == "false" ]] && [ -z "${FLINK_ENV_JAVA_OPTS}" ] && [ -z "${FLINK_ENV_JAVA_OPTS_TM}" ]; then + if [ -z "${FLINK_ENV_JAVA_OPTS}" ] && [ -z "${FLINK_ENV_JAVA_OPTS_TM}" ]; then export JVM_ARGS="$JVM_ARGS -XX:+UseG1GC" fi diff --git a/flink-dist/src/main/resources/flink-conf.yaml b/flink-dist/src/main/resources/flink-conf.yaml index d7c897bbe9cb..a3bc57dad07e 100644 --- a/flink-dist/src/main/resources/flink-conf.yaml +++ b/flink-dist/src/main/resources/flink-conf.yaml @@ -173,17 +173,6 @@ jobmanager.execution.failover-strategy: region # # io.tmp.dirs: /tmp -# Specify whether TaskManager's managed memory should be allocated when starting -# up (true) or when memory is requested. -# -# We recommend to set this value to 'true' only in setups for pure batch -# processing (DataSet API). Streaming setups currently do not use the TaskManager's -# managed memory: The 'rocksdb' state backend uses RocksDB's own memory management, -# while the 'memory' and 'filesystem' backends explicitly keep data as objects -# to save on serialization cost. -# -# taskmanager.memory.preallocate: false - # The classloading resolve order. Possible values are 'child-first' (Flink's default) # and 'parent-first' (Java's default). # diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java index c2e2c111927c..1501ef4bea83 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java @@ -19,10 +19,8 @@ package org.apache.flink.runtime.memory; import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.core.memory.HybridMemorySegment; import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.core.memory.MemorySegmentFactory; import org.apache.flink.core.memory.MemoryType; import org.apache.flink.util.MathUtils; import org.apache.flink.util.Preconditions; @@ -30,8 +28,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.nio.ByteBuffer; -import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Collection; import java.util.ConcurrentModificationException; @@ -42,6 +38,9 @@ import java.util.NoSuchElementException; import java.util.Set; +import static org.apache.flink.core.memory.MemorySegmentFactory.allocateUnpooledOffHeapMemory; +import static org.apache.flink.core.memory.MemorySegmentFactory.allocateUnpooledSegment; + /** * The memory manager governs the memory that Flink uses for sorting, hashing, and caching. Memory * is represented in segments of equal size. Operators allocate the memory by requesting a number @@ -49,14 +48,8 @@ * *

The memory may be represented as on-heap byte arrays or as off-heap memory regions * (both via {@link HybridMemorySegment}). Which kind of memory the MemoryManager serves can - * be passed as an argument to the initialization. - * - *

The memory manager can either pre-allocate all memory, or allocate the memory on demand. In the - * former version, memory will be occupied and reserved from start on, which means that no OutOfMemoryError - * can come while requesting memory. Released memory will also return to the MemoryManager's pool. - * On-demand allocation means that the memory manager only keeps track how many memory segments are - * currently allocated (bookkeeping only). Releasing a memory segment will not add it back to the pool, - * but make it re-claimable by the garbage collector. + * be passed as an argument to the initialization. Releasing a memory segment will make it re-claimable + * by the garbage collector. */ public class MemoryManager { @@ -72,9 +65,6 @@ public class MemoryManager { /** The lock used on the shared structures. */ private final Object lock = new Object(); - /** The memory pool from which we draw memory segments. Specific to on-heap or off-heap memory */ - private final MemoryPool memoryPool; - /** Memory segments allocated per memory owner. */ private final HashMap> allocatedSegments; @@ -90,8 +80,8 @@ public class MemoryManager { /** Number of slots of the task manager. */ private final int numberOfSlots; - /** Flag marking whether the memory manager immediately allocates the memory. */ - private final boolean isPreAllocated; + /** Type of the managed memory. */ + private final MemoryType memoryType; /** The number of memory pages that have not been allocated and are available for lazy allocation. */ private int numNonAllocatedPages; @@ -99,7 +89,6 @@ public class MemoryManager { /** Flag whether the close() has already been invoked. */ private boolean isShutDown; - /** * Creates a memory manager with the given capacity and given page size. * @@ -107,37 +96,32 @@ public class MemoryManager { * @param numberOfSlots The number of slots of the task manager. * @param pageSize The size of the pages handed out by the memory manager. * @param memoryType The type of memory (heap / off-heap) that the memory manager should allocate. - * @param preAllocateMemory True, if the memory manager should immediately allocate all memory, false - * if it should allocate and release the memory as needed. */ public MemoryManager( long memorySize, int numberOfSlots, int pageSize, - MemoryType memoryType, - boolean preAllocateMemory) { - sanityCheck(memorySize, pageSize, memoryType, preAllocateMemory); + MemoryType memoryType) { + sanityCheck(memorySize, pageSize, memoryType); this.allocatedSegments = new HashMap<>(); this.memorySize = memorySize; this.numberOfSlots = numberOfSlots; this.pageSize = pageSize; - this.isPreAllocated = preAllocateMemory; this.totalNumPages = calculateTotalNumberOfPages(memorySize, pageSize); - this.numNonAllocatedPages = preAllocateMemory ? 0 : this.totalNumPages; - this.memoryPool = createMemoryPool(preAllocateMemory ? this.totalNumPages : 0, pageSize, memoryType); + this.numNonAllocatedPages = this.totalNumPages; + this.memoryType = memoryType; LOG.debug("Initialized MemoryManager with total memory size {}, number of slots {}, page size {}, " + - "memory type {}, pre allocate memory {} and number of non allocated pages {}.", + "memory type {} and number of non allocated pages {}.", memorySize, numberOfSlots, pageSize, memoryType, - preAllocateMemory, numNonAllocatedPages); } - private static void sanityCheck(long memorySize, int pageSize, MemoryType memoryType, boolean preAllocateMemory) { + private static void sanityCheck(long memorySize, int pageSize, MemoryType memoryType) { Preconditions.checkNotNull(memoryType); Preconditions.checkArgument(memorySize > 0L, "Size of total memory must be positive."); Preconditions.checkArgument( @@ -146,15 +130,6 @@ private static void sanityCheck(long memorySize, int pageSize, MemoryType memory Preconditions.checkArgument( MathUtils.isPowerOf2(pageSize), "The given page size is not a power of two."); - warnAboutPreallocationForOffHeap(memoryType, preAllocateMemory); - } - - private static void warnAboutPreallocationForOffHeap(MemoryType memoryType, boolean preAllocateMemory) { - if (memoryType == MemoryType.OFF_HEAP && !preAllocateMemory) { - LOG.warn("It is advisable to set '{}' to true when the memory type '{}' is set to true.", - TaskManagerOptions.MANAGED_MEMORY_PRE_ALLOCATE.key(), - TaskManagerOptions.MEMORY_OFF_HEAP.key()); - } } private static int calculateTotalNumberOfPages(long memorySize, int pageSize) { @@ -170,17 +145,6 @@ private static int calculateTotalNumberOfPages(long memorySize, int pageSize) { return totalNumPages; } - private static MemoryPool createMemoryPool(int segmentsToAllocate, int pageSize, MemoryType memoryType) { - switch (memoryType) { - case HEAP: - return new HybridHeapMemoryPool(segmentsToAllocate, pageSize); - case OFF_HEAP: - return new HybridOffHeapMemoryPool(segmentsToAllocate, pageSize); - default: - throw new IllegalArgumentException("unrecognized memory type: " + memoryType); - } - } - // ------------------------------------------------------------------------ // Shutdown // ------------------------------------------------------------------------ @@ -205,8 +169,6 @@ public void shutdown() { seg.free(); } } - - memoryPool.clear(); } } // -------------------- END CRITICAL SECTION ------------------- @@ -230,9 +192,7 @@ public boolean isShutdown() { @VisibleForTesting public boolean verifyEmpty() { synchronized (lock) { - return isPreAllocated ? - memoryPool.getNumberOfAvailableMemorySegments() == totalNumPages : - numNonAllocatedPages == totalNumPages; + return numNonAllocatedPages == totalNumPages; } } @@ -241,9 +201,7 @@ public boolean verifyEmpty() { // ------------------------------------------------------------------------ /** - * Allocates a set of memory segments from this memory manager. If the memory manager pre-allocated the - * segments, they will be taken from the pool of memory segments. Otherwise, they will be allocated - * as part of this call. + * Allocates a set of memory segments from this memory manager. * * @param owner The owner to associate with the memory segment, for the fallback release. * @param numPages The number of pages to allocate. @@ -258,9 +216,7 @@ public List allocatePages(Object owner, int numPages) throws Memo } /** - * Allocates a set of memory segments from this memory manager. If the memory manager pre-allocated the - * segments, they will be taken from the pool of memory segments. Otherwise, they will be allocated - * as part of this call. + * Allocates a set of memory segments from this memory manager. * * @param owner The owner to associate with the memory segment, for the fallback release. * @param target The list into which to put the allocated memory pages. @@ -286,12 +242,9 @@ public void allocatePages(Object owner, List target, int numPages throw new IllegalStateException("Memory manager has been shut down."); } - // in the case of pre-allocated memory, the 'numNonAllocatedPages' is zero, in the - // lazy case, the 'freeSegments.size()' is zero. - if (numPages > (memoryPool.getNumberOfAvailableMemorySegments() + numNonAllocatedPages)) { - throw new MemoryAllocationException("Could not allocate " + numPages + " pages. Only " + - (memoryPool.getNumberOfAvailableMemorySegments() + numNonAllocatedPages) - + " pages are remaining."); + if (numPages > numNonAllocatedPages) { + throw new MemoryAllocationException( + String.format("Could not allocate %d pages. Only %d pages are remaining.", numPages, numNonAllocatedPages)); } Set segmentsForOwner = allocatedSegments.get(owner); @@ -300,31 +253,21 @@ public void allocatePages(Object owner, List target, int numPages allocatedSegments.put(owner, segmentsForOwner); } - if (isPreAllocated) { - for (int i = numPages; i > 0; i--) { - MemorySegment segment = memoryPool.requestSegmentFromPool(owner); - target.add(segment); - segmentsForOwner.add(segment); - } - } - else { - for (int i = numPages; i > 0; i--) { - MemorySegment segment = memoryPool.allocateNewSegment(owner); - target.add(segment); - segmentsForOwner.add(segment); - } - numNonAllocatedPages -= numPages; + for (int i = numPages; i > 0; i--) { + MemorySegment segment = allocateManagedSegment(memoryType, owner); + target.add(segment); + segmentsForOwner.add(segment); } + numNonAllocatedPages -= numPages; } // -------------------- END CRITICAL SECTION ------------------- } /** - * Tries to release the memory for the specified segment. If the segment has already been released or - * is null, the request is simply ignored. + * Tries to release the memory for the specified segment. * - *

If the memory manager manages pre-allocated memory, the memory segment goes back to the memory pool. - * Otherwise, the segment is only freed and made eligible for reclamation by the GC. + *

If the segment has already been released or is null, the request is simply ignored. + * The segment is only freed and made eligible for reclamation by the GC. * * @param segment The segment to be released. * @throws IllegalArgumentException Thrown, if the given segment is of an incompatible type. @@ -358,14 +301,8 @@ public void release(MemorySegment segment) { } } - if (isPreAllocated) { - // release the memory in any case - memoryPool.returnSegmentToPool(segment); - } - else { - segment.free(); - numNonAllocatedPages++; - } + segment.free(); + numNonAllocatedPages++; } catch (Throwable t) { throw new RuntimeException("Error removing book-keeping reference to allocated memory segment.", t); @@ -377,8 +314,7 @@ public void release(MemorySegment segment) { /** * Tries to release many memory segments together. * - *

If the memory manager manages pre-allocated memory, the memory segment goes back to the memory pool. - * Otherwise, the segment is only freed and made eligible for reclamation by the GC. + *

The segment is only freed and made eligible for reclamation by the GC. * * @param segments The segments to be released. * @throws NullPointerException Thrown, if the given collection is null. @@ -431,13 +367,8 @@ public void release(Collection segments) { } } - if (isPreAllocated) { - memoryPool.returnSegmentToPool(seg); - } - else { - seg.free(); - numNonAllocatedPages++; - } + seg.free(); + numNonAllocatedPages++; } catch (Throwable t) { throw new RuntimeException( @@ -484,17 +415,10 @@ public void releaseAll(Object owner) { } // free each segment - if (isPreAllocated) { - for (MemorySegment seg : segments) { - memoryPool.returnSegmentToPool(seg); - } - } - else { - for (MemorySegment seg : segments) { - seg.free(); - } - numNonAllocatedPages += segments.size(); + for (MemorySegment seg : segments) { + seg.free(); } + numNonAllocatedPages += segments.size(); segments.clear(); } @@ -540,122 +464,14 @@ public int computeNumberOfPages(double fraction) { return (int) (totalNumPages * fraction / numberOfSlots); } - - // ------------------------------------------------------------------------ - // Memory Pools - // ------------------------------------------------------------------------ - - abstract static class MemoryPool { - - abstract int getNumberOfAvailableMemorySegments(); - - abstract MemorySegment allocateNewSegment(Object owner); - - abstract MemorySegment requestSegmentFromPool(Object owner); - - abstract void returnSegmentToPool(MemorySegment segment); - - abstract void clear(); - } - - static final class HybridHeapMemoryPool extends MemoryPool { - - /** The collection of available memory segments. */ - private final ArrayDeque availableMemory; - - private final int segmentSize; - - HybridHeapMemoryPool(int numInitialSegments, int segmentSize) { - this.availableMemory = new ArrayDeque<>(numInitialSegments); - this.segmentSize = segmentSize; - - for (int i = 0; i < numInitialSegments; i++) { - this.availableMemory.add(new byte[segmentSize]); - } - } - - @Override - MemorySegment allocateNewSegment(Object owner) { - return MemorySegmentFactory.allocateUnpooledSegment(segmentSize, owner); - } - - @Override - MemorySegment requestSegmentFromPool(Object owner) { - byte[] buf = availableMemory.remove(); - return MemorySegmentFactory.wrapPooledHeapMemory(buf, owner); - } - - @Override - void returnSegmentToPool(MemorySegment segment) { - if (segment.getClass() == HybridMemorySegment.class) { - HybridMemorySegment heapSegment = (HybridMemorySegment) segment; - availableMemory.add(heapSegment.getArray()); - heapSegment.free(); - } - else { - throw new IllegalArgumentException("Memory segment is not a " + HybridMemorySegment.class.getSimpleName()); - } - } - - @Override - protected int getNumberOfAvailableMemorySegments() { - return availableMemory.size(); - } - - @Override - void clear() { - availableMemory.clear(); - } - } - - static final class HybridOffHeapMemoryPool extends MemoryPool { - - /** The collection of available memory segments. */ - private final ArrayDeque availableMemory; - - private final int segmentSize; - - HybridOffHeapMemoryPool(int numInitialSegments, int segmentSize) { - this.availableMemory = new ArrayDeque<>(numInitialSegments); - this.segmentSize = segmentSize; - - for (int i = 0; i < numInitialSegments; i++) { - this.availableMemory.add(ByteBuffer.allocateDirect(segmentSize)); - } - } - - @Override - MemorySegment allocateNewSegment(Object owner) { - return MemorySegmentFactory.allocateUnpooledOffHeapMemory(segmentSize, owner); - } - - @Override - MemorySegment requestSegmentFromPool(Object owner) { - ByteBuffer buf = availableMemory.remove(); - return MemorySegmentFactory.wrapPooledOffHeapMemory(buf, owner); - } - - @Override - void returnSegmentToPool(MemorySegment segment) { - if (segment.getClass() == HybridMemorySegment.class) { - HybridMemorySegment hybridSegment = (HybridMemorySegment) segment; - ByteBuffer buf = hybridSegment.getOffHeapBuffer(); - availableMemory.add(buf); - hybridSegment.free(); - } - else { - throw new IllegalArgumentException("Memory segment is not a " + HybridMemorySegment.class.getSimpleName()); - } - } - - @Override - protected int getNumberOfAvailableMemorySegments() { - return availableMemory.size(); - } - - @Override - void clear() { - availableMemory.clear(); + private MemorySegment allocateManagedSegment(MemoryType memoryType, Object owner) { + switch (memoryType) { + case HEAP: + return allocateUnpooledSegment(pageSize, owner); + case OFF_HEAP: + return allocateUnpooledOffHeapMemory(pageSize, owner); + default: + throw new IllegalArgumentException("unrecognized memory type: " + memoryType); } } } 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 752deabed2c4..33a7ea93de29 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 @@ -326,10 +326,9 @@ public static TaskManagerServices fromConfiguration( * * @param taskManagerServicesConfiguration to create the memory manager from * @return Memory manager - * @throws Exception */ private static MemoryManager createMemoryManager( - TaskManagerServicesConfiguration taskManagerServicesConfiguration) throws Exception { + TaskManagerServicesConfiguration taskManagerServicesConfiguration) { // computing the amount of memory to use depends on how much memory is available // it strictly needs to happen AFTER the network stack has been initialized @@ -340,14 +339,8 @@ private static MemoryManager createMemoryManager( final long memorySize; - boolean preAllocateMemory = taskManagerServicesConfiguration.isPreAllocateMemory(); - if (configuredMemory > 0) { - if (preAllocateMemory) { - LOG.info("Using {} MB for managed memory." , configuredMemory); - } else { - LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory); - } + LOG.info("Limiting managed memory to {} MB." , configuredMemory); memorySize = configuredMemory << 20; // megabytes to bytes } else { // similar to #calculateNetworkBufferMemory(TaskManagerServicesConfiguration tmConfig) @@ -357,13 +350,7 @@ private static MemoryManager createMemoryManager( long freeHeapMemoryWithDefrag = taskManagerServicesConfiguration.getFreeHeapMemoryWithDefrag(); // network buffers allocated off-heap -> use memoryFraction of the available heap: long relativeMemSize = (long) (freeHeapMemoryWithDefrag * memoryFraction); - if (preAllocateMemory) { - LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." , - memoryFraction , relativeMemSize >> 20); - } else { - LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " + - "memory will be allocated lazily." , memoryFraction , relativeMemSize >> 20); - } + LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB)." , memoryFraction , relativeMemSize >> 20); memorySize = relativeMemSize; } else if (memType == MemoryType.OFF_HEAP) { long maxJvmHeapMemory = taskManagerServicesConfiguration.getMaxJvmHeapMemory(); @@ -372,13 +359,7 @@ private static MemoryManager createMemoryManager( // maxJvmHeap = jvmTotalNoNet - jvmTotalNoNet * memoryFraction = jvmTotalNoNet * (1 - memoryFraction) // directMemorySize = jvmTotalNoNet * memoryFraction long directMemorySize = (long) (maxJvmHeapMemory / (1.0 - memoryFraction) * memoryFraction); - if (preAllocateMemory) { - LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." , - memoryFraction, directMemorySize >> 20); - } else { - LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," + - " memory will be allocated lazily.", memoryFraction, directMemorySize >> 20); - } + LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB).", memoryFraction, directMemorySize >> 20); memorySize = directMemorySize; } else { throw new RuntimeException("No supported memory type detected."); @@ -386,27 +367,11 @@ private static MemoryManager createMemoryManager( } // now start the memory manager - final MemoryManager memoryManager; - try { - memoryManager = new MemoryManager( - memorySize, - taskManagerServicesConfiguration.getNumberOfSlots(), - taskManagerServicesConfiguration.getPageSize(), - memType, - preAllocateMemory); - } catch (OutOfMemoryError e) { - if (memType == MemoryType.HEAP) { - throw new Exception("OutOfMemory error (" + e.getMessage() + - ") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e); - } else if (memType == MemoryType.OFF_HEAP) { - throw new Exception("OutOfMemory error (" + e.getMessage() + - ") while allocating the TaskManager off-heap memory (" + memorySize + - " bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e); - } else { - throw e; - } - } - return memoryManager; + return new MemoryManager( + memorySize, + taskManagerServicesConfiguration.getNumberOfSlots(), + taskManagerServicesConfiguration.getPageSize(), + memType); } /** 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 ea6032f389bd..42aaabf42c08 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 @@ -73,8 +73,6 @@ public class TaskManagerServicesConfiguration { private final MemoryType memoryType; - private final boolean preAllocateMemory; - private final float memoryFraction; private final int pageSize; @@ -101,7 +99,6 @@ public TaskManagerServicesConfiguration( int numberOfSlots, long configuredMemory, MemoryType memoryType, - boolean preAllocateMemory, float memoryFraction, int pageSize, long timerServiceShutdownTimeout, @@ -122,7 +119,6 @@ public TaskManagerServicesConfiguration( this.configuredMemory = configuredMemory; this.memoryType = checkNotNull(memoryType); - this.preAllocateMemory = preAllocateMemory; this.memoryFraction = memoryFraction; this.pageSize = pageSize; @@ -207,10 +203,6 @@ long getConfiguredMemory() { return configuredMemory; } - boolean isPreAllocateMemory() { - return preAllocateMemory; - } - public int getPageSize() { return pageSize; } @@ -263,8 +255,6 @@ public static TaskManagerServicesConfiguration fromConfiguration( final QueryableStateConfiguration queryableStateConfig = QueryableStateConfiguration.fromConfiguration(configuration); - boolean preAllocateMemory = configuration.getBoolean(TaskManagerOptions.MANAGED_MEMORY_PRE_ALLOCATE); - long timerServiceShutdownTimeout = AkkaUtils.getTimeout(configuration).toMillis(); final RetryingRegistrationConfiguration retryingRegistrationConfiguration = RetryingRegistrationConfiguration.fromConfiguration(configuration); @@ -283,7 +273,6 @@ public static TaskManagerServicesConfiguration fromConfiguration( ConfigurationParserUtils.getSlot(configuration), ConfigurationParserUtils.getManagedMemorySize(configuration), ConfigurationParserUtils.getMemoryType(configuration), - preAllocateMemory, ConfigurationParserUtils.getManagedMemoryFraction(configuration), ConfigurationParserUtils.getPageSize(configuration), timerServiceShutdownTimeout, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerBuilder.java index 4bc7336f98a2..1a33b7715a09 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerBuilder.java @@ -29,7 +29,6 @@ public class MemoryManagerBuilder { private long memorySize = DEFAULT_MEMORY_SIZE; private int numberOfSlots = 1; private int pageSize = DEFAULT_PAGE_SIZE; - private boolean preAllocateMemory = true; private MemoryManagerBuilder() { @@ -50,13 +49,8 @@ public MemoryManagerBuilder setPageSize(int pageSize) { return this; } - public MemoryManagerBuilder setPreAllocateMemory(boolean preAllocateMemory) { - this.preAllocateMemory = preAllocateMemory; - return this; - } - public MemoryManager build() { - return new MemoryManager(memorySize, numberOfSlots, pageSize, MemoryType.HEAP, preAllocateMemory); + return new MemoryManager(memorySize, numberOfSlots, pageSize, MemoryType.HEAP); } public static MemoryManagerBuilder newBuilder() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerLazyAllocationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerLazyAllocationTest.java deleted file mode 100644 index 0d5bba07ed54..000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerLazyAllocationTest.java +++ /dev/null @@ -1,201 +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.memory; - -import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; -import org.apache.flink.runtime.operators.testutils.DummyInvokable; - -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.List; -import java.util.Random; - -import static org.junit.Assert.fail; - -/** - * Tests for the memory manager, in the mode where it pre-allocates all memory. - */ -public class MemoryManagerLazyAllocationTest { - - private static final long RANDOM_SEED = 643196033469871L; - - private static final int MEMORY_SIZE = 1024 * 1024 * 72; // 72 MiBytes - - private static final int PAGE_SIZE = 1024 * 32; // 32 KiBytes - - private static final int NUM_PAGES = MEMORY_SIZE / PAGE_SIZE; - - private MemoryManager memoryManager; - - private Random random; - - @Before - public void setUp() { - this.memoryManager = MemoryManagerBuilder - .newBuilder() - .setMemorySize(MEMORY_SIZE) - .setPageSize(PAGE_SIZE) - .build(); - this.random = new Random(RANDOM_SEED); - } - - @After - public void tearDown() { - if (!this.memoryManager.verifyEmpty()) { - fail("Memory manager is not complete empty and valid at the end of the test."); - } - this.memoryManager = null; - this.random = null; - } - - @Test - public void allocateAllSingle() { - try { - final AbstractInvokable mockInvoke = new DummyInvokable(); - List segments = new ArrayList(); - - try { - for (int i = 0; i < NUM_PAGES; i++) { - segments.add(this.memoryManager.allocatePages(mockInvoke, 1).get(0)); - } - } - catch (MemoryAllocationException e) { - fail("Unable to allocate memory"); - } - - for (MemorySegment seg : segments) { - this.memoryManager.release(seg); - } - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void allocateAllMulti() { - try { - final AbstractInvokable mockInvoke = new DummyInvokable(); - final List segments = new ArrayList(); - - try { - for (int i = 0; i < NUM_PAGES / 2; i++) { - segments.addAll(this.memoryManager.allocatePages(mockInvoke, 2)); - } - } catch (MemoryAllocationException e) { - Assert.fail("Unable to allocate memory"); - } - - this.memoryManager.release(segments); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void allocateMultipleOwners() { - final int numOwners = 17; - - try { - AbstractInvokable[] owners = new AbstractInvokable[numOwners]; - - @SuppressWarnings("unchecked") - List[] mems = (List[]) new List[numOwners]; - - for (int i = 0; i < numOwners; i++) { - owners[i] = new DummyInvokable(); - mems[i] = new ArrayList(64); - } - - // allocate all memory to the different owners - for (int i = 0; i < NUM_PAGES; i++) { - final int owner = this.random.nextInt(numOwners); - mems[owner].addAll(this.memoryManager.allocatePages(owners[owner], 1)); - } - - // free one owner at a time - for (int i = 0; i < numOwners; i++) { - this.memoryManager.releaseAll(owners[i]); - owners[i] = null; - Assert.assertTrue("Released memory segments have not been destroyed.", allMemorySegmentsFreed(mems[i])); - mems[i] = null; - - // check that the owner owners were not affected - for (int k = i + 1; k < numOwners; k++) { - Assert.assertTrue("Non-released memory segments are accidentaly destroyed.", allMemorySegmentsValid(mems[k])); - } - } - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void allocateTooMuch() { - try { - final AbstractInvokable mockInvoke = new DummyInvokable(); - - List segs = this.memoryManager.allocatePages(mockInvoke, NUM_PAGES); - - try { - this.memoryManager.allocatePages(mockInvoke, 1); - Assert.fail("Expected MemoryAllocationException."); - } catch (MemoryAllocationException maex) { - // expected - } - - Assert.assertTrue("The previously allocated segments were not valid any more.", - allMemorySegmentsValid(segs)); - - this.memoryManager.releaseAll(mockInvoke); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - private boolean allMemorySegmentsValid(List memSegs) { - for (MemorySegment seg : memSegs) { - if (seg.isFreed()) { - return false; - } - } - return true; - } - - private boolean allMemorySegmentsFreed(List memSegs) { - for (MemorySegment seg : memSegs) { - if (!seg.isFreed()) { - return false; - } - } - return true; - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java index 70505258ff74..3ba291b3918b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java @@ -34,7 +34,7 @@ import static org.junit.Assert.fail; /** - * Tests for the memory manager, in the mode where it pre-allocates all memory. + * Tests for the memory manager. */ public class MemoryManagerTest { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index 9f099bafab7e..7c4d4c96566f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -277,7 +277,6 @@ public void testShouldShutDownTaskManagerServicesInPostStop() throws Exception { .newBuilder() .setMemorySize(4096) .setPageSize(4096) - .setPreAllocateMemory(false) .build(); nettyShuffleEnvironment.start(); 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 d2ace9562e7a..4f777211b21e 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 @@ -21,7 +21,6 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; 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.blob.BlobCacheService; @@ -47,7 +46,6 @@ import java.net.InetAddress; import java.net.ServerSocket; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -120,7 +118,6 @@ public void testIODirectoryNotWritable() throws Exception { @Test public void testMemoryConfigWrong() throws Exception { Configuration cfg = new Configuration(); - cfg.setBoolean(TaskManagerOptions.MANAGED_MEMORY_PRE_ALLOCATE, true); // something invalid cfg.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "-42m"); @@ -135,23 +132,6 @@ public void testMemoryConfigWrong() throws Exception { } catch (IllegalConfigurationException e) { // splendid! } - - // something ridiculously high - final long memSize = (((long) Integer.MAX_VALUE - 1) * - MemorySize.parse(TaskManagerOptions.MEMORY_SEGMENT_SIZE.defaultValue()).getBytes()) >> 20; - cfg.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, memSize + "m"); - try { - - startTaskManager( - cfg, - rpcService, - highAvailabilityServices); - - fail("Should fail synchronously with an exception"); - } catch (Exception e) { - // splendid! - assertTrue(e.getCause() instanceof OutOfMemoryError); - } } /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java index 1185139b818f..7bf73225f509 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java @@ -57,7 +57,6 @@ public TaskManagerServicesBuilder() { .newBuilder() .setMemorySize(MemoryManager.MIN_PAGE_SIZE) .setPageSize(MemoryManager.MIN_PAGE_SIZE) - .setPreAllocateMemory(false) .build(); ioManager = mock(IOManager.class); shuffleEnvironment = mock(ShuffleEnvironment.class); 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 6b809bea9ca1..50d151518996 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 @@ -221,8 +221,7 @@ public void perJobYarnClusterOffHeap() throws Exception { "-yjm", "768m", "-ytm", taskManagerMemoryMB + "m", "-yD", "taskmanager.memory.off-heap=true", - "-yD", "taskmanager.memory.size=" + offHeapMemory + "m", - "-yD", "taskmanager.memory.preallocate=true", exampleJarLocation.getAbsolutePath()}, + "-yD", "taskmanager.memory.size=" + offHeapMemory + "m", exampleJarLocation.getAbsolutePath()}, /* test succeeded after this string */ "Program execution finished", /* prohibited strings: (to verify the parallelism) */ 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 e87927897acd..8d6916af802b 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 @@ -133,9 +133,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine Date: Wed, 11 Sep 2019 16:55:51 +0200 Subject: [PATCH 177/746] [FLINK-13984] Separate on-heap and off-heap managed memory pools Refactor MemoryManager to support allocating from two memory pools of different types (heap/off-heap) at the same time. --- .../flink/runtime/memory/MemoryManager.java | 531 +++++++++++------- .../taskexecutor/TaskManagerServices.java | 5 +- .../runtime/util/KeyedBudgetManager.java | 294 ++++++++++ .../runtime/memory/MemoryManagerBuilder.java | 18 +- .../runtime/memory/MemoryManagerTest.java | 78 ++- .../runtime/util/KeyedBudgetManagerTest.java | 261 +++++++++ 6 files changed, 968 insertions(+), 219 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyedBudgetManager.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyedBudgetManagerTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java index 1501ef4bea83..94c0e30c4b1f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java @@ -22,21 +22,31 @@ import org.apache.flink.core.memory.HybridMemorySegment; import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.core.memory.MemoryType; +import org.apache.flink.runtime.util.KeyedBudgetManager; +import org.apache.flink.runtime.util.KeyedBudgetManager.AcquisitionResult; import org.apache.flink.util.MathUtils; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.ConcurrentModificationException; -import java.util.HashMap; +import java.util.EnumMap; +import java.util.EnumSet; import java.util.HashSet; import java.util.Iterator; import java.util.List; +import java.util.Map; +import java.util.Map.Entry; import java.util.NoSuchElementException; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicReference; import static org.apache.flink.core.memory.MemorySegmentFactory.allocateUnpooledOffHeapMemory; import static org.apache.flink.core.memory.MemorySegmentFactory.allocateUnpooledSegment; @@ -47,7 +57,7 @@ * of memory segments. * *

The memory may be represented as on-heap byte arrays or as off-heap memory regions - * (both via {@link HybridMemorySegment}). Which kind of memory the MemoryManager serves can + * (both via {@link HybridMemorySegment}). Which kinds of memory the MemoryManager serves and their sizes can * be passed as an argument to the initialization. Releasing a memory segment will make it re-claimable * by the garbage collector. */ @@ -62,63 +72,43 @@ public class MemoryManager { // ------------------------------------------------------------------------ - /** The lock used on the shared structures. */ - private final Object lock = new Object(); - /** Memory segments allocated per memory owner. */ - private final HashMap> allocatedSegments; - - /** The size of the memory segments. */ - private final int pageSize; - - /** The initial total size, for verification. */ - private final int totalNumPages; - - /** The total size of the memory managed by this memory manager. */ - private final long memorySize; + private final Map> allocatedSegments; /** Number of slots of the task manager. */ private final int numberOfSlots; - /** Type of the managed memory. */ - private final MemoryType memoryType; - - /** The number of memory pages that have not been allocated and are available for lazy allocation. */ - private int numNonAllocatedPages; + private final KeyedBudgetManager budgetByType; /** Flag whether the close() has already been invoked. */ - private boolean isShutDown; + private volatile boolean isShutDown; /** - * Creates a memory manager with the given capacity and given page size. + * Creates a memory manager with the given memory types, capacity and given page size. * - * @param memorySize The total size of the memory to be managed by this memory manager. + * @param memorySizeByType The total size of the memory to be managed by this memory manager for each type (heap / off-heap). * @param numberOfSlots The number of slots of the task manager. * @param pageSize The size of the pages handed out by the memory manager. - * @param memoryType The type of memory (heap / off-heap) that the memory manager should allocate. */ public MemoryManager( - long memorySize, + Map memorySizeByType, int numberOfSlots, - int pageSize, - MemoryType memoryType) { - sanityCheck(memorySize, pageSize, memoryType); + int pageSize) { + for (Entry sizeForType : memorySizeByType.entrySet()) { + sanityCheck(sizeForType.getValue(), pageSize, sizeForType.getKey()); + } - this.allocatedSegments = new HashMap<>(); - this.memorySize = memorySize; + this.allocatedSegments = new ConcurrentHashMap<>(); this.numberOfSlots = numberOfSlots; - this.pageSize = pageSize; - this.totalNumPages = calculateTotalNumberOfPages(memorySize, pageSize); - this.numNonAllocatedPages = this.totalNumPages; - this.memoryType = memoryType; - - LOG.debug("Initialized MemoryManager with total memory size {}, number of slots {}, page size {}, " + - "memory type {} and number of non allocated pages {}.", - memorySize, + this.budgetByType = new KeyedBudgetManager<>(memorySizeByType, pageSize); + verifyIntTotalNumberOfPages(memorySizeByType, budgetByType.maxTotalNumberOfPages()); + + LOG.debug( + "Initialized MemoryManager with total memory size {} ({}), number of slots {}, page size {}.", + budgetByType.totalAvailableBudget(), + memorySizeByType, numberOfSlots, - pageSize, - memoryType, - numNonAllocatedPages); + pageSize); } private static void sanityCheck(long memorySize, int pageSize, MemoryType memoryType) { @@ -132,17 +122,16 @@ private static void sanityCheck(long memorySize, int pageSize, MemoryType memory "The given page size is not a power of two."); } - private static int calculateTotalNumberOfPages(long memorySize, int pageSize) { - long numPagesLong = memorySize / pageSize; + private static void verifyIntTotalNumberOfPages(Map memorySizeByType, long numberOfPagesLong) { Preconditions.checkArgument( - numPagesLong <= Integer.MAX_VALUE, - "The given number of memory bytes (%s) corresponds to more than MAX_INT pages.", memorySize); + numberOfPagesLong <= Integer.MAX_VALUE, + "The given number of memory bytes (%d: %s) corresponds to more than MAX_INT pages.", + numberOfPagesLong, + memorySizeByType); @SuppressWarnings("NumericCastThatLosesPrecision") - int totalNumPages = (int) numPagesLong; + int totalNumPages = (int) numberOfPagesLong; Preconditions.checkArgument(totalNumPages >= 1, "The given amount of memory amounted to less than one page."); - - return totalNumPages; } // ------------------------------------------------------------------------ @@ -156,22 +145,20 @@ private static int calculateTotalNumberOfPages(long memorySize, int pageSize) { * code that allocated them from the memory manager. */ public void shutdown() { - // -------------------- BEGIN CRITICAL SECTION ------------------- - synchronized (lock) { - if (!isShutDown) { - // mark as shutdown and release memory - isShutDown = true; - numNonAllocatedPages = 0; - - // go over all allocated segments and release them - for (Set segments : allocatedSegments.values()) { - for (MemorySegment seg : segments) { - seg.free(); - } + if (!isShutDown) { + // mark as shutdown and release memory + isShutDown = true; + budgetByType.releaseAll(); + + // go over all allocated segments and release them + for (Set segments : allocatedSegments.values()) { + for (MemorySegment seg : segments) { + seg.free(); } + segments.clear(); } + allocatedSegments.clear(); } - // -------------------- END CRITICAL SECTION ------------------- } /** @@ -191,9 +178,7 @@ public boolean isShutdown() { */ @VisibleForTesting public boolean verifyEmpty() { - synchronized (lock) { - return numNonAllocatedPages == totalNumPages; - } + return budgetByType.totalAvailableBudget() == budgetByType.maxTotalBudget(); } // ------------------------------------------------------------------------ @@ -203,12 +188,17 @@ public boolean verifyEmpty() { /** * Allocates a set of memory segments from this memory manager. * + *

The returned segments can have any memory type. The total allocated memory for each type will not exceed its + * size limit, announced in the constructor. + * * @param owner The owner to associate with the memory segment, for the fallback release. * @param numPages The number of pages to allocate. * @return A list with the memory segments. * @throws MemoryAllocationException Thrown, if this memory manager does not have the requested amount * of memory pages any more. + * @deprecated use {@link #allocatePages(AllocationRequest)} */ + @Deprecated public List allocatePages(Object owner, int numPages) throws MemoryAllocationException { List segments = new ArrayList<>(numPages); allocatePages(owner, segments, numPages); @@ -218,103 +208,120 @@ public List allocatePages(Object owner, int numPages) throws Memo /** * Allocates a set of memory segments from this memory manager. * + *

The allocated segments can have any memory type. The total allocated memory for each type will not exceed its + * size limit, announced in the constructor. + * * @param owner The owner to associate with the memory segment, for the fallback release. * @param target The list into which to put the allocated memory pages. - * @param numPages The number of pages to allocate. + * @param numberOfPages The number of pages to allocate. * @throws MemoryAllocationException Thrown, if this memory manager does not have the requested amount * of memory pages any more. + * @deprecated use {@link #allocatePages(AllocationRequest)} */ - public void allocatePages(Object owner, List target, int numPages) - throws MemoryAllocationException { + @Deprecated + public void allocatePages( + Object owner, + Collection target, + int numberOfPages) throws MemoryAllocationException { + allocatePages(AllocationRequest + .newBuilder(owner) + .ofAllTypes() + .numberOfPages(numberOfPages) + .withOutput(target) + .build()); + } + + /** + * Allocates a set of memory segments from this memory manager. + * + *

The allocated segments can have any memory type. The total allocated memory for each type will not exceed its + * size limit, announced in the constructor. + * + * @param request The allocation request which contains all the parameters. + * @return A collection with the allocated memory segments. + * @throws MemoryAllocationException Thrown, if this memory manager does not have the requested amount + * of memory pages any more. + */ + public Collection allocatePages(AllocationRequest request) throws MemoryAllocationException { + Object owner = request.getOwner(); + Collection target = request.output; + int numberOfPages = request.getNumberOfPages(); + // sanity check - if (owner == null) { - throw new IllegalArgumentException("The memory owner must not be null."); - } + Preconditions.checkNotNull(owner, "The memory owner must not be null."); + Preconditions.checkState(!isShutDown, "Memory manager has been shut down."); // reserve array space, if applicable if (target instanceof ArrayList) { - ((ArrayList) target).ensureCapacity(numPages); + ((ArrayList) target).ensureCapacity(numberOfPages); } - // -------------------- BEGIN CRITICAL SECTION ------------------- - synchronized (lock) { - if (isShutDown) { - throw new IllegalStateException("Memory manager has been shut down."); - } + AcquisitionResult acquiredBudget = budgetByType.acquirePagedBudget(request.getTypes(), numberOfPages); + if (acquiredBudget.isFailure()) { + throw new MemoryAllocationException( + String.format( + "Could not allocate %d pages. Only %d pages are remaining.", + numberOfPages, + acquiredBudget.getTotalAvailableForAllQueriedKeys())); + } - if (numPages > numNonAllocatedPages) { - throw new MemoryAllocationException( - String.format("Could not allocate %d pages. Only %d pages are remaining.", numPages, numNonAllocatedPages)); + allocatedSegments.compute(owner, (o, currentSegmentsForOwner) -> { + Set segmentsForOwner = currentSegmentsForOwner == null ? + new HashSet<>(numberOfPages) : currentSegmentsForOwner; + for (MemoryType memoryType : acquiredBudget.getAcquiredPerKey().keySet()) { + for (long i = acquiredBudget.getAcquiredPerKey().get(memoryType); i > 0; i--) { + MemorySegment segment = allocateManagedSegment(memoryType, owner); + target.add(segment); + segmentsForOwner.add(segment); + } } + return segmentsForOwner; + }); - Set segmentsForOwner = allocatedSegments.get(owner); - if (segmentsForOwner == null) { - segmentsForOwner = new HashSet(numPages); - allocatedSegments.put(owner, segmentsForOwner); - } + Preconditions.checkState(!isShutDown, "Memory manager has been concurrently shut down."); - for (int i = numPages; i > 0; i--) { - MemorySegment segment = allocateManagedSegment(memoryType, owner); - target.add(segment); - segmentsForOwner.add(segment); - } - numNonAllocatedPages -= numPages; - } - // -------------------- END CRITICAL SECTION ------------------- + return target; } /** * Tries to release the memory for the specified segment. * - *

If the segment has already been released or is null, the request is simply ignored. - * The segment is only freed and made eligible for reclamation by the GC. + *

If the segment has already been released, it is only freed. If it is null or has no owner, the request is simply ignored. + * The segment is only freed and made eligible for reclamation by the GC. The segment will be returned to + * the memory pool of its type, increasing its available limit for the later allocations. * * @param segment The segment to be released. * @throws IllegalArgumentException Thrown, if the given segment is of an incompatible type. */ public void release(MemorySegment segment) { + Preconditions.checkState(!isShutDown, "Memory manager has been shut down."); + // check if segment is null or has already been freed if (segment == null || segment.getOwner() == null) { return; } - final Object owner = segment.getOwner(); - - // -------------------- BEGIN CRITICAL SECTION ------------------- - synchronized (lock) { - // prevent double return to this memory manager - if (segment.isFreed()) { - return; - } - if (isShutDown) { - throw new IllegalStateException("Memory manager has been shut down."); - } - - // remove the reference in the map for the owner - try { - Set segsForOwner = this.allocatedSegments.get(owner); - - if (segsForOwner != null) { - segsForOwner.remove(segment); - if (segsForOwner.isEmpty()) { - this.allocatedSegments.remove(owner); - } - } - + // remove the reference in the map for the owner + try { + allocatedSegments.computeIfPresent(segment.getOwner(), (o, segsForOwner) -> { segment.free(); - numNonAllocatedPages++; - } - catch (Throwable t) { - throw new RuntimeException("Error removing book-keeping reference to allocated memory segment.", t); - } + if (segsForOwner.remove(segment)) { + budgetByType.releasePageForKey(getSegmentType(segment)); + } + //noinspection ReturnOfNull + return segsForOwner.isEmpty() ? null : segsForOwner; + }); + } + catch (Throwable t) { + throw new RuntimeException("Error removing book-keeping reference to allocated memory segment.", t); } - // -------------------- END CRITICAL SECTION ------------------- } /** * Tries to release many memory segments together. * - *

The segment is only freed and made eligible for reclamation by the GC. + *

The segment is only freed and made eligible for reclamation by the GC. Each segment will be returned to + * the memory pool of its type, increasing its available limit for the later allocations. * * @param segments The segments to be released. * @throws NullPointerException Thrown, if the given collection is null. @@ -325,69 +332,81 @@ public void release(Collection segments) { return; } - // -------------------- BEGIN CRITICAL SECTION ------------------- - synchronized (lock) { - if (isShutDown) { - throw new IllegalStateException("Memory manager has been shut down."); - } + Preconditions.checkState(!isShutDown, "Memory manager has been shut down."); - // since concurrent modifications to the collection - // can disturb the release, we need to try potentially multiple times - boolean successfullyReleased = false; - do { - final Iterator segmentsIterator = segments.iterator(); + EnumMap releasedMemory = new EnumMap<>(MemoryType.class); - Object lastOwner = null; - Set segsForOwner = null; + // since concurrent modifications to the collection + // can disturb the release, we need to try potentially multiple times + boolean successfullyReleased = false; + do { + // We could just pre-sort the segments by owner and release them in a loop by owner. + // It would simplify the code but require this additional step and memory for the sorted map of segments by owner. + // Current approach is more complicated but it traverses the input segments only once w/o any additional buffer. + // Later, we can check whether the simpler approach actually leads to any performance penalty and + // if not, we can change it to the simpler approach for the better readability. + Iterator segmentsIterator = segments.iterator(); - try { - // go over all segments - while (segmentsIterator.hasNext()) { - - final MemorySegment seg = segmentsIterator.next(); - if (seg == null || seg.isFreed()) { - continue; - } - - final Object owner = seg.getOwner(); - - try { - // get the list of segments by this owner only if it is a different owner than for - // the previous one (or it is the first segment) - if (lastOwner != owner) { - lastOwner = owner; - segsForOwner = this.allocatedSegments.get(owner); - } - - // remove the segment from the list - if (segsForOwner != null) { - segsForOwner.remove(seg); - if (segsForOwner.isEmpty()) { - this.allocatedSegments.remove(owner); - } - } - - seg.free(); - numNonAllocatedPages++; - } - catch (Throwable t) { - throw new RuntimeException( - "Error removing book-keeping reference to allocated memory segment.", t); - } - } + //noinspection ProhibitedExceptionCaught + try { + MemorySegment segment = null; + while (segment == null && segmentsIterator.hasNext()) { + segment = segmentsIterator.next(); + } + while (segment != null) { + segment = releaseSegmentsForOwnerUntilNextOwner(segment, segmentsIterator, releasedMemory); + } + segments.clear(); + // the only way to exit the loop + successfullyReleased = true; + } catch (ConcurrentModificationException | NoSuchElementException e) { + // this may happen in the case where an asynchronous + // call releases the memory. fall through the loop and try again + } + } while (!successfullyReleased); - segments.clear(); + budgetByType.releaseBudgetForKeys(releasedMemory); + } - // the only way to exit the loop - successfullyReleased = true; - } - catch (ConcurrentModificationException | NoSuchElementException e) { - // this may happen in the case where an asynchronous - // call releases the memory. fall through the loop and try again + private MemorySegment releaseSegmentsForOwnerUntilNextOwner( + MemorySegment firstSeg, + Iterator segmentsIterator, + EnumMap releasedMemory) { + AtomicReference nextOwnerMemorySegment = new AtomicReference<>(); + Object owner = firstSeg.getOwner(); + allocatedSegments.compute(owner, (o, segsForOwner) -> { + freeSegment(firstSeg, segsForOwner, releasedMemory); + while (segmentsIterator.hasNext()) { + MemorySegment segment = segmentsIterator.next(); + try { + if (segment == null || segment.isFreed()) { + continue; + } + Object nextOwner = segment.getOwner(); + if (nextOwner != owner) { + nextOwnerMemorySegment.set(segment); + break; + } + freeSegment(segment, segsForOwner, releasedMemory); + } catch (Throwable t) { + throw new RuntimeException( + "Error removing book-keeping reference to allocated memory segment.", t); } - } while (!successfullyReleased); + } + //noinspection ReturnOfNull + return segsForOwner == null || segsForOwner.isEmpty() ? null : segsForOwner; + }); + return nextOwnerMemorySegment.get(); + } + + private void freeSegment( + MemorySegment segment, + @Nullable Collection segments, + EnumMap releasedMemory) { + segment.free(); + if (segments != null && segments.remove(segment)) { + releaseSegment(segment, releasedMemory); } - // -------------------- END CRITICAL SECTION ------------------- } /** @@ -400,29 +419,25 @@ public void releaseAll(Object owner) { return; } - // -------------------- BEGIN CRITICAL SECTION ------------------- - synchronized (lock) { - if (isShutDown) { - throw new IllegalStateException("Memory manager has been shut down."); - } - - // get all segments - final Set segments = allocatedSegments.remove(owner); + Preconditions.checkState(!isShutDown, "Memory manager has been shut down."); - // all segments may have been freed previously individually - if (segments == null || segments.isEmpty()) { - return; - } + // get all segments + Set segments = allocatedSegments.remove(owner); - // free each segment - for (MemorySegment seg : segments) { - seg.free(); - } - numNonAllocatedPages += segments.size(); + // all segments may have been freed previously individually + if (segments == null || segments.isEmpty()) { + return; + } - segments.clear(); + // free each segment + EnumMap releasedMemory = new EnumMap<>(MemoryType.class); + for (MemorySegment segment : segments) { + segment.free(); + releaseSegment(segment, releasedMemory); } - // -------------------- END CRITICAL SECTION ------------------- + budgetByType.releaseBudgetForKeys(releasedMemory); + + segments.clear(); } // ------------------------------------------------------------------------ @@ -435,7 +450,8 @@ public void releaseAll(Object owner) { * @return The size of the pages handled by the memory manager. */ public int getPageSize() { - return pageSize; + //noinspection NumericCastThatLosesPrecision + return (int) budgetByType.getDefaultPageSize(); } /** @@ -444,7 +460,17 @@ public int getPageSize() { * @return The total size of memory. */ public long getMemorySize() { - return memorySize; + return budgetByType.maxTotalBudget(); + } + + /** + * Returns the total size of the certain type of memory handled by this memory manager. + * + * @param memoryType The type of memory. + * @return The total size of memory. + */ + public long getMemorySizeByType(MemoryType memoryType) { + return budgetByType.maxTotalBudgetForKey(memoryType); } /** @@ -461,17 +487,114 @@ public int computeNumberOfPages(double fraction) { } //noinspection NumericCastThatLosesPrecision - return (int) (totalNumPages * fraction / numberOfSlots); + return (int) (budgetByType.maxTotalNumberOfPages() * fraction / numberOfSlots); } private MemorySegment allocateManagedSegment(MemoryType memoryType, Object owner) { switch (memoryType) { case HEAP: - return allocateUnpooledSegment(pageSize, owner); + return allocateUnpooledSegment(getPageSize(), owner); case OFF_HEAP: - return allocateUnpooledOffHeapMemory(pageSize, owner); + return allocateUnpooledOffHeapMemory(getPageSize(), owner); default: throw new IllegalArgumentException("unrecognized memory type: " + memoryType); } } + + private void releaseSegment(MemorySegment segment, EnumMap releasedMemory) { + releasedMemory.compute(getSegmentType(segment), (t, v) -> v == null ? getPageSize() : v + getPageSize()); + } + + private static MemoryType getSegmentType(MemorySegment segment) { + return segment.isOffHeap() ? MemoryType.OFF_HEAP : MemoryType.HEAP; + } + + /** Memory segment allocation request. */ + @SuppressWarnings("WeakerAccess") + public static class AllocationRequest { + /** Owner of the segment to track by. */ + private final Object owner; + + /** Collection to add the allocated segments to. */ + private final Collection output; + + /** Number of pages to allocate. */ + private final int numberOfPages; + + /** Allowed types of memory to allocate. */ + private final Set types; + + private AllocationRequest( + Object owner, + Collection output, + int numberOfPages, + Set types) { + this.owner = owner; + this.output = output; + this.numberOfPages = numberOfPages; + this.types = types; + } + + public Object getOwner() { + return owner; + } + + public int getNumberOfPages() { + return numberOfPages; + } + + public Set getTypes() { + return Collections.unmodifiableSet(types); + } + + public static Builder newBuilder(Object owner) { + return new Builder(owner); + } + + public static AllocationRequest ofAllTypes(Object owner, int numberOfPages) { + return newBuilder(owner).ofAllTypes().numberOfPages(numberOfPages).build(); + } + + public static AllocationRequest ofType(Object owner, int numberOfPages, MemoryType type) { + return newBuilder(owner).ofType(type).numberOfPages(numberOfPages).build(); + } + } + + /** A builder for the {@link AllocationRequest}. */ + @SuppressWarnings("WeakerAccess") + public static class Builder { + private final Object owner; + private Collection output = new ArrayList<>(); + private int numberOfPages = 1; + private Set types = EnumSet.noneOf(MemoryType.class); + + public Builder(Object owner) { + this.owner = owner; + } + + public Builder withOutput(Collection output) { + //noinspection AssignmentOrReturnOfFieldWithMutableType + this.output = output; + return this; + } + + public Builder numberOfPages(int numberOfPages) { + this.numberOfPages = numberOfPages; + return this; + } + + public Builder ofType(MemoryType type) { + types.add(type); + return this; + } + + public Builder ofAllTypes() { + types = EnumSet.allOf(MemoryType.class); + return this; + } + + public AllocationRequest build() { + return new AllocationRequest(owner, output, numberOfPages, types); + } + } } 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 33a7ea93de29..d91431181921 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 @@ -368,10 +368,9 @@ private static MemoryManager createMemoryManager( // now start the memory manager return new MemoryManager( - memorySize, + Collections.singletonMap(memType, memorySize), taskManagerServicesConfiguration.getNumberOfSlots(), - taskManagerServicesConfiguration.getPageSize(), - memType); + taskManagerServicesConfiguration.getPageSize()); } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyedBudgetManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyedBudgetManager.java new file mode 100644 index 000000000000..a2bae84f8099 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/KeyedBudgetManager.java @@ -0,0 +1,294 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.util; + +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nullable; +import javax.annotation.concurrent.GuardedBy; +import javax.annotation.concurrent.ThreadSafe; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; + +/** + * Manages {@code long} available budget per key (allocation/release). + * + *

This manager gets a certain maximum {@code long} budget per key. + * Users can acquire some budget for some key and release it later. + * The manager keeps track of acquired/released budget and prevents from over-allocating. + * + *

There is also a paged type of allocation where a certain number of pages can be acquired from a set of keys. + * The page has its budget size. The manager acquires randomly from all keys of a given set. + * At the end, sum of pages acquired from each key is either requested number of pages or none. + * Only integer number of pages are acquired from each key respecting its available budget (no page spans two or more keys) + * or nothing is acquired reporting the maximum number of pages which could be acquired per each given key at the moment. + * + * @param type of the budget key + */ +@ThreadSafe +public class KeyedBudgetManager { + private final Map maxBudgetByKey; + + private final long defaultPageSize; + + private final long totalNumberOfPages; + + @GuardedBy("lock") + private final Map availableBudgetByKey; + + private final Object lock = new Object(); + + public KeyedBudgetManager(Map maxBudgetByKey, long defaultPageSize) { + Preconditions.checkNotNull(maxBudgetByKey); + Preconditions.checkArgument(defaultPageSize > 0L, "The default page size has to be greater than zero"); + + this.maxBudgetByKey = new HashMap<>(maxBudgetByKey); + this.availableBudgetByKey = new HashMap<>(maxBudgetByKey); + this.defaultPageSize = defaultPageSize; + this.totalNumberOfPages = calculateTotalNumberOfPages(maxBudgetByKey, defaultPageSize); + } + + public long getDefaultPageSize() { + return defaultPageSize; + } + + /** + * Tries to acquire budget for a given key. + * + *

No budget is acquired if it was not possible to fully acquire the requested budget. + * + * @param key the key to acquire budget from + * @param size the size of budget to acquire from the given key + * @return the fully acquired budget for the key or max possible budget to acquire + * if it was not possible to acquire the requested budget. + */ + public long acquireBudgetForKey(K key, long size) { + Preconditions.checkNotNull(key); + AcquisitionResult result = acquirePagedBudgetForKeys(Collections.singletonList(key), size, 1L); + return result.isSuccess() ? + result.getAcquiredPerKey().get(key) : result.getTotalAvailableForAllQueriedKeys(); + } + + /** + * Tries to acquire budget for given keys which equals to the number of pages times default page size. + * + *

See also {@link #acquirePagedBudgetForKeys(Iterable, long, long)} + */ + public AcquisitionResult acquirePagedBudget(Iterable keys, long numberOfPages) { + return acquirePagedBudgetForKeys(keys, numberOfPages, defaultPageSize); + } + + /** + * Tries to acquire budget which equals to the number of pages times page size. + * + *

The budget will be acquired only from the given keys. Only integer number of pages will be acquired from each key. + * If the next page does not fit into the available budget of some key, it will try to be acquired from another key. + * The acquisition is successful if the acquired number of pages for each key sums up to the requested number of pages. + * The function does not make any preference about which keys from the given keys to acquire from. + * + * @param keys the keys to acquire budget from + * @param numberOfPages the total number of pages to acquire from the given keys + * @param pageSize the size of budget to acquire per page + * @return the acquired number of pages for each key if the acquisition is successful or + * the total number of pages which were available for the given keys. + */ + AcquisitionResult acquirePagedBudgetForKeys(Iterable keys, long numberOfPages, long pageSize) { + Preconditions.checkNotNull(keys); + Preconditions.checkArgument(numberOfPages >= 0L, "The requested number of pages has to be positive"); + Preconditions.checkArgument(pageSize > 0L, "The page size has to be greater than zero"); + + synchronized (lock) { + long leftPagesToReserve = numberOfPages; + Map pagesToReserveByKey = new HashMap<>(); + for (K key : keys) { + long availableBudgetOfCurrentKey = availableBudgetByKey.getOrDefault(key, 0L); + long availablePagesOfCurrentKey = availableBudgetOfCurrentKey / pageSize; + if (leftPagesToReserve <= availablePagesOfCurrentKey) { + pagesToReserveByKey.put(key, leftPagesToReserve); + leftPagesToReserve = 0L; + break; + } else if (availablePagesOfCurrentKey > 0L) { + pagesToReserveByKey.put(key, availablePagesOfCurrentKey); + leftPagesToReserve -= availablePagesOfCurrentKey; + } + } + boolean possibleToAcquire = leftPagesToReserve == 0L; + if (possibleToAcquire) { + for (Entry pagesToReserveForKey : pagesToReserveByKey.entrySet()) { + //noinspection ConstantConditions + availableBudgetByKey.compute( + pagesToReserveForKey.getKey(), + (k, v) -> v - (pagesToReserveForKey.getValue() * pageSize)); + } + } + return possibleToAcquire ? + AcquisitionResult.success(pagesToReserveByKey) : AcquisitionResult.failure(numberOfPages - leftPagesToReserve); + } + } + + public void releasePageForKey(K key) { + releaseBudgetForKey(key, defaultPageSize); + } + + public void releaseBudgetForKey(K key, long size) { + Preconditions.checkNotNull(key); + Preconditions.checkArgument(size >= 0L, "The budget to release has to be positive"); + + releaseBudgetForKeys(Collections.singletonMap(key, size)); + } + + public void releaseBudgetForKeys(Map sizeByKey) { + Preconditions.checkNotNull(sizeByKey); + + synchronized (lock) { + for (Entry toReleaseForKey : sizeByKey.entrySet()) { + long toRelease = toReleaseForKey.getValue(); + Preconditions.checkArgument( + toRelease >= 0L, + "The budget to release for key %s has to be positive", + toReleaseForKey.getKey()); + if (toRelease == 0L) { + continue; + } + K keyToReleaseFor = toReleaseForKey.getKey(); + long maxBudgetForKey = maxBudgetByKey.get(keyToReleaseFor); + availableBudgetByKey.compute(keyToReleaseFor, (k, currentBudget) -> { + if (currentBudget == null) { + throw new IllegalArgumentException("The budget key is not supported: " + keyToReleaseFor); + } else if (currentBudget + toRelease > maxBudgetForKey) { + throw new IllegalStateException( + String.format( + "The budget to release %d exceeds the limit %d for key %s", + toRelease, + maxBudgetForKey, + keyToReleaseFor)); + } else { + return currentBudget + toRelease; + } + }); + } + } + } + + public void releaseAll() { + synchronized (lock) { + availableBudgetByKey.putAll(maxBudgetByKey); + } + } + + public long maxTotalBudget() { + return maxBudgetByKey.values().stream().mapToLong(b -> b).sum(); + } + + public long maxTotalNumberOfPages() { + return totalNumberOfPages; + } + + public long maxTotalBudgetForKey(K key) { + Preconditions.checkNotNull(key); + return maxBudgetByKey.get(key); + } + + public long totalAvailableBudget() { + return availableBudgetForKeys(maxBudgetByKey.keySet()); + } + + long availableBudgetForKeys(Iterable keys) { + Preconditions.checkNotNull(keys); + synchronized (lock) { + long totalSize = 0L; + for (K key : keys) { + totalSize += availableBudgetForKey(key); + } + return totalSize; + } + } + + long availableBudgetForKey(K key) { + Preconditions.checkNotNull(key); + synchronized (lock) { + return availableBudgetByKey.getOrDefault(key, 0L); + } + } + + private static long calculateTotalNumberOfPages(Map budgetByType, long pageSize) { + long numPages = 0L; + for (long sizeForType : budgetByType.values()) { + numPages += sizeForType / pageSize; + } + return numPages; + } + + /** + * Result of budget acquisition to return from acquisition functions. + * + *

The result of acquisition is either success: {@link AcquisitionResult#isSuccess()} and this class contains + * acquired budget/pages per key: {@link AcquisitionResult#getAcquiredPerKey()} or + * it is failure: {@link AcquisitionResult#isFailure()} and this class contains total max available budget for all + * queried keys: {@link AcquisitionResult#getTotalAvailableForAllQueriedKeys()} which was not enough to + * acquire the requested number of pages. + */ + public static class AcquisitionResult { + @Nullable + private final Map acquiredBudgetPerKey; + + @Nullable + private final Long totalAvailableBudgetForAllQueriedKeys; + + private AcquisitionResult( + @Nullable Map acquiredBudgetPerKey, + @Nullable Long totalAvailableBudgetForAllQueriedKeys) { + this.acquiredBudgetPerKey = acquiredBudgetPerKey; + this.totalAvailableBudgetForAllQueriedKeys = totalAvailableBudgetForAllQueriedKeys; + } + + public static AcquisitionResult success(Map acquiredBudgetPerKey) { + return new AcquisitionResult<>(acquiredBudgetPerKey, null); + } + + public static AcquisitionResult failure(long totalAvailableBudgetForAllQueriedKeys) { + return new AcquisitionResult<>(null, totalAvailableBudgetForAllQueriedKeys); + } + + public boolean isSuccess() { + return acquiredBudgetPerKey != null; + } + + public boolean isFailure() { + return totalAvailableBudgetForAllQueriedKeys != null; + } + + public Map getAcquiredPerKey() { + if (acquiredBudgetPerKey == null) { + throw new IllegalStateException("The acquisition failed. Nothing was acquired."); + } + return Collections.unmodifiableMap(acquiredBudgetPerKey); + } + + public long getTotalAvailableForAllQueriedKeys() { + if (totalAvailableBudgetForAllQueriedKeys == null) { + throw new IllegalStateException("The acquisition succeeded. All requested pages were acquired."); + } + return totalAvailableBudgetForAllQueriedKeys; + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerBuilder.java index 1a33b7715a09..5386282d50ec 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerBuilder.java @@ -20,22 +20,30 @@ import org.apache.flink.core.memory.MemoryType; +import java.util.EnumMap; +import java.util.Map; + import static org.apache.flink.runtime.memory.MemoryManager.DEFAULT_PAGE_SIZE; /** Builder class for {@link MemoryManager}. */ public class MemoryManagerBuilder { - private static final int DEFAULT_MEMORY_SIZE = 32 * DEFAULT_PAGE_SIZE; + private static final long DEFAULT_MEMORY_SIZE = 32L * DEFAULT_PAGE_SIZE; - private long memorySize = DEFAULT_MEMORY_SIZE; + private final Map memoryPools = new EnumMap<>(MemoryType.class); private int numberOfSlots = 1; private int pageSize = DEFAULT_PAGE_SIZE; private MemoryManagerBuilder() { - + memoryPools.put(MemoryType.HEAP, DEFAULT_MEMORY_SIZE); } public MemoryManagerBuilder setMemorySize(long memorySize) { - this.memorySize = memorySize; + this.memoryPools.put(MemoryType.HEAP, memorySize); + return this; + } + + public MemoryManagerBuilder setMemorySize(MemoryType memoryType, long memorySize) { + this.memoryPools.put(memoryType, memorySize); return this; } @@ -50,7 +58,7 @@ public MemoryManagerBuilder setPageSize(int pageSize) { } public MemoryManager build() { - return new MemoryManager(memorySize, numberOfSlots, pageSize, MemoryType.HEAP); + return new MemoryManager(memoryPools, numberOfSlots, pageSize); } public static MemoryManagerBuilder newBuilder() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java index 3ba291b3918b..824ce58a6a72 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java @@ -19,7 +19,9 @@ package org.apache.flink.runtime.memory; import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.core.memory.MemoryType; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.memory.MemoryManager.AllocationRequest; import org.apache.flink.runtime.operators.testutils.DummyInvokable; import org.junit.After; @@ -28,9 +30,16 @@ import org.junit.Test; import java.util.ArrayList; +import java.util.Collection; +import java.util.EnumMap; import java.util.List; +import java.util.Map; import java.util.Random; +import static org.apache.flink.runtime.memory.MemoryManager.AllocationRequest.ofAllTypes; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.number.OrderingComparison.lessThanOrEqualTo; +import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; /** @@ -54,7 +63,8 @@ public class MemoryManagerTest { public void setUp() { this.memoryManager = MemoryManagerBuilder .newBuilder() - .setMemorySize(MEMORY_SIZE) + .setMemorySize(MemoryType.HEAP, MEMORY_SIZE / 2) + .setMemorySize(MemoryType.OFF_HEAP, MEMORY_SIZE / 2) .setPageSize(PAGE_SIZE) .build(); this.random = new Random(RANDOM_SEED); @@ -161,12 +171,7 @@ public void allocateTooMuch() { List segs = this.memoryManager.allocatePages(mockInvoke, NUM_PAGES); - try { - this.memoryManager.allocatePages(mockInvoke, 1); - Assert.fail("Expected MemoryAllocationException."); - } catch (MemoryAllocationException maex) { - // expected - } + testCannotAllocateAnymore(ofAllTypes(mockInvoke, 1)); Assert.assertTrue("The previously allocated segments were not valid any more.", allMemorySegmentsValid(segs)); @@ -179,6 +184,21 @@ public void allocateTooMuch() { } } + @Test + public void doubleReleaseReturnsMemoryOnlyOnce() throws MemoryAllocationException { + final AbstractInvokable mockInvoke = new DummyInvokable(); + + Collection segs = this.memoryManager.allocatePages(ofAllTypes(mockInvoke, NUM_PAGES)); + MemorySegment segment = segs.iterator().next(); + + this.memoryManager.release(segment); + this.memoryManager.release(segment); + + testCannotAllocateAnymore(ofAllTypes(mockInvoke, 2)); + + this.memoryManager.releaseAll(mockInvoke); + } + private boolean allMemorySegmentsValid(List memSegs) { for (MemorySegment seg : memSegs) { if (seg.isFreed()) { @@ -196,4 +216,48 @@ private boolean allMemorySegmentsFreed(List memSegs) { } return true; } + + @Test + @SuppressWarnings("NumericCastThatLosesPrecision") + public void testAllocateMixedMemoryType() throws MemoryAllocationException { + int totalHeapPages = (int) memoryManager.getMemorySizeByType(MemoryType.HEAP) / PAGE_SIZE; + int totalOffHeapPages = (int) memoryManager.getMemorySizeByType(MemoryType.OFF_HEAP) / PAGE_SIZE; + int pagesToAllocate = totalHeapPages + totalOffHeapPages / 2; + + Object owner = new Object(); + Collection segments = memoryManager.allocatePages(ofAllTypes(owner, pagesToAllocate)); + Map split = calcMemoryTypeSplitForSegments(segments); + + assertThat(split.get(MemoryType.HEAP), lessThanOrEqualTo(totalHeapPages)); + assertThat(split.get(MemoryType.OFF_HEAP), lessThanOrEqualTo(totalOffHeapPages)); + assertThat(split.get(MemoryType.HEAP) + split.get(MemoryType.OFF_HEAP), is(pagesToAllocate)); + + memoryManager.release(segments); + } + + private static Map calcMemoryTypeSplitForSegments(Iterable segments) { + int heapPages = 0; + int offHeapPages = 0; + for (MemorySegment memorySegment : segments) { + if (memorySegment.isOffHeap()) { + offHeapPages++; + } else { + heapPages++; + } + } + Map split = new EnumMap<>(MemoryType.class); + split.put(MemoryType.HEAP, heapPages); + split.put(MemoryType.OFF_HEAP, offHeapPages); + return split; + } + + private void testCannotAllocateAnymore(AllocationRequest request) { + try { + memoryManager.allocatePages(request); + Assert.fail("Expected MemoryAllocationException. " + + "We should not be able to allocate after allocating or(and) reserving all memory of a certain type."); + } catch (MemoryAllocationException maex) { + // expected + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyedBudgetManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyedBudgetManagerTest.java new file mode 100644 index 000000000000..bb420367c9d9 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyedBudgetManagerTest.java @@ -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. + */ + +package org.apache.flink.runtime.util; + +import org.apache.flink.runtime.util.KeyedBudgetManager.AcquisitionResult; +import org.apache.flink.util.Preconditions; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.concurrent.Future; +import java.util.stream.Collectors; +import java.util.stream.LongStream; + +import static org.hamcrest.CoreMatchers.is; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; + +/** + * Test suite for {@link KeyedBudgetManager}. + */ +@SuppressWarnings("MagicNumber") +public class KeyedBudgetManagerTest { + private static final String[] TEST_KEYS = {"k1", "k2", "k3", "k4"}; + private static final long[] TEST_BUDGETS = {15, 17, 22, 11}; + private static final Executor NEW_THREAD_EXECUTOR = r -> new Thread(r).start(); + + private KeyedBudgetManager keyedBudgetManager; + + @Before + public void setup() { + keyedBudgetManager = createSimpleKeyedBudget(); + } + + @After + public void teardown() { + keyedBudgetManager.releaseAll(); + checkNoKeyBudgetChange(); + } + + @Test + public void testSuccessfulAcquisitionForKey() { + long acquired = keyedBudgetManager.acquireBudgetForKey("k1", 10L); + + assertThat(acquired, is(10L)); + checkOneKeyBudgetChange("k1", 5L); + } + + @Test + public void testFailedAcquisitionForKey() { + long maxPossibleBudgetToAcquire = keyedBudgetManager.acquireBudgetForKey("k1", 20L); + + assertThat(maxPossibleBudgetToAcquire, is(15L)); + checkNoKeyBudgetChange(); + } + + @Test + public void testSuccessfulReleaseForKey() { + keyedBudgetManager.acquireBudgetForKey("k1", 10L); + keyedBudgetManager.releaseBudgetForKey("k1", 5L); + + checkOneKeyBudgetChange("k1", 10L); + } + + @Test + public void testFailedReleaseForKey() { + keyedBudgetManager.acquireBudgetForKey("k1", 10L); + try { + keyedBudgetManager.releaseBudgetForKey("k1", 15L); + fail("IllegalStateException is expected to fail over-sized release"); + } catch (IllegalStateException e) { + // expected + } + + checkOneKeyBudgetChange("k1", 5L); + } + + @Test + public void testSuccessfulAcquisitionForKeys() { + AcquisitionResult acquired = acquireForMultipleKeys(5L); + + assertThat(checkAcquisitionSuccess(acquired, 4L), is(true)); + + assertThat(keyedBudgetManager.availableBudgetForKey("k1"), is(15L)); + assertThat(keyedBudgetManager.availableBudgetForKeys(Arrays.asList("k2", "k3")), is(19L)); + assertThat(keyedBudgetManager.totalAvailableBudget(), is(45L)); + } + + @Test + public void testConcurrentAcquisitionForKeys() throws ExecutionException, InterruptedException { + long pageSize = 5L; + CompletableFuture> allocation1 = acquireForMultipleKeysAsync(pageSize); + CompletableFuture availableBudgetForKeysFuture = getAvailableBudgetForKeysAsync(); + CompletableFuture> allocation2 = acquireForMultipleKeysAsync(pageSize); + Arrays + .asList(allocation1, allocation2, availableBudgetForKeysFuture) + .forEach(KeyedBudgetManagerTest::waitForFutureSilently); + + boolean firstSucceeded = checkFirstAcquisitionSucceeded(allocation1, allocation2); + boolean secondSucceeded = checkFirstAcquisitionSucceeded(allocation2, allocation1); + assertThat(firstSucceeded || secondSucceeded, is(true)); + + long availableBudgetForKeys = availableBudgetForKeysFuture.get(); + assertThat(availableBudgetForKeys == 39L || availableBudgetForKeys == 19L, is(true)); + } + + @Test + public void testConcurrentReleaseForKeys() throws ExecutionException, InterruptedException { + long pageSize = 5L; + Map sizeByKey = acquireForMultipleKeys(pageSize) + .getAcquiredPerKey() + .entrySet() + .stream() + .collect(Collectors.toMap(Entry::getKey, e -> e.getValue() * pageSize)); + + CompletableFuture release1 = releaseKeysAsync(sizeByKey); + CompletableFuture availableBudgetForKeysFuture = getAvailableBudgetForKeysAsync(); + CompletableFuture release2 = releaseKeysAsync(sizeByKey); + Arrays + .asList(release1, availableBudgetForKeysFuture, release2) + .forEach(KeyedBudgetManagerTest::waitForFutureSilently); + + boolean firstSucceeded = !release1.isCompletedExceptionally() && release2.isCompletedExceptionally(); + boolean secondSucceeded = !release2.isCompletedExceptionally() && release1.isCompletedExceptionally(); + assertThat(firstSucceeded || secondSucceeded, is(true)); + + long availableBudgetForKeys = availableBudgetForKeysFuture.get(); + assertThat(availableBudgetForKeys == 39L || availableBudgetForKeys == 19L, is(true)); + + checkNoKeyBudgetChange(); + } + + @Test + public void testFailedAcquisitionForKeys() { + AcquisitionResult acquired = + keyedBudgetManager.acquirePagedBudgetForKeys(Arrays.asList("k2", "k3"), 6, 6); + + assertThat(acquired.isFailure(), is(true)); + assertThat(acquired.getTotalAvailableForAllQueriedKeys(), is(5L)); + checkNoKeyBudgetChange(); + } + + @Test + public void testSuccessfulReleaseForKeys() { + keyedBudgetManager.acquirePagedBudgetForKeys(Arrays.asList("k2", "k3"), 4, 8); + keyedBudgetManager.releaseBudgetForKeys(createdBudgetMap(new String[] {"k2", "k3"}, new long[] {7, 10})); + + assertThat(keyedBudgetManager.availableBudgetForKeys(Arrays.asList("k2", "k3")), is(24L)); + assertThat(keyedBudgetManager.availableBudgetForKeys(Arrays.asList("k1", "k4")), is(26L)); + assertThat(keyedBudgetManager.totalAvailableBudget(), is(50L)); + } + + @Test + public void testSuccessfulReleaseForKeysWithMixedRequests() { + keyedBudgetManager.acquirePagedBudgetForKeys(Arrays.asList("k2", "k3"), 4, 8); + keyedBudgetManager.acquirePagedBudgetForKeys(Arrays.asList("k1", "k4"), 6, 3); + keyedBudgetManager.releaseBudgetForKeys(createdBudgetMap(new String[] {"k2", "k3"}, new long[] {7, 10})); + + assertThat(keyedBudgetManager.availableBudgetForKeys(Arrays.asList("k2", "k3")), is(24L)); + assertThat(keyedBudgetManager.availableBudgetForKeys(Arrays.asList("k1", "k4")), is(8L)); + assertThat(keyedBudgetManager.totalAvailableBudget(), is(32L)); + } + + private void checkNoKeyBudgetChange() { + checkKeysBudgetChange(Collections.emptyMap()); + } + + private void checkOneKeyBudgetChange( + @SuppressWarnings("SameParameterValue") String key, + long budget) { + checkKeysBudgetChange(Collections.singletonMap(key, budget)); + } + + private void checkKeysBudgetChange( + Map changedBudgetPerKey) { + long totalExpectedBudget = 0L; + for (int i = 0; i < TEST_KEYS.length; i++) { + long expectedBudget = changedBudgetPerKey.containsKey(TEST_KEYS[i]) ? + changedBudgetPerKey.get(TEST_KEYS[i]) : TEST_BUDGETS[i]; + assertThat(keyedBudgetManager.availableBudgetForKey(TEST_KEYS[i]), is(expectedBudget)); + totalExpectedBudget += expectedBudget; + } + assertThat(keyedBudgetManager.maxTotalBudget(), is(LongStream.of(TEST_BUDGETS).sum())); + assertThat(keyedBudgetManager.totalAvailableBudget(), is(totalExpectedBudget)); + } + + private CompletableFuture> acquireForMultipleKeysAsync(long pageSize) { + return CompletableFuture.supplyAsync(() -> acquireForMultipleKeys(pageSize), NEW_THREAD_EXECUTOR); + } + + private CompletableFuture getAvailableBudgetForKeysAsync() { + return CompletableFuture.supplyAsync(() -> keyedBudgetManager.availableBudgetForKeys(Arrays.asList("k2", "k3")), NEW_THREAD_EXECUTOR); + } + + private AcquisitionResult acquireForMultipleKeys(long pageSize) { + return keyedBudgetManager.acquirePagedBudgetForKeys(Arrays.asList("k2", "k3"), 4, pageSize); + } + + private CompletableFuture releaseKeysAsync(Map sizeByKey) { + return CompletableFuture.runAsync(() -> keyedBudgetManager.releaseBudgetForKeys(sizeByKey), NEW_THREAD_EXECUTOR); + } + + private static boolean checkFirstAcquisitionSucceeded( + Future> allocation1, + Future> allocation2) throws ExecutionException, InterruptedException { + return checkAcquisitionSuccess(allocation1.get(), 4L) && allocation2.get().isFailure(); + } + + private static boolean checkAcquisitionSuccess( + AcquisitionResult acquired, + @SuppressWarnings("SameParameterValue") long numberOfPageToAcquire) { + return acquired.isSuccess() && + acquired.getAcquiredPerKey().values().stream().mapToLong(b -> b).sum() == numberOfPageToAcquire; + } + + private static KeyedBudgetManager createSimpleKeyedBudget() { + return new KeyedBudgetManager<>(createdBudgetMap(TEST_KEYS, TEST_BUDGETS), 1L); + } + + private static Map createdBudgetMap(String[] keys, long[] budgets) { + Preconditions.checkArgument(keys.length == budgets.length); + Map keydBudgets = new HashMap<>(); + for (int i = 0; i < keys.length; i++) { + keydBudgets.put(keys[i], budgets[i]); + } + return keydBudgets; + } + + private static void waitForFutureSilently(Future future) { + try { + future.get(); + } catch (InterruptedException | ExecutionException e) { + // silent + } + } +} From 036505f2afcde1ca4f0922cddafc3fe796ce45cb Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Thu, 12 Sep 2019 17:08:26 +0200 Subject: [PATCH 178/746] [FLINK-14399] Implement reservation of memory chunks in MemoryManager MemoryManager allocates paged segments from the provided memory pools of different types (on-/off-heap). Additionally, it can manage reservation and release of arbitrarily sized chunks of memory from the same memory pools respecting their overall limit. The way, how the memory is allocated, used and freed, is then up to the memory user. MemoryManager is just a book-keeping and limit checking component. --- .../flink/runtime/memory/MemoryManager.java | 122 +++++++++++++++++- .../memory/MemoryReservationException.java | 31 +++++ .../runtime/memory/MemoryManagerTest.java | 57 ++++++++ 3 files changed, 204 insertions(+), 6 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryReservationException.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java index 94c0e30c4b1f..07e22467132b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java @@ -52,13 +52,16 @@ import static org.apache.flink.core.memory.MemorySegmentFactory.allocateUnpooledSegment; /** - * The memory manager governs the memory that Flink uses for sorting, hashing, and caching. Memory - * is represented in segments of equal size. Operators allocate the memory by requesting a number - * of memory segments. + * The memory manager governs the memory that Flink uses for sorting, hashing, and caching. Memory is represented + * either in {@link MemorySegment}s of equal size and arbitrary type or in reserved chunks of certain size and {@link MemoryType}. + * Operators allocate the memory either by requesting a number of memory segments or by reserving chunks. + * Any allocated memory has to be released to be reused later. * - *

The memory may be represented as on-heap byte arrays or as off-heap memory regions - * (both via {@link HybridMemorySegment}). Which kinds of memory the MemoryManager serves and their sizes can - * be passed as an argument to the initialization. Releasing a memory segment will make it re-claimable + *

Which {@link MemoryType}s the MemoryManager serves and their total sizes can be passed as an argument + * to the constructor. + * + *

The memory segments may be represented as on-heap byte arrays or as off-heap memory regions + * (both via {@link HybridMemorySegment}). Releasing a memory segment will make it re-claimable * by the garbage collector. */ public class MemoryManager { @@ -75,6 +78,9 @@ public class MemoryManager { /** Memory segments allocated per memory owner. */ private final Map> allocatedSegments; + /** Reserved memory per memory owner. */ + private final Map> reservedMemory; + /** Number of slots of the task manager. */ private final int numberOfSlots; @@ -99,6 +105,7 @@ public MemoryManager( } this.allocatedSegments = new ConcurrentHashMap<>(); + this.reservedMemory = new ConcurrentHashMap<>(); this.numberOfSlots = numberOfSlots; this.budgetByType = new KeyedBudgetManager<>(memorySizeByType, pageSize); verifyIntTotalNumberOfPages(memorySizeByType, budgetByType.maxTotalNumberOfPages()); @@ -148,6 +155,7 @@ public void shutdown() { if (!isShutDown) { // mark as shutdown and release memory isShutDown = true; + reservedMemory.clear(); budgetByType.releaseAll(); // go over all allocated segments and release them @@ -440,6 +448,108 @@ public void releaseAll(Object owner) { segments.clear(); } + /** + * Reserves memory of a certain type for an owner from this memory manager. + * + * @param owner The owner to associate with the memory reservation, for the fallback release. + * @param memoryType type of memory to reserve (heap / off-heap). + * @param size size of memory to reserve. + * @throws MemoryReservationException Thrown, if this memory manager does not have the requested amount + * of memory any more. + */ + public void reserveMemory(Object owner, MemoryType memoryType, long size) throws MemoryReservationException { + checkMemoryReservationPreconditions(owner, memoryType, size); + if (size == 0L) { + return; + } + + long acquiredMemory = budgetByType.acquireBudgetForKey(memoryType, size); + if (acquiredMemory < size) { + throw new MemoryReservationException( + String.format("Could not allocate %d bytes. Only %d bytes are remaining.", size, acquiredMemory)); + } + + reservedMemory.compute(owner, (o, reservations) -> { + Map newReservations = reservations; + if (reservations == null) { + newReservations = new EnumMap<>(MemoryType.class); + newReservations.put(memoryType, size); + } else { + reservations.compute( + memoryType, + (mt, currentlyReserved) -> currentlyReserved == null ? size : currentlyReserved + size); + } + return newReservations; + }); + + Preconditions.checkState(!isShutDown, "Memory manager has been concurrently shut down."); + } + + /** + * Releases memory of a certain type from an owner to this memory manager. + * + * @param owner The owner to associate with the memory reservation, for the fallback release. + * @param memoryType type of memory to release (heap / off-heap). + * @param size size of memory to release. + */ + public void releaseMemory(Object owner, MemoryType memoryType, long size) { + checkMemoryReservationPreconditions(owner, memoryType, size); + if (size == 0L) { + return; + } + + reservedMemory.compute(owner, (o, reservations) -> { + if (reservations != null) { + reservations.compute( + memoryType, + (mt, currentlyReserved) -> { + if (currentlyReserved == null || currentlyReserved < size) { + LOG.warn( + "Trying to release more memory {} than it was reserved {} so far for the owner {}", + size, + currentlyReserved == null ? 0 : currentlyReserved, + owner); + //noinspection ReturnOfNull + return null; + } else { + return currentlyReserved - size; + } + }); + } + //noinspection ReturnOfNull + return reservations == null || reservations.isEmpty() ? null : reservations; + }); + budgetByType.releaseBudgetForKey(memoryType, size); + } + + private void checkMemoryReservationPreconditions(Object owner, MemoryType memoryType, long size) { + Preconditions.checkNotNull(owner, "The memory owner must not be null."); + Preconditions.checkNotNull(memoryType, "The memory type must not be null."); + Preconditions.checkState(!isShutDown, "Memory manager has been shut down."); + Preconditions.checkArgument(size >= 0L, "The memory size (%s) has to have non-negative size", size); + } + + /** + * Releases all memory of a certain type from an owner to this memory manager. + * + * @param owner The owner to associate with the memory reservation, for the fallback release. + * @param memoryType type of memory to release (heap / off-heap). + */ + public void releaseAllMemory(Object owner, MemoryType memoryType) { + checkMemoryReservationPreconditions(owner, memoryType, 0L); + + reservedMemory.compute(owner, (o, reservations) -> { + if (reservations != null) { + Long size = reservations.remove(memoryType); + if (size != null) { + budgetByType.releaseBudgetForKey(memoryType, size); + } + } + //noinspection ReturnOfNull + return reservations == null || reservations.isEmpty() ? null : reservations; + }); + } + // ------------------------------------------------------------------------ // Properties, sizes and size conversions // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryReservationException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryReservationException.java new file mode 100644 index 000000000000..6a7ddf94a8fa --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryReservationException.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.memory; + +/** + * An exception to be thrown when a memory reservation operation is not successful. + */ +public class MemoryReservationException extends Exception { + + private static final long serialVersionUID = 1L; + + MemoryReservationException(String message) { + super(message); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java index 824ce58a6a72..75fb970c9d9d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java @@ -37,6 +37,7 @@ import java.util.Random; import static org.apache.flink.runtime.memory.MemoryManager.AllocationRequest.ofAllTypes; +import static org.apache.flink.runtime.memory.MemoryManager.AllocationRequest.ofType; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.number.OrderingComparison.lessThanOrEqualTo; import static org.junit.Assert.assertThat; @@ -251,6 +252,52 @@ private static Map calcMemoryTypeSplitForSegments(Iterable< return split; } + @Test + public void testMemoryReservation() throws MemoryReservationException { + Object owner = new Object(); + + memoryManager.reserveMemory(owner, MemoryType.HEAP, PAGE_SIZE); + memoryManager.reserveMemory(owner, MemoryType.OFF_HEAP, memoryManager.getMemorySizeByType(MemoryType.OFF_HEAP)); + + memoryManager.releaseMemory(owner, MemoryType.HEAP, PAGE_SIZE); + memoryManager.releaseAllMemory(owner, MemoryType.OFF_HEAP); + } + + @Test + public void testCannotReserveBeyondTheLimit() throws MemoryReservationException { + Object owner = new Object(); + memoryManager.reserveMemory(owner, MemoryType.OFF_HEAP, memoryManager.getMemorySizeByType(MemoryType.OFF_HEAP)); + testCannotReserveAnymore(MemoryType.OFF_HEAP, 1L); + memoryManager.releaseAllMemory(owner, MemoryType.OFF_HEAP); + } + + @Test + public void testMemoryTooBigReservation() { + long size = memoryManager.getMemorySizeByType(MemoryType.HEAP) + PAGE_SIZE; + testCannotReserveAnymore(MemoryType.HEAP, size); + } + + @Test + public void testMemoryAllocationAndReservation() throws MemoryAllocationException, MemoryReservationException { + MemoryType type = MemoryType.OFF_HEAP; + @SuppressWarnings("NumericCastThatLosesPrecision") + int totalPagesForType = (int) memoryManager.getMemorySizeByType(type) / PAGE_SIZE; + + // allocate half memory for segments + Object owner1 = new Object(); + memoryManager.allocatePages(ofType(owner1, totalPagesForType / 2, MemoryType.OFF_HEAP)); + + // reserve the other half of memory + Object owner2 = new Object(); + memoryManager.reserveMemory(owner2, type, (long) PAGE_SIZE * totalPagesForType / 2); + + testCannotAllocateAnymore(ofType(new Object(), 1, type)); + testCannotReserveAnymore(type, 1L); + + memoryManager.releaseAll(owner1); + memoryManager.releaseAllMemory(owner2, type); + } + private void testCannotAllocateAnymore(AllocationRequest request) { try { memoryManager.allocatePages(request); @@ -260,4 +307,14 @@ private void testCannotAllocateAnymore(AllocationRequest request) { // expected } } + + private void testCannotReserveAnymore(MemoryType type, long size) { + try { + memoryManager.reserveMemory(new Object(), type, size); + Assert.fail("Expected MemoryAllocationException. " + + "We should not be able to any more memory after allocating or(and) reserving all memory of a certain type."); + } catch (MemoryReservationException maex) { + // expected + } + } } From 7334e0aae60c0aea79e992e42f0822d3eec3a7ff Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Mon, 21 Oct 2019 15:17:20 +0200 Subject: [PATCH 179/746] [hotfix] Enable memory manager with zero memory or zero pages --- .../java/org/apache/flink/runtime/memory/MemoryManager.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java index 07e22467132b..472e57104678 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java @@ -120,7 +120,7 @@ public MemoryManager( private static void sanityCheck(long memorySize, int pageSize, MemoryType memoryType) { Preconditions.checkNotNull(memoryType); - Preconditions.checkArgument(memorySize > 0L, "Size of total memory must be positive."); + Preconditions.checkArgument(memorySize >= 0L, "Size of total memory must be non-negative."); Preconditions.checkArgument( pageSize >= MIN_PAGE_SIZE, "The page size must be at least %d bytes.", MIN_PAGE_SIZE); @@ -135,10 +135,6 @@ private static void verifyIntTotalNumberOfPages(Map memorySize "The given number of memory bytes (%d: %s) corresponds to more than MAX_INT pages.", numberOfPagesLong, memorySizeByType); - - @SuppressWarnings("NumericCastThatLosesPrecision") - int totalNumPages = (int) numberOfPagesLong; - Preconditions.checkArgument(totalNumPages >= 1, "The given amount of memory amounted to less than one page."); } // ------------------------------------------------------------------------ From 56126bd43f6539dea39419e8f81779419ccea1c4 Mon Sep 17 00:00:00 2001 From: yanghua Date: Mon, 21 Oct 2019 16:22:35 +0800 Subject: [PATCH 180/746] [FLINK-14447] Network metrics doc table render confusion --- 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 696b508ae430..7f6598ba187f 100644 --- a/docs/monitoring/metrics.md +++ b/docs/monitoring/metrics.md @@ -1033,8 +1033,8 @@ Thus, in order to infer the metric identifier: Gauge - Task - buffers + Task + buffers inputQueueLength The number of queued input buffers. (ignores LocalInputChannels which are using blocking subpartitions) Gauge diff --git a/docs/monitoring/metrics.zh.md b/docs/monitoring/metrics.zh.md index 389c65cabf6c..a2b552d88042 100644 --- a/docs/monitoring/metrics.zh.md +++ b/docs/monitoring/metrics.zh.md @@ -1033,8 +1033,8 @@ Thus, in order to infer the metric identifier: Gauge - Task - buffers + Task + buffers inputQueueLength The number of queued input buffers. (ignores LocalInputChannels which are using blocking subpartitions) Gauge From ee433955575abed85a043e3b5a837028ed88ccd4 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Fri, 18 Oct 2019 16:27:32 +0200 Subject: [PATCH 181/746] [hotfix][kafka,test] Drop unused field in FailingIdentityMapper --- .../connectors/kafka/testutils/FailingIdentityMapper.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java index c25eefb247d1..bd412c911c9a 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/FailingIdentityMapper.java @@ -42,7 +42,6 @@ public class FailingIdentityMapper extends RichMapFunction implements private static final long serialVersionUID = 6334389850158707313L; public static volatile boolean failedBefore; - public static volatile boolean hasBeenCheckpointedBeforeFailure; private final int failCount; private int numElementsTotal; @@ -74,7 +73,6 @@ public T map(T value) throws Exception { Thread.sleep(10); if (failer && numElementsTotal >= failCount) { - hasBeenCheckpointedBeforeFailure = hasBeenCheckpointed; failedBefore = true; throw new Exception("Artificial Test Failure"); } From 8a75dec1caa6803c79d4b0713cf27c4c17c46a22 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Fri, 18 Oct 2019 16:28:31 +0200 Subject: [PATCH 182/746] [hotfix][kafka,test] Remove unused statement from testOneToOneAtLeastOnce --- .../flink/streaming/connectors/kafka/KafkaProducerTestBase.java | 1 - 1 file changed, 1 deletion(-) 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 a971386c44f0..99a19d2fc98f 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 @@ -277,7 +277,6 @@ public int partition(Integer record, byte[] key, byte[] value, String targetTopi }); } - FailingIdentityMapper.failedBefore = false; try { env.execute("One-to-one at least once test"); fail("Job should fail!"); From 5ecb35198cb81c4d9cdb6b8a68698db8ca0ec034 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Mon, 21 Oct 2019 10:16:50 +0200 Subject: [PATCH 183/746] [hotfix][dist] Rename local variables in GlobalConfiguration --- .../flink/configuration/GlobalConfiguration.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/configuration/GlobalConfiguration.java b/flink-core/src/main/java/org/apache/flink/configuration/GlobalConfiguration.java index c0debab7c521..be0ffa00f84e 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/GlobalConfiguration.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/GlobalConfiguration.java @@ -139,23 +139,23 @@ private static Configuration enrichWithEnvironmentVariables(Configuration config } private static void enrichWithEnvironmentVariable(String environmentVariable, Configuration configuration) { - String pluginsDirFromEnv = System.getenv(environmentVariable); + String valueFromEnv = System.getenv(environmentVariable); - if (pluginsDirFromEnv == null) { + if (valueFromEnv == null) { return; } - String pluginsDirFromConfig = configuration.getString(environmentVariable, pluginsDirFromEnv); + String valueFromConfig = configuration.getString(environmentVariable, valueFromEnv); - if (!pluginsDirFromEnv.equals(pluginsDirFromConfig)) { + if (!valueFromEnv.equals(valueFromConfig)) { throw new IllegalConfigurationException( - "The given configuration file already contains a value (" + pluginsDirFromEnv + + "The given configuration file already contains a value (" + valueFromEnv + ") for the key (" + environmentVariable + - ") that would have been overwritten with (" + pluginsDirFromConfig + + ") that would have been overwritten with (" + valueFromConfig + ") by an environment with the same name."); } - configuration.setString(environmentVariable, pluginsDirFromEnv); + configuration.setString(environmentVariable, valueFromEnv); } /** From c31e44e5402c5fd7deb10b83534740ac7f66d0f8 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Fri, 18 Oct 2019 16:38:26 +0200 Subject: [PATCH 184/746] [FLINK-14235][kafka,tests] Change source in at-least-once test from finite to infinite Previously it was possible that the source would end before a first chcekpoint could complete. If that was the case, any exceptions thrown during checkpointing are swallowed, which could explain the apparent data loss from FLINK-14235. --- .../kafka/KafkaProducerTestBase.java | 20 ++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) 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 99a19d2fc98f..10e4f552e738 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 @@ -255,7 +255,7 @@ protected void testOneToOneAtLeastOnce(boolean regularSink) throws Exception { // process exactly failAfterElements number of elements and then shutdown Kafka broker and fail application DataStream inputStream = env - .fromCollection(getIntegersSequence(numElements)) + .addSource(new InfiniteIntegerSource()) .map(new BrokerRestartingMapper<>(failAfterElements)); StreamSink kafkaSink = kafkaServer.getProducerSink(topic, keyedSerializationSchema, properties, new FlinkKafkaPartitioner() { @@ -538,4 +538,22 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { public void initializeState(FunctionInitializationContext context) throws Exception { } } + + private static final class InfiniteIntegerSource implements SourceFunction { + + private volatile boolean running = true; + private int counter = 0; + + @Override + public void run(SourceContext ctx) throws Exception { + while (running) { + ctx.collect(counter++); + } + } + + @Override + public void cancel() { + running = false; + } + } } From 8e0b35cc09bc158b0b152b72e569943c1b2e1c97 Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Fri, 20 Sep 2019 18:35:33 +0200 Subject: [PATCH 185/746] [hotfix] Use MemorySegmentFactory in tests instead of HybridMemorySegment constructors --- .../core/memory/HybridMemorySegment.java | 25 ---------- .../core/memory/MemorySegmentFactory.java | 40 ++++----------- .../core/memory/CrossSegmentTypeTest.java | 50 ++++++------------- .../core/memory/EndiannessAccessChecks.java | 5 +- .../HybridOffHeapMemorySegmentTest.java | 6 +-- .../memory/HybridOnHeapMemorySegmentTest.java | 6 +-- .../core/memory/MemorySegmentChecksTest.java | 12 +---- .../memory/MemorySegmentUndersizedTest.java | 8 +-- .../memory/OperationsOnFreedSegmentTest.java | 28 +++++------ 9 files changed, 53 insertions(+), 127 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/HybridMemorySegment.java b/flink-core/src/main/java/org/apache/flink/core/memory/HybridMemorySegment.java index 115718e4195e..170f22d33254 100644 --- a/flink-core/src/main/java/org/apache/flink/core/memory/HybridMemorySegment.java +++ b/flink-core/src/main/java/org/apache/flink/core/memory/HybridMemorySegment.java @@ -52,20 +52,6 @@ public final class HybridMemorySegment extends MemorySegment { */ private final ByteBuffer offHeapBuffer; - /** - * Creates a new memory segment that represents the memory backing the given direct byte buffer. - * Note that the given ByteBuffer must be direct {@link java.nio.ByteBuffer#allocateDirect(int)}, - * otherwise this method with throw an IllegalArgumentException. - * - *

The owner referenced by this memory segment is null. - * - * @param buffer The byte buffer whose memory is represented by this memory segment. - * @throws IllegalArgumentException Thrown, if the given ByteBuffer is not direct. - */ - HybridMemorySegment(ByteBuffer buffer) { - this(buffer, null); - } - /** * Creates a new memory segment that represents the memory backing the given direct byte buffer. * Note that the given ByteBuffer must be direct {@link java.nio.ByteBuffer#allocateDirect(int)}, @@ -82,17 +68,6 @@ public final class HybridMemorySegment extends MemorySegment { this.offHeapBuffer = buffer; } - /** - * Creates a new memory segment that represents the memory of the byte array. - * - *

The owner referenced by this memory segment is null. - * - * @param buffer The byte array whose memory is represented by this memory segment. - */ - HybridMemorySegment(byte[] buffer) { - this(buffer, null); - } - /** * Creates a new memory segment that represents the memory of the byte array. * diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java index 9a20e0e97832..53865f43dc97 100644 --- a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java +++ b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java @@ -42,7 +42,7 @@ public final class MemorySegmentFactory { * @return A new memory segment that targets the given heap memory region. */ public static MemorySegment wrap(byte[] buffer) { - return new HybridMemorySegment(buffer); + return new HybridMemorySegment(buffer, null); } /** @@ -79,40 +79,22 @@ public static MemorySegment allocateUnpooledSegment(int size, Object owner) { * represents that memory. * * @param size The size of the off-heap memory segment to allocate. - * @param owner The owner to associate with the off-heap memory segment. * @return A new memory segment, backed by unpooled off-heap memory. */ - public static MemorySegment allocateUnpooledOffHeapMemory(int size, Object owner) { - ByteBuffer memory = ByteBuffer.allocateDirect(size); - return wrapPooledOffHeapMemory(memory, owner); - } - - /** - * Creates a memory segment that wraps the given byte array. - * - *

This method is intended to be used for components which pool memory and create - * memory segments around long-lived memory regions. - * - * @param memory The heap memory to be represented by the memory segment. - * @param owner The owner to associate with the memory segment. - * @return A new memory segment representing the given heap memory. - */ - public static MemorySegment wrapPooledHeapMemory(byte[] memory, Object owner) { - return new HybridMemorySegment(memory, owner); + public static MemorySegment allocateUnpooledOffHeapMemory(int size) { + return allocateUnpooledOffHeapMemory(size, null); } /** - * Creates a memory segment that wraps the off-heap memory backing the given ByteBuffer. - * Note that the ByteBuffer needs to be a direct ByteBuffer. - * - *

This method is intended to be used for components which pool memory and create - * memory segments around long-lived memory regions. + * Allocates some unpooled off-heap memory and creates a new memory segment that + * represents that memory. * - * @param memory The byte buffer with the off-heap memory to be represented by the memory segment. - * @param owner The owner to associate with the memory segment. - * @return A new memory segment representing the given off-heap memory. + * @param size The size of the off-heap memory segment to allocate. + * @param owner The owner to associate with the off-heap memory segment. + * @return A new memory segment, backed by unpooled off-heap memory. */ - public static MemorySegment wrapPooledOffHeapMemory(ByteBuffer memory, Object owner) { + public static MemorySegment allocateUnpooledOffHeapMemory(int size, Object owner) { + ByteBuffer memory = ByteBuffer.allocateDirect(size); return new HybridMemorySegment(memory, owner); } @@ -127,7 +109,7 @@ public static MemorySegment wrapPooledOffHeapMemory(ByteBuffer memory, Object ow * @return A new memory segment representing the given off-heap memory. */ public static MemorySegment wrapOffHeapMemory(ByteBuffer memory) { - return new HybridMemorySegment(memory); + return new HybridMemorySegment(memory, null); } } diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/CrossSegmentTypeTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/CrossSegmentTypeTest.java index ccff2ba14795..1fc352f2f6ba 100644 --- a/flink-core/src/test/java/org/apache/flink/core/memory/CrossSegmentTypeTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/memory/CrossSegmentTypeTest.java @@ -20,7 +20,6 @@ import org.junit.Test; -import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Random; @@ -43,17 +42,8 @@ public class CrossSegmentTypeTest { @Test public void testCompareBytesMixedSegments() { - MemorySegment[] segs1 = { - new HeapMemorySegment(new byte[pageSize]), - new HybridMemorySegment(new byte[pageSize]), - new HybridMemorySegment(ByteBuffer.allocateDirect(pageSize)) - }; - - MemorySegment[] segs2 = { - new HeapMemorySegment(new byte[pageSize]), - new HybridMemorySegment(new byte[pageSize]), - new HybridMemorySegment(ByteBuffer.allocateDirect(pageSize)) - }; + MemorySegment[] segs1 = createSegments(pageSize); + MemorySegment[] segs2 = createSegments(pageSize); Random rnd = new Random(); @@ -107,18 +97,8 @@ private void testCompare(MemorySegment seg1, MemorySegment seg2, Random random) @Test public void testSwapBytesMixedSegments() { final int halfPageSize = pageSize / 2; - - MemorySegment[] segs1 = { - new HeapMemorySegment(new byte[pageSize]), - new HybridMemorySegment(new byte[pageSize]), - new HybridMemorySegment(ByteBuffer.allocateDirect(pageSize)) - }; - - MemorySegment[] segs2 = { - new HeapMemorySegment(new byte[halfPageSize]), - new HybridMemorySegment(new byte[halfPageSize]), - new HybridMemorySegment(ByteBuffer.allocateDirect(halfPageSize)) - }; + MemorySegment[] segs1 = createSegments(pageSize); + MemorySegment[] segs2 = createSegments(halfPageSize); Random rnd = new Random(); @@ -162,17 +142,8 @@ private void testSwap(MemorySegment seg1, MemorySegment seg2, Random random, int @Test public void testCopyMixedSegments() { - MemorySegment[] segs1 = { - new HeapMemorySegment(new byte[pageSize]), - new HybridMemorySegment(new byte[pageSize]), - new HybridMemorySegment(ByteBuffer.allocateDirect(pageSize)) - }; - - MemorySegment[] segs2 = { - new HeapMemorySegment(new byte[pageSize]), - new HybridMemorySegment(new byte[pageSize]), - new HybridMemorySegment(ByteBuffer.allocateDirect(pageSize)) - }; + MemorySegment[] segs1 = createSegments(pageSize); + MemorySegment[] segs2 = createSegments(pageSize); Random rnd = new Random(); @@ -183,6 +154,15 @@ public void testCopyMixedSegments() { } } + private static MemorySegment[] createSegments(int size) { + MemorySegment[] segments = { + new HeapMemorySegment(new byte[size]), + MemorySegmentFactory.allocateUnpooledSegment(size), + MemorySegmentFactory.allocateUnpooledOffHeapMemory(size) + }; + return segments; + } + private void testCopy(MemorySegment seg1, MemorySegment seg2, Random random) { assertEquals(pageSize, seg1.size()); assertEquals(pageSize, seg2.size()); diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/EndiannessAccessChecks.java b/flink-core/src/test/java/org/apache/flink/core/memory/EndiannessAccessChecks.java index c04d4353bc7c..feead4b4b74b 100644 --- a/flink-core/src/test/java/org/apache/flink/core/memory/EndiannessAccessChecks.java +++ b/flink-core/src/test/java/org/apache/flink/core/memory/EndiannessAccessChecks.java @@ -20,7 +20,6 @@ import org.junit.Test; -import java.nio.ByteBuffer; import java.util.Random; import static org.junit.Assert.assertEquals; @@ -38,12 +37,12 @@ public void testHeapSegment() { @Test public void testHybridOnHeapSegment() { - testBigAndLittleEndianAccessUnaligned(new HybridMemorySegment(new byte[11111])); + testBigAndLittleEndianAccessUnaligned(MemorySegmentFactory.wrap(new byte[11111])); } @Test public void testHybridOffHeapSegment() { - testBigAndLittleEndianAccessUnaligned(new HybridMemorySegment(ByteBuffer.allocateDirect(11111))); + testBigAndLittleEndianAccessUnaligned(MemorySegmentFactory.allocateUnpooledOffHeapMemory(11111)); } private void testBigAndLittleEndianAccessUnaligned(MemorySegment segment) { diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapMemorySegmentTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapMemorySegmentTest.java index 254254c76698..b93f798ce8bc 100644 --- a/flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapMemorySegmentTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapMemorySegmentTest.java @@ -41,18 +41,18 @@ public HybridOffHeapMemorySegmentTest(int pageSize) { @Override MemorySegment createSegment(int size) { - return new HybridMemorySegment(ByteBuffer.allocateDirect(size)); + return MemorySegmentFactory.allocateUnpooledOffHeapMemory(size); } @Override MemorySegment createSegment(int size, Object owner) { - return new HybridMemorySegment(ByteBuffer.allocateDirect(size), owner); + return MemorySegmentFactory.allocateUnpooledOffHeapMemory(size, owner); } @Test public void testHybridHeapSegmentSpecifics() { final ByteBuffer buffer = ByteBuffer.allocateDirect(411); - HybridMemorySegment seg = new HybridMemorySegment(buffer); + HybridMemorySegment seg = new HybridMemorySegment(buffer, null); assertFalse(seg.isFreed()); assertTrue(seg.isOffHeap()); diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/HybridOnHeapMemorySegmentTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/HybridOnHeapMemorySegmentTest.java index 667274418e1e..7cfc3019fcbc 100644 --- a/flink-core/src/test/java/org/apache/flink/core/memory/HybridOnHeapMemorySegmentTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/memory/HybridOnHeapMemorySegmentTest.java @@ -41,18 +41,18 @@ public HybridOnHeapMemorySegmentTest(int pageSize) { @Override MemorySegment createSegment(int size) { - return new HybridMemorySegment(new byte[size]); + return MemorySegmentFactory.allocateUnpooledSegment(size); } @Override MemorySegment createSegment(int size, Object owner) { - return new HybridMemorySegment(new byte[size], owner); + return MemorySegmentFactory.allocateUnpooledSegment(size, owner); } @Test public void testHybridHeapSegmentSpecifics() { final byte[] buffer = new byte[411]; - HybridMemorySegment seg = new HybridMemorySegment(buffer); + HybridMemorySegment seg = new HybridMemorySegment(buffer, null); assertFalse(seg.isFreed()); assertFalse(seg.isOffHeap()); diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentChecksTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentChecksTest.java index ccf3a4b4f046..09619cdd2ad9 100644 --- a/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentChecksTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentChecksTest.java @@ -39,21 +39,11 @@ public void testHeapNullBuffer2() { new HeapMemorySegment(null, new Object()); } - @Test(expected = NullPointerException.class) - public void testHybridHeapNullBuffer1() { - new HybridMemorySegment((byte[]) null); - } - @Test(expected = NullPointerException.class) public void testHybridHeapNullBuffer2() { new HybridMemorySegment((byte[]) null, new Object()); } - @Test(expected = NullPointerException.class) - public void testHybridOffHeapNullBuffer1() { - new HybridMemorySegment((ByteBuffer) null); - } - @Test(expected = NullPointerException.class) public void testHybridOffHeapNullBuffer2() { new HybridMemorySegment((ByteBuffer) null, new Object()); @@ -61,7 +51,7 @@ public void testHybridOffHeapNullBuffer2() { @Test(expected = IllegalArgumentException.class) public void testHybridNonDirectBuffer() { - new HybridMemorySegment(ByteBuffer.allocate(1024)); + new HybridMemorySegment(ByteBuffer.allocate(1024), new Object()); } @Test(expected = IllegalArgumentException.class) diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentUndersizedTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentUndersizedTest.java index 5f807428f596..efece684f16c 100644 --- a/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentUndersizedTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentUndersizedTest.java @@ -47,7 +47,7 @@ public void testZeroSizeHeapSegment() { @Test public void testZeroSizeHeapHybridSegment() { - MemorySegment segment = new HybridMemorySegment(new byte[0]); + MemorySegment segment = MemorySegmentFactory.allocateUnpooledSegment(0); testZeroSizeBuffer(segment); testSegmentWithSizeLargerZero(segment); @@ -55,7 +55,7 @@ public void testZeroSizeHeapHybridSegment() { @Test public void testZeroSizeOffHeapHybridSegment() { - MemorySegment segment = new HybridMemorySegment(ByteBuffer.allocateDirect(0)); + MemorySegment segment = MemorySegmentFactory.allocateUnpooledOffHeapMemory(0); testZeroSizeBuffer(segment); testSegmentWithSizeLargerZero(segment); @@ -68,12 +68,12 @@ public void testSizeOneHeapSegment() { @Test public void testSizeOneHeapHybridSegment() { - testSegmentWithSizeLargerZero(new HybridMemorySegment(new byte[1])); + testSegmentWithSizeLargerZero(MemorySegmentFactory.allocateUnpooledSegment(1)); } @Test public void testSizeOneOffHeapHybridSegment() { - testSegmentWithSizeLargerZero(new HybridMemorySegment(ByteBuffer.allocateDirect(1))); + testSegmentWithSizeLargerZero(MemorySegmentFactory.allocateUnpooledOffHeapMemory(1)); } private static void testZeroSizeBuffer(MemorySegment segment) { diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/OperationsOnFreedSegmentTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/OperationsOnFreedSegmentTest.java index ad9d63f3c9a8..d6fa6763a7a4 100644 --- a/flink-core/src/test/java/org/apache/flink/core/memory/OperationsOnFreedSegmentTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/memory/OperationsOnFreedSegmentTest.java @@ -42,19 +42,19 @@ public class OperationsOnFreedSegmentTest { @Test public void testSingleSegmentOperationsHeapSegment() throws Exception { testOpsOnFreedSegment(new HeapMemorySegment(new byte[PAGE_SIZE])); - testOpsOnFreedSegment(new HybridMemorySegment(new byte[PAGE_SIZE])); - testOpsOnFreedSegment(new HybridMemorySegment(ByteBuffer.allocateDirect(PAGE_SIZE))); + testOpsOnFreedSegment(MemorySegmentFactory.wrap(new byte[PAGE_SIZE])); + testOpsOnFreedSegment(MemorySegmentFactory.allocateUnpooledOffHeapMemory(PAGE_SIZE)); } @Test public void testCompare() { MemorySegment aliveHeap = new HeapMemorySegment(new byte[PAGE_SIZE]); - MemorySegment aliveHybridHeap = new HybridMemorySegment(new byte[PAGE_SIZE]); - MemorySegment aliveHybridOffHeap = new HybridMemorySegment(ByteBuffer.allocateDirect(PAGE_SIZE)); + MemorySegment aliveHybridHeap = MemorySegmentFactory.wrap(new byte[PAGE_SIZE]); + MemorySegment aliveHybridOffHeap = MemorySegmentFactory.allocateUnpooledOffHeapMemory(PAGE_SIZE); MemorySegment freedHeap = new HeapMemorySegment(new byte[PAGE_SIZE]); - MemorySegment freedHybridHeap = new HybridMemorySegment(new byte[PAGE_SIZE]); - MemorySegment freedHybridOffHeap = new HybridMemorySegment(ByteBuffer.allocateDirect(PAGE_SIZE)); + MemorySegment freedHybridHeap = MemorySegmentFactory.wrap(new byte[PAGE_SIZE]); + MemorySegment freedHybridOffHeap = MemorySegmentFactory.allocateUnpooledOffHeapMemory(PAGE_SIZE); freedHeap.free(); freedHybridHeap.free(); freedHybridOffHeap.free(); @@ -87,12 +87,12 @@ public void testCompare() { @Test public void testCopyTo() { MemorySegment aliveHeap = new HeapMemorySegment(new byte[PAGE_SIZE]); - MemorySegment aliveHybridHeap = new HybridMemorySegment(new byte[PAGE_SIZE]); - MemorySegment aliveHybridOffHeap = new HybridMemorySegment(ByteBuffer.allocateDirect(PAGE_SIZE)); + MemorySegment aliveHybridHeap = MemorySegmentFactory.wrap(new byte[PAGE_SIZE]); + MemorySegment aliveHybridOffHeap = MemorySegmentFactory.allocateUnpooledOffHeapMemory(PAGE_SIZE); MemorySegment freedHeap = new HeapMemorySegment(new byte[PAGE_SIZE]); - MemorySegment freedHybridHeap = new HybridMemorySegment(new byte[PAGE_SIZE]); - MemorySegment freedHybridOffHeap = new HybridMemorySegment(ByteBuffer.allocateDirect(PAGE_SIZE)); + MemorySegment freedHybridHeap = MemorySegmentFactory.wrap(new byte[PAGE_SIZE]); + MemorySegment freedHybridOffHeap = MemorySegmentFactory.allocateUnpooledOffHeapMemory(PAGE_SIZE); freedHeap.free(); freedHybridHeap.free(); freedHybridOffHeap.free(); @@ -125,12 +125,12 @@ public void testCopyTo() { @Test public void testSwap() { MemorySegment aliveHeap = new HeapMemorySegment(new byte[PAGE_SIZE]); - MemorySegment aliveHybridHeap = new HybridMemorySegment(new byte[PAGE_SIZE]); - MemorySegment aliveHybridOffHeap = new HybridMemorySegment(ByteBuffer.allocateDirect(PAGE_SIZE)); + MemorySegment aliveHybridHeap = MemorySegmentFactory.wrap(new byte[PAGE_SIZE]); + MemorySegment aliveHybridOffHeap = MemorySegmentFactory.allocateUnpooledOffHeapMemory(PAGE_SIZE); MemorySegment freedHeap = new HeapMemorySegment(new byte[PAGE_SIZE]); - MemorySegment freedHybridHeap = new HybridMemorySegment(new byte[PAGE_SIZE]); - MemorySegment freedHybridOffHeap = new HybridMemorySegment(ByteBuffer.allocateDirect(PAGE_SIZE)); + MemorySegment freedHybridHeap = MemorySegmentFactory.wrap(new byte[PAGE_SIZE]); + MemorySegment freedHybridOffHeap = MemorySegmentFactory.allocateUnpooledOffHeapMemory(PAGE_SIZE); freedHeap.free(); freedHybridHeap.free(); freedHybridOffHeap.free(); From d154ae5f1379991ed1a9d3416b167f5b61401ad7 Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Tue, 15 Oct 2019 15:00:50 +0200 Subject: [PATCH 186/746] [hotfix] Minor refactoring of OperationsOnFreedSegmentTest --- .../memory/OperationsOnFreedSegmentTest.java | 80 +++++++------------ 1 file changed, 27 insertions(+), 53 deletions(-) diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/OperationsOnFreedSegmentTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/OperationsOnFreedSegmentTest.java index d6fa6763a7a4..8e7c83b95b98 100644 --- a/flink-core/src/test/java/org/apache/flink/core/memory/OperationsOnFreedSegmentTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/memory/OperationsOnFreedSegmentTest.java @@ -27,6 +27,7 @@ import java.io.DataOutput; import java.io.DataOutputStream; import java.nio.ByteBuffer; +import java.util.function.BiConsumer; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -41,9 +42,9 @@ public class OperationsOnFreedSegmentTest { @Test public void testSingleSegmentOperationsHeapSegment() throws Exception { - testOpsOnFreedSegment(new HeapMemorySegment(new byte[PAGE_SIZE])); - testOpsOnFreedSegment(MemorySegmentFactory.wrap(new byte[PAGE_SIZE])); - testOpsOnFreedSegment(MemorySegmentFactory.allocateUnpooledOffHeapMemory(PAGE_SIZE)); + for (MemorySegment segment : createTestSegments()) { + testOpsOnFreedSegment(segment); + } } @Test @@ -86,78 +87,51 @@ public void testCompare() { @Test public void testCopyTo() { - MemorySegment aliveHeap = new HeapMemorySegment(new byte[PAGE_SIZE]); - MemorySegment aliveHybridHeap = MemorySegmentFactory.wrap(new byte[PAGE_SIZE]); - MemorySegment aliveHybridOffHeap = MemorySegmentFactory.allocateUnpooledOffHeapMemory(PAGE_SIZE); + testAliveVsFree(this::testCopy); + } - MemorySegment freedHeap = new HeapMemorySegment(new byte[PAGE_SIZE]); - MemorySegment freedHybridHeap = MemorySegmentFactory.wrap(new byte[PAGE_SIZE]); - MemorySegment freedHybridOffHeap = MemorySegmentFactory.allocateUnpooledOffHeapMemory(PAGE_SIZE); - freedHeap.free(); - freedHybridHeap.free(); - freedHybridOffHeap.free(); + @Test + public void testSwap() { + testAliveVsFree(this::testSwap); + } - MemorySegment[] alive = { aliveHeap, aliveHybridHeap, aliveHybridOffHeap }; - MemorySegment[] free = { freedHeap, freedHybridHeap, freedHybridOffHeap }; + private static void testAliveVsFree(BiConsumer testOperation) { + MemorySegment[] alive = createTestSegments(); + MemorySegment[] free = createTestSegments(); + for (MemorySegment segment : free) { + segment.free(); + } // alive with free for (MemorySegment seg1 : alive) { for (MemorySegment seg2 : free) { - testCopy(seg1, seg2); + testOperation.accept(seg1, seg2); } } // free with alive for (MemorySegment seg1 : free) { for (MemorySegment seg2 : alive) { - testCopy(seg1, seg2); + testOperation.accept(seg1, seg2); } } // free with free for (MemorySegment seg1 : free) { for (MemorySegment seg2 : free) { - testCopy(seg1, seg2); + testOperation.accept(seg1, seg2); } } } - @Test - public void testSwap() { - MemorySegment aliveHeap = new HeapMemorySegment(new byte[PAGE_SIZE]); - MemorySegment aliveHybridHeap = MemorySegmentFactory.wrap(new byte[PAGE_SIZE]); - MemorySegment aliveHybridOffHeap = MemorySegmentFactory.allocateUnpooledOffHeapMemory(PAGE_SIZE); - - MemorySegment freedHeap = new HeapMemorySegment(new byte[PAGE_SIZE]); - MemorySegment freedHybridHeap = MemorySegmentFactory.wrap(new byte[PAGE_SIZE]); - MemorySegment freedHybridOffHeap = MemorySegmentFactory.allocateUnpooledOffHeapMemory(PAGE_SIZE); - freedHeap.free(); - freedHybridHeap.free(); - freedHybridOffHeap.free(); - - MemorySegment[] alive = { aliveHeap, aliveHybridHeap, aliveHybridOffHeap }; - MemorySegment[] free = { freedHeap, freedHybridHeap, freedHybridOffHeap }; - - // alive with free - for (MemorySegment seg1 : alive) { - for (MemorySegment seg2 : free) { - testSwap(seg1, seg2); - } - } + private static MemorySegment[] createTestSegments() { + MemorySegment heap = new HeapMemorySegment(new byte[PAGE_SIZE]); + MemorySegment hybridHeap = MemorySegmentFactory.wrap(new byte[PAGE_SIZE]); + MemorySegment hybridOffHeap = MemorySegmentFactory.allocateUnpooledOffHeapMemory(PAGE_SIZE); - // free with alive - for (MemorySegment seg1 : free) { - for (MemorySegment seg2 : alive) { - testSwap(seg1, seg2); - } - } + MemorySegment[] segments = { heap, hybridHeap, hybridOffHeap }; - // free with free - for (MemorySegment seg1 : free) { - for (MemorySegment seg2 : free) { - testSwap(seg1, seg2); - } - } + return segments; } private void testOpsOnFreedSegment(MemorySegment segment) throws Exception { @@ -943,8 +917,8 @@ private void testOpsOnFreedSegment(MemorySegment segment) throws Exception { // --------- ByteBuffer ----------- for (ByteBuffer bbuf : new ByteBuffer[] { - ByteBuffer.allocate(55), - ByteBuffer.allocateDirect(55) }) { + ByteBuffer.allocate(55), + ByteBuffer.allocateDirect(55) }) { try { segment.get(0, bbuf, 17); From afe216121e93b902a07e4e104ac94a73cf6a2877 Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Thu, 19 Sep 2019 18:00:23 +0200 Subject: [PATCH 187/746] [FLINK-13985] Use unsafe memory for managed memory Makes allocation of off-heap memory independent of direct buffers and limiting option -XX:MaxDirectMemorySize. This implementation uses sun.misc.Unsafe for that purpose. - Add allocate/release unsafe memory methods to MemoryUtils - Add a MemoryUtils#createMemoryGcCleaner to release memory in phantom reference queue upon GC of the memory owning object (based on sun.misc.Cleaner similar to java.nio.DirectByteBuffer(int cap)). - Add MemoryUtils#wrapUnsafeMemoryWithByteBuffer which uses the private constructor java.nio.DirectByteBuffer(long address, int cap) to wrap unsafe memory but w/o checking -XX:MaxDirectMemorySize. - Add an optional custom action to call in HybridMemorySegment#free for memory cleanup - Change MemorySegmentFactory#allocateUnpooledOffHeapMemory to - allocate unsafe memory, - wrap it with ByteBuffer with MemoryUtils#wrapUnsafeMemoryWithByteBuffer - hook a phantom reference queue cleaner upon the buffer GC and - create HybridMemorySegment with this buffer and its custom cleaner. --- .../core/memory/HybridMemorySegment.java | 47 +++++++---- .../core/memory/MemorySegmentFactory.java | 21 ++++- .../apache/flink/core/memory/MemoryUtils.java | 81 +++++++++++++++++++ .../core/memory/CrossSegmentTypeTest.java | 3 +- .../core/memory/EndiannessAccessChecks.java | 5 ++ .../HybridOffHeapDirectMemorySegmentTest.java | 43 ++++++++++ .../HybridOffHeapMemorySegmentTest.java | 21 +---- .../HybridOffHeapUnsafeMemorySegmentTest.java | 43 ++++++++++ .../core/memory/MemorySegmentChecksTest.java | 4 +- .../core/memory/MemorySegmentTestBase.java | 5 ++ .../memory/MemorySegmentUndersizedTest.java | 13 +++ .../memory/OperationsOnFreedSegmentTest.java | 3 +- .../flink/runtime/memory/MemoryManager.java | 4 +- 13 files changed, 254 insertions(+), 39 deletions(-) create mode 100644 flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapDirectMemorySegmentTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapUnsafeMemorySegmentTest.java diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/HybridMemorySegment.java b/flink-core/src/main/java/org/apache/flink/core/memory/HybridMemorySegment.java index 170f22d33254..c8889f53aa36 100644 --- a/flink-core/src/main/java/org/apache/flink/core/memory/HybridMemorySegment.java +++ b/flink-core/src/main/java/org/apache/flink/core/memory/HybridMemorySegment.java @@ -20,6 +20,8 @@ import org.apache.flink.annotation.Internal; +import javax.annotation.Nullable; + import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; @@ -30,7 +32,9 @@ import java.nio.ReadOnlyBufferException; /** - * This class represents a piece of memory managed by Flink. The memory can be on-heap or off-heap, + * This class represents a piece of memory managed by Flink. + * + *

The memory can be on-heap, off-heap direct or off-heap unsafe, * this is transparently handled by this class. * *

This class specializes byte access and byte copy calls for heap memory, while reusing the @@ -44,28 +48,34 @@ */ @Internal public final class HybridMemorySegment extends MemorySegment { - /** - * The direct byte buffer that allocated the off-heap memory. This memory segment holds a + * The direct byte buffer that wraps the off-heap memory. This memory segment holds a * reference to that buffer, so as long as this memory segment lives, the memory will not be * released. */ + @Nullable private final ByteBuffer offHeapBuffer; + /** The cleaner is called to free the underlying native memory. */ + @Nullable + private final Runnable cleaner; + /** - * Creates a new memory segment that represents the memory backing the given direct byte buffer. - * Note that the given ByteBuffer must be direct {@link java.nio.ByteBuffer#allocateDirect(int)}, - * otherwise this method with throw an IllegalArgumentException. - * - *

The memory segment references the given owner. - * - * @param buffer The byte buffer whose memory is represented by this memory segment. - * @param owner The owner references by this memory segment. - * @throws IllegalArgumentException Thrown, if the given ByteBuffer is not direct. - */ - HybridMemorySegment(ByteBuffer buffer, Object owner) { + * Creates a new memory segment that represents the memory backing the given direct byte buffer. + * Note that the given ByteBuffer must be direct {@link java.nio.ByteBuffer#allocateDirect(int)}, + * otherwise this method with throw an IllegalArgumentException. + * + *

The memory segment references the given owner. + * + * @param buffer The byte buffer whose memory is represented by this memory segment. + * @param owner The owner references by this memory segment. + * @param cleaner optional action to run upon freeing the segment. + * @throws IllegalArgumentException Thrown, if the given ByteBuffer is not direct. + */ + HybridMemorySegment(ByteBuffer buffer, @Nullable Object owner, @Nullable Runnable cleaner) { super(checkBufferAndGetAddress(buffer), buffer.capacity(), owner); this.offHeapBuffer = buffer; + this.cleaner = cleaner; } /** @@ -79,6 +89,7 @@ public final class HybridMemorySegment extends MemorySegment { HybridMemorySegment(byte[] buffer, Object owner) { super(buffer, owner); this.offHeapBuffer = null; + this.cleaner = null; } // ------------------------------------------------------------------------- @@ -121,6 +132,14 @@ public ByteBuffer wrap(int offset, int length) { } } + @Override + public void free() { + super.free(); + if (cleaner != null) { + cleaner.run(); + } + } + // ------------------------------------------------------------------------ // Random Access get() and put() methods // ------------------------------------------------------------------------ diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java index 53865f43dc97..c297a2648047 100644 --- a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java +++ b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java @@ -95,7 +95,24 @@ public static MemorySegment allocateUnpooledOffHeapMemory(int size) { */ public static MemorySegment allocateUnpooledOffHeapMemory(int size, Object owner) { ByteBuffer memory = ByteBuffer.allocateDirect(size); - return new HybridMemorySegment(memory, owner); + return new HybridMemorySegment(memory, owner, null); + } + + /** + * Allocates an off-heap unsafe memory and creates a new memory segment to represent that memory. + * + *

Creation of this segment schedules its memory freeing operation when its java wrapping object is about + * to be garbage collected, similar to {@link java.nio.DirectByteBuffer#DirectByteBuffer(int)}. + * The difference is that this memory allocation is out of option -XX:MaxDirectMemorySize limitation. + * + * @param size The size of the off-heap unsafe memory segment to allocate. + * @param owner The owner to associate with the off-heap unsafe memory segment. + * @return A new memory segment, backed by off-heap unsafe memory. + */ + public static MemorySegment allocateOffHeapUnsafeMemory(int size, Object owner) { + long address = MemoryUtils.allocateUnsafe(size); + ByteBuffer offHeapBuffer = MemoryUtils.wrapUnsafeMemoryWithByteBuffer(address, size); + return new HybridMemorySegment(offHeapBuffer, owner, MemoryUtils.createMemoryGcCleaner(offHeapBuffer, address)); } /** @@ -109,7 +126,7 @@ public static MemorySegment allocateUnpooledOffHeapMemory(int size, Object owner * @return A new memory segment representing the given off-heap memory. */ public static MemorySegment wrapOffHeapMemory(ByteBuffer memory) { - return new HybridMemorySegment(memory, null); + return new HybridMemorySegment(memory, null, null); } } diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java b/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java index 1cb507ed823d..76a295d8ef2d 100644 --- a/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java +++ b/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java @@ -19,8 +19,11 @@ package org.apache.flink.core.memory; import org.apache.flink.annotation.Internal; +import org.apache.flink.util.ExceptionUtils; +import java.lang.reflect.Constructor; import java.lang.reflect.Field; +import java.nio.ByteBuffer; import java.nio.ByteOrder; /** @@ -36,6 +39,8 @@ public class MemoryUtils { /** The native byte order of the platform on which the system currently runs. */ public static final ByteOrder NATIVE_BYTE_ORDER = ByteOrder.nativeOrder(); + private static final Constructor DIRECT_BUFFER_CONSTRUCTOR = getDirectBufferPrivateConstructor(); + @SuppressWarnings("restriction") private static sun.misc.Unsafe getUnsafe() { try { @@ -57,4 +62,80 @@ private static sun.misc.Unsafe getUnsafe() { /** Should not be instantiated. */ private MemoryUtils() {} + + private static Constructor getDirectBufferPrivateConstructor() { + //noinspection OverlyBroadCatchBlock + try { + Constructor constructor = + ByteBuffer.allocateDirect(1).getClass().getDeclaredConstructor(long.class, int.class); + constructor.setAccessible(true); + return constructor; + } catch (NoSuchMethodException e) { + ExceptionUtils.rethrow( + e, + "The private constructor java.nio.DirectByteBuffer.(long, int) is not available."); + } catch (SecurityException e) { + ExceptionUtils.rethrow( + e, + "The private constructor java.nio.DirectByteBuffer.(long, int) is not available, " + + "permission denied by security manager"); + } catch (Throwable t) { + ExceptionUtils.rethrow( + t, + "Unclassified error while trying to access private constructor " + + "java.nio.DirectByteBuffer.(long, int)."); + } + throw new RuntimeException("unexpected to avoid returning null"); + } + + /** + * Allocates unsafe native memory. + * + * @param size size of the unsafe memory to allocate. + * @return address of the allocated unsafe memory + */ + static long allocateUnsafe(long size) { + return UNSAFE.allocateMemory(Math.max(1L, size)); + } + + /** + * Creates a cleaner to release the unsafe memory by VM GC. + * + *

When memory owner becomes phantom reachable, + * GC will release the underlying unsafe memory if not released yet. + * + * @param owner memory owner which phantom reaching is to monitor by GC and release the unsafe memory + * @param address address of the unsafe memory to release + * @return action to run to release the unsafe memory manually + */ + @SuppressWarnings("UseOfSunClasses") + static Runnable createMemoryGcCleaner(Object owner, long address) { + // The release call is wrapped with the sun.misc.Cleaner + // which will schedule it before GC is run for the owner object (not reachable in user code). + // but only if sun.misc.Cleaner::clean has not been already called explicitly by user before. + // If sun.misc.Cleaner::clean is called after GC it will not call the release. + // This way we guarantee that there will always be a release at some point but only once. + return sun.misc.Cleaner.create(owner, () -> releaseUnsafe(address))::clean; + } + + private static void releaseUnsafe(long address) { + UNSAFE.freeMemory(address); + } + + /** + * Wraps the unsafe native memory with a {@link ByteBuffer}. + * + * @param address address of the unsafe memory to wrap + * @param size size of the unsafe memory to wrap + * @return a {@link ByteBuffer} which is a view of the given unsafe memory + */ + static ByteBuffer wrapUnsafeMemoryWithByteBuffer(long address, int size) { + //noinspection OverlyBroadCatchBlock + try { + return (ByteBuffer) DIRECT_BUFFER_CONSTRUCTOR.newInstance(address, size); + } catch (Throwable t) { + ExceptionUtils.rethrow(t, "Failed to wrap unsafe off-heap memory with ByteBuffer"); + } + throw new RuntimeException("unexpected to avoid returning null"); + } } diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/CrossSegmentTypeTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/CrossSegmentTypeTest.java index 1fc352f2f6ba..51804d02a977 100644 --- a/flink-core/src/test/java/org/apache/flink/core/memory/CrossSegmentTypeTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/memory/CrossSegmentTypeTest.java @@ -158,7 +158,8 @@ private static MemorySegment[] createSegments(int size) { MemorySegment[] segments = { new HeapMemorySegment(new byte[size]), MemorySegmentFactory.allocateUnpooledSegment(size), - MemorySegmentFactory.allocateUnpooledOffHeapMemory(size) + MemorySegmentFactory.allocateUnpooledOffHeapMemory(size), + MemorySegmentFactory.allocateOffHeapUnsafeMemory(size, null) }; return segments; } diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/EndiannessAccessChecks.java b/flink-core/src/test/java/org/apache/flink/core/memory/EndiannessAccessChecks.java index feead4b4b74b..c2db44e7ac60 100644 --- a/flink-core/src/test/java/org/apache/flink/core/memory/EndiannessAccessChecks.java +++ b/flink-core/src/test/java/org/apache/flink/core/memory/EndiannessAccessChecks.java @@ -45,6 +45,11 @@ public void testHybridOffHeapSegment() { testBigAndLittleEndianAccessUnaligned(MemorySegmentFactory.allocateUnpooledOffHeapMemory(11111)); } + @Test + public void testHybridOffHeapUnsafeSegment() { + testBigAndLittleEndianAccessUnaligned(MemorySegmentFactory.allocateOffHeapUnsafeMemory(11111, null)); + } + private void testBigAndLittleEndianAccessUnaligned(MemorySegment segment) { final Random rnd = new Random(); diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapDirectMemorySegmentTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapDirectMemorySegmentTest.java new file mode 100644 index 000000000000..f72dbda1b3ef --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapDirectMemorySegmentTest.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.core.memory; + +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +/** + * Tests for the {@link HybridMemorySegment} in off-heap mode using direct memory. + */ +@RunWith(Parameterized.class) +public class HybridOffHeapDirectMemorySegmentTest extends HybridOffHeapMemorySegmentTest { + + public HybridOffHeapDirectMemorySegmentTest(int pageSize) { + super(pageSize); + } + + @Override + MemorySegment createSegment(int size) { + return MemorySegmentFactory.allocateUnpooledOffHeapMemory(size); + } + + @Override + MemorySegment createSegment(int size, Object owner) { + return MemorySegmentFactory.allocateUnpooledOffHeapMemory(size, owner); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapMemorySegmentTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapMemorySegmentTest.java index b93f798ce8bc..67c05c307ce8 100644 --- a/flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapMemorySegmentTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapMemorySegmentTest.java @@ -19,8 +19,6 @@ package org.apache.flink.core.memory; import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; import java.nio.ByteBuffer; @@ -32,27 +30,16 @@ /** * Tests for the {@link HybridMemorySegment} in off-heap mode. */ -@RunWith(Parameterized.class) -public class HybridOffHeapMemorySegmentTest extends MemorySegmentTestBase { +public abstract class HybridOffHeapMemorySegmentTest extends MemorySegmentTestBase { - public HybridOffHeapMemorySegmentTest(int pageSize) { + HybridOffHeapMemorySegmentTest(int pageSize) { super(pageSize); } - @Override - MemorySegment createSegment(int size) { - return MemorySegmentFactory.allocateUnpooledOffHeapMemory(size); - } - - @Override - MemorySegment createSegment(int size, Object owner) { - return MemorySegmentFactory.allocateUnpooledOffHeapMemory(size, owner); - } - @Test public void testHybridHeapSegmentSpecifics() { - final ByteBuffer buffer = ByteBuffer.allocateDirect(411); - HybridMemorySegment seg = new HybridMemorySegment(buffer, null); + HybridMemorySegment seg = (HybridMemorySegment) createSegment(411); + ByteBuffer buffer = seg.getOffHeapBuffer(); assertFalse(seg.isFreed()); assertTrue(seg.isOffHeap()); diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapUnsafeMemorySegmentTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapUnsafeMemorySegmentTest.java new file mode 100644 index 000000000000..f16720351723 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapUnsafeMemorySegmentTest.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.core.memory; + +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +/** + * Tests for the {@link HybridMemorySegment} in off-heap mode using unsafe memory. + */ +@RunWith(Parameterized.class) +public class HybridOffHeapUnsafeMemorySegmentTest extends HybridOffHeapMemorySegmentTest { + + public HybridOffHeapUnsafeMemorySegmentTest(int pageSize) { + super(pageSize); + } + + @Override + MemorySegment createSegment(int size) { + return MemorySegmentFactory.allocateOffHeapUnsafeMemory(size, null); + } + + @Override + MemorySegment createSegment(int size, Object owner) { + return MemorySegmentFactory.allocateOffHeapUnsafeMemory(size, owner); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentChecksTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentChecksTest.java index 09619cdd2ad9..3e3e267a44c8 100644 --- a/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentChecksTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentChecksTest.java @@ -46,12 +46,12 @@ public void testHybridHeapNullBuffer2() { @Test(expected = NullPointerException.class) public void testHybridOffHeapNullBuffer2() { - new HybridMemorySegment((ByteBuffer) null, new Object()); + new HybridMemorySegment(null, new Object(), () -> {}); } @Test(expected = IllegalArgumentException.class) public void testHybridNonDirectBuffer() { - new HybridMemorySegment(ByteBuffer.allocate(1024), new Object()); + new HybridMemorySegment(ByteBuffer.allocate(1024), new Object(), () -> {}); } @Test(expected = IllegalArgumentException.class) diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentTestBase.java b/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentTestBase.java index 0b8f1d090c20..b618cc2fccec 100644 --- a/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentTestBase.java +++ b/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentTestBase.java @@ -326,6 +326,10 @@ public void testEqualTo() { MemorySegment seg1 = createSegment(pageSize); MemorySegment seg2 = createSegment(pageSize); + byte[] referenceArray = new byte[pageSize]; + seg1.put(0, referenceArray); + seg2.put(0, referenceArray); + int i = new Random().nextInt(pageSize - 8); seg1.put(i, (byte) 10); @@ -1665,6 +1669,7 @@ public void testBulkByteAccess() { { final MemorySegment segment = createSegment(pageSize); byte[] expected = new byte[pageSize]; + segment.put(0, expected, 0, pageSize); for (int i = 0; i < 200; i++) { int numBytes = random.nextInt(pageSize - 10) + 1; diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentUndersizedTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentUndersizedTest.java index efece684f16c..1363703e4dd6 100644 --- a/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentUndersizedTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentUndersizedTest.java @@ -61,6 +61,14 @@ public void testZeroSizeOffHeapHybridSegment() { testSegmentWithSizeLargerZero(segment); } + @Test + public void testZeroSizeOffHeapUnsafeHybridSegment() { + MemorySegment segment = MemorySegmentFactory.allocateOffHeapUnsafeMemory(0, null); + + testZeroSizeBuffer(segment); + testSegmentWithSizeLargerZero(segment); + } + @Test public void testSizeOneHeapSegment() { testSegmentWithSizeLargerZero(new HeapMemorySegment(new byte[1])); @@ -76,6 +84,11 @@ public void testSizeOneOffHeapHybridSegment() { testSegmentWithSizeLargerZero(MemorySegmentFactory.allocateUnpooledOffHeapMemory(1)); } + @Test + public void testSizeOneOffHeapUnsafeHybridSegment() { + testSegmentWithSizeLargerZero(MemorySegmentFactory.allocateOffHeapUnsafeMemory(1, null)); + } + private static void testZeroSizeBuffer(MemorySegment segment) { // ------ bytes ------ diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/OperationsOnFreedSegmentTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/OperationsOnFreedSegmentTest.java index 8e7c83b95b98..bf27fc1ce0fb 100644 --- a/flink-core/src/test/java/org/apache/flink/core/memory/OperationsOnFreedSegmentTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/memory/OperationsOnFreedSegmentTest.java @@ -128,8 +128,9 @@ private static MemorySegment[] createTestSegments() { MemorySegment heap = new HeapMemorySegment(new byte[PAGE_SIZE]); MemorySegment hybridHeap = MemorySegmentFactory.wrap(new byte[PAGE_SIZE]); MemorySegment hybridOffHeap = MemorySegmentFactory.allocateUnpooledOffHeapMemory(PAGE_SIZE); + MemorySegment hybridOffHeapUnsafe = MemorySegmentFactory.allocateOffHeapUnsafeMemory(PAGE_SIZE, null); - MemorySegment[] segments = { heap, hybridHeap, hybridOffHeap }; + MemorySegment[] segments = { heap, hybridHeap, hybridOffHeap, hybridOffHeapUnsafe }; return segments; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java index 472e57104678..a6bc31135eaf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java @@ -48,7 +48,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicReference; -import static org.apache.flink.core.memory.MemorySegmentFactory.allocateUnpooledOffHeapMemory; +import static org.apache.flink.core.memory.MemorySegmentFactory.allocateOffHeapUnsafeMemory; import static org.apache.flink.core.memory.MemorySegmentFactory.allocateUnpooledSegment; /** @@ -601,7 +601,7 @@ private MemorySegment allocateManagedSegment(MemoryType memoryType, Object owner case HEAP: return allocateUnpooledSegment(getPageSize(), owner); case OFF_HEAP: - return allocateUnpooledOffHeapMemory(getPageSize(), owner); + return allocateOffHeapUnsafeMemory(getPageSize(), owner); default: throw new IllegalArgumentException("unrecognized memory type: " + memoryType); } From 6d0a82729f56a3944f4d68d95dd512f29688a01c Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Tue, 22 Oct 2019 14:59:24 +0200 Subject: [PATCH 188/746] [hotfix] KeyedBudgetManagerTest extends TestLogger --- .../org/apache/flink/runtime/util/KeyedBudgetManagerTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyedBudgetManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyedBudgetManagerTest.java index bb420367c9d9..0d431bdfe8eb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyedBudgetManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyedBudgetManagerTest.java @@ -21,6 +21,7 @@ import org.apache.flink.runtime.util.KeyedBudgetManager.AcquisitionResult; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.TestLogger; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -45,7 +46,7 @@ * Test suite for {@link KeyedBudgetManager}. */ @SuppressWarnings("MagicNumber") -public class KeyedBudgetManagerTest { +public class KeyedBudgetManagerTest extends TestLogger { private static final String[] TEST_KEYS = {"k1", "k2", "k3", "k4"}; private static final long[] TEST_BUDGETS = {15, 17, 22, 11}; private static final Executor NEW_THREAD_EXECUTOR = r -> new Thread(r).start(); From 5135cea708939bacd98539d669d274af91f996e6 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Wed, 23 Oct 2019 10:54:20 +0800 Subject: [PATCH 189/746] [FLINK-13539][table-api] Improve CSV table factory doesn't need to require format.fields This closes #9421 --- docs/dev/table/connect.md | 6 +- .../flink/table/descriptors/OldCsv.java | 25 +++- .../table/descriptors/OldCsvValidator.java | 17 ++- .../table/sinks/CsvTableSinkFactoryBase.java | 16 ++- .../sources/CsvTableSourceFactoryBase.java | 23 +-- .../factories/CsvTableSinkFactoryTest.java | 135 ++++++++++++++++++ 6 files changed, 204 insertions(+), 18 deletions(-) create mode 100644 flink-table/flink-table-api-java-bridge/src/test/java/org/apache/flink/table/factories/CsvTableSinkFactoryTest.java diff --git a/docs/dev/table/connect.md b/docs/dev/table/connect.md index 520662026e2a..e760059cd507 100644 --- a/docs/dev/table/connect.md +++ b/docs/dev/table/connect.md @@ -1857,6 +1857,7 @@ Use the old one for stream/batch filesystem operations for now. new OldCsv() .field("field1", Types.STRING) // required: ordered format fields .field("field2", Types.TIMESTAMP) + .deriveSchema() // or use the table's schema .fieldDelimiter(",") // optional: string delimiter "," by default .lineDelimiter("\n") // optional: string delimiter "\n" by default .quoteCharacter('"') // optional: single character for string values, empty by default @@ -1892,6 +1893,7 @@ format: type: VARCHAR - name: field2 type: TIMESTAMP + derive-schema: true # or use the table's schema field-delimiter: "," # optional: string delimiter "," by default line-delimiter: "\n" # optional: string delimiter "\n" by default quote-character: '"' # optional: single character for string values, empty by default @@ -1912,7 +1914,9 @@ CREATE TABLE MyUserTable ( 'format.fields.0.type' = 'FLOAT', 'format.fields.1.name' = 'rideTime', 'format.fields.1.type' = 'TIMESTAMP', - + + 'format.derive-schema' = 'true', -- or use the table's schema' + 'format.field-delimiter' = ',', -- optional: string delimiter "," by default 'format.line-delimiter' = '\n', -- optional: string delimiter "\n" by default 'format.quote-character' = '"', -- optional: single character for string values, empty by default diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/descriptors/OldCsv.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/descriptors/OldCsv.java index f7cdc9f6d5d5..378d466110c7 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/descriptors/OldCsv.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/descriptors/OldCsv.java @@ -31,6 +31,7 @@ import java.util.Optional; import java.util.stream.Collectors; +import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_DERIVE_SCHEMA; import static org.apache.flink.table.descriptors.OldCsvValidator.FORMAT_COMMENT_PREFIX; import static org.apache.flink.table.descriptors.OldCsvValidator.FORMAT_FIELDS; import static org.apache.flink.table.descriptors.OldCsvValidator.FORMAT_FIELD_DELIMITER; @@ -62,6 +63,7 @@ public class OldCsv extends FormatDescriptor { private Optional commentPrefix = Optional.empty(); private Optional isIgnoreFirstLine = Optional.empty(); private Optional lenient = Optional.empty(); + private Optional deriveSchema = Optional.empty(); public OldCsv() { super(FORMAT_TYPE_VALUE, 1); @@ -168,6 +170,19 @@ public OldCsv ignoreParseErrors() { return this; } + /** + * Derives the format schema from the table's schema. Required if no format schema is defined. + * + *

This allows for defining schema information only once. + * + *

The names, types, and fields' order of the format are determined by the table's + * schema. + */ + public OldCsv deriveSchema() { + this.deriveSchema = Optional.of(true); + return this; + } + @Override protected Map toFormatProperties() { DescriptorProperties properties = new DescriptorProperties(); @@ -175,15 +190,19 @@ protected Map toFormatProperties() { fieldDelim.ifPresent(s -> properties.putString(FORMAT_FIELD_DELIMITER, s)); lineDelim.ifPresent(s -> properties.putString(FORMAT_LINE_DELIMITER, s)); - List subKeys = Arrays.asList( + if (deriveSchema.isPresent() && deriveSchema.get()) { + properties.putBoolean(FORMAT_DERIVE_SCHEMA, true); + } else { + List subKeys = Arrays.asList( DescriptorProperties.TABLE_SCHEMA_NAME, DescriptorProperties.TABLE_SCHEMA_TYPE); - List> subValues = schema.entrySet().stream() + List> subValues = schema.entrySet().stream() .map(e -> Arrays.asList(e.getKey(), e.getValue())) .collect(Collectors.toList()); - properties.putIndexedFixedProperties(FORMAT_FIELDS, subKeys, subValues); + properties.putIndexedFixedProperties(FORMAT_FIELDS, subKeys, subValues); + } quoteCharacter.ifPresent(character -> properties.putCharacter(FORMAT_QUOTE_CHARACTER, character)); commentPrefix.ifPresent(s -> properties.putString(FORMAT_COMMENT_PREFIX, s)); diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/descriptors/OldCsvValidator.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/descriptors/OldCsvValidator.java index 1264494f9b11..9f8476db96cb 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/descriptors/OldCsvValidator.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/descriptors/OldCsvValidator.java @@ -19,6 +19,7 @@ package org.apache.flink.table.descriptors; import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.ValidationException; /** * Validator for {@link OldCsv}. @@ -49,6 +50,20 @@ public void validate(DescriptorProperties properties) { properties.validateString(FORMAT_COMMENT_PREFIX, true, 1); properties.validateBoolean(FORMAT_IGNORE_FIRST_LINE, true); properties.validateBoolean(FORMAT_IGNORE_PARSE_ERRORS, true); - properties.validateTableSchema(FORMAT_FIELDS, false); + properties.validateBoolean(FormatDescriptorValidator.FORMAT_DERIVE_SCHEMA, true); + + final boolean hasSchema = properties.hasPrefix(FORMAT_FIELDS); + final boolean isDerived = properties + .getOptionalBoolean(FormatDescriptorValidator.FORMAT_DERIVE_SCHEMA) + .orElse(false); + if (isDerived && hasSchema) { + throw new ValidationException( + "Format cannot define a schema and derive from the table's schema at the same time."); + } else if (hasSchema) { + properties.validateTableSchema(FORMAT_FIELDS, false); + } else if (!isDerived) { + throw new ValidationException( + "A definition of a schema or derivation from the table's schema is required."); + } } } diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSinkFactoryBase.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSinkFactoryBase.java index 9c83b4a3897e..735136bf5ecf 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSinkFactoryBase.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSinkFactoryBase.java @@ -23,6 +23,7 @@ import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.descriptors.DescriptorProperties; import org.apache.flink.table.descriptors.FileSystemValidator; +import org.apache.flink.table.descriptors.FormatDescriptorValidator; import org.apache.flink.table.descriptors.OldCsvValidator; import org.apache.flink.table.descriptors.SchemaValidator; import org.apache.flink.table.factories.TableFactory; @@ -66,6 +67,7 @@ public List supportedProperties() { // format properties.add(FORMAT_FIELDS + ".#." + DescriptorProperties.TABLE_SCHEMA_TYPE); properties.add(FORMAT_FIELDS + ".#." + DescriptorProperties.TABLE_SCHEMA_NAME); + properties.add(FormatDescriptorValidator.FORMAT_DERIVE_SCHEMA); properties.add(FORMAT_FIELD_DELIMITER); properties.add(CONNECTOR_PATH); // schema @@ -87,12 +89,16 @@ protected CsvTableSink createTableSink( new SchemaValidator(isStreaming, false, false).validate(params); // build - TableSchema formatSchema = params.getTableSchema(FORMAT_FIELDS); TableSchema tableSchema = params.getTableSchema(SCHEMA); - - if (!formatSchema.equals(tableSchema)) { - throw new TableException( + boolean isDerived = params + .getOptionalBoolean(FormatDescriptorValidator.FORMAT_DERIVE_SCHEMA) + .orElse(false); + if (!isDerived) { + TableSchema formatSchema = params.getTableSchema(FORMAT_FIELDS); + if (!formatSchema.equals(tableSchema)) { + throw new TableException( "Encodings that differ from the schema are not supported yet for CsvTableSink."); + } } String path = params.getString(CONNECTOR_PATH); @@ -100,7 +106,7 @@ protected CsvTableSink createTableSink( CsvTableSink csvTableSink = new CsvTableSink(path, fieldDelimiter); - return (CsvTableSink) csvTableSink.configure(formatSchema.getFieldNames(), formatSchema.getFieldTypes()); + return (CsvTableSink) csvTableSink.configure(tableSchema.getFieldNames(), tableSchema.getFieldTypes()); } } diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvTableSourceFactoryBase.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvTableSourceFactoryBase.java index 4dc8e4034012..e1cf4051a565 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvTableSourceFactoryBase.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sources/CsvTableSourceFactoryBase.java @@ -23,6 +23,7 @@ import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.descriptors.DescriptorProperties; import org.apache.flink.table.descriptors.FileSystemValidator; +import org.apache.flink.table.descriptors.FormatDescriptorValidator; import org.apache.flink.table.descriptors.OldCsvValidator; import org.apache.flink.table.descriptors.SchemaValidator; import org.apache.flink.table.factories.TableFactory; @@ -71,6 +72,7 @@ public List supportedProperties() { // format properties.add(FORMAT_FIELDS + ".#." + DescriptorProperties.TABLE_SCHEMA_TYPE); properties.add(FORMAT_FIELDS + ".#." + DescriptorProperties.TABLE_SCHEMA_NAME); + properties.add(FormatDescriptorValidator.FORMAT_DERIVE_SCHEMA); properties.add(FORMAT_FIELD_DELIMITER); properties.add(FORMAT_LINE_DELIMITER); properties.add(FORMAT_QUOTE_CHARACTER); @@ -99,22 +101,27 @@ protected CsvTableSource createTableSource( // build CsvTableSource.Builder csvTableSourceBuilder = new CsvTableSource.Builder(); - TableSchema formatSchema = params.getTableSchema(FORMAT_FIELDS); TableSchema tableSchema = params.getTableSchema(SCHEMA); - - // the CsvTableSource needs some rework first - // for now the schema must be equal to the encoding - if (!formatSchema.equals(tableSchema)) { - throw new TableException( + boolean isDerived = params + .getOptionalBoolean(FormatDescriptorValidator.FORMAT_DERIVE_SCHEMA) + .orElse(false); + + if (!isDerived) { + TableSchema formatSchema = params.getTableSchema(FORMAT_FIELDS); + // the CsvTableSource needs some rework first + // for now the schema must be equal to the encoding + if (!formatSchema.equals(tableSchema)) { + throw new TableException( "Encodings that differ from the schema are not supported yet for CsvTableSources."); + } } params.getOptionalString(CONNECTOR_PATH).ifPresent(csvTableSourceBuilder::path); params.getOptionalString(FORMAT_FIELD_DELIMITER).ifPresent(csvTableSourceBuilder::fieldDelimiter); params.getOptionalString(FORMAT_LINE_DELIMITER).ifPresent(csvTableSourceBuilder::lineDelimiter); - for (int i = 0; i < formatSchema.getFieldCount(); ++i) { - csvTableSourceBuilder.field(formatSchema.getFieldNames()[i], formatSchema.getFieldTypes()[i]); + for (int i = 0; i < tableSchema.getFieldCount(); ++i) { + csvTableSourceBuilder.field(tableSchema.getFieldNames()[i], tableSchema.getFieldTypes()[i]); } params.getOptionalCharacter(FORMAT_QUOTE_CHARACTER).ifPresent(csvTableSourceBuilder::quoteCharacter); params.getOptionalString(FORMAT_COMMENT_PREFIX).ifPresent(csvTableSourceBuilder::commentPrefix); diff --git a/flink-table/flink-table-api-java-bridge/src/test/java/org/apache/flink/table/factories/CsvTableSinkFactoryTest.java b/flink-table/flink-table-api-java-bridge/src/test/java/org/apache/flink/table/factories/CsvTableSinkFactoryTest.java new file mode 100644 index 000000000000..aa1a9e50d87e --- /dev/null +++ b/flink-table/flink-table-api-java-bridge/src/test/java/org/apache/flink/table/factories/CsvTableSinkFactoryTest.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.factories; + +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.descriptors.DescriptorProperties; +import org.apache.flink.table.sinks.CsvTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.table.sources.CsvTableSource; +import org.apache.flink.table.sources.TableSource; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.HashMap; +import java.util.Map; + +import static org.apache.flink.table.descriptors.OldCsvValidator.FORMAT_FIELDS; +import static org.apache.flink.table.descriptors.Schema.SCHEMA; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * Tests for CsvTableSourceFactory and CsvTableSinkFactory. + */ +@RunWith(Parameterized.class) +public class CsvTableSinkFactoryTest { + + private static TableSchema testingSchema = TableSchema.builder() + .field("myfield", DataTypes.STRING()) + .field("myfield2", DataTypes.INT()) + .field("myfield3", DataTypes.MAP(DataTypes.STRING(), DataTypes.INT())) + .field("myfield4", DataTypes.ROW(DataTypes.FIELD("nested_f1", DataTypes.BIGINT()))) + .field("myfield5", Types.PRIMITIVE_ARRAY(Types.SHORT)) + .build(); + + @Parameterized.Parameter + public boolean deriveSchema; + + @Parameterized.Parameters(name = "deriveSchema = {0}") + public static Boolean[] getDeriveSchema() { + return new Boolean[]{true, false}; + } + + @Test + public void testAppendTableSinkFactory() { + DescriptorProperties descriptor = createDescriptor(testingSchema); + descriptor.putString("update-mode", "append"); + TableSink sink = createTableSink(descriptor); + + assertTrue(sink instanceof CsvTableSink); + assertEquals(testingSchema.toRowDataType(), sink.getConsumedDataType()); + } + + @Test + public void testBatchTableSinkFactory() { + DescriptorProperties descriptor = createDescriptor(testingSchema); + TableSink sink = createTableSink(descriptor); + + assertTrue(sink instanceof CsvTableSink); + assertEquals(testingSchema.toRowDataType(), sink.getConsumedDataType()); + } + + @Test + public void testAppendTableSourceFactory() { + DescriptorProperties descriptor = createDescriptor(testingSchema); + descriptor.putString("update-mode", "append"); + TableSource sink = createTableSource(descriptor); + + assertTrue(sink instanceof CsvTableSource); + assertEquals(testingSchema.toRowDataType(), sink.getProducedDataType()); + } + + @Test + public void testBatchTableSourceFactory() { + DescriptorProperties descriptor = createDescriptor(testingSchema); + TableSource sink = createTableSource(descriptor); + + assertTrue(sink instanceof CsvTableSource); + assertEquals(testingSchema.toRowDataType(), sink.getProducedDataType()); + } + + private DescriptorProperties createDescriptor(TableSchema schema) { + Map properties = new HashMap<>(); + properties.put("connector.type", "filesystem"); + properties.put("connector.property-version", "1"); + properties.put("connector.path", "/path/to/csv"); + + // schema + properties.put("format.type", "csv"); + properties.put("format.property-version", "1"); + properties.put("format.field-delimiter", ";"); + + DescriptorProperties descriptor = new DescriptorProperties(true); + descriptor.putProperties(properties); + descriptor.putTableSchema(SCHEMA, schema); + if (deriveSchema) { + descriptor.putBoolean("format.derive-schema", deriveSchema); + } else { + descriptor.putTableSchema(FORMAT_FIELDS, testingSchema); + } + return descriptor; + } + + private static TableSource createTableSource(DescriptorProperties descriptor) { + return TableFactoryService + .find(TableSourceFactory.class, descriptor.asMap()) + .createTableSource(descriptor.asMap()); + } + + private static TableSink createTableSink(DescriptorProperties descriptor) { + return TableFactoryService + .find(TableSinkFactory.class, descriptor.asMap()) + .createTableSink(descriptor.asMap()); + } +} From ae043e0279b034229c57587a38c2321710ee8816 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Mon, 14 Oct 2019 15:31:37 +0800 Subject: [PATCH 190/746] [FLINK-14202][table][python] Optimize the execution plan for Python Calc when there is a condition Optimizes the Python UDF execution plan when there is a condition which contains Python UDFs by calculating the condition firstly and then check whether to call the other UDFs. This can eliminate the Python UDF execution for the input rows which could be filtered by the condition. Brief changes: - Refactor PythonScalarFunctionSplitRule by splitting it into multi rules to make it more understandable. - Handle cases where condition is not null in the SplitRule. This closes #9907. --- flink-python/pyflink/table/tests/test_udf.py | 10 +- .../plan/nodes/common/CommonPythonCalc.scala | 95 +----- .../physical/batch/BatchExecPythonCalc.scala | 13 +- .../stream/StreamExecPythonCalc.scala | 21 +- .../plan/rules/FlinkBatchRuleSets.scala | 5 +- .../plan/rules/FlinkStreamRuleSets.scala | 5 +- .../rules/logical/PythonCalcSplitRule.scala | 322 ++++++++++++++++++ .../PythonScalarFunctionSplitRule.scala | 210 ------------ .../rules/logical/PythonCalcSplitRuleTest.xml | 235 +++++++++++++ .../PythonScalarFunctionSplitRuleTest.xml | 164 --------- ...st.scala => PythonCalcSplitRuleTest.scala} | 16 +- .../table/plan/nodes/CommonPythonCalc.scala | 3 +- .../datastream/DataStreamPythonCalc.scala | 54 +-- .../table/plan/rules/FlinkRuleSets.scala | 5 +- .../rules/logical/PythonCalcSplitRule.scala | 322 ++++++++++++++++++ .../PythonScalarFunctionSplitRule.scala | 211 ------------ .../plan/ExpressionReductionRulesTest.scala | 11 +- ...st.scala => PythonCalcSplitRuleTest.scala} | 78 ++++- 18 files changed, 1006 insertions(+), 774 deletions(-) create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRule.scala delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PythonScalarFunctionSplitRule.scala create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRuleTest.xml delete mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonScalarFunctionSplitRuleTest.xml rename flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/{PythonScalarFunctionSplitRuleTest.scala => PythonCalcSplitRuleTest.scala} (91%) create mode 100644 flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/PythonCalcSplitRule.scala delete mode 100644 flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/PythonScalarFunctionSplitRule.scala rename flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/{PythonScalarFunctionSplitRuleTest.scala => PythonCalcSplitRuleTest.scala} (79%) diff --git a/flink-python/pyflink/table/tests/test_udf.py b/flink-python/pyflink/table/tests/test_udf.py index d529681f2e91..12f4e8c96642 100644 --- a/flink-python/pyflink/table/tests/test_udf.py +++ b/flink-python/pyflink/table/tests/test_udf.py @@ -71,15 +71,17 @@ def test_chaining_scalar_function(self): "subtract_one", udf(SubtractOne(), DataTypes.BIGINT(), DataTypes.BIGINT())) self.t_env.register_function("add", add) - table_sink = source_sink_utils.TestAppendSink(['a'], [DataTypes.BIGINT()]) + table_sink = source_sink_utils.TestAppendSink( + ['a', 'b', 'c'], + [DataTypes.BIGINT(), DataTypes.BIGINT(), DataTypes.INT()]) 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))") \ + t = self.t_env.from_elements([(1, 2, 1), (2, 5, 2), (3, 1, 3)], ['a', 'b', 'c']) + t.select("add(add_one(a), subtract_one(b)), c, 1") \ .insert_into("Results") self.t_env.execute("test") actual = source_sink_utils.results() - self.assert_equals(actual, ["3", "7", "4"]) + self.assert_equals(actual, ["3,1,1", "7,2,1", "4,3,1"]) def test_udf_in_join_condition(self): t1 = self.t_env.from_elements([(2, "Hi")], ['a', 'b']) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonCalc.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonCalc.scala index 4ca9d8f76419..a29ca11ebfb8 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonCalc.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonCalc.scala @@ -18,19 +18,15 @@ package org.apache.flink.table.planner.plan.nodes.common -import org.apache.calcite.plan.RelOptCluster -import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rex._ import org.apache.calcite.sql.`type`.SqlTypeName import org.apache.flink.api.dag.Transformation import org.apache.flink.streaming.api.operators.OneInputStreamOperator import org.apache.flink.streaming.api.transformations.OneInputTransformation -import org.apache.flink.table.api.TableConfig import org.apache.flink.table.dataformat.BaseRow import org.apache.flink.table.functions.FunctionLanguage import org.apache.flink.table.functions.python.{PythonFunction, PythonFunctionInfo, SimplePythonFunction} import org.apache.flink.table.planner.calcite.FlinkTypeFactory -import org.apache.flink.table.planner.codegen.{CalcCodeGenerator, CodeGeneratorContext} import org.apache.flink.table.planner.functions.utils.ScalarSqlFunction import org.apache.flink.table.planner.plan.nodes.common.CommonPythonCalc.PYTHON_SCALAR_FUNCTION_OPERATOR_NAME import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo @@ -54,8 +50,7 @@ trait CommonPythonCalc { val udfInputOffsets = inputNodes.toArray .map(_._1) - .filter(_.isInstanceOf[RexInputRef]) - .map(_.asInstanceOf[RexInputRef].getIndex) + .collect { case inputRef: RexInputRef => inputRef.getIndex } (udfInputOffsets, pythonFunctionInfos) } @@ -116,35 +111,20 @@ trait CommonPythonCalc { .asInstanceOf[OneInputStreamOperator[BaseRow, BaseRow]] } - private def createPythonOneInputTransformation( + def createPythonOneInputTransformation( inputTransform: Transformation[BaseRow], calcProgram: RexProgram, - name: String) = { + name: String): OneInputTransformation[BaseRow, BaseRow] = { val pythonRexCalls = calcProgram.getProjectList .map(calcProgram.expandLocalRef) - .filter(_.isInstanceOf[RexCall]) - .map(_.asInstanceOf[RexCall]) + .collect { case call: RexCall => call } .toArray val forwardedFields: Array[Int] = calcProgram.getProjectList .map(calcProgram.expandLocalRef) - .filter(_.isInstanceOf[RexInputRef]) - .map(_.asInstanceOf[RexInputRef].getIndex) + .collect { case inputRef: RexInputRef => inputRef.getIndex } .toArray - val resultProjectList = { - 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 - } - } - val (pythonUdfInputOffsets, pythonFunctionInfos) = extractPythonScalarFunctionInfos(pythonRexCalls) @@ -162,76 +142,13 @@ trait CommonPythonCalc { pythonFunctionInfos, forwardedFields) - val pythonInputTransform = new OneInputTransformation( + new OneInputTransformation( inputTransform, name, pythonOperator, pythonOperatorResultTyeInfo, inputTransform.getParallelism ) - (pythonInputTransform, pythonOperatorResultTyeInfo, resultProjectList) - } - - private def createProjectionRexProgram( - inputRowType: RowType, - outputRelData: RelDataType, - projectList: mutable.Buffer[RexNode], - cluster: RelOptCluster) = { - val factory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory] - val inputRelData = factory.createFieldTypeFromLogicalType(inputRowType) - RexProgram.create(inputRelData, projectList, null, outputRelData, cluster.getRexBuilder) - } - - protected def createOneInputTransformation( - inputTransform: Transformation[BaseRow], - inputsContainSingleton: Boolean, - calcProgram: RexProgram, - name: String, - config : TableConfig, - ctx : CodeGeneratorContext, - cluster: RelOptCluster, - rowType: RelDataType, - opName: String): OneInputTransformation[BaseRow, BaseRow] = { - val (pythonInputTransform, pythonOperatorResultTyeInfo, resultProjectList) = - createPythonOneInputTransformation(inputTransform, calcProgram, name) - - if (inputsContainSingleton) { - pythonInputTransform.setParallelism(1) - pythonInputTransform.setMaxParallelism(1) - } - - val onlyFilter = resultProjectList.zipWithIndex.forall { case (rexNode, index) => - rexNode.isInstanceOf[RexInputRef] && rexNode.asInstanceOf[RexInputRef].getIndex == index - } - - if (onlyFilter) { - pythonInputTransform - } else { - // After executing python OneInputTransformation, the order of the output fields - // is Python Call after the forwarding fields, so in the case of sequential changes, - // a calc is needed to adjust the order. - val outputType = FlinkTypeFactory.toLogicalRowType(rowType) - val rexProgram = createProjectionRexProgram( - pythonOperatorResultTyeInfo.toRowType, rowType, resultProjectList, cluster) - val substituteOperator = CalcCodeGenerator.generateCalcOperator( - ctx, - cluster, - pythonInputTransform, - outputType, - config, - rexProgram, - None, - retainHeader = true, - opName - ) - - new OneInputTransformation( - pythonInputTransform, - name, - substituteOperator, - BaseRowTypeInfo.of(outputType), - pythonInputTransform.getParallelism) - } } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecPythonCalc.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecPythonCalc.scala index 7adf0ee9cc9c..2111484c08c9 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecPythonCalc.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecPythonCalc.scala @@ -25,7 +25,6 @@ import org.apache.calcite.rel.core.Calc import org.apache.calcite.rex.RexProgram import org.apache.flink.api.dag.Transformation import org.apache.flink.table.dataformat.BaseRow -import org.apache.flink.table.planner.codegen.CodeGeneratorContext import org.apache.flink.table.planner.delegation.BatchPlanner import org.apache.flink.table.planner.plan.nodes.common.CommonPythonCalc @@ -53,17 +52,9 @@ class BatchExecPythonCalc( override protected def translateToPlanInternal(planner: BatchPlanner): Transformation[BaseRow] = { val inputTransform = getInputNodes.get(0).translateToPlan(planner) .asInstanceOf[Transformation[BaseRow]] - val config = planner.getTableConfig - val ctx = CodeGeneratorContext(config) - createOneInputTransformation( + createPythonOneInputTransformation( inputTransform, - inputsContainSingleton = false, calcProgram, - getRelDetailedDescription, - config, - ctx, - cluster, - getRowType, - "BatchExecCalc") + "BatchExecPythonCalc") } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecPythonCalc.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecPythonCalc.scala index 0bebc5ea3690..4a5a2c9dd321 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecPythonCalc.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecPythonCalc.scala @@ -25,10 +25,8 @@ import org.apache.calcite.rel.core.Calc import org.apache.calcite.rex.RexProgram import org.apache.flink.api.dag.Transformation import org.apache.flink.table.dataformat.BaseRow -import org.apache.flink.table.planner.codegen.CodeGeneratorContext import org.apache.flink.table.planner.delegation.StreamPlanner import org.apache.flink.table.planner.plan.nodes.common.CommonPythonCalc -import org.apache.flink.table.runtime.operators.AbstractProcessStreamOperator /** * Stream physical RelNode for Python ScalarFunctions. @@ -55,18 +53,15 @@ class StreamExecPythonCalc( planner: StreamPlanner): Transformation[BaseRow] = { val inputTransform = getInputNodes.get(0).translateToPlan(planner) .asInstanceOf[Transformation[BaseRow]] - val config = planner.getTableConfig - val ctx = CodeGeneratorContext(config).setOperatorBaseClass( - classOf[AbstractProcessStreamOperator[BaseRow]]) - createOneInputTransformation( + val ret = createPythonOneInputTransformation( inputTransform, - inputsContainSingleton(), calcProgram, - getRelDetailedDescription, - config, - ctx, - cluster, - getRowType, - "StreamExecCalc") + "StreamExecPythonCalc") + + if (inputsContainSingleton()) { + ret.setParallelism(1) + ret.setMaxParallelism(1) + } + ret } } 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 9dd7ef45064e..27191457fb43 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 @@ -354,7 +354,10 @@ object FlinkBatchRuleSets { // merge calc after calc transpose FlinkCalcMergeRule.INSTANCE, // Rule that splits python ScalarFunctions from java/scala ScalarFunctions - PythonScalarFunctionSplitRule.INSTANCE + PythonCalcSplitRule.SPLIT_CONDITION, + PythonCalcSplitRule.SPLIT_PROJECT, + PythonCalcSplitRule.PUSH_CONDITION, + PythonCalcSplitRule.REWRITE_PROJECT ) /** 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 e4431831a028..33c3ee5ae5e5 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 @@ -339,7 +339,10 @@ object FlinkStreamRuleSets { // merge calc after calc transpose FlinkCalcMergeRule.INSTANCE, // Rule that splits python ScalarFunctions from java/scala ScalarFunctions. - PythonScalarFunctionSplitRule.INSTANCE + PythonCalcSplitRule.SPLIT_CONDITION, + PythonCalcSplitRule.SPLIT_PROJECT, + PythonCalcSplitRule.PUSH_CONDITION, + PythonCalcSplitRule.REWRITE_PROJECT ) /** diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRule.scala new file mode 100644 index 000000000000..874952511139 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRule.scala @@ -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.flink.table.planner.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.planner.functions.utils.ScalarSqlFunction +import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCalc +import org.apache.flink.table.planner.plan.utils.PythonUtil.containsFunctionOf +import org.apache.flink.table.planner.plan.utils.{InputRefVisitor, RexDefaultVisitor} + +import scala.collection.JavaConverters._ +import scala.collection.JavaConversions._ +import scala.collection.mutable + +/** + * Base rule that splits [[FlinkLogicalCalc]] into multiple [[FlinkLogicalCalc]]s. + * It is mainly to ensure that each [[FlinkLogicalCalc]] only contains Java/Scala + * [[ScalarFunction]]s or Python [[ScalarFunction]]s. + */ +abstract class PythonCalcSplitRuleBase(description: String) + extends RelOptRule( + operand(classOf[FlinkLogicalCalc], any), + description) { + + 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 extractedFunctionOffset = input.getRowType.getFieldCount + val splitter = new ScalarFunctionSplitter( + extractedFunctionOffset, + extractedRexCalls, + isConvertPythonFunction(program)) + + val (bottomCalcCondition, topCalcCondition, topCalcProjects) = split(program, splitter) + val accessedFields = extractRefInputFields( + topCalcProjects, topCalcCondition, 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, + bottomCalcCondition.orNull, + bottomCalcFieldNames, + rexBuilder)) + + val inputRewriter = new ExtractedFunctionInputRewriter(extractedFunctionOffset, accessedFields) + val topCalc = new FlinkLogicalCalc( + calc.getCluster, + calc.getTraitSet, + bottomCalc, + RexProgram.create( + bottomCalc.getRowType, + topCalcProjects.map(_.accept(inputRewriter)), + topCalcCondition.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) + } + + /** + * Returns true if converting Python functions. + */ + def isConvertPythonFunction(program: RexProgram): Boolean + + /** + * Splits the specified [[RexProgram]] using the specified [[ScalarFunctionSplitter]]. + * It returns a triple of (bottom calc condition, top calc condition, top calc projects) + * as the split result. + */ + def split(program: RexProgram, splitter: ScalarFunctionSplitter) + : (Option[RexNode], Option[RexNode], Seq[RexNode]) +} + +/** + * Rule that splits [[FlinkLogicalCalc]]s which contain Python functions in the condition + * into multiple [[FlinkLogicalCalc]]s. After this rule is applied, there will be no + * Python functions in the condition of the [[FlinkLogicalCalc]]s. + */ +object PythonCalcSplitConditionRule extends PythonCalcSplitRuleBase( + "PythonCalcSplitConditionRule") { + + override def matches(call: RelOptRuleCall): Boolean = { + val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc] + + // matches if it contains Python functions in condition + Option(calc.getProgram.getCondition) + .map(calc.getProgram.expandLocalRef) + .exists(containsFunctionOf(_, FunctionLanguage.PYTHON)) + } + + override def isConvertPythonFunction(program: RexProgram): Boolean = true + + override def split(program: RexProgram, splitter: ScalarFunctionSplitter) + : (Option[RexNode], Option[RexNode], Seq[RexNode]) = { + (None, Option(program.getCondition).map(program.expandLocalRef(_).accept(splitter)), + program.getProjectList.map(program.expandLocalRef)) + } +} + +/** + * Rule that splits [[FlinkLogicalCalc]]s which contain both Java functions and Python functions + * in the projection into multiple [[FlinkLogicalCalc]]s. After this rule is applied, it will + * only contain Python functions or Java functions in the projection of each [[FlinkLogicalCalc]]. + */ +object PythonCalcSplitProjectionRule extends PythonCalcSplitRuleBase( + "PythonCalcSplitProjectionRule") { + + override def matches(call: RelOptRuleCall): Boolean = { + val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc] + val projects = calc.getProgram.getProjectList.map(calc.getProgram.expandLocalRef) + + // matches if it contains both Python functions and Java functions in the projection + projects.exists(containsFunctionOf(_, FunctionLanguage.PYTHON)) && + projects.exists(containsFunctionOf(_, FunctionLanguage.JVM)) + } + + override def isConvertPythonFunction(program: RexProgram): Boolean = { + program.getProjectList + .map(program.expandLocalRef) + .exists(containsFunctionOf(_, FunctionLanguage.JVM, recursive = false)) + } + + override def split(program: RexProgram, splitter: ScalarFunctionSplitter) + : (Option[RexNode], Option[RexNode], Seq[RexNode]) = { + (Option(program.getCondition).map(program.expandLocalRef), None, + program.getProjectList.map(program.expandLocalRef(_).accept(splitter))) + } +} + +/** + * Rule that pushes the condition of [[FlinkLogicalCalc]]s before it for the + * [[FlinkLogicalCalc]]s which contain Python functions in the projection. + */ +object PythonCalcPushConditionRule extends PythonCalcSplitRuleBase( + "PythonCalcPushConditionRule") { + + override def matches(call: RelOptRuleCall): Boolean = { + val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc] + val projects = calc.getProgram.getProjectList.map(calc.getProgram.expandLocalRef) + + // matches if all the following conditions hold true: + // 1) the condition is not null + // 2) it contains Python functions in the projection + calc.getProgram.getCondition != null && + projects.exists(containsFunctionOf(_, FunctionLanguage.PYTHON)) + } + + override def isConvertPythonFunction(program: RexProgram): Boolean = false + + override def split(program: RexProgram, splitter: ScalarFunctionSplitter) + : (Option[RexNode], Option[RexNode], Seq[RexNode]) = { + (Option(program.getCondition).map(program.expandLocalRef), None, + program.getProjectList.map(program.expandLocalRef)) + } +} + +/** + * Rule that ensures that it only contains [[RexInputRef]]s at the beginning of + * the project list and [[RexCall]]s at the end of the project list for [[FlinkLogicalCalc]]s + * which contain Python functions in the projection. This rule exists to keep + * DataStreamPythonCalc as simple as possible and ensures that it only needs to + * handle the Python function execution. + */ +object PythonCalcRewriteProjectionRule extends PythonCalcSplitRuleBase( + "PythonCalcRewriteProjectionRule") { + + override def matches(call: RelOptRuleCall): Boolean = { + val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc] + val projects = calc.getProgram.getProjectList.map(calc.getProgram.expandLocalRef) + + // matches if all the following conditions hold true: + // 1) it contains Python functions in the projection + // 2) it contains RexNodes besides RexInputRef and RexCall or + // not all the RexCalls lying at the end of the project list + projects.exists(containsFunctionOf(_, FunctionLanguage.PYTHON)) && + (projects.exists(expr => !expr.isInstanceOf[RexCall] && !expr.isInstanceOf[RexInputRef]) || + projects.indexWhere(_.isInstanceOf[RexCall]) < + projects.lastIndexWhere(_.isInstanceOf[RexInputRef])) + } + + override def isConvertPythonFunction(program: RexProgram): Boolean = true + + override def split(program: RexProgram, splitter: ScalarFunctionSplitter) + : (Option[RexNode], Option[RexNode], Seq[RexNode]) = { + (None, None, program.getProjectList.map(program.expandLocalRef(_).accept(splitter))) + } +} + +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.scalarFunction.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 PythonCalcSplitRule { + /** + * These rules should be applied sequentially in the order of + * SPLIT_CONDITION, SPLIT_PROJECT, PUSH_CONDITION and REWRITE_PROJECT. + */ + val SPLIT_CONDITION: RelOptRule = PythonCalcSplitConditionRule + val SPLIT_PROJECT: RelOptRule = PythonCalcSplitProjectionRule + val PUSH_CONDITION: RelOptRule = PythonCalcPushConditionRule + val REWRITE_PROJECT: RelOptRule = PythonCalcRewriteProjectionRule +} diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PythonScalarFunctionSplitRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PythonScalarFunctionSplitRule.scala deleted file mode 100644 index fa219ca930eb..000000000000 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PythonScalarFunctionSplitRule.scala +++ /dev/null @@ -1,210 +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.{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, ScalarFunction} -import org.apache.flink.table.planner.functions.utils.ScalarSqlFunction -import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCalc -import org.apache.flink.table.planner.plan.utils.PythonUtil.containsFunctionOf -import org.apache.flink.table.planner.plan.utils.{InputRefVisitor, RexDefaultVisitor} - -import scala.collection.JavaConversions._ -import scala.collection.JavaConverters._ -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.scalarFunction.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-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRuleTest.xml new file mode 100644 index 000000000000..dac9ca4ec1c5 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRuleTest.xml @@ -0,0 +1,235 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 0]]> + + + + (pyFunc2($0, $2), 0)]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) +]]> + + + + (f0, 0)]) + +- FlinkLogicalCalc(select=[a, b, c, pyFunc2(a, c) AS f0]) + +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +]]> + + + + diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonScalarFunctionSplitRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonScalarFunctionSplitRuleTest.xml deleted file mode 100644 index 66a06cd9f546..000000000000 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PythonScalarFunctionSplitRuleTest.xml +++ /dev/null @@ -1,164 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - 0]]> - - - (pyFunc2($0, $2), 0)]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) -]]> - - - (f1, 0)]) -+- FlinkLogicalCalc(select=[c, pyFunc1(a, b) AS f0, pyFunc2(a, c) AS f1]) - +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -]]> - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonScalarFunctionSplitRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRuleTest.scala similarity index 91% rename from flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonScalarFunctionSplitRuleTest.scala rename to flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRuleTest.scala index d99d39ff48b7..d54db3bf6463 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonScalarFunctionSplitRuleTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRuleTest.scala @@ -29,9 +29,9 @@ import org.apache.flink.table.planner.utils.TableTestBase import org.junit.{Before, Test} /** - * Test for [[PythonScalarFunctionSplitRule]]. + * Test for [[PythonCalcSplitRule]]. */ -class PythonScalarFunctionSplitRuleTest extends TableTestBase { +class PythonCalcSplitRuleTest extends TableTestBase { private val util = batchTestUtil() @@ -108,4 +108,16 @@ class PythonScalarFunctionSplitRuleTest extends TableTestBase { val sqlQuery = "SELECT pyFunc1(f1, f2), f0 + 1 FROM MyTable2" util.verifyPlan(sqlQuery) } + + @Test + def testLiteral(): Unit = { + val sqlQuery = "SELECT a, b, pyFunc1(a, c), 1 FROM MyTable" + util.verifyPlan(sqlQuery) + } + + @Test + def testReorderPythonCalc(): Unit = { + val sqlQuery = "SELECT a, pyFunc1(a, c), b FROM MyTable" + util.verifyPlan(sqlQuery) + } } 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 aca06762ed1d..6784c212dacd 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 @@ -41,8 +41,7 @@ trait CommonPythonCalc { val udfInputOffsets = inputNodes.toArray .map(_._1) - .filter(_.isInstanceOf[RexInputRef]) - .map(_.asInstanceOf[RexInputRef].getIndex) + .collect { case inputRef: RexInputRef => inputRef.getIndex } (udfInputOffsets, pythonFunctionInfos) } 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 2a8dce24ef9c..d2511c8e4575 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,20 +21,17 @@ 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.{RexCall, RexInputRef, RexLocalRef, RexNode, RexProgram} +import org.apache.calcite.rex.{RexCall, RexInputRef, 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.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 @@ -75,38 +72,20 @@ class DataStreamPythonCalc( private lazy val pythonRexCalls = calcProgram.getProjectList .map(calcProgram.expandLocalRef) - .filter(_.isInstanceOf[RexCall]) - .map(_.asInstanceOf[RexCall]) + .collect { case call: RexCall => call } .toArray private lazy val forwardedFields: Array[Int] = calcProgram.getProjectList .map(calcProgram.expandLocalRef) - .filter(_.isInstanceOf[RexInputRef]) - .map(_.asInstanceOf[RexInputRef].getIndex) + .collect { case inputRef: RexInputRef => inputRef.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] = { - val config = planner.getConfig - val inputDataStream = getInput.asInstanceOf[DataStreamRel].translateToPlan(planner, queryConfig) @@ -116,7 +95,7 @@ class DataStreamPythonCalc( forwardedFields.map(inputSchema.fieldTypeInfos.get(_)) ++ pythonRexCalls.map(node => FlinkTypeFactory.toTypeInfo(node.getType)): _*) - // Constructs the Python operator + // construct the Python operator val pythonOperatorInputRowType = TypeConversions.fromLegacyInfoToDataType( inputSchema.typeInfo).getLogicalType.asInstanceOf[RowType] val pythonOperatorOutputRowType = TypeConversions.fromLegacyInfoToDataType( @@ -124,36 +103,13 @@ class DataStreamPythonCalc( val pythonOperator = getPythonScalarFunctionOperator( pythonOperatorInputRowType, pythonOperatorOutputRowType, pythonUdfInputOffsets) - val pythonDataStream = inputDataStream + 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( 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 c13ed65ea465..ea3273815740 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 @@ -147,7 +147,10 @@ object FlinkRuleSets { * RuleSet to do rewrite on FlinkLogicalRel */ val LOGICAL_REWRITE_RULES: RuleSet = RuleSets.ofList( - PythonScalarFunctionSplitRule.INSTANCE + PythonCalcSplitRule.SPLIT_CONDITION, + PythonCalcSplitRule.SPLIT_PROJECT, + PythonCalcSplitRule.PUSH_CONDITION, + PythonCalcSplitRule.REWRITE_PROJECT ) /** diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/PythonCalcSplitRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/PythonCalcSplitRule.scala new file mode 100644 index 000000000000..8a55ec131b69 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/PythonCalcSplitRule.scala @@ -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.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 + +/** + * Base rule that splits [[FlinkLogicalCalc]] into multiple [[FlinkLogicalCalc]]s. + * It is mainly to ensure that each [[FlinkLogicalCalc]] only contains Java/Scala + * [[ScalarFunction]]s or Python [[ScalarFunction]]s. + */ +abstract class PythonCalcSplitRuleBase(description: String) + extends RelOptRule( + operand(classOf[FlinkLogicalCalc], any), + description) { + + 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 extractedFunctionOffset = input.getRowType.getFieldCount + val splitter = new ScalarFunctionSplitter( + extractedFunctionOffset, + extractedRexCalls, + isConvertPythonFunction(program)) + + val (bottomCalcCondition, topCalcCondition, topCalcProjects) = split(program, splitter) + val accessedFields = extractRefInputFields( + topCalcProjects, topCalcCondition, 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, + bottomCalcCondition.orNull, + bottomCalcFieldNames, + rexBuilder)) + + val inputRewriter = new ExtractedFunctionInputRewriter(extractedFunctionOffset, accessedFields) + val topCalc = new FlinkLogicalCalc( + calc.getCluster, + calc.getTraitSet, + bottomCalc, + RexProgram.create( + bottomCalc.getRowType, + topCalcProjects.map(_.accept(inputRewriter)), + topCalcCondition.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) + } + + /** + * Returns true if converting Python functions. + */ + def isConvertPythonFunction(program: RexProgram): Boolean + + /** + * Splits the specified [[RexProgram]] using the specified [[ScalarFunctionSplitter]]. + * It returns a triple of (bottom calc condition, top calc condition, top calc projects) + * as the split result. + */ + def split(program: RexProgram, splitter: ScalarFunctionSplitter) + : (Option[RexNode], Option[RexNode], Seq[RexNode]) +} + +/** + * Rule that splits [[FlinkLogicalCalc]]s which contain Python functions in the condition + * into multiple [[FlinkLogicalCalc]]s. After this rule is applied, there will be no + * Python functions in the condition of the [[FlinkLogicalCalc]]s. + */ +object PythonCalcSplitConditionRule extends PythonCalcSplitRuleBase( + "PythonCalcSplitConditionRule") { + + override def matches(call: RelOptRuleCall): Boolean = { + val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc] + + // matches if it contains Python functions in condition + Option(calc.getProgram.getCondition) + .map(calc.getProgram.expandLocalRef) + .exists(containsFunctionOf(_, FunctionLanguage.PYTHON)) + } + + override def isConvertPythonFunction(program: RexProgram): Boolean = true + + override def split(program: RexProgram, splitter: ScalarFunctionSplitter) + : (Option[RexNode], Option[RexNode], Seq[RexNode]) = { + (None, Option(program.getCondition).map(program.expandLocalRef(_).accept(splitter)), + program.getProjectList.map(program.expandLocalRef)) + } +} + +/** + * Rule that splits [[FlinkLogicalCalc]]s which contain both Java functions and Python functions + * in the projection into multiple [[FlinkLogicalCalc]]s. After this rule is applied, it will + * only contain Python functions or Java functions in the projection of each [[FlinkLogicalCalc]]. + */ +object PythonCalcSplitProjectionRule extends PythonCalcSplitRuleBase( + "PythonCalcSplitProjectionRule") { + + override def matches(call: RelOptRuleCall): Boolean = { + val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc] + val projects = calc.getProgram.getProjectList.map(calc.getProgram.expandLocalRef) + + // matches if it contains both Python functions and Java functions in the projection + projects.exists(containsFunctionOf(_, FunctionLanguage.PYTHON)) && + projects.exists(containsFunctionOf(_, FunctionLanguage.JVM)) + } + + override def isConvertPythonFunction(program: RexProgram): Boolean = { + program.getProjectList + .map(program.expandLocalRef) + .exists(containsFunctionOf(_, FunctionLanguage.JVM, recursive = false)) + } + + override def split(program: RexProgram, splitter: ScalarFunctionSplitter) + : (Option[RexNode], Option[RexNode], Seq[RexNode]) = { + (Option(program.getCondition).map(program.expandLocalRef), None, + program.getProjectList.map(program.expandLocalRef(_).accept(splitter))) + } +} + +/** + * Rule that pushes the condition of [[FlinkLogicalCalc]]s before it for the + * [[FlinkLogicalCalc]]s which contain Python functions in the projection. + */ +object PythonCalcPushConditionRule extends PythonCalcSplitRuleBase( + "PythonCalcPushConditionRule") { + + override def matches(call: RelOptRuleCall): Boolean = { + val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc] + val projects = calc.getProgram.getProjectList.map(calc.getProgram.expandLocalRef) + + // matches if all the following conditions hold true: + // 1) the condition is not null + // 2) it contains Python functions in the projection + calc.getProgram.getCondition != null && + projects.exists(containsFunctionOf(_, FunctionLanguage.PYTHON)) + } + + override def isConvertPythonFunction(program: RexProgram): Boolean = false + + override def split(program: RexProgram, splitter: ScalarFunctionSplitter) + : (Option[RexNode], Option[RexNode], Seq[RexNode]) = { + (Option(program.getCondition).map(program.expandLocalRef), None, + program.getProjectList.map(program.expandLocalRef)) + } +} + +/** + * Rule that ensures that it only contains [[RexInputRef]]s at the beginning of + * the project list and [[RexCall]]s at the end of the project list for [[FlinkLogicalCalc]]s + * which contain Python functions in the projection. This rule exists to keep + * DataStreamPythonCalc as simple as possible and ensures that it only needs to + * handle the Python function execution. + */ +object PythonCalcRewriteProjectionRule extends PythonCalcSplitRuleBase( + "PythonCalcRewriteProjectionRule") { + + override def matches(call: RelOptRuleCall): Boolean = { + val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc] + val projects = calc.getProgram.getProjectList.map(calc.getProgram.expandLocalRef) + + // matches if all the following conditions hold true: + // 1) it contains Python functions in the projection + // 2) it contains RexNodes besides RexInputRef and RexCall or + // not all the RexCalls lying at the end of the project list + projects.exists(containsFunctionOf(_, FunctionLanguage.PYTHON)) && + (projects.exists(expr => !expr.isInstanceOf[RexCall] && !expr.isInstanceOf[RexInputRef]) || + projects.indexWhere(_.isInstanceOf[RexCall]) < + projects.lastIndexWhere(_.isInstanceOf[RexInputRef])) + } + + override def isConvertPythonFunction(program: RexProgram): Boolean = true + + override def split(program: RexProgram, splitter: ScalarFunctionSplitter) + : (Option[RexNode], Option[RexNode], Seq[RexNode]) = { + (None, None, program.getProjectList.map(program.expandLocalRef(_).accept(splitter))) + } +} + +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 PythonCalcSplitRule { + /** + * These rules should be applied sequentially in the order of + * SPLIT_CONDITION, SPLIT_PROJECT, PUSH_CONDITION and REWRITE_PROJECT. + */ + val SPLIT_CONDITION: RelOptRule = PythonCalcSplitConditionRule + val SPLIT_PROJECT: RelOptRule = PythonCalcSplitProjectionRule + val PUSH_CONDITION: RelOptRule = PythonCalcPushConditionRule + val REWRITE_PROJECT: RelOptRule = PythonCalcRewriteProjectionRule +} 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 deleted file mode 100644 index deb56c55de8a..000000000000 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/PythonScalarFunctionSplitRule.scala +++ /dev/null @@ -1,211 +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.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/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 ce29bed319a3..5209bdf6cd2a 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 @@ -509,10 +509,13 @@ class ExpressionReductionRulesTest extends TableTestBase { .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") + "DataStreamCalc", + unaryNode( + "DataStreamPythonCalc", + streamTableNode(table), + term("select", "a", "b", "c", "DeterministicPythonFunc$() AS f0") + ), + term("select", "a", "b", "c", "f0 AS d", "null:VARCHAR(65536) AS e") ) util.verifyTable(result, expected) 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/PythonCalcSplitRuleTest.scala similarity index 79% rename from flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/PythonScalarFunctionSplitRuleTest.scala rename to flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/PythonCalcSplitRuleTest.scala index 3db939873337..5d59e3767cad 100644 --- 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/PythonCalcSplitRuleTest.scala @@ -26,7 +26,7 @@ import org.apache.flink.table.utils.TableTestUtil._ import org.apache.flink.table.utils.TableTestBase import org.junit.Test -class PythonScalarFunctionSplitRuleTest extends TableTestBase { +class PythonCalcSplitRuleTest extends TableTestBase { @Test def testPythonFunctionAsInputOfJavaFunction(): Unit = { @@ -87,12 +87,20 @@ class PythonScalarFunctionSplitRuleTest extends TableTestBase { "DataStreamCalc", unaryNode( "DataStreamPythonCalc", - streamTableNode(table), - term("select", "c", "pyFunc1(a, b) AS f0", "pyFunc2(a, c) AS f1") + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamPythonCalc", + streamTableNode(table), + term("select", "a", "b", "c", "pyFunc2(a, c) AS f0") + ), + term("select", "c", "a", "b"), + term("where", ">(f0, 0)") ), - term("select", "f0 AS _c0", "+(c, 1) AS _c1"), - term("where", ">(f1, 0)") - ) + term("select", "c", "pyFunc1(a, b) AS f0") + ), + term("select", "f0 AS _c0", "+(c, 1) AS _c1") + ) util.verifyTable(resultTable, expected) } @@ -109,14 +117,18 @@ class PythonScalarFunctionSplitRuleTest extends TableTestBase { .select("pyFunc1(a, b)") val expected = unaryNode( - "DataStreamCalc", + "DataStreamPythonCalc", unaryNode( - "DataStreamPythonCalc", - streamTableNode(table), - term("select", "pyFunc1(a, b) AS f0", "pyFunc2(a, c) AS f1") + "DataStreamCalc", + unaryNode( + "DataStreamPythonCalc", + streamTableNode(table), + term("select", "a", "b", "pyFunc2(a, c) AS f0") + ), + term("select", "a", "b"), + term("where", "f0") ), - term("select", "f0 AS _c0"), - term("where", "f1") + term("select", "pyFunc1(a, b) AS _c0") ) util.verifyTable(resultTable, expected) @@ -213,6 +225,48 @@ class PythonScalarFunctionSplitRuleTest extends TableTestBase { util.verifyTable(resultTable, expected) } + + @Test + def testLiteral(): 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.select("a, b, pyFunc1(a, c), 1") + + val expected = unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamPythonCalc", + streamTableNode(table), + term("select", "a", "b", "pyFunc1(a, c) AS f0") + ), + term("select", "a", "b", "f0 AS _c2", "1 AS _c3") + ) + + util.verifyTable(resultTable, expected) + } + + @Test + def testReorderPythonCalc(): 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.select("a, pyFunc1(a, c), b") + + val expected = unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamPythonCalc", + streamTableNode(table), + term("select", "a", "b", "pyFunc1(a, c) AS f0") + ), + term("select", "a", "f0 AS _c1", "b") + ) + + util.verifyTable(resultTable, expected) + } } class PythonScalarFunction(name: String) extends ScalarFunction { From 86297253d813a34b73ecf2d3fd883c0a91cef835 Mon Sep 17 00:00:00 2001 From: zhangzhanchum Date: Tue, 22 Oct 2019 17:39:56 +0800 Subject: [PATCH 191/746] [hotfix][typo] fix typo in YarnClusterDescriptor --- .../main/java/org/apache/flink/yarn/YarnClusterDescriptor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 85d420fa2de8..26d641798bf4 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 @@ -951,7 +951,7 @@ private ApplicationReport startAppMaster( if (UserGroupInformation.isSecurityEnabled()) { // set HDFS delegation tokens when security is enabled - LOG.info("Adding delegation token to the AM container.."); + LOG.info("Adding delegation token to the AM container."); Utils.setTokensFor(amContainer, paths, yarnConfiguration); } From a99b84929eab0e8644498fea39c926a0d74ed28c Mon Sep 17 00:00:00 2001 From: Congxian Qiu Date: Wed, 23 Oct 2019 17:38:08 +0800 Subject: [PATCH 192/746] [FLINK-14336][checkpointing] Log exceptions during async checkpoint --- .../apache/flink/streaming/runtime/tasks/StreamTask.java | 6 ++++++ 1 file changed, 6 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 864be69f793a..91a999a883db 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 @@ -1115,6 +1115,12 @@ public void run() { checkpointMetaData.getCheckpointId()); } } catch (Exception e) { + if (LOG.isDebugEnabled()) { + LOG.debug("{} - asynchronous part of checkpoint {} could not be completed.", + owner.getName(), + checkpointMetaData.getCheckpointId(), + e); + } handleExecutionException(e); } finally { owner.cancelables.unregisterCloseable(this); From 406de1db13c54d68eab3dd5d4940d4be6ba7cb45 Mon Sep 17 00:00:00 2001 From: Yun Gao Date: Wed, 23 Oct 2019 17:51:49 +0800 Subject: [PATCH 193/746] [hotfix][streaming] Fix/clarify side output comment --- .../flink/streaming/runtime/io/RecordWriterOutput.java | 4 ++-- .../flink/streaming/runtime/tasks/OperatorChain.java | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java index 31fbcefe021c..0624f0fbd7e2 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java @@ -82,7 +82,7 @@ public RecordWriterOutput( @Override public void collect(StreamRecord record) { if (this.outputTag != null) { - // we are only responsible for emitting to the main input + // we are not responsible for emitting to the main output. return; } @@ -92,7 +92,7 @@ public void collect(StreamRecord record) { @Override public void collect(OutputTag outputTag, StreamRecord record) { if (this.outputTag == null || !this.outputTag.equals(outputTag)) { - // we are only responsible for emitting to the side-output specified by our + // we are not responsible for emitting to the side-output specified by this // OutputTag. return; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java index d15db0ddaa1b..824b0fc68a76 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java @@ -539,7 +539,7 @@ public ChainingOutput( @Override public void collect(StreamRecord record) { if (this.outputTag != null) { - // we are only responsible for emitting to the main input + // we are not responsible for emitting to the main output. return; } @@ -549,7 +549,7 @@ public void collect(StreamRecord record) { @Override public void collect(OutputTag outputTag, StreamRecord record) { if (this.outputTag == null || !this.outputTag.equals(outputTag)) { - // we are only responsible for emitting to the side-output specified by our + // we are not responsible for emitting to the side-output specified by this // OutputTag. return; } @@ -628,7 +628,7 @@ public CopyingChainingOutput( @Override public void collect(StreamRecord record) { if (this.outputTag != null) { - // we are only responsible for emitting to the main input + // we are not responsible for emitting to the main output. return; } @@ -638,7 +638,7 @@ public void collect(StreamRecord record) { @Override public void collect(OutputTag outputTag, StreamRecord record) { if (this.outputTag == null || !this.outputTag.equals(outputTag)) { - // we are only responsible for emitting to the side-output specified by our + // we are not responsible for emitting to the side-output specified by this // OutputTag. return; } From 1cf261e64ff24165f5758263a1e86f38571e32b8 Mon Sep 17 00:00:00 2001 From: Alberto Romero Date: Wed, 23 Oct 2019 11:03:55 +0100 Subject: [PATCH 194/746] [FLINK-13519][es][docs] Correct scala Elasticsearch 6.x example --- docs/dev/connectors/elasticsearch.md | 20 +++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/docs/dev/connectors/elasticsearch.md b/docs/dev/connectors/elasticsearch.md index 23cdb8e26945..9eae491d91da 100644 --- a/docs/dev/connectors/elasticsearch.md +++ b/docs/dev/connectors/elasticsearch.md @@ -255,15 +255,17 @@ httpHosts.add(new HttpHost("10.2.3.1", 9200, "http")) val esSinkBuilder = new ElasticsearchSink.Builder[String]( httpHosts, new ElasticsearchSinkFunction[String] { - def createIndexRequest(element: String): IndexRequest = { - val json = new java.util.HashMap[String, String] - json.put("data", element) - - return Requests.indexRequest() - .index("my-index") - .type("my-type") - .source(json) - } + def process(element: String, ctx: RuntimeContext, indexer: RequestIndexer) { + val json = new java.util.HashMap[String, String] + json.put("data", element) + + val rqst: IndexRequest = Requests.indexRequest + .index("my-index") + .`type`("my-type") + .source(json) + + indexer.add(rqst) + } } ) From 3fc304015968eb9c4c0be759b25492e6087c89c1 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 23 Oct 2019 12:08:07 +0200 Subject: [PATCH 195/746] [hotfix][es][docs] Sync english/chinese docs --- docs/dev/connectors/elasticsearch.zh.md | 20 +++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/docs/dev/connectors/elasticsearch.zh.md b/docs/dev/connectors/elasticsearch.zh.md index 915860fbd251..d17952e5d4b1 100644 --- a/docs/dev/connectors/elasticsearch.zh.md +++ b/docs/dev/connectors/elasticsearch.zh.md @@ -255,15 +255,17 @@ httpHosts.add(new HttpHost("10.2.3.1", 9200, "http")) val esSinkBuilder = new ElasticsearchSink.Builder[String]( httpHosts, new ElasticsearchSinkFunction[String] { - def createIndexRequest(element: String): IndexRequest = { - val json = new java.util.HashMap[String, String] - json.put("data", element) - - return Requests.indexRequest() - .index("my-index") - .type("my-type") - .source(json) - } + def process(element: String, ctx: RuntimeContext, indexer: RequestIndexer) { + val json = new java.util.HashMap[String, String] + json.put("data", element) + + val rqst: IndexRequest = Requests.indexRequest + .index("my-index") + .`type`("my-type") + .source(json) + + indexer.add(rqst) + } } ) From 048ddc104d6917170a701a5ea4a6e83a50817c9d Mon Sep 17 00:00:00 2001 From: cyq89051127 Date: Sat, 25 May 2019 08:58:13 +0800 Subject: [PATCH 196/746] [FLINK-12622][scala][docs] Fix ProcessWindowFunction example --- docs/dev/stream/operators/windows.md | 4 ++-- docs/dev/stream/operators/windows.zh.md | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/dev/stream/operators/windows.md b/docs/dev/stream/operators/windows.md index d317b2ee84fa..a71177d0da02 100644 --- a/docs/dev/stream/operators/windows.md +++ b/docs/dev/stream/operators/windows.md @@ -759,7 +759,7 @@ input class MyProcessWindowFunction extends ProcessWindowFunction[(String, Long), String, String, TimeWindow] { - def process(key: String, context: Context, input: Iterable[(String, Long)], out: Collector[String]): () = { + def process(key: String, context: Context, input: Iterable[(String, Long)], out: Collector[String]) = { var count = 0L for (in <- input) { count = count + 1 @@ -938,7 +938,7 @@ class AverageAggregate extends AggregateFunction[(String, Long), (Long, Long), D class MyProcessWindowFunction extends ProcessWindowFunction[Double, (String, Double), String, TimeWindow] { - def process(key: String, context: Context, averages: Iterable[Double], out: Collector[(String, Double)]): () = { + def process(key: String, context: Context, averages: Iterable[Double], out: Collector[(String, Double)]) = { val average = averages.iterator.next() out.collect((key, average)) } diff --git a/docs/dev/stream/operators/windows.zh.md b/docs/dev/stream/operators/windows.zh.md index 45ef36f12ad4..72269a45596d 100644 --- a/docs/dev/stream/operators/windows.zh.md +++ b/docs/dev/stream/operators/windows.zh.md @@ -759,7 +759,7 @@ input class MyProcessWindowFunction extends ProcessWindowFunction[(String, Long), String, String, TimeWindow] { - def process(key: String, context: Context, input: Iterable[(String, Long)], out: Collector[String]): () = { + def process(key: String, context: Context, input: Iterable[(String, Long)], out: Collector[String]) = { var count = 0L for (in <- input) { count = count + 1 @@ -938,7 +938,7 @@ class AverageAggregate extends AggregateFunction[(String, Long), (Long, Long), D class MyProcessWindowFunction extends ProcessWindowFunction[Double, (String, Double), String, TimeWindow] { - def process(key: String, context: Context, averages: Iterable[Double], out: Collector[(String, Double)]): () = { + def process(key: String, context: Context, averages: Iterable[Double], out: Collector[(String, Double)]) = { val average = averages.iterator.next() out.collect((key, average)) } From 569c3a206ad1909a89c28bbb95c45cbc35c1f104 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Wed, 16 Oct 2019 13:57:39 +0200 Subject: [PATCH 197/746] [hotfix][table] JoinedRow failed on toString as the default implementation invokes the unsupported hashCode. --- .../java/org/apache/flink/table/dataformat/JoinedRow.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/JoinedRow.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/JoinedRow.java index 92e29a513e75..e3023a8c0931 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/JoinedRow.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/JoinedRow.java @@ -281,4 +281,12 @@ public int hashCode() { throw new UnsupportedOperationException( "JoinedRow do not support hashCode, please hash fields one by one!"); } + + @Override + public String toString() { + return "JoinedRow{" + + "row1=" + row1 + + ", row2=" + row2 + + '}'; + } } From 3e455e5d9ac904943e11aa6695cb340a9c5cb850 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Thu, 26 Sep 2019 17:36:27 +0200 Subject: [PATCH 198/746] [FLINK-14199] [runtime] Mailbox explicitly requires a description to ease debugging --- .../operators/async/AsyncWaitOperator.java | 30 ++-- .../streaming/runtime/tasks/StreamTask.java | 137 ++++++++++-------- .../runtime/tasks/mailbox/MailboxSender.java | 16 +- .../runtime/tasks/mailbox/TaskMailbox.java | 20 ++- .../tasks/mailbox/TaskMailboxImpl.java | 25 ++-- .../mailbox/execution/MailboxExecutor.java | 75 +++++++++- .../execution/MailboxExecutorImpl.java | 7 +- .../mailbox/execution/MailboxProcessor.java | 22 ++- .../runtime/tasks/StreamTaskTest.java | 2 +- .../tasks/mailbox/TaskMailboxImplTest.java | 52 +++---- .../execution/MailboxExecutorImplTest.java | 43 ++++-- .../execution/TaskMailboxProcessorTest.java | 17 +-- .../execution/TestMailboxExecutor.java | 64 -------- 13 files changed, 285 insertions(+), 225 deletions(-) delete mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/TestMailboxExecutor.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java index 85161aacee1f..5fbc19e0fe67 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java @@ -293,7 +293,7 @@ private void outputCompletedElement() { } // if there are more completed elements, emit them with subsequent mails if (queue.hasCompletedElements()) { - mailboxExecutor.execute(this::outputCompletedElement); + mailboxExecutor.execute(this::outputCompletedElement, "AsyncWaitOperator#outputCompletedElement"); } } } @@ -346,19 +346,23 @@ public void complete(Collection results) { private void processInMailbox(Collection results) { // move further processing into the mailbox thread - mailboxExecutor.execute(() -> { - // Cancel the timer once we've completed the stream record buffer entry. This will remove the registered - // timer task - if (timeoutTimer != null) { - // canceling in mailbox thread avoids https://issues.apache.org/jira/browse/FLINK-13635 - timeoutTimer.cancel(true); - } + mailboxExecutor.execute( + () -> processResults(results), + "Result in AsyncWaitOperator of input %s", results); + } + + private void processResults(Collection results) { + // Cancel the timer once we've completed the stream record buffer entry. This will remove the registered + // timer task + if (timeoutTimer != null) { + // canceling in mailbox thread avoids https://issues.apache.org/jira/browse/FLINK-13635 + timeoutTimer.cancel(true); + } - // update the queue entry with the result - resultFuture.complete(results); - // now output all elements from the queue that have been completed (in the correct order) - outputCompletedElement(); - }); + // update the queue entry with the result + resultFuture.complete(results); + // now output all elements from the queue that have been completed (in the correct order) + outputCompletedElement(); } @Override 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 91a999a883db..07c0a59d0137 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 @@ -723,32 +723,41 @@ public Future triggerCheckpointAsync( CheckpointOptions checkpointOptions, boolean advanceToEndOfEventTime) { - return mailboxProcessor.getMainMailboxExecutor().submit(() -> { - try { - // No alignment if we inject a checkpoint - CheckpointMetrics checkpointMetrics = new CheckpointMetrics() - .setBytesBufferedInAlignment(0L) - .setAlignmentDurationNanos(0L); - - boolean success = performCheckpoint(checkpointMetaData, checkpointOptions, checkpointMetrics, advanceToEndOfEventTime); - if (!success) { - declineCheckpoint(checkpointMetaData.getCheckpointId()); - } - return success; - } catch (Exception e) { - // propagate exceptions only if the task is still in "running" state - if (isRunning) { - Exception exception = new Exception("Could not perform checkpoint " + checkpointMetaData.getCheckpointId() + - " for operator " + getName() + '.', e); - handleCheckpointException(exception); - throw exception; - } else { - LOG.debug("Could not perform checkpoint {} for operator {} while the " + - "invokable was not in state running.", checkpointMetaData.getCheckpointId(), getName(), e); - return false; - } + return mailboxProcessor.getMainMailboxExecutor().submit( + () -> triggerCheckpoint(checkpointMetaData, checkpointOptions, advanceToEndOfEventTime), + "checkpoint %s with %s", + checkpointMetaData, + checkpointOptions); + } + + private boolean triggerCheckpoint( + CheckpointMetaData checkpointMetaData, + CheckpointOptions checkpointOptions, + boolean advanceToEndOfEventTime) throws Exception { + try { + // No alignment if we inject a checkpoint + CheckpointMetrics checkpointMetrics = new CheckpointMetrics() + .setBytesBufferedInAlignment(0L) + .setAlignmentDurationNanos(0L); + + boolean success = performCheckpoint(checkpointMetaData, checkpointOptions, checkpointMetrics, advanceToEndOfEventTime); + if (!success) { + declineCheckpoint(checkpointMetaData.getCheckpointId()); + } + return success; + } catch (Exception e) { + // propagate exceptions only if the task is still in "running" state + if (isRunning) { + Exception exception = new Exception("Could not perform checkpoint " + checkpointMetaData.getCheckpointId() + + " for operator " + getName() + '.', e); + handleCheckpointException(exception); + throw exception; + } else { + LOG.debug("Could not perform checkpoint {} for operator {} while the " + + "invokable was not in state running.", checkpointMetaData.getCheckpointId(), getName(), e); + return false; } - }); + } } @Override @@ -875,37 +884,37 @@ public ExecutorService getAsyncOperationsThreadPool() { @Override public Future notifyCheckpointCompleteAsync(long checkpointId) { - return mailboxProcessor.getMailboxExecutor(TaskMailbox.MAX_PRIORITY).submit(() -> { - try { - boolean success = false; - synchronized (lock) { - if (isRunning) { - LOG.debug("Notification of complete checkpoint for task {}", getName()); + return mailboxProcessor.getMailboxExecutor(TaskMailbox.MAX_PRIORITY).submit( + () -> notifyCheckpointComplete(checkpointId), + "checkpoint %d complete", checkpointId); + } + + private void notifyCheckpointComplete(long checkpointId) { + try { + boolean success = false; + synchronized (lock) { + if (isRunning) { + LOG.debug("Notification of complete checkpoint for task {}", getName()); - for (StreamOperator operator : operatorChain.getAllOperators()) { - if (operator != null) { - operator.notifyCheckpointComplete(checkpointId); - } + for (StreamOperator operator : operatorChain.getAllOperators()) { + if (operator != null) { + operator.notifyCheckpointComplete(checkpointId); } - success = true; - } else { - LOG.debug("Ignoring notification of complete checkpoint for not-running task {}", getName()); } + success = true; + } else { + LOG.debug("Ignoring notification of complete checkpoint for not-running task {}", getName()); } - getEnvironment().getTaskStateManager().notifyCheckpointComplete(checkpointId); - if (success && isSynchronousSavepointId(checkpointId)) { - finishTask(); - // Reset to "notify" the internal synchronous savepoint mailbox loop. - resetSynchronousSavepointId(); - } - return null; - } catch (Exception e) { - handleException(new RuntimeException( - "Error while confirming checkpoint", - e)); - throw e; } - }); + getEnvironment().getTaskStateManager().notifyCheckpointComplete(checkpointId); + if (success && isSynchronousSavepointId(checkpointId)) { + finishTask(); + // Reset to "notify" the internal synchronous savepoint mailbox loop. + resetSynchronousSavepointId(); + } + } catch (Exception e) { + handleException(new RuntimeException("Error while confirming checkpoint", e)); + } } private void tryShutdownTimerService() { @@ -1432,18 +1441,26 @@ private class TimerInvocationContext implements SystemProcessingTimeService.Sche @Override public void invoke(ProcessingTimeCallback callback, long timestamp) { try { - mailboxProcessor.getMailboxExecutor(TaskMailbox.MAX_PRIORITY).execute(() -> { - synchronized (getCheckpointLock()) { - try { - callback.onProcessingTime(timestamp); - } catch (Throwable t) { - handleAsyncException("Caught exception while processing timer.", new TimerException(t)); - } - } - }); + mailboxProcessor.getMailboxExecutor(TaskMailbox.MAX_PRIORITY).execute( + () -> invokeProcessingTimeCallback(callback, timestamp), + "Timer callback for %s @ %d", + callback, + timestamp); } catch (Throwable t) { handleAsyncException("Caught exception while processing timer.", new TimerException(t)); } } + + private void invokeProcessingTimeCallback(ProcessingTimeCallback callback, long timestamp) { + synchronized (getCheckpointLock()) { + try { + callback.onProcessingTime(timestamp); + } catch (Throwable t) { + handleAsyncException( + "Caught exception while processing timer.", + new TimerException(t)); + } + } + } } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxSender.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxSender.java index 8603031f6eb0..bb2ef4664c1e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxSender.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxSender.java @@ -29,17 +29,29 @@ public interface MailboxSender { /** * Enqueues the given letter to the mailbox and blocks until there is capacity for a successful put. * + *

An optional description can be added to ease debugging and error-reporting. Any object can be passed on which + * {@link Object#toString()} is lazily invoked. In most cases, it should be a {@link String} or + * {@link org.apache.flink.streaming.runtime.tasks.mailbox.LazyString}. If no explicit description is taken, the + * command itself is used and {@code toString()} will be invoked on it. + * * @param letter the letter to enqueue. + * @param description the optional description for the command that is used for debugging and error-reporting. * @throws MailboxStateException if the mailbox is quiesced or closed. */ - void putMail(@Nonnull Runnable letter) throws MailboxStateException; + void putMail(@Nonnull Runnable letter, Object description) throws MailboxStateException; /** * Adds the given action to the head of the mailbox. * + *

An optional description can be added to ease debugging and error-reporting. Any object can be passed on which + * {@link Object#toString()} is lazily invoked. In most cases, it should be a {@link String} or + * {@link org.apache.flink.streaming.runtime.tasks.mailbox.LazyString}. If no explicit description is taken, the + * command itself is used and {@code toString()} will be invoked on it. + * * @param priorityLetter action to enqueue to the head of the mailbox. + * @param description the optional description for the command that is used for debugging and error-reporting. * @throws MailboxStateException if the mailbox is quiesced or closed. */ - void putFirst(@Nonnull Runnable priorityLetter) throws MailboxStateException; + void putFirst(@Nonnull Runnable priorityLetter, Object description) throws MailboxStateException; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java index 0a8561556be8..9c166a909736 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java @@ -81,8 +81,8 @@ enum State { /** * Returns a mailbox view bound to the given priority. * - *

Enqueuing letters (e.g., {@link Mailbox#putMail(Runnable)} and {@link Mailbox#putFirst(Runnable)}) will mark these letters - * to belong to the bound operator. + *

Enqueuing letters (e.g., {@link Mailbox#putMail(Runnable, Object description)} and + * {@link Mailbox#putFirst(Runnable, Object description)}) will mark these letters to belong to the bound operator. * *

Similarly, only letters from the operator or any downstream operator are retrieved by {@link Mailbox#tryTakeMail()} * and {@link Mailbox#takeMail()}. @@ -102,19 +102,31 @@ enum State { /** * Enqueues the given letter to the mailbox and blocks until there is capacity for a successful put. * + *

An optional description can be added to ease debugging and error-reporting. Any object can be passed on which + * {@link Object#toString()} is lazily invoked. In most cases, it should be a {@link String} or + * {@link org.apache.flink.streaming.runtime.tasks.mailbox.LazyString}. If no explicit description is taken, the + * command itself is used and {@code toString()} will be invoked on it. + * * @param letter the letter to enqueue. * @param priority the priority of the letter. + * @param description the optional description for the command that is used for debugging and error-reporting. * @throws MailboxStateException if the mailbox is quiesced or closed. */ - void putMail(@Nonnull Runnable letter, int priority) throws MailboxStateException; + void putMail(@Nonnull Runnable letter, int priority, Object description) throws MailboxStateException; /** * Adds the given action to the head of the mailbox. * + *

An optional description can be added to ease debugging and error-reporting. Any object can be passed on which + * {@link Object#toString()} is lazily invoked. In most cases, it should be a {@link String} or + * {@link org.apache.flink.streaming.runtime.tasks.mailbox.LazyString}. If no explicit description is taken, the + * command itself is used and {@code toString()} will be invoked on it. + * * @param priorityLetter action to enqueue to the head of the mailbox. + * @param description the optional description for the command that is used for debugging and error-reporting. * @throws MailboxStateException if the mailbox is quiesced or closed. */ - void putFirst(@Nonnull Runnable priorityLetter) throws MailboxStateException; + void putFirst(@Nonnull Runnable priorityLetter, Object description) throws MailboxStateException; /** * Returns an optional with either the oldest letter with a minimum priority from the mailbox (head of queue) if the diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java index c4a3f2506731..e656a1daf6e5 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java @@ -111,11 +111,11 @@ public Optional tryTakeMail(int priority) throws MailboxStateException //------------------------------------------------------------------------------------------------------------------ @Override - public void putMail(@Nonnull Runnable letter, int priority) throws MailboxStateException { + public void putMail(@Nonnull Runnable letter, int priority, Object description) throws MailboxStateException { final ReentrantLock lock = this.lock; lock.lock(); try { - putTailInternal(new Mail(letter, priority)); + putTailInternal(new Mail(letter, priority, description)); } finally { lock.unlock(); } @@ -124,11 +124,11 @@ public void putMail(@Nonnull Runnable letter, int priority) throws MailboxStateE //------------------------------------------------------------------------------------------------------------------ @Override - public void putFirst(@Nonnull Runnable priorityLetter) throws MailboxStateException { + public void putFirst(@Nonnull Runnable priorityLetter, Object description) throws MailboxStateException { final ReentrantLock lock = this.lock; lock.lock(); try { - putHeadInternal(new Mail(priorityLetter, MAX_PRIORITY)); + putHeadInternal(new Mail(priorityLetter, MAX_PRIORITY, description)); } finally { lock.unlock(); } @@ -276,13 +276,13 @@ public Runnable takeMail() throws InterruptedException, MailboxStateException { } @Override - public void putMail(@Nonnull Runnable letter) throws MailboxStateException { - TaskMailboxImpl.this.putMail(letter, priority); + public void putMail(@Nonnull Runnable letter, Object description) throws MailboxStateException { + TaskMailboxImpl.this.putMail(letter, priority, description); } @Override - public void putFirst(@Nonnull Runnable priorityLetter) throws MailboxStateException { - TaskMailboxImpl.this.putFirst(priorityLetter); + public void putFirst(@Nonnull Runnable priorityLetter, Object description) throws MailboxStateException { + TaskMailboxImpl.this.putFirst(priorityLetter, description); } } @@ -292,10 +292,12 @@ public void putFirst(@Nonnull Runnable priorityLetter) throws MailboxStateExcept static class Mail { private final Runnable runnable; private final int priority; + private final Object description; - public Mail(Runnable runnable, int priority) { + public Mail(Runnable runnable, int priority, Object description) { this.runnable = runnable; this.priority = priority; + this.description = description == null ? runnable : description; } public int getPriority() { @@ -305,5 +307,10 @@ public int getPriority() { Runnable getRunnable() { return runnable; } + + @Override + public String toString() { + return description.toString(); + } } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutor.java index 1ee296bc0dd4..f15bc2dc03f4 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutor.java @@ -32,34 +32,76 @@ /** * Interface for an {@link Executor} build around a {@link Mailbox}-based execution model. */ -public interface MailboxExecutor extends Executor { +public interface MailboxExecutor { /** * Executes the given command at some time in the future in the mailbox thread. * + *

An optional description can (and should) be added to ease debugging and error-reporting. The description + * may contain placeholder that refer to the provided description arguments using {@link java.util.Formatter} + * syntax. The actual description is only formatted on demand. + * * @param command the runnable task to add to the mailbox for execution. + * @param descriptionFormat the optional description for the command that is used for debugging and error-reporting. + * @param descriptionArgs the parameters used to format the final description string. + * @throws RejectedExecutionException if this task cannot be accepted for execution, e.g. because the mailbox is + * quiesced or closed. + */ + void execute(@Nonnull Runnable command, String descriptionFormat, Object... descriptionArgs); + + /** + * Executes the given command at the next possible time in the mailbox thread. Repeated calls will result in the + * last executed command being executed first. + * + *

An optional description can (and should) be added to ease debugging and error-reporting. The description + * may contain placeholder that refer to the provided description arguments using {@link java.util.Formatter} + * syntax. The actual description is only formatted on demand. + * + * @param command the runnable task to add to the mailbox for execution. + * @param descriptionFormat the optional description for the command that is used for debugging and error-reporting. + * @param descriptionArgs the parameters used to format the final description string. * @throws RejectedExecutionException if this task cannot be accepted for execution, e.g. because the mailbox is * quiesced or closed. */ - @Override - void execute(@Nonnull Runnable command) throws RejectedExecutionException; + void executeFirst(@Nonnull Runnable command, String descriptionFormat, Object... descriptionArgs); /** * Submits the given command for execution in the future in the mailbox thread and returns a Future representing * that command. The Future's {@code get} method will return {@code null} upon successful completion. * + *

An optional description can (and should) be added to ease debugging and error-reporting. The description + * may contain placeholder that refer to the provided description arguments using {@link java.util.Formatter} + * syntax. The actual description is only formatted on demand. + * * @param command the command to submit + * @param descriptionFormat the optional description for the command that is used for debugging and error-reporting. + * @param descriptionArgs the parameters used to format the final description string. * @return a Future representing pending completion of the task * @throws RejectedExecutionException if this task cannot be accepted for execution, e.g. because the mailbox is * quiesced or closed. */ - default @Nonnull Future submit(@Nonnull Runnable command) { - return submit(Executors.callable(command, null)); + default @Nonnull Future submit(@Nonnull Runnable command, String descriptionFormat, Object... descriptionArgs) { + return submit(Executors.callable(command, null), descriptionFormat, descriptionArgs); } - default @Nonnull Future submit(@Nonnull Callable task) { - FutureTask future = new FutureTask<>(task); - execute(future); + /** + * Submits the given command for execution in the future in the mailbox thread and returns a Future representing + * that command. The Future's {@code get} method will return {@code null} upon successful completion. + * + *

An optional description can (and should) be added to ease debugging and error-reporting. The description + * may contain placeholder that refer to the provided description arguments using {@link java.util.Formatter} + * syntax. The actual description is only formatted on demand. + * + * @param command the command to submit + * @param descriptionFormat the optional description for the command that is used for debugging and error-reporting. + * @param descriptionArgs the parameters used to format the final description string. + * @return a Future representing pending completion of the task + * @throws RejectedExecutionException if this task cannot be accepted for execution, e.g. because the mailbox is + * quiesced or closed. + */ + default @Nonnull Future submit(@Nonnull Callable command, String descriptionFormat, Object... descriptionArgs) { + FutureTask future = new FutureTask<>(command); + execute(future, descriptionFormat, descriptionArgs); return future; } @@ -91,4 +133,21 @@ public interface MailboxExecutor extends Executor { * @return only true if called from the mailbox thread. */ boolean isMailboxThread(); + + /** + * Provides an {@link Executor} view on this {@code MailboxExecutor}, where submitted tasks will receive the + * given description. The {@link Executor} can be used with {@link java.util.concurrent.CompletableFuture}. + * + *

An optional description can (and should) be added to ease debugging and error-reporting. The description + * may contain placeholder that refer to the provided description arguments using {@link java.util.Formatter} + * syntax. The actual description is only formatted on demand. + * + * @param descriptionFormat the optional description for all commands that is used for debugging and + * error-reporting. + * @param descriptionArgs the parameters used to format the final description string. + * @return an {@code Executor} view on this {@code MailboxExecutor} + */ + default Executor asExecutor(String descriptionFormat, Object... descriptionArgs) { + return command -> execute(command, descriptionFormat, descriptionArgs); + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java index eccdd77fd372..9ff150e9d075 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java @@ -49,9 +49,12 @@ public MailboxExecutorImpl(@Nonnull Mailbox mailbox, @Nonnull Thread taskMailbox } @Override - public void execute(@Nonnull Runnable command) { + public void execute( + @Nonnull final Runnable command, + final String descriptionFormat, + final Object... descriptionArgs) { try { - mailbox.putMail(command); + mailbox.putMail(command, description); } catch (MailboxStateException mbex) { throw new RejectedExecutionException(mbex); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java index 9fe362bb28d2..89d29f7c79cc 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java @@ -157,24 +157,22 @@ public void runMailboxLoop() throws Exception { * @param throwable to report by rethrowing from the mailbox loop. */ public void reportThrowable(Throwable throwable) { - sendPriorityLetter(() -> { - throw new WrappingRuntimeException(throwable); - }); + sendPriorityLetter( + () -> { + throw new WrappingRuntimeException(throwable); + }, + "Report throwable %s", throwable); } /** * This method must be called to end the stream task when all actions for the tasks have been performed. */ public void allActionsCompleted() { - sendPriorityLetter(mailboxPoisonLetter); + sendPriorityLetter(mailboxPoisonLetter, "poison letter"); } - private void sendPriorityLetter(Runnable priorityLetter) { - try { - mailbox.putFirst(priorityLetter); - } catch (MailboxStateException me) { - LOG.debug("Action context could not submit priority letter to mailbox.", me); - } + private void sendPriorityLetter(Runnable priorityLetter, String descriptionFormat, Object... descriptionArgs) { + mainMailboxExecutor.executeFirst(priorityLetter, descriptionFormat, descriptionArgs); } /** @@ -244,7 +242,7 @@ private boolean isMailboxLoopRunning() { private void ensureControlFlowSignalCheck() { // Make sure that mailbox#hasMail is true via a dummy letter so that the flag change is noticed. if (!mailbox.hasMail()) { - sendPriorityLetter(() -> {}); + sendPriorityLetter(() -> {}, "signal check"); } } @@ -281,7 +279,7 @@ public void resume() { if (isMailboxThread()) { resumeInternal(); } else { - sendPriorityLetter(this::resumeInternal); + sendPriorityLetter(this::resumeInternal, "resume default action"); } } 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 222e45b70fef..61953b43201f 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 @@ -779,7 +779,7 @@ protected void init() throws Exception { @Override protected void processInput(DefaultActionContext context) throws Exception { - mailboxProcessor.getMailboxExecutor(0).execute(latch::trigger); + mailboxProcessor.getMailboxExecutor(0).execute(latch::trigger, "trigger"); context.allActionsCompleted(); } }); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImplTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImplTest.java index 7c3a2d3d4022..d702590958cb 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImplTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImplTest.java @@ -18,7 +18,6 @@ package org.apache.flink.streaming.runtime.tasks.mailbox; -import org.apache.flink.util.function.BiConsumerWithException; import org.apache.flink.util.function.FunctionWithException; import org.apache.flink.util.function.RunnableWithException; import org.apache.flink.util.function.ThrowingRunnable; @@ -65,10 +64,10 @@ public void testPutAsHead() throws Exception { Runnable instanceC = () -> {}; Runnable instanceD = () -> {}; - taskMailbox.putMail(instanceC, DEFAULT_PRIORITY); - taskMailbox.putFirst(instanceB); - taskMailbox.putMail(instanceD, DEFAULT_PRIORITY); - taskMailbox.putFirst(instanceA); + taskMailbox.putMail(instanceC, DEFAULT_PRIORITY, "instanceC, DEFAULT_PRIORITY"); + taskMailbox.putFirst(instanceB, "instanceB"); + taskMailbox.putMail(instanceD, DEFAULT_PRIORITY, "instanceD, DEFAULT_PRIORITY"); + taskMailbox.putFirst(instanceA, "instanceA"); Assert.assertSame(instanceA, taskMailbox.takeMail(DEFAULT_PRIORITY)); Assert.assertSame(instanceB, taskMailbox.takeMail(DEFAULT_PRIORITY)); @@ -86,7 +85,7 @@ public void testContracts() throws Exception { for (int i = 0; i < 10; ++i) { Runnable letter = NO_OP; testObjects.add(letter); - taskMailbox.putMail(letter, DEFAULT_PRIORITY); + taskMailbox.putMail(letter, DEFAULT_PRIORITY, "letter, DEFAULT_PRIORITY"); Assert.assertTrue(taskMailbox.hasMail()); } @@ -101,7 +100,7 @@ public void testContracts() throws Exception { */ @Test public void testConcurrentPutTakeBlocking() throws Exception { - testPutTake(Mailbox::takeMail, Mailbox::putMail); + testPutTake(Mailbox::takeMail); } /** @@ -115,8 +114,7 @@ public void testConcurrentPutTakeNonBlockingAndWait() throws Exception { optionalLetter = mailbox.tryTakeMail(); } return optionalLetter.get(); - }), - MailboxSender::putMail); + })); } /** @@ -139,8 +137,8 @@ public void testQuiesceUnblocks() throws Exception { @Test public void testLifeCycleQuiesce() throws Exception { - taskMailbox.putMail(NO_OP, DEFAULT_PRIORITY); - taskMailbox.putMail(NO_OP, DEFAULT_PRIORITY); + taskMailbox.putMail(NO_OP, DEFAULT_PRIORITY, "NO_OP, DEFAULT_PRIORITY"); + taskMailbox.putMail(NO_OP, DEFAULT_PRIORITY, "NO_OP, DEFAULT_PRIORITY"); taskMailbox.quiesce(); testLifecyclePuttingInternal(); taskMailbox.takeMail(DEFAULT_PRIORITY); @@ -168,21 +166,21 @@ public void testLifeCycleClose() throws Exception { private void testLifecyclePuttingInternal() throws Exception { try { - taskMailbox.putMail(NO_OP, DEFAULT_PRIORITY); + taskMailbox.putMail(NO_OP, DEFAULT_PRIORITY, "NO_OP, DEFAULT_PRIORITY"); Assert.fail(); } catch (MailboxStateException ignore) { } try { - taskMailbox.putFirst(NO_OP); + taskMailbox.putFirst(NO_OP, "NO_OP"); Assert.fail(); } catch (MailboxStateException ignore) { } } private void testAllPuttingUnblocksInternal(Consumer unblockMethod) throws InterruptedException { - testUnblocksInternal(() -> taskMailbox.putMail(NO_OP, DEFAULT_PRIORITY), unblockMethod); + testUnblocksInternal(() -> taskMailbox.putMail(NO_OP, DEFAULT_PRIORITY, "NO_OP, DEFAULT_PRIORITY"), unblockMethod); setUp(); - testUnblocksInternal(() -> taskMailbox.putFirst(NO_OP), unblockMethod); + testUnblocksInternal(() -> taskMailbox.putFirst(NO_OP, "NO_OP"), unblockMethod); } private void testUnblocksInternal( @@ -225,9 +223,7 @@ private void testUnblocksInternal( /** * Test producer-consumer pattern through the mailbox in a concurrent setting (n-writer / 1-reader). */ - private void testPutTake( - FunctionWithException takeMethod, - BiConsumerWithException putMethod) throws Exception { + private void testPutTake(FunctionWithException takeMethod) throws Exception { final int numThreads = 10; final int numLettersPerThread = 1000; final int[] results = new int[numThreads]; @@ -245,7 +241,7 @@ private void testPutTake( final int threadId = i; writerThreads[i] = new Thread(ThrowingRunnable.unchecked(() -> { for (int k = 0; k < numLettersPerThread; ++k) { - putMethod.accept(mailbox, () -> ++results[threadId]); + mailbox.putMail(() -> ++results[threadId], "result " + k); } })); } @@ -258,7 +254,7 @@ private void testPutTake( writerThread.join(); } - taskMailbox.putMail(POISON_LETTER, DEFAULT_PRIORITY); + taskMailbox.putMail(POISON_LETTER, DEFAULT_PRIORITY, "POISON_LETTER, DEFAULT_PRIORITY"); readerThread.join(); for (int perThreadResult : results) { @@ -293,10 +289,10 @@ public void testPutAsHeadInDownstream() throws Exception { Runnable instanceC = () -> {}; Runnable instanceD = () -> {}; - taskMailbox.getDownstreamMailbox(1).putMail(instanceC); - taskMailbox.getDownstreamMailbox(2).putMail(instanceB); - taskMailbox.getDownstreamMailbox(1).putMail(instanceD); - taskMailbox.getDownstreamMailbox(2).putFirst(instanceA); + taskMailbox.getDownstreamMailbox(1).putMail(instanceC, "instanceC"); + taskMailbox.getDownstreamMailbox(2).putMail(instanceB, "instanceB"); + taskMailbox.getDownstreamMailbox(1).putMail(instanceD, "instanceD"); + taskMailbox.getDownstreamMailbox(2).putFirst(instanceA, "instanceA"); Assert.assertSame(instanceA, taskMailbox.getDownstreamMailbox(2).takeMail()); Assert.assertSame(instanceB, taskMailbox.getDownstreamMailbox(2).takeMail()); @@ -316,10 +312,10 @@ public void testPutInDownstreamAndReadingFromTaskMailbox() throws Exception { Runnable instanceC = () -> {}; Runnable instanceD = () -> {}; - taskMailbox.getDownstreamMailbox(1).putMail(instanceC); - taskMailbox.getDownstreamMailbox(2).putMail(instanceB); - taskMailbox.getDownstreamMailbox(1).putMail(instanceD); - taskMailbox.getDownstreamMailbox(2).putFirst(instanceA); + taskMailbox.getDownstreamMailbox(1).putMail(instanceC, "instanceC"); + taskMailbox.getDownstreamMailbox(2).putMail(instanceB, "instanceB"); + taskMailbox.getDownstreamMailbox(1).putMail(instanceD, "instanceD"); + taskMailbox.getDownstreamMailbox(2).putFirst(instanceA, "instanceA"); // same order for non-priority and priority on top Assert.assertSame(instanceA, taskMailbox.takeMail(TaskMailbox.MIN_PRIORITY)); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImplTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImplTest.java index 4dd0156c27ae..087407200201 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImplTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImplTest.java @@ -35,6 +35,9 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + /** * Tests for {@link MailboxExecutorImpl}. */ @@ -71,26 +74,28 @@ public void tearDown() { @Test public void testOperations() throws Exception { final TestRunnable testRunnable = new TestRunnable(); - mailboxExecutor.execute(testRunnable); + mailboxExecutor.execute(testRunnable, "testRunnable"); Assert.assertEquals(testRunnable, mailbox.tryTakeMail(DEFAULT_PRIORITY).get()); - CompletableFuture.runAsync(() -> mailboxExecutor.execute(testRunnable), otherThreadExecutor).get(); + CompletableFuture.runAsync( + () -> mailboxExecutor.execute(testRunnable, "testRunnable"), + otherThreadExecutor).get(); Assert.assertEquals(testRunnable, mailbox.takeMail(DEFAULT_PRIORITY)); final TestRunnable yieldRun = new TestRunnable(); final TestRunnable leftoverRun = new TestRunnable(); - mailboxExecutor.execute(yieldRun); + mailboxExecutor.execute(yieldRun, "yieldRun"); Future leftoverFuture = CompletableFuture.supplyAsync( - () -> mailboxExecutor.submit(leftoverRun), - otherThreadExecutor).get(); + () -> mailboxExecutor.submit(leftoverRun, "leftoverRun"), + otherThreadExecutor).get(); - Assert.assertTrue(mailboxExecutor.tryYield()); + assertTrue(mailboxExecutor.tryYield()); Assert.assertEquals(Thread.currentThread(), yieldRun.wasExecutedBy()); - Assert.assertFalse(leftoverFuture.isDone()); + assertFalse(leftoverFuture.isDone()); List leftoverTasks = mailbox.close(); Assert.assertEquals(1, leftoverTasks.size()); - Assert.assertFalse(leftoverFuture.isCancelled()); + assertFalse(leftoverFuture.isCancelled()); FutureUtils.cancelRunnableFutures(leftoverTasks); - Assert.assertTrue(leftoverFuture.isCancelled()); + assertTrue(leftoverFuture.isCancelled()); try { mailboxExecutor.tryYield(); @@ -108,9 +113,12 @@ public void testOperations() throws Exception { @Test public void testTryYield() throws Exception { final TestRunnable testRunnable = new TestRunnable(); - CompletableFuture.runAsync(() -> mailboxExecutor.execute(testRunnable), otherThreadExecutor).get(); - Assert.assertTrue(mailboxExecutor.tryYield()); - Assert.assertFalse(mailbox.tryTakeMail(DEFAULT_PRIORITY).isPresent()); + CompletableFuture.runAsync( + () -> mailboxExecutor.execute(testRunnable, "testRunnable"), + otherThreadExecutor) + .get(); + assertTrue(mailboxExecutor.tryYield()); + assertFalse(mailbox.tryTakeMail(DEFAULT_PRIORITY).isPresent()); Assert.assertEquals(Thread.currentThread(), testRunnable.wasExecutedBy()); } @@ -120,7 +128,7 @@ public void testYield() throws Exception { final TestRunnable testRunnable = new TestRunnable(); final Thread submitThread = new Thread(() -> { try { - mailboxExecutor.execute(testRunnable); + mailboxExecutor.execute(testRunnable, "testRunnable"); } catch (Exception e) { exceptionReference.set(e); } @@ -134,6 +142,15 @@ public void testYield() throws Exception { Assert.assertEquals(Thread.currentThread(), testRunnable.wasExecutedBy()); } + @Test + public void testExecutorView() throws Exception { + CompletableFuture future = CompletableFuture.runAsync(() -> {}, mailboxExecutor.asExecutor("runAsync")); + assertFalse(future.isDone()); + + mailboxExecutor.yield(); + assertTrue(future.isDone()); + } + /** * Test {@link Runnable} that tracks execution. */ diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/TaskMailboxProcessorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/TaskMailboxProcessorTest.java index 7e7c90e9d281..466c4f7e7df6 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/TaskMailboxProcessorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/TaskMailboxProcessorTest.java @@ -41,7 +41,7 @@ public void testRejectIfNotOpen() { MailboxProcessor mailboxProcessor = new MailboxProcessor((ctx) -> {}); mailboxProcessor.prepareClose(); try { - mailboxProcessor.getMailboxExecutor(DEFAULT_PRIORITY).execute(() -> {}); + mailboxProcessor.getMailboxExecutor(DEFAULT_PRIORITY).execute(() -> {}, "dummy"); Assert.fail("Should not be able to accept runnables if not opened."); } catch (RejectedExecutionException expected) { } @@ -51,11 +51,11 @@ public void testRejectIfNotOpen() { public void testShutdown() { MailboxProcessor mailboxProcessor = new MailboxProcessor((ctx) -> {}); FutureTask testRunnableFuture = new FutureTask<>(() -> {}, null); - mailboxProcessor.getMailboxExecutor(DEFAULT_PRIORITY).execute(testRunnableFuture); + mailboxProcessor.getMailboxExecutor(DEFAULT_PRIORITY).execute(testRunnableFuture, "testRunnableFuture"); mailboxProcessor.prepareClose(); try { - mailboxProcessor.getMailboxExecutor(DEFAULT_PRIORITY).execute(() -> {}); + mailboxProcessor.getMailboxExecutor(DEFAULT_PRIORITY).execute(() -> {}, "dummy"); Assert.fail("Should not be able to accept runnables if not opened."); } catch (RejectedExecutionException expected) { } @@ -81,7 +81,7 @@ public void runDefaultAction(DefaultActionContext context) throws Exception { }; MailboxProcessor mailboxProcessor = start(mailboxThread); - mailboxProcessor.getMailboxExecutor(DEFAULT_PRIORITY).execute(() -> stop.set(true)); + mailboxProcessor.getMailboxExecutor(DEFAULT_PRIORITY).execute(() -> stop.set(true), "stop"); stop(mailboxThread); } @@ -130,7 +130,7 @@ public void runDefaultAction(DefaultActionContext context) { Assert.assertEquals(blockAfterInvocations, counter.get()); SuspendedMailboxDefaultAction suspendedMailboxDefaultAction = suspendedActionRef.get(); - mailboxProcessor.getMailboxExecutor(DEFAULT_PRIORITY).execute(suspendedMailboxDefaultAction::resume); + mailboxProcessor.getMailboxExecutor(DEFAULT_PRIORITY).execute(suspendedMailboxDefaultAction::resume, "resume"); stop(mailboxThread); Assert.assertEquals(totalInvocations, counter.get()); } @@ -170,13 +170,12 @@ public void runDefaultAction(DefaultActionContext context) { while (!Thread.currentThread().isInterrupted()) { final SuspendedMailboxDefaultAction resume = - suspendedActionRef.getAndSet(null); + suspendedActionRef.getAndSet(null); if (resume != null) { - mailboxProcessor.getMailboxExecutor(DEFAULT_PRIORITY).execute(resume::resume); + mailboxProcessor.getMailboxExecutor(DEFAULT_PRIORITY).execute(resume::resume, "resume"); } else { try { - mailboxProcessor.getMailboxExecutor(DEFAULT_PRIORITY).execute(() -> { - }); + mailboxProcessor.getMailboxExecutor(DEFAULT_PRIORITY).execute(() -> {}, "dummy"); } catch (RejectedExecutionException ignore) { } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/TestMailboxExecutor.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/TestMailboxExecutor.java deleted file mode 100644 index 979fe58d5c40..000000000000 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/TestMailboxExecutor.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.streaming.runtime.tasks.mailbox.execution; - -import javax.annotation.Nonnull; - -import java.util.concurrent.RejectedExecutionException; - -/** - * Dummy implementation of {@link MailboxExecutor} for testing. - */ -public class TestMailboxExecutor implements MailboxExecutor { - - private final Object lock; - - public TestMailboxExecutor(Object lock) { - this.lock = lock; - } - - public TestMailboxExecutor() { - this(new Object()); - } - - @Override - public void execute(@Nonnull Runnable command) throws RejectedExecutionException { - synchronized (lock) { - command.run(); - lock.notifyAll(); - } - } - - @Override - public void yield() throws InterruptedException { - synchronized (lock) { - lock.wait(1); - } - } - - @Override - public boolean tryYield() { - return false; - } - - @Override - public boolean isMailboxThread() { - return true; - } -} From 529e1ccc68f38cb627c2b0baabcd26c2f218a15e Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Tue, 15 Oct 2019 20:11:35 +0200 Subject: [PATCH 199/746] [FLINK-14199] [runtime] Improving error-reporting when a Mailbox mail failed to execute Promoting Mail to internal class and using it primarily in Mailbox API. Moving priority association entirely to MailboxExecutor. Removing priority views on TaskMailbox - the MailboxExecutor will now put and pull mails with its priority. Consolidating Mailbox hierarchy: Only the Mailbox interface for MailboxExecutor and the TaskMailbox interface for MailProcessor remain. --- .../streaming/runtime/tasks/mailbox/Mail.java | 71 +++++++ .../runtime/tasks/mailbox/Mailbox.java | 43 ++++- .../tasks/mailbox/MailboxReceiver.java | 50 ----- .../runtime/tasks/mailbox/MailboxSender.java | 57 ------ .../runtime/tasks/mailbox/TaskMailbox.java | 77 +------- .../tasks/mailbox/TaskMailboxImpl.java | 86 +-------- .../execution/MailboxExecutorImpl.java | 33 +++- .../mailbox/execution/MailboxProcessor.java | 18 +- .../tasks/mailbox/TaskMailboxImplTest.java | 177 ++++++++---------- .../execution/MailboxExecutorImplTest.java | 10 +- 10 files changed, 239 insertions(+), 383 deletions(-) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/Mail.java delete mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxReceiver.java delete mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxSender.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/Mail.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/Mail.java new file mode 100644 index 000000000000..958db0fc4755 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/Mail.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.runtime.tasks.mailbox; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.util.Preconditions; + +/** + * An executable bound to a specific operator in the chain, such that it can be picked for downstream mailbox. + */ +@Internal +public class Mail { + /** + * The action to execute. + */ + private final Runnable runnable; + /** + * The priority of the mail. The priority does not determine the order, but helps to hide upstream mails from + * downstream processors to avoid live/deadlocks. + */ + private final int priority; + /** + * The description of the mail that is used for debugging and error-reporting. + */ + private final String descriptionFormat; + + private final Object[] descriptionArgs; + + public Mail(Runnable runnable, int priority, String descriptionFormat, Object... descriptionArgs) { + this.runnable = Preconditions.checkNotNull(runnable); + this.priority = priority; + this.descriptionFormat = descriptionFormat == null ? runnable.toString() : descriptionFormat; + this.descriptionArgs = Preconditions.checkNotNull(descriptionArgs); + } + + public int getPriority() { + return priority; + } + + public Runnable getRunnable() { + return runnable; + } + + @Override + public String toString() { + return String.format(descriptionFormat, descriptionArgs); + } + + public void run() { + try { + runnable.run(); + } catch (Exception e) { + throw new IllegalStateException("Cannot process mail " + toString(), e); + } + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/Mailbox.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/Mailbox.java index 365e3588555c..6bc7501121b7 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/Mailbox.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/Mailbox.java @@ -18,13 +18,52 @@ package org.apache.flink.streaming.runtime.tasks.mailbox; +import javax.annotation.Nonnull; + +import java.util.Optional; + /** * A mailbox is basically a queue for inter-thread message exchange in form of {@link Runnable} objects between multiple * producer threads and a single consumer. * - *

This interface combines the {@link MailboxReceiver} and {@link MailboxSender} side without life-cycle methods. + *

This interface combines the {@link Mailbox} and {@link Mailbox} side without life-cycle methods. * * @see TaskMailbox */ -public interface Mailbox extends MailboxReceiver, MailboxSender { +public interface Mailbox { + /** + * Returns an optional with either the oldest letter from the mailbox (head of queue) if the mailbox is not empty or + * an empty optional otherwise. + * + * @return an optional with either the oldest letter from the mailbox (head of queue) if the mailbox is not empty or + * an empty optional otherwise. + * @throws MailboxStateException if mailbox is already closed. + */ + Optional tryTake(int priority) throws MailboxStateException; + + /** + * This method returns the oldest letter from the mailbox (head of queue) or blocks until a letter is available. + * + * @return the oldest letter from the mailbox (head of queue). + * @throws InterruptedException on interruption. + * @throws MailboxStateException if mailbox is already closed. + */ + @Nonnull + Mail take(int priority) throws InterruptedException, MailboxStateException; + + /** + * Enqueues the given letter to the mailbox and blocks until there is capacity for a successful put. + * + * @param mail the mail to enqueue. + * @throws MailboxStateException if the mailbox is quiesced or closed. + */ + void put(Mail mail) throws MailboxStateException; + + /** + * Adds the given action to the head of the mailbox. + * + * @param mail the mail to enqueue. + * @throws MailboxStateException if the mailbox is quiesced or closed. + */ + void putFirst(Mail mail) throws MailboxStateException; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxReceiver.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxReceiver.java deleted file mode 100644 index eb633ba31e1d..000000000000 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxReceiver.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.streaming.runtime.tasks.mailbox; - -import javax.annotation.Nonnull; - -import java.util.Optional; - -/** - * Consumer-facing side of the {@link Mailbox} interface. This is used to dequeue letters. The mailbox returns letters - * in the order by which they were enqueued. A mailbox should only be consumed by one thread at a time. - */ -public interface MailboxReceiver { - - /** - * Returns an optional with either the oldest letter from the mailbox (head of queue) if the mailbox is not empty or - * an empty optional otherwise. - * - * @return an optional with either the oldest letter from the mailbox (head of queue) if the mailbox is not empty or - * an empty optional otherwise. - * @throws MailboxStateException if mailbox is already closed. - */ - Optional tryTakeMail() throws MailboxStateException; - - /** - * This method returns the oldest letter from the mailbox (head of queue) or blocks until a letter is available. - * - * @return the oldest letter from the mailbox (head of queue). - * @throws InterruptedException on interruption. - * @throws MailboxStateException if mailbox is already closed. - */ - @Nonnull - Runnable takeMail() throws InterruptedException, MailboxStateException; -} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxSender.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxSender.java deleted file mode 100644 index bb2ef4664c1e..000000000000 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxSender.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.streaming.runtime.tasks.mailbox; - -import javax.annotation.Nonnull; - -/** - * Producer-facing side of the {@link Mailbox} interface. This is used to enqueue letters. Multiple producers threads - * can put to the same mailbox. - */ -public interface MailboxSender { - - /** - * Enqueues the given letter to the mailbox and blocks until there is capacity for a successful put. - * - *

An optional description can be added to ease debugging and error-reporting. Any object can be passed on which - * {@link Object#toString()} is lazily invoked. In most cases, it should be a {@link String} or - * {@link org.apache.flink.streaming.runtime.tasks.mailbox.LazyString}. If no explicit description is taken, the - * command itself is used and {@code toString()} will be invoked on it. - * - * @param letter the letter to enqueue. - * @param description the optional description for the command that is used for debugging and error-reporting. - * @throws MailboxStateException if the mailbox is quiesced or closed. - */ - void putMail(@Nonnull Runnable letter, Object description) throws MailboxStateException; - - /** - * Adds the given action to the head of the mailbox. - * - *

An optional description can be added to ease debugging and error-reporting. Any object can be passed on which - * {@link Object#toString()} is lazily invoked. In most cases, it should be a {@link String} or - * {@link org.apache.flink.streaming.runtime.tasks.mailbox.LazyString}. If no explicit description is taken, the - * command itself is used and {@code toString()} will be invoked on it. - * - * @param priorityLetter action to enqueue to the head of the mailbox. - * @param description the optional description for the command that is used for debugging and error-reporting. - * @throws MailboxStateException if the mailbox is quiesced or closed. - */ - void putFirst(@Nonnull Runnable priorityLetter, Object description) throws MailboxStateException; - -} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java index 9c166a909736..6238f595b51d 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java @@ -21,7 +21,6 @@ import javax.annotation.Nonnull; import java.util.List; -import java.util.Optional; /** * A task mailbox wraps the basic {@link Mailbox} interface with a lifecycle of open -> (quiesced) -> closed. @@ -30,7 +29,7 @@ * *

Additionally, letters have a priority that can be used to retrieve only relevant letters. */ -public interface TaskMailbox { +public interface TaskMailbox extends Mailbox { /** * The minimal priority for letters. The priority is used when no operator is associated with the letter. */ @@ -71,82 +70,8 @@ enum State { @Nonnull State getState(); - /** - * Returns a mailbox view bound to all mails. - * - * @return the mailbox - */ - Mailbox getMainMailbox(); - - /** - * Returns a mailbox view bound to the given priority. - * - *

Enqueuing letters (e.g., {@link Mailbox#putMail(Runnable, Object description)} and - * {@link Mailbox#putFirst(Runnable, Object description)}) will mark these letters to belong to the bound operator. - * - *

Similarly, only letters from the operator or any downstream operator are retrieved by {@link Mailbox#tryTakeMail()} - * and {@link Mailbox#takeMail()}. - * - *

Note, that the lifecycle of the view is strictly coupled to the lifecycle of this task mailbox. - * - * @param priority the operator to which to bind - * @return the bound mailbox - */ - Mailbox getDownstreamMailbox(int priority); - /** * Returns true if the mailbox contains mail. */ boolean hasMail(); - - /** - * Enqueues the given letter to the mailbox and blocks until there is capacity for a successful put. - * - *

An optional description can be added to ease debugging and error-reporting. Any object can be passed on which - * {@link Object#toString()} is lazily invoked. In most cases, it should be a {@link String} or - * {@link org.apache.flink.streaming.runtime.tasks.mailbox.LazyString}. If no explicit description is taken, the - * command itself is used and {@code toString()} will be invoked on it. - * - * @param letter the letter to enqueue. - * @param priority the priority of the letter. - * @param description the optional description for the command that is used for debugging and error-reporting. - * @throws MailboxStateException if the mailbox is quiesced or closed. - */ - void putMail(@Nonnull Runnable letter, int priority, Object description) throws MailboxStateException; - - /** - * Adds the given action to the head of the mailbox. - * - *

An optional description can be added to ease debugging and error-reporting. Any object can be passed on which - * {@link Object#toString()} is lazily invoked. In most cases, it should be a {@link String} or - * {@link org.apache.flink.streaming.runtime.tasks.mailbox.LazyString}. If no explicit description is taken, the - * command itself is used and {@code toString()} will be invoked on it. - * - * @param priorityLetter action to enqueue to the head of the mailbox. - * @param description the optional description for the command that is used for debugging and error-reporting. - * @throws MailboxStateException if the mailbox is quiesced or closed. - */ - void putFirst(@Nonnull Runnable priorityLetter, Object description) throws MailboxStateException; - - /** - * Returns an optional with either the oldest letter with a minimum priority from the mailbox (head of queue) if the - * mailbox is not empty or an empty optional otherwise. - * - * @param priority the minimum priority of the letter. - * @return an optional with either the oldest letter from the mailbox (head of queue) if the mailbox is not empty or - * an empty optional otherwise. - * @throws MailboxStateException if mailbox is already closed. - */ - Optional tryTakeMail(int priority) throws MailboxStateException; - - /** - * This method returns the oldest letter with a minimum priority from the mailbox (head of queue) or blocks until a - * letter is available. - * - * @param priority the minimum priority of the letter. - * @return the oldest letter from the mailbox (head of queue). - * @throws InterruptedException on interruption. - * @throws MailboxStateException if mailbox is already closed. - */ - @Nonnull Runnable takeMail(int priority) throws InterruptedException, MailboxStateException; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java index e656a1daf6e5..645d57df525e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java @@ -83,7 +83,7 @@ public boolean hasMail() { } @Override - public Optional tryTakeMail(int priority) throws MailboxStateException { + public Optional tryTake(int priority) throws MailboxStateException { final ReentrantLock lock = this.lock; lock.lock(); try { @@ -94,11 +94,11 @@ public Optional tryTakeMail(int priority) throws MailboxStateException } @Override - public @Nonnull Runnable takeMail(int priorty) throws InterruptedException, MailboxStateException { + public @Nonnull Mail take(int priorty) throws InterruptedException, MailboxStateException { final ReentrantLock lock = this.lock; lock.lockInterruptibly(); try { - Runnable headLetter; + Mail headLetter; while ((headLetter = takeHeadInternal(priorty)) == null) { notEmpty.await(); } @@ -111,11 +111,11 @@ public Optional tryTakeMail(int priority) throws MailboxStateException //------------------------------------------------------------------------------------------------------------------ @Override - public void putMail(@Nonnull Runnable letter, int priority, Object description) throws MailboxStateException { + public void put(@Nonnull Mail mail) throws MailboxStateException { final ReentrantLock lock = this.lock; lock.lock(); try { - putTailInternal(new Mail(letter, priority, description)); + putTailInternal(mail); } finally { lock.unlock(); } @@ -124,11 +124,11 @@ public void putMail(@Nonnull Runnable letter, int priority, Object description) //------------------------------------------------------------------------------------------------------------------ @Override - public void putFirst(@Nonnull Runnable priorityLetter, Object description) throws MailboxStateException { + public void putFirst(@Nonnull Mail mail) throws MailboxStateException { final ReentrantLock lock = this.lock; lock.lock(); try { - putHeadInternal(new Mail(priorityLetter, MAX_PRIORITY, description)); + putHeadInternal(mail); } finally { lock.unlock(); } @@ -157,7 +157,7 @@ private void incrementCountAndCheckOverflow() { } @Nullable - private Runnable takeHeadInternal(int priority) throws MailboxStateException { + private Mail takeHeadInternal(int priority) throws MailboxStateException { assert lock.isHeldByCurrentThread(); checkTakeStateConditions(); Iterator iterator = queue.iterator(); @@ -166,7 +166,7 @@ private Runnable takeHeadInternal(int priority) throws MailboxStateException { if (mail.getPriority() >= priority) { --count; iterator.remove(); - return mail.getRunnable(); + return mail; } } return null; @@ -245,72 +245,4 @@ public List close() { public State getState() { return state; } - - @Override - public Mailbox getMainMailbox() { - return new DownstreamMailbox(TaskMailbox.MIN_PRIORITY); - } - - @Override - public Mailbox getDownstreamMailbox(int priority) { - Preconditions.checkArgument(priority >= 0, "The priority of a downstream mailbox should be non-negative"); - return new DownstreamMailbox(priority); - } - - class DownstreamMailbox implements Mailbox { - private final int priority; - - DownstreamMailbox(int priority) { - this.priority = priority; - } - - @Override - public Optional tryTakeMail() throws MailboxStateException { - return TaskMailboxImpl.this.tryTakeMail(priority); - } - - @Nonnull - @Override - public Runnable takeMail() throws InterruptedException, MailboxStateException { - return TaskMailboxImpl.this.takeMail(priority); - } - - @Override - public void putMail(@Nonnull Runnable letter, Object description) throws MailboxStateException { - TaskMailboxImpl.this.putMail(letter, priority, description); - } - - @Override - public void putFirst(@Nonnull Runnable priorityLetter, Object description) throws MailboxStateException { - TaskMailboxImpl.this.putFirst(priorityLetter, description); - } - } - - /** - * An executable bound to a specific operator in the chain, such that it can be picked for downstream mailbox. - */ - static class Mail { - private final Runnable runnable; - private final int priority; - private final Object description; - - public Mail(Runnable runnable, int priority, Object description) { - this.runnable = runnable; - this.priority = priority; - this.description = description == null ? runnable : description; - } - - public int getPriority() { - return priority; - } - - Runnable getRunnable() { - return runnable; - } - - @Override - public String toString() { - return description.toString(); - } - } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java index 9ff150e9d075..66797d6e61f3 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.runtime.tasks.mailbox.execution; +import org.apache.flink.streaming.runtime.tasks.mailbox.Mail; import org.apache.flink.streaming.runtime.tasks.mailbox.Mailbox; import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxStateException; @@ -39,13 +40,16 @@ public class MailboxExecutorImpl implements MailboxExecutor { @Nonnull private final Mailbox mailbox; - public MailboxExecutorImpl(@Nonnull Mailbox mailbox) { - this(mailbox, Thread.currentThread()); + private final int priority; + + public MailboxExecutorImpl(@Nonnull Mailbox mailbox, int priority) { + this(mailbox, Thread.currentThread(), priority); } - public MailboxExecutorImpl(@Nonnull Mailbox mailbox, @Nonnull Thread taskMailboxThread) { + public MailboxExecutorImpl(@Nonnull Mailbox mailbox, @Nonnull Thread taskMailboxThread, int priority) { this.mailbox = mailbox; this.taskMailboxThread = taskMailboxThread; + this.priority = priority; } @Override @@ -54,7 +58,19 @@ public void execute( final String descriptionFormat, final Object... descriptionArgs) { try { - mailbox.putMail(command, description); + mailbox.put(new Mail(command, priority, descriptionFormat, descriptionArgs)); + } catch (MailboxStateException mbex) { + throw new RejectedExecutionException(mbex); + } + } + + @Override + public void executeFirst( + @Nonnull final Runnable command, + final String descriptionFormat, + final Object... descriptionArgs) { + try { + mailbox.putFirst(new Mail(command, priority, descriptionFormat, descriptionArgs)); } catch (MailboxStateException mbex) { throw new RejectedExecutionException(mbex); } @@ -64,8 +80,7 @@ public void execute( public void yield() throws InterruptedException, IllegalStateException { checkIsMailboxThread(); try { - Runnable runnable = mailbox.takeMail(); - runnable.run(); + mailbox.take(priority).run(); } catch (MailboxStateException e) { throw new IllegalStateException("Mailbox can no longer supply runnables for yielding.", e); } @@ -75,9 +90,9 @@ public void yield() throws InterruptedException, IllegalStateException { public boolean tryYield() throws IllegalStateException { checkIsMailboxThread(); try { - Optional runnableOptional = mailbox.tryTakeMail(); - if (runnableOptional.isPresent()) { - runnableOptional.get().run(); + Optional optionalMail = mailbox.tryTake(priority); + if (optionalMail.isPresent()) { + optionalMail.get().run(); return true; } else { return false; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java index 89d29f7c79cc..0dcb8a257046 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java @@ -19,6 +19,7 @@ package org.apache.flink.streaming.runtime.tasks.mailbox.execution; import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.streaming.runtime.tasks.mailbox.Mail; import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxStateException; import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox; import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailboxImpl; @@ -88,7 +89,7 @@ public MailboxProcessor(MailboxDefaultAction mailboxDefaultAction) { this.mailboxDefaultAction = Preconditions.checkNotNull(mailboxDefaultAction); this.mailbox = new TaskMailboxImpl(); this.mailboxThread = Thread.currentThread(); - this.mainMailboxExecutor = new MailboxExecutorImpl(mailbox.getMainMailbox(), mailboxThread); + this.mainMailboxExecutor = new MailboxExecutorImpl(mailbox, mailboxThread, TaskMailbox.MIN_PRIORITY); this.mailboxPoisonLetter = () -> mailboxLoopRunning = false; this.mailboxLoopRunning = true; this.suspendedDefaultAction = null; @@ -106,7 +107,7 @@ public MailboxExecutor getMainMailboxExecutor() { * @param priority */ public MailboxExecutor getMailboxExecutor(int priority) { - return new MailboxExecutorImpl(mailbox.getDownstreamMailbox(priority), mailboxThread); + return new MailboxExecutorImpl(mailbox, mailboxThread, priority); } /** @@ -193,20 +194,15 @@ private boolean processMail(TaskMailbox mailbox) throws MailboxStateException, I // TODO consider batched draining into list and/or limit number of executed letters // Take letters in a non-blockingly and execute them. - Optional maybeLetter; - while (isMailboxLoopRunning() && (maybeLetter = mailbox.tryTakeMail(MIN_PRIORITY)).isPresent()) { - try { - maybeLetter.get().run(); - } catch (Exception e) { - e.printStackTrace(); - } + Optional maybeLetter; + while (isMailboxLoopRunning() && (maybeLetter = mailbox.tryTake(MIN_PRIORITY)).isPresent()) { + maybeLetter.get().run(); } // If the default action is currently not available, we can run a blocking mailbox execution until the default // action becomes available again. while (isDefaultActionUnavailable() && isMailboxLoopRunning()) { - Runnable letter = mailbox.takeMail(MIN_PRIORITY); - letter.run(); + mailbox.take(MIN_PRIORITY).run(); } return isMailboxLoopRunning(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImplTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImplTest.java index d702590958cb..61256122dd71 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImplTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImplTest.java @@ -33,6 +33,8 @@ import java.util.concurrent.CountDownLatch; import java.util.function.Consumer; +import static org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox.MAX_PRIORITY; + /** * Unit tests for {@link TaskMailboxImpl}. */ @@ -59,38 +61,38 @@ public void tearDown() { @Test public void testPutAsHead() throws Exception { - Runnable instanceA = () -> {}; - Runnable instanceB = () -> {}; - Runnable instanceC = () -> {}; - Runnable instanceD = () -> {}; + Mail mailA = new Mail(() -> {}, MAX_PRIORITY, "mailA"); + Mail mailB = new Mail(() -> {}, MAX_PRIORITY, "mailB"); + Mail mailC = new Mail(() -> {}, DEFAULT_PRIORITY, "mailC, DEFAULT_PRIORITY"); + Mail mailD = new Mail(() -> {}, DEFAULT_PRIORITY, "mailD, DEFAULT_PRIORITY"); - taskMailbox.putMail(instanceC, DEFAULT_PRIORITY, "instanceC, DEFAULT_PRIORITY"); - taskMailbox.putFirst(instanceB, "instanceB"); - taskMailbox.putMail(instanceD, DEFAULT_PRIORITY, "instanceD, DEFAULT_PRIORITY"); - taskMailbox.putFirst(instanceA, "instanceA"); + taskMailbox.put(mailC); + taskMailbox.putFirst(mailB); + taskMailbox.put(mailD); + taskMailbox.putFirst(mailA); - Assert.assertSame(instanceA, taskMailbox.takeMail(DEFAULT_PRIORITY)); - Assert.assertSame(instanceB, taskMailbox.takeMail(DEFAULT_PRIORITY)); - Assert.assertSame(instanceC, taskMailbox.takeMail(DEFAULT_PRIORITY)); - Assert.assertSame(instanceD, taskMailbox.takeMail(DEFAULT_PRIORITY)); + Assert.assertSame(mailA, taskMailbox.take(DEFAULT_PRIORITY)); + Assert.assertSame(mailB, taskMailbox.take(DEFAULT_PRIORITY)); + Assert.assertSame(mailC, taskMailbox.take(DEFAULT_PRIORITY)); + Assert.assertSame(mailD, taskMailbox.take(DEFAULT_PRIORITY)); - Assert.assertFalse(taskMailbox.tryTakeMail(DEFAULT_PRIORITY).isPresent()); + Assert.assertFalse(taskMailbox.tryTake(DEFAULT_PRIORITY).isPresent()); } @Test public void testContracts() throws Exception { - final Queue testObjects = new LinkedList<>(); + final Queue testObjects = new LinkedList<>(); Assert.assertFalse(taskMailbox.hasMail()); for (int i = 0; i < 10; ++i) { - Runnable letter = NO_OP; - testObjects.add(letter); - taskMailbox.putMail(letter, DEFAULT_PRIORITY, "letter, DEFAULT_PRIORITY"); + final Mail mail = new Mail(NO_OP, DEFAULT_PRIORITY, "letter, DEFAULT_PRIORITY"); + testObjects.add(mail); + taskMailbox.put(mail); Assert.assertTrue(taskMailbox.hasMail()); } while (!testObjects.isEmpty()) { - Assert.assertEquals(testObjects.remove(), taskMailbox.tryTakeMail(DEFAULT_PRIORITY).get()); + Assert.assertEquals(testObjects.remove(), taskMailbox.take(DEFAULT_PRIORITY)); Assert.assertEquals(!testObjects.isEmpty(), taskMailbox.hasMail()); } } @@ -100,7 +102,7 @@ public void testContracts() throws Exception { */ @Test public void testConcurrentPutTakeBlocking() throws Exception { - testPutTake(Mailbox::takeMail); + testPutTake(mailbox -> mailbox.take(DEFAULT_PRIORITY)); } /** @@ -109,11 +111,11 @@ public void testConcurrentPutTakeBlocking() throws Exception { @Test public void testConcurrentPutTakeNonBlockingAndWait() throws Exception { testPutTake((mailbox -> { - Optional optionalLetter = mailbox.tryTakeMail(); - while (!optionalLetter.isPresent()) { - optionalLetter = mailbox.tryTakeMail(); + Optional optionalMail = mailbox.tryTake(DEFAULT_PRIORITY); + while (!optionalMail.isPresent()) { + optionalMail = mailbox.tryTake(DEFAULT_PRIORITY); } - return optionalLetter.get(); + return optionalMail.get(); })); } @@ -124,7 +126,7 @@ public void testConcurrentPutTakeNonBlockingAndWait() throws Exception { public void testCloseUnblocks() throws InterruptedException { testAllPuttingUnblocksInternal(TaskMailbox::close); setUp(); - testUnblocksInternal(() -> taskMailbox.takeMail(DEFAULT_PRIORITY), TaskMailbox::close); + testUnblocksInternal(() -> taskMailbox.take(DEFAULT_PRIORITY), TaskMailbox::close); } /** @@ -137,13 +139,13 @@ public void testQuiesceUnblocks() throws Exception { @Test public void testLifeCycleQuiesce() throws Exception { - taskMailbox.putMail(NO_OP, DEFAULT_PRIORITY, "NO_OP, DEFAULT_PRIORITY"); - taskMailbox.putMail(NO_OP, DEFAULT_PRIORITY, "NO_OP, DEFAULT_PRIORITY"); + taskMailbox.put(new Mail(NO_OP, DEFAULT_PRIORITY, "NO_OP, DEFAULT_PRIORITY")); + taskMailbox.put(new Mail(NO_OP, DEFAULT_PRIORITY, "NO_OP, DEFAULT_PRIORITY")); taskMailbox.quiesce(); testLifecyclePuttingInternal(); - taskMailbox.takeMail(DEFAULT_PRIORITY); - Assert.assertTrue(taskMailbox.tryTakeMail(DEFAULT_PRIORITY).isPresent()); - Assert.assertFalse(taskMailbox.tryTakeMail(DEFAULT_PRIORITY).isPresent()); + taskMailbox.take(DEFAULT_PRIORITY); + Assert.assertTrue(taskMailbox.tryTake(DEFAULT_PRIORITY).isPresent()); + Assert.assertFalse(taskMailbox.tryTake(DEFAULT_PRIORITY).isPresent()); } @Test @@ -152,13 +154,13 @@ public void testLifeCycleClose() throws Exception { testLifecyclePuttingInternal(); try { - taskMailbox.takeMail(DEFAULT_PRIORITY); + taskMailbox.take(DEFAULT_PRIORITY); Assert.fail(); } catch (MailboxStateException ignore) { } try { - taskMailbox.tryTakeMail(DEFAULT_PRIORITY); + taskMailbox.tryTake(DEFAULT_PRIORITY); Assert.fail(); } catch (MailboxStateException ignore) { } @@ -166,21 +168,23 @@ public void testLifeCycleClose() throws Exception { private void testLifecyclePuttingInternal() throws Exception { try { - taskMailbox.putMail(NO_OP, DEFAULT_PRIORITY, "NO_OP, DEFAULT_PRIORITY"); + taskMailbox.put(new Mail(NO_OP, DEFAULT_PRIORITY, "NO_OP, DEFAULT_PRIORITY")); Assert.fail(); } catch (MailboxStateException ignore) { } try { - taskMailbox.putFirst(NO_OP, "NO_OP"); + taskMailbox.putFirst(new Mail(NO_OP, MAX_PRIORITY, "NO_OP")); Assert.fail(); } catch (MailboxStateException ignore) { } } private void testAllPuttingUnblocksInternal(Consumer unblockMethod) throws InterruptedException { - testUnblocksInternal(() -> taskMailbox.putMail(NO_OP, DEFAULT_PRIORITY, "NO_OP, DEFAULT_PRIORITY"), unblockMethod); + testUnblocksInternal( + () -> taskMailbox.put(new Mail(NO_OP, DEFAULT_PRIORITY, "NO_OP, DEFAULT_PRIORITY")), + unblockMethod); setUp(); - testUnblocksInternal(() -> taskMailbox.putFirst(NO_OP, "NO_OP"), unblockMethod); + testUnblocksInternal(() -> taskMailbox.putFirst(new Mail(NO_OP, MAX_PRIORITY, "NO_OP")), unblockMethod); } private void testUnblocksInternal( @@ -223,16 +227,15 @@ private void testUnblocksInternal( /** * Test producer-consumer pattern through the mailbox in a concurrent setting (n-writer / 1-reader). */ - private void testPutTake(FunctionWithException takeMethod) throws Exception { + private void testPutTake(FunctionWithException takeMethod) throws Exception { final int numThreads = 10; final int numLettersPerThread = 1000; final int[] results = new int[numThreads]; Thread[] writerThreads = new Thread[numThreads]; - Mailbox mailbox = taskMailbox.getDownstreamMailbox(DEFAULT_PRIORITY); Thread readerThread = new Thread(ThrowingRunnable.unchecked(() -> { - Runnable letter; - while ((letter = takeMethod.apply(mailbox)) != POISON_LETTER) { - letter.run(); + Mail mail; + while ((mail = takeMethod.apply(taskMailbox)).getRunnable() != POISON_LETTER) { + mail.run(); } })); @@ -241,7 +244,7 @@ private void testPutTake(FunctionWithException tak final int threadId = i; writerThreads[i] = new Thread(ThrowingRunnable.unchecked(() -> { for (int k = 0; k < numLettersPerThread; ++k) { - mailbox.putMail(() -> ++results[threadId], "result " + k); + taskMailbox.put(new Mail(() -> ++results[threadId], DEFAULT_PRIORITY, "result " + k)); } })); } @@ -254,7 +257,7 @@ private void testPutTake(FunctionWithException tak writerThread.join(); } - taskMailbox.putMail(POISON_LETTER, DEFAULT_PRIORITY, "POISON_LETTER, DEFAULT_PRIORITY"); + taskMailbox.put(new Mail(POISON_LETTER, DEFAULT_PRIORITY, "POISON_LETTER, DEFAULT_PRIORITY")); readerThread.join(); for (int perThreadResult : results) { @@ -262,66 +265,46 @@ private void testPutTake(FunctionWithException tak } } - /** - * Tests the downstream view of {@link TaskMailbox}. - */ - public static class DownstreamMailboxTest { - /** - * Object under test. - */ - private TaskMailboxImpl taskMailbox; - - @Before - public void setUp() { - taskMailbox = new TaskMailboxImpl(); - } - - @After - public void tearDown() { - taskMailbox.close(); - } - - @Test - public void testPutAsHeadInDownstream() throws Exception { - - Runnable instanceA = () -> {}; - Runnable instanceB = () -> {}; - Runnable instanceC = () -> {}; - Runnable instanceD = () -> {}; - - taskMailbox.getDownstreamMailbox(1).putMail(instanceC, "instanceC"); - taskMailbox.getDownstreamMailbox(2).putMail(instanceB, "instanceB"); - taskMailbox.getDownstreamMailbox(1).putMail(instanceD, "instanceD"); - taskMailbox.getDownstreamMailbox(2).putFirst(instanceA, "instanceA"); - - Assert.assertSame(instanceA, taskMailbox.getDownstreamMailbox(2).takeMail()); - Assert.assertSame(instanceB, taskMailbox.getDownstreamMailbox(2).takeMail()); - Assert.assertFalse(taskMailbox.getDownstreamMailbox(2).tryTakeMail().isPresent()); + @Test + public void testPutAsHeadWithPriority() throws Exception { - Assert.assertSame(instanceC, taskMailbox.getDownstreamMailbox(1).takeMail()); - Assert.assertSame(instanceD, taskMailbox.getDownstreamMailbox(1).takeMail()); + Mail mailA = new Mail(() -> {}, 2, "mailA"); + Mail mailB = new Mail(() -> {}, 2, "mailB"); + Mail mailC = new Mail(() -> {}, 1, "mailC"); + Mail mailD = new Mail(() -> {}, 1, "mailD"); - Assert.assertFalse(taskMailbox.getDownstreamMailbox(1).tryTakeMail().isPresent()); - } + taskMailbox.put(mailC); + taskMailbox.put(mailB); + taskMailbox.put(mailD); + taskMailbox.putFirst(mailA); - @Test - public void testPutInDownstreamAndReadingFromTaskMailbox() throws Exception { + Assert.assertSame(mailA, taskMailbox.take(2)); + Assert.assertSame(mailB, taskMailbox.take(2)); + Assert.assertFalse(taskMailbox.tryTake(2).isPresent()); - Runnable instanceA = () -> {}; - Runnable instanceB = () -> {}; - Runnable instanceC = () -> {}; - Runnable instanceD = () -> {}; + Assert.assertSame(mailC, taskMailbox.take(1)); + Assert.assertSame(mailD, taskMailbox.take(1)); - taskMailbox.getDownstreamMailbox(1).putMail(instanceC, "instanceC"); - taskMailbox.getDownstreamMailbox(2).putMail(instanceB, "instanceB"); - taskMailbox.getDownstreamMailbox(1).putMail(instanceD, "instanceD"); - taskMailbox.getDownstreamMailbox(2).putFirst(instanceA, "instanceA"); + Assert.assertFalse(taskMailbox.tryTake(1).isPresent()); + } - // same order for non-priority and priority on top - Assert.assertSame(instanceA, taskMailbox.takeMail(TaskMailbox.MIN_PRIORITY)); - Assert.assertSame(instanceC, taskMailbox.takeMail(TaskMailbox.MIN_PRIORITY)); - Assert.assertSame(instanceB, taskMailbox.takeMail(TaskMailbox.MIN_PRIORITY)); - Assert.assertSame(instanceD, taskMailbox.takeMail(TaskMailbox.MIN_PRIORITY)); - } + @Test + public void testPutWithPriorityAndReadingFromMainMailbox() throws Exception { + + Mail mailA = new Mail(() -> {}, 2, "mailA"); + Mail mailB = new Mail(() -> {}, 2, "mailB"); + Mail mailC = new Mail(() -> {}, 1, "mailC"); + Mail mailD = new Mail(() -> {}, 1, "mailD"); + + taskMailbox.put(mailC); + taskMailbox.put(mailB); + taskMailbox.put(mailD); + taskMailbox.putFirst(mailA); + + // same order for non-priority and priority on top + Assert.assertSame(mailA, taskMailbox.take(TaskMailbox.MIN_PRIORITY)); + Assert.assertSame(mailC, taskMailbox.take(TaskMailbox.MIN_PRIORITY)); + Assert.assertSame(mailB, taskMailbox.take(TaskMailbox.MIN_PRIORITY)); + Assert.assertSame(mailD, taskMailbox.take(TaskMailbox.MIN_PRIORITY)); } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImplTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImplTest.java index 087407200201..023c56ad5e64 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImplTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImplTest.java @@ -51,7 +51,7 @@ public class MailboxExecutorImplTest { @Before public void setUp() throws Exception { this.mailbox = new TaskMailboxImpl(); - this.mailboxExecutor = new MailboxExecutorImpl(mailbox.getDownstreamMailbox(DEFAULT_PRIORITY)); + this.mailboxExecutor = new MailboxExecutorImpl(mailbox, DEFAULT_PRIORITY); this.otherThreadExecutor = Executors.newSingleThreadScheduledExecutor(); } @@ -75,11 +75,13 @@ public void tearDown() { public void testOperations() throws Exception { final TestRunnable testRunnable = new TestRunnable(); mailboxExecutor.execute(testRunnable, "testRunnable"); - Assert.assertEquals(testRunnable, mailbox.tryTakeMail(DEFAULT_PRIORITY).get()); + Assert.assertEquals(testRunnable, mailbox.take(DEFAULT_PRIORITY).getRunnable()); + CompletableFuture.runAsync( () -> mailboxExecutor.execute(testRunnable, "testRunnable"), otherThreadExecutor).get(); - Assert.assertEquals(testRunnable, mailbox.takeMail(DEFAULT_PRIORITY)); + Assert.assertEquals(testRunnable, mailbox.take(DEFAULT_PRIORITY).getRunnable()); + final TestRunnable yieldRun = new TestRunnable(); final TestRunnable leftoverRun = new TestRunnable(); mailboxExecutor.execute(yieldRun, "yieldRun"); @@ -118,7 +120,7 @@ public void testTryYield() throws Exception { otherThreadExecutor) .get(); assertTrue(mailboxExecutor.tryYield()); - assertFalse(mailbox.tryTakeMail(DEFAULT_PRIORITY).isPresent()); + assertFalse(mailboxExecutor.tryYield()); Assert.assertEquals(Thread.currentThread(), testRunnable.wasExecutedBy()); } From 070fae5453b37a0d1eddbc80f357a49a03791e73 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Tue, 22 Oct 2019 16:13:39 +0200 Subject: [PATCH 200/746] [FLINK-14199] [runtime] Improving the performance of named mails by adding overloaded submission methods to MailboxExecutor.java that do not create a new array for empty arguments. --- .../mailbox/execution/MailboxExecutor.java | 64 ++++++++++++++++++- .../execution/MailboxExecutorImpl.java | 2 +- 2 files changed, 64 insertions(+), 2 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutor.java index f15bc2dc03f4..4b54eceefee5 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutor.java @@ -33,6 +33,26 @@ * Interface for an {@link Executor} build around a {@link Mailbox}-based execution model. */ public interface MailboxExecutor { + /** + * A constant for empty args to save on object allocation. + */ + Object[] EMPTY_ARGS = new Object[0]; + + /** + * Executes the given command at some time in the future in the mailbox thread. + * + *

An optional description can (and should) be added to ease debugging and error-reporting. The description + * may contain placeholder that refer to the provided description arguments using {@link java.util.Formatter} + * syntax. The actual description is only formatted on demand. + * + * @param command the runnable task to add to the mailbox for execution. + * @param description the optional description for the command that is used for debugging and error-reporting. + * @throws RejectedExecutionException if this task cannot be accepted for execution, e.g. because the mailbox is + * quiesced or closed. + */ + default void execute(@Nonnull Runnable command, String description) { + execute(command, description, EMPTY_ARGS); + } /** * Executes the given command at some time in the future in the mailbox thread. @@ -81,7 +101,29 @@ public interface MailboxExecutor { * quiesced or closed. */ default @Nonnull Future submit(@Nonnull Runnable command, String descriptionFormat, Object... descriptionArgs) { - return submit(Executors.callable(command, null), descriptionFormat, descriptionArgs); + FutureTask future = new FutureTask<>(Executors.callable(command, null)); + execute(future, descriptionFormat, descriptionArgs); + return future; + } + + /** + * Submits the given command for execution in the future in the mailbox thread and returns a Future representing + * that command. The Future's {@code get} method will return {@code null} upon successful completion. + * + *

An optional description can (and should) be added to ease debugging and error-reporting. The description + * may contain placeholder that refer to the provided description arguments using {@link java.util.Formatter} + * syntax. The actual description is only formatted on demand. + * + * @param command the command to submit + * @param description the optional description for the command that is used for debugging and error-reporting. + * @return a Future representing pending completion of the task + * @throws RejectedExecutionException if this task cannot be accepted for execution, e.g. because the mailbox is + * quiesced or closed. + */ + default @Nonnull Future submit(@Nonnull Runnable command, String description) { + FutureTask future = new FutureTask<>(Executors.callable(command, null)); + execute(future, description, EMPTY_ARGS); + return future; } /** @@ -105,6 +147,26 @@ public interface MailboxExecutor { return future; } + /** + * Submits the given command for execution in the future in the mailbox thread and returns a Future representing + * that command. The Future's {@code get} method will return {@code null} upon successful completion. + * + *

An optional description can (and should) be added to ease debugging and error-reporting. The description + * may contain placeholder that refer to the provided description arguments using {@link java.util.Formatter} + * syntax. The actual description is only formatted on demand. + * + * @param command the command to submit + * @param description the optional description for the command that is used for debugging and error-reporting. + * @return a Future representing pending completion of the task + * @throws RejectedExecutionException if this task cannot be accepted for execution, e.g. because the mailbox is + * quiesced or closed. + */ + default @Nonnull Future submit(@Nonnull Callable command, String description) { + FutureTask future = new FutureTask<>(command); + execute(future, description, EMPTY_ARGS); + return future; + } + /** * This methods starts running the command at the head of the mailbox and is intended to be used by the mailbox * thread to yield from a currently ongoing action to another command. The method blocks until another command to diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java index 66797d6e61f3..f44bb6bac5f3 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java @@ -30,7 +30,7 @@ /** * Implementation of an executor service build around a mailbox-based execution model. */ -public class MailboxExecutorImpl implements MailboxExecutor { +public final class MailboxExecutorImpl implements MailboxExecutor { /** Reference to the thread that executes the mailbox letters. */ @Nonnull From bfdab9f9e1d245622601daca68550fd382e3f74f Mon Sep 17 00:00:00 2001 From: David Hrbacek Date: Tue, 24 Sep 2019 15:10:36 +0200 Subject: [PATCH 201/746] [FLINK-14169][history-server] Refactor HistoryServerTest for reusability --- .../webmonitor/history/HistoryServerTest.java | 70 ++++++++++++------- 1 file changed, 43 insertions(+), 27 deletions(-) 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 75113000379f..53f8086df09a 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 @@ -41,7 +41,7 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; -import org.junit.ClassRule; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TemporaryFolder; @@ -68,12 +68,13 @@ @Category(AlsoRunWithSchedulerNG.class) public class HistoryServerTest extends TestLogger { - @ClassRule - public static final TemporaryFolder TMP = new TemporaryFolder(); - private static final JsonFactory JACKSON_FACTORY = new JsonFactory() .enable(JsonGenerator.Feature.AUTO_CLOSE_TARGET) .disable(JsonGenerator.Feature.AUTO_CLOSE_JSON_CONTENT); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + @Rule + public final TemporaryFolder tmpFolder = new TemporaryFolder(); private MiniClusterWithClientResource cluster; private File jmDirectory; @@ -89,8 +90,8 @@ public static Collection parameters() { @Before public void setUp() throws Exception { - jmDirectory = TMP.newFolder("jm_" + versionLessThan14); - hsDirectory = TMP.newFolder("hs_" + versionLessThan14); + jmDirectory = tmpFolder.newFolder("jm_" + versionLessThan14); + hsDirectory = tmpFolder.newFolder("hs_" + versionLessThan14); Configuration clusterConfig = new Configuration(); clusterConfig.setString(JobManagerOptions.ARCHIVE_DIR, jmDirectory.toURI().toString()); @@ -117,40 +118,53 @@ public void testHistoryServerIntegration() throws Exception { for (int x = 0; x < numJobs; x++) { runJob(); } + final int numLegacyJobs = 1; createLegacyArchive(jmDirectory.toPath()); - CountDownLatch numExpectedArchivedJobs = new CountDownLatch(numJobs + 1); - - Configuration historyServerConfig = new Configuration(); - historyServerConfig.setString(HistoryServerOptions.HISTORY_SERVER_ARCHIVE_DIRS, jmDirectory.toURI().toString()); - historyServerConfig.setString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR, hsDirectory.getAbsolutePath()); - historyServerConfig.setLong(HistoryServerOptions.HISTORY_SERVER_ARCHIVE_REFRESH_INTERVAL, 100L); + waitForArchivesCreation(numJobs + numLegacyJobs); - historyServerConfig.setInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT, 0); + CountDownLatch numExpectedArchivedJobs = new CountDownLatch(numJobs + numLegacyJobs); - // the job is archived asynchronously after env.execute() returns - File[] archives = jmDirectory.listFiles(); - while (archives == null || archives.length != numJobs + 1) { - Thread.sleep(50); - archives = jmDirectory.listFiles(); - } + Configuration historyServerConfig = createTestConfiguration(); HistoryServer hs = new HistoryServer(historyServerConfig, numExpectedArchivedJobs); + try { hs.start(); String baseUrl = "http://localhost:" + hs.getWebPort(); numExpectedArchivedJobs.await(10L, TimeUnit.SECONDS); - ObjectMapper mapper = new ObjectMapper(); - String response = getFromHTTP(baseUrl + JobsOverviewHeaders.URL); - MultipleJobsDetails overview = mapper.readValue(response, MultipleJobsDetails.class); + Assert.assertEquals(numJobs + numLegacyJobs, getJobsOverview(baseUrl).getJobs().size()); - Assert.assertEquals(numJobs + 1, overview.getJobs().size()); } finally { hs.stop(); } } + private void waitForArchivesCreation(int numJobs) throws InterruptedException { + // the job is archived asynchronously after env.execute() returns + File[] archives = jmDirectory.listFiles(); + while (archives == null || archives.length != numJobs) { + Thread.sleep(50); + archives = jmDirectory.listFiles(); + } + } + + private Configuration createTestConfiguration() { + Configuration historyServerConfig = new Configuration(); + historyServerConfig.setString(HistoryServerOptions.HISTORY_SERVER_ARCHIVE_DIRS, jmDirectory.toURI().toString()); + historyServerConfig.setString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR, hsDirectory.getAbsolutePath()); + historyServerConfig.setLong(HistoryServerOptions.HISTORY_SERVER_ARCHIVE_REFRESH_INTERVAL, 100L); + + historyServerConfig.setInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT, 0); + return historyServerConfig; + } + + private MultipleJobsDetails getJobsOverview(String baseUrl) throws Exception { + String response = getFromHTTP(baseUrl + JobsOverviewHeaders.URL); + return OBJECT_MAPPER.readValue(response, MultipleJobsDetails.class); + } + private static void runJob() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.fromElements(1, 2, 3).addSink(new DiscardingSink<>()); @@ -174,15 +188,15 @@ public static String getFromHTTP(String url) throws Exception { return IOUtils.toString(is, connection.getContentEncoding() != null ? connection.getContentEncoding() : "UTF-8"); } - private static void createLegacyArchive(Path directory) throws IOException { - JobID jobID = JobID.generate(); + private static String createLegacyArchive(Path directory) throws IOException { + JobID jobId = JobID.generate(); StringWriter sw = new StringWriter(); try (JsonGenerator gen = JACKSON_FACTORY.createGenerator(sw)) { try (JsonObject root = new JsonObject(gen)) { try (JsonArray finished = new JsonArray(gen, "finished")) { try (JsonObject job = new JsonObject(gen)) { - gen.writeStringField("jid", jobID.toString()); + gen.writeStringField("jid", jobId.toString()); gen.writeStringField("name", "testjob"); gen.writeStringField("state", JobStatus.FINISHED.name()); @@ -215,7 +229,9 @@ private static void createLegacyArchive(Path directory) throws IOException { ArchivedJson archivedJson = new ArchivedJson("/joboverview", json); - FsJobArchivist.archiveJob(new org.apache.flink.core.fs.Path(directory.toUri()), jobID, Collections.singleton(archivedJson)); + FsJobArchivist.archiveJob(new org.apache.flink.core.fs.Path(directory.toUri()), jobId, Collections.singleton(archivedJson)); + + return jobId.toString(); } private static final class JsonObject implements AutoCloseable { From 64fd87295501f5aeece8059d4d52da7b3291a608 Mon Sep 17 00:00:00 2001 From: David Hrbacek Date: Tue, 24 Sep 2019 15:10:36 +0200 Subject: [PATCH 202/746] [FLINK-14169][history-server] Cleanup expired jobs from history server - added logic for removing deleted job archives from history server --- .../history_server_configuration.html | 5 + .../configuration/HistoryServerOptions.java | 10 ++ .../webmonitor/history/HistoryServer.java | 22 ++++- .../history/HistoryServerArchiveFetcher.java | 96 ++++++++++++++++--- .../webmonitor/history/HistoryServerTest.java | 86 ++++++++++++++++- 5 files changed, 196 insertions(+), 23 deletions(-) diff --git a/docs/_includes/generated/history_server_configuration.html b/docs/_includes/generated/history_server_configuration.html index 7691c0abe62a..f35257084291 100644 --- a/docs/_includes/generated/history_server_configuration.html +++ b/docs/_includes/generated/history_server_configuration.html @@ -7,6 +7,11 @@ + +

historyserver.archive.clean-expired-jobs
+ false + Whether HistoryServer should cleanup jobs that are no longer present `historyserver.archive.fs.dir`. +
historyserver.archive.fs.dir
(none) diff --git a/flink-core/src/main/java/org/apache/flink/configuration/HistoryServerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/HistoryServerOptions.java index a4df23ce1486..19db864fcac4 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/HistoryServerOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/HistoryServerOptions.java @@ -46,6 +46,16 @@ public class HistoryServerOptions { " monitor these directories for archived jobs. You can configure the JobManager to archive jobs to a" + " directory via `jobmanager.archive.fs.dir`."); + /** + * If this option is enabled then deleted job archives are also deleted from HistoryServer. + */ + public static final ConfigOption HISTORY_SERVER_CLEANUP_EXPIRED_JOBS = + key("historyserver.archive.clean-expired-jobs") + .defaultValue(false) + .withDescription( + String.format("Whether HistoryServer should cleanup jobs" + + " that are no longer present `%s`.", HISTORY_SERVER_ARCHIVE_DIRS.key())); + /** * The local directory used by the HistoryServer web-frontend. */ diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java index 46bccb8ff0e2..dedccdc121d5 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServer.java @@ -62,6 +62,7 @@ import java.util.concurrent.Callable; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Consumer; /** * The HistoryServer provides a WebInterface and REST API to retrieve information about finished jobs for which @@ -134,12 +135,23 @@ public Integer call() throws Exception { } public HistoryServer(Configuration config) throws IOException, FlinkException { - this(config, new CountDownLatch(0)); + this(config, (event) -> {}); } - public HistoryServer(Configuration config, CountDownLatch numArchivedJobs) throws IOException, FlinkException { + /** + * Creates HistoryServer instance. + * @param config configuration + * @param jobArchiveEventListener Listener for job archive operations. First param is operation, second + * param is id of the job. + * @throws IOException When creation of SSL factory failed + * @throws FlinkException When configuration error occurred + */ + public HistoryServer( + Configuration config, + Consumer jobArchiveEventListener + ) throws IOException, FlinkException { Preconditions.checkNotNull(config); - Preconditions.checkNotNull(numArchivedJobs); + Preconditions.checkNotNull(jobArchiveEventListener); this.config = config; if (HistoryServerUtils.isSSLEnabled(config)) { @@ -163,6 +175,8 @@ public HistoryServer(Configuration config, CountDownLatch numArchivedJobs) throw } webDir = new File(webDirectory); + boolean cleanupExpiredArchives = config.getBoolean(HistoryServerOptions.HISTORY_SERVER_CLEANUP_EXPIRED_JOBS); + String refreshDirectories = config.getString(HistoryServerOptions.HISTORY_SERVER_ARCHIVE_DIRS); if (refreshDirectories == null) { throw new FlinkException(HistoryServerOptions.HISTORY_SERVER_ARCHIVE_DIRS + " was not configured."); @@ -184,7 +198,7 @@ public HistoryServer(Configuration config, CountDownLatch numArchivedJobs) throw } long refreshIntervalMillis = config.getLong(HistoryServerOptions.HISTORY_SERVER_ARCHIVE_REFRESH_INTERVAL); - archiveFetcher = new HistoryServerArchiveFetcher(refreshIntervalMillis, refreshDirs, webDir, numArchivedJobs); + archiveFetcher = new HistoryServerArchiveFetcher(refreshIntervalMillis, refreshDirs, webDir, jobArchiveEventListener, cleanupExpiredArchives); this.shutdownHook = ShutdownHookUtil.addShutdownHook( HistoryServer.this::stop, 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 79add41fba51..bd85e8945e8e 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 @@ -53,10 +53,10 @@ import java.util.List; import java.util.Set; import java.util.TimerTask; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -69,6 +69,37 @@ */ class HistoryServerArchiveFetcher { + /** + * Possible job archive operations in history-server. + */ + public enum ArchiveEventType { + /** Job archive was found in one refresh location and created in history server. */ + CREATED, + /** Job archive was deleted from one of refresh locations and deleted from history server.*/ + DELETED + } + + /** + * Representation of job archive event. + */ + public static class ArchiveEvent { + private final String jobID; + private final ArchiveEventType operation; + + ArchiveEvent(String jobID, ArchiveEventType operation) { + this.jobID = jobID; + this.operation = operation; + } + + public String getJobID() { + return jobID; + } + + public ArchiveEventType getType() { + return operation; + } + } + private static final Logger LOG = LoggerFactory.getLogger(HistoryServerArchiveFetcher.class); private static final JsonFactory jacksonFactory = new JsonFactory(); @@ -79,9 +110,15 @@ class HistoryServerArchiveFetcher { private final JobArchiveFetcherTask fetcherTask; private final long refreshIntervalMillis; - HistoryServerArchiveFetcher(long refreshIntervalMillis, List refreshDirs, File webDir, CountDownLatch numArchivedJobs) { + HistoryServerArchiveFetcher( + long refreshIntervalMillis, + List refreshDirs, + File webDir, + Consumer jobArchiveEventListener, + boolean cleanupExpiredArchives + ) { this.refreshIntervalMillis = refreshIntervalMillis; - this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, webDir, numArchivedJobs); + this.fetcherTask = new JobArchiveFetcherTask(refreshDirs, webDir, jobArchiveEventListener, cleanupExpiredArchives); if (LOG.isInfoEnabled()) { for (HistoryServer.RefreshLocation refreshDir : refreshDirs) { LOG.info("Monitoring directory {} for archived jobs.", refreshDir.getPath()); @@ -112,7 +149,8 @@ void stop() { static class JobArchiveFetcherTask extends TimerTask { private final List refreshDirs; - private final CountDownLatch numArchivedJobs; + private final Consumer jobArchiveEventListener; + private final boolean processArchiveDeletion; /** Cache of all available jobs identified by their id. */ private final Set cachedArchives; @@ -123,9 +161,15 @@ static class JobArchiveFetcherTask extends TimerTask { private static final String JSON_FILE_ENDING = ".json"; - JobArchiveFetcherTask(List refreshDirs, File webDir, CountDownLatch numArchivedJobs) { + JobArchiveFetcherTask( + List refreshDirs, + File webDir, + Consumer jobArchiveEventListener, + boolean processArchiveDeletion + ) { this.refreshDirs = checkNotNull(refreshDirs); - this.numArchivedJobs = numArchivedJobs; + this.jobArchiveEventListener = jobArchiveEventListener; + this.processArchiveDeletion = processArchiveDeletion; this.cachedArchives = new HashSet<>(); this.webDir = checkNotNull(webDir); this.webJobDir = new File(webDir, "jobs"); @@ -137,6 +181,8 @@ static class JobArchiveFetcherTask extends TimerTask { @Override public void run() { try { + List events = new ArrayList<>(); + Set jobsToRemove = new HashSet<>(cachedArchives); for (HistoryServer.RefreshLocation refreshLocation : refreshDirs) { Path refreshDir = refreshLocation.getPath(); FileSystem refreshFS = refreshLocation.getFs(); @@ -152,7 +198,6 @@ public void run() { if (jobArchives == null) { continue; } - int numFetchedArchives = 0; for (FileStatus jobArchive : jobArchives) { Path jobArchivePath = jobArchive.getPath(); String jobID = jobArchivePath.getName(); @@ -163,6 +208,7 @@ public void run() { refreshDir, jobID, iae); continue; } + jobsToRemove.remove(jobID); if (!cachedArchives.contains(jobID)) { try { for (ArchivedJson archive : FsJobArchivist.getArchivedJsons(jobArchive.getPath())) { @@ -200,8 +246,8 @@ public void run() { fw.flush(); } } + events.add(new ArchiveEvent(jobID, ArchiveEventType.CREATED)); cachedArchives.add(jobID); - numFetchedArchives++; } catch (IOException e) { LOG.error("Failure while fetching/processing job archive for job {}.", jobID, e); // Make sure we do not include this job in the overview @@ -221,17 +267,39 @@ public void run() { } } } - if (numFetchedArchives > 0) { - updateJobOverview(webOverviewDir, webDir); - for (int x = 0; x < numFetchedArchives; x++) { - numArchivedJobs.countDown(); - } - } } + + if (!jobsToRemove.isEmpty() && processArchiveDeletion) { + events.addAll(cleanupExpiredJobs(jobsToRemove)); + } + if (!events.isEmpty()) { + updateJobOverview(webOverviewDir, webDir); + } + events.forEach(jobArchiveEventListener::accept); } catch (Exception e) { LOG.error("Critical failure while fetching/processing job archives.", e); } } + + private List cleanupExpiredJobs(Set jobsToRemove) { + + List deleteLog = new ArrayList<>(); + LOG.info("Archive directories for jobs {} were deleted.", jobsToRemove); + + cachedArchives.removeAll(jobsToRemove); + jobsToRemove.forEach(removedJobID -> { + try { + Files.deleteIfExists(new File(webOverviewDir, removedJobID + JSON_FILE_ENDING).toPath()); + FileUtils.deleteDirectory(new File(webJobDir, removedJobID)); + } catch (IOException e) { + LOG.error("Failure while removing job overview for job {}.", removedJobID, e); + } + deleteLog.add(new ArchiveEvent(removedJobID, ArchiveEventType.DELETED)); + }); + + return deleteLog; + } + } private static String convertLegacyJobOverview(String legacyOverview) throws IOException { 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 53f8086df09a..f4ea68351b86 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 @@ -24,6 +24,7 @@ import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.runtime.history.FsJobArchivist; import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.messages.webmonitor.JobDetails; import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails; import org.apache.flink.runtime.rest.messages.JobsOverviewHeaders; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; @@ -54,6 +55,7 @@ import java.io.StringWriter; import java.net.HttpURLConnection; import java.net.URL; +import java.nio.file.Files; import java.nio.file.Path; import java.util.Arrays; import java.util.Collection; @@ -125,9 +127,13 @@ public void testHistoryServerIntegration() throws Exception { CountDownLatch numExpectedArchivedJobs = new CountDownLatch(numJobs + numLegacyJobs); - Configuration historyServerConfig = createTestConfiguration(); + Configuration historyServerConfig = createTestConfiguration(false); - HistoryServer hs = new HistoryServer(historyServerConfig, numExpectedArchivedJobs); + HistoryServer hs = new HistoryServer(historyServerConfig, (event) -> { + if (event.getType() == HistoryServerArchiveFetcher.ArchiveEventType.CREATED) { + numExpectedArchivedJobs.countDown(); + } + }); try { hs.start(); @@ -135,7 +141,75 @@ public void testHistoryServerIntegration() throws Exception { numExpectedArchivedJobs.await(10L, TimeUnit.SECONDS); Assert.assertEquals(numJobs + numLegacyJobs, getJobsOverview(baseUrl).getJobs().size()); + } finally { + hs.stop(); + } + } + + @Test + public void testCleanExpiredJob() throws Exception { + runArchiveExpirationTest(true); + } + + @Test + public void testRemainExpiredJob() throws Exception { + runArchiveExpirationTest(false); + } + + private void runArchiveExpirationTest(boolean cleanupExpiredJobs) throws Exception { + int numExpiredJobs = cleanupExpiredJobs ? 1 : 0; + int numJobs = 3; + for (int x = 0; x < numJobs; x++) { + runJob(); + } + waitForArchivesCreation(numJobs); + + CountDownLatch numExpectedArchivedJobs = new CountDownLatch(numJobs); + CountDownLatch numExpectedExpiredJobs = new CountDownLatch(numExpiredJobs); + + Configuration historyServerConfig = createTestConfiguration(cleanupExpiredJobs); + + HistoryServer hs = + new HistoryServer( + historyServerConfig, + (event) -> { + switch (event.getType()){ + case CREATED: + numExpectedArchivedJobs.countDown(); + break; + case DELETED: + numExpectedExpiredJobs.countDown(); + break; + } + }); + try { + hs.start(); + String baseUrl = "http://localhost:" + hs.getWebPort(); + numExpectedArchivedJobs.await(10L, TimeUnit.SECONDS); + + Collection jobs = getJobsOverview(baseUrl).getJobs(); + Assert.assertEquals(numJobs, jobs.size()); + + String jobIdToDelete = jobs.stream() + .findFirst() + .map(JobDetails::getJobId) + .map(JobID::toString) + .orElseThrow(() -> new IllegalStateException("Expected at least one existing job")); + + // delete one archive from jm + Files.deleteIfExists(jmDirectory.toPath().resolve(jobIdToDelete)); + + numExpectedExpiredJobs.await(10L, TimeUnit.SECONDS); + + // check that archive is present in hs + Collection jobsAfterDeletion = getJobsOverview(baseUrl).getJobs(); + Assert.assertEquals(numJobs - numExpiredJobs, jobsAfterDeletion.size()); + Assert.assertEquals(1 - numExpiredJobs, jobsAfterDeletion.stream() + .map(JobDetails::getJobId) + .map(JobID::toString) + .filter(jobId -> jobId.equals(jobIdToDelete)) + .count()); } finally { hs.stop(); } @@ -150,17 +224,19 @@ private void waitForArchivesCreation(int numJobs) throws InterruptedException { } } - private Configuration createTestConfiguration() { + private Configuration createTestConfiguration(boolean cleanupExpiredJobs) { Configuration historyServerConfig = new Configuration(); historyServerConfig.setString(HistoryServerOptions.HISTORY_SERVER_ARCHIVE_DIRS, jmDirectory.toURI().toString()); historyServerConfig.setString(HistoryServerOptions.HISTORY_SERVER_WEB_DIR, hsDirectory.getAbsolutePath()); historyServerConfig.setLong(HistoryServerOptions.HISTORY_SERVER_ARCHIVE_REFRESH_INTERVAL, 100L); + historyServerConfig.setBoolean(HistoryServerOptions.HISTORY_SERVER_CLEANUP_EXPIRED_JOBS, cleanupExpiredJobs); + historyServerConfig.setInteger(HistoryServerOptions.HISTORY_SERVER_WEB_PORT, 0); return historyServerConfig; } - private MultipleJobsDetails getJobsOverview(String baseUrl) throws Exception { + private static MultipleJobsDetails getJobsOverview(String baseUrl) throws Exception { String response = getFromHTTP(baseUrl + JobsOverviewHeaders.URL); return OBJECT_MAPPER.readValue(response, MultipleJobsDetails.class); } @@ -172,7 +248,7 @@ private static void runJob() throws Exception { env.execute(); } - public static String getFromHTTP(String url) throws Exception { + static String getFromHTTP(String url) throws Exception { URL u = new URL(url); HttpURLConnection connection = (HttpURLConnection) u.openConnection(); connection.setConnectTimeout(100000); From 518f16c33ee39959670f31ebf662ce1eb241b1a7 Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Wed, 16 Oct 2019 17:05:42 -0700 Subject: [PATCH 203/746] [FLINK-14416][table] Add Module interface and ModuleManager Add Module interface and ModuleManager to Flink SQL. This closes #9937. --- .../pyflink/table/table_environment.py | 9 ++ .../tests/test_environment_completeness.py | 9 +- .../gateway/local/ExecutionContext.java | 6 +- .../table/api/java/BatchTableEnvironment.java | 6 +- .../internal/StreamTableEnvironmentImpl.java | 9 +- .../StreamTableEnvironmentImplTest.java | 5 +- .../flink/table/api/TableEnvironment.java | 27 ++++ .../api/internal/TableEnvironmentImpl.java | 26 +++- .../flink/table/catalog/FunctionCatalog.java | 5 +- .../flink/table/module/ModuleManager.java | 118 ++++++++++++++++++ .../table/catalog/FunctionCatalogTest.java | 4 +- .../table/utils/TableEnvironmentMock.java | 12 +- .../api/scala/BatchTableEnvironment.scala | 7 +- .../internal/StreamTableEnvironmentImpl.scala | 9 +- .../StreamTableEnvironmentImplTest.scala | 8 +- .../org/apache/flink/table/module/Module.java | 55 ++++++++ .../ModuleAlreadyExistException.java | 28 +++++ .../exceptions/ModuleNotFoundException.java | 28 +++++ .../sqlexec/SqlToOperationConverterTest.java | 4 +- .../AggCallSelectivityEstimatorTest.scala | 7 +- .../metadata/FlinkRelMdHandlerTestBase.scala | 6 +- .../metadata/SelectivityEstimatorTest.scala | 7 +- .../plan/utils/RexNodeExtractorTest.scala | 8 +- .../table/planner/utils/TableTestBase.scala | 17 ++- .../api/internal/BatchTableEnvImpl.scala | 6 +- .../table/api/internal/TableEnvImpl.scala | 21 +++- .../internal/BatchTableEnvironmentImpl.scala | 7 +- .../internal/BatchTableEnvironmentImpl.scala | 7 +- .../PushFilterIntoTableSourceScanRule.scala | 3 +- .../sqlexec/SqlToOperationConverterTest.java | 4 +- .../stream/StreamTableEnvironmentTest.scala | 8 +- .../table/api/stream/sql/AggregateTest.scala | 6 +- .../table/plan/RexProgramExtractorTest.scala | 5 +- .../table/utils/MockTableEnvironment.scala | 9 +- .../flink/table/utils/TableTestBase.scala | 13 +- 35 files changed, 447 insertions(+), 62 deletions(-) create mode 100644 flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/module/ModuleManager.java create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/Module.java create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/exceptions/ModuleAlreadyExistException.java create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/exceptions/ModuleNotFoundException.java diff --git a/flink-python/pyflink/table/table_environment.py b/flink-python/pyflink/table/table_environment.py index e4af611d15c7..6c88d9e72c20 100644 --- a/flink-python/pyflink/table/table_environment.py +++ b/flink-python/pyflink/table/table_environment.py @@ -209,6 +209,15 @@ def list_catalogs(self): j_catalog_name_array = self._j_tenv.listCatalogs() return [item for item in j_catalog_name_array] + def list_modules(self): + """ + Gets the names of all modules registered in this environment. + + :return: List of module names. + """ + j_module_name_array = self._j_tenv.listModules() + return [item for item in j_module_name_array] + def list_databases(self): """ Gets the names of all databases in the current catalog. diff --git a/flink-python/pyflink/table/tests/test_environment_completeness.py b/flink-python/pyflink/table/tests/test_environment_completeness.py index 89459891f8f3..d50bebbb4a3e 100644 --- a/flink-python/pyflink/table/tests/test_environment_completeness.py +++ b/flink-python/pyflink/table/tests/test_environment_completeness.py @@ -41,7 +41,14 @@ def excluded_methods(cls): # 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', 'listTables', 'getCompletionHints', 'create'} + return { + 'registerCatalog', + 'getCatalog', + 'listTables', + 'getCompletionHints', + 'create', + 'loadModule', + 'unloadModule'} if __name__ == '__main__': 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 73c9ccebd9f4..023efbc210e7 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 @@ -77,6 +77,7 @@ import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.functions.TableFunction; import org.apache.flink.table.functions.UserDefinedFunction; +import org.apache.flink.table.module.ModuleManager; import org.apache.flink.table.planner.delegation.ExecutorBase; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.table.sources.TableSource; @@ -293,8 +294,8 @@ private static TableEnvironment createStreamTableEnvironment( final CatalogManager catalogManager = new CatalogManager( settings.getBuiltInCatalogName(), new GenericInMemoryCatalog(settings.getBuiltInCatalogName(), settings.getBuiltInDatabaseName())); - - final FunctionCatalog functionCatalog = new FunctionCatalog(catalogManager); + final ModuleManager moduleManager = new ModuleManager(); + final FunctionCatalog functionCatalog = new FunctionCatalog(catalogManager, moduleManager); final Map plannerProperties = settings.toPlannerProperties(); final Planner planner = ComponentFactoryService.find(PlannerFactory.class, plannerProperties) @@ -302,6 +303,7 @@ private static TableEnvironment createStreamTableEnvironment( return new StreamTableEnvironmentImpl( catalogManager, + moduleManager, functionCatalog, config, env, diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/BatchTableEnvironment.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/BatchTableEnvironment.java index 7b7692d9d48b..38d0063c4788 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/BatchTableEnvironment.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/BatchTableEnvironment.java @@ -33,6 +33,7 @@ import org.apache.flink.table.descriptors.ConnectorDescriptor; import org.apache.flink.table.functions.AggregateFunction; import org.apache.flink.table.functions.TableFunction; +import org.apache.flink.table.module.ModuleManager; import org.apache.flink.table.sinks.TableSink; import java.lang.reflect.Constructor; @@ -327,13 +328,14 @@ static BatchTableEnvironment create(ExecutionEnvironment executionEnvironment) { static BatchTableEnvironment create(ExecutionEnvironment executionEnvironment, TableConfig tableConfig) { try { Class clazz = Class.forName("org.apache.flink.table.api.java.internal.BatchTableEnvironmentImpl"); - Constructor con = clazz.getConstructor(ExecutionEnvironment.class, TableConfig.class, CatalogManager.class); + Constructor con = clazz.getConstructor(ExecutionEnvironment.class, TableConfig.class, CatalogManager.class, ModuleManager.class); String defaultCatalog = "default_catalog"; CatalogManager catalogManager = new CatalogManager( defaultCatalog, new GenericInMemoryCatalog(defaultCatalog, "default_database") ); - return (BatchTableEnvironment) con.newInstance(executionEnvironment, tableConfig, catalogManager); + ModuleManager moduleManager = new ModuleManager(); + return (BatchTableEnvironment) con.newInstance(executionEnvironment, tableConfig, catalogManager, moduleManager); } catch (Throwable t) { throw new TableException("Create BatchTableEnvironment failed.", t); } 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 06b20f529adc..4addf4fe5e18 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 @@ -53,6 +53,7 @@ import org.apache.flink.table.functions.TableAggregateFunction; import org.apache.flink.table.functions.TableFunction; import org.apache.flink.table.functions.UserFunctionsTypeHelper; +import org.apache.flink.table.module.ModuleManager; import org.apache.flink.table.operations.JavaDataStreamQueryOperation; import org.apache.flink.table.operations.OutputConversionModifyOperation; import org.apache.flink.table.sources.TableSource; @@ -80,13 +81,14 @@ public final class StreamTableEnvironmentImpl extends TableEnvironmentImpl imple public StreamTableEnvironmentImpl( CatalogManager catalogManager, + ModuleManager moduleManager, FunctionCatalog functionCatalog, TableConfig tableConfig, StreamExecutionEnvironment executionEnvironment, Planner planner, Executor executor, boolean isStreamingMode) { - super(catalogManager, tableConfig, executor, functionCatalog, planner, isStreamingMode); + super(catalogManager, moduleManager, tableConfig, executor, functionCatalog, planner, isStreamingMode); this.executionEnvironment = executionEnvironment; } @@ -104,7 +106,9 @@ public static StreamTableEnvironment create( settings.getBuiltInCatalogName(), new GenericInMemoryCatalog(settings.getBuiltInCatalogName(), settings.getBuiltInDatabaseName())); - FunctionCatalog functionCatalog = new FunctionCatalog(catalogManager); + ModuleManager moduleManager = new ModuleManager(); + + FunctionCatalog functionCatalog = new FunctionCatalog(catalogManager, moduleManager); Map executorProperties = settings.toExecutorProperties(); Executor executor = lookupExecutor(executorProperties, executionEnvironment); @@ -115,6 +119,7 @@ public static StreamTableEnvironment create( return new StreamTableEnvironmentImpl( catalogManager, + moduleManager, functionCatalog, tableConfig, executionEnvironment, diff --git a/flink-table/flink-table-api-java-bridge/src/test/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImplTest.java b/flink-table/flink-table-api-java-bridge/src/test/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImplTest.java index 1348ec238e8f..6b1724de54ca 100644 --- a/flink-table/flink-table-api-java-bridge/src/test/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImplTest.java +++ b/flink-table/flink-table-api-java-bridge/src/test/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImplTest.java @@ -30,6 +30,7 @@ import org.apache.flink.table.catalog.GenericInMemoryCatalog; import org.apache.flink.table.delegation.Executor; import org.apache.flink.table.delegation.Planner; +import org.apache.flink.table.module.ModuleManager; import org.apache.flink.table.operations.ModifyOperation; import org.apache.flink.table.operations.Operation; import org.apache.flink.types.Row; @@ -92,9 +93,11 @@ private StreamTableEnvironmentImpl getStreamTableEnvironment( StreamExecutionEnvironment env, DataStreamSource elements) { CatalogManager catalogManager = new CatalogManager("cat", new GenericInMemoryCatalog("cat", "db")); + ModuleManager moduleManager = new ModuleManager(); return new StreamTableEnvironmentImpl( catalogManager, - new FunctionCatalog(catalogManager), + moduleManager, + new FunctionCatalog(catalogManager, moduleManager), new TableConfig(), env, new TestPlanner(elements.getTransformation()), diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java index 23589b31f80c..344e7ffe796d 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java @@ -27,6 +27,9 @@ import org.apache.flink.table.descriptors.ConnectTableDescriptor; import org.apache.flink.table.descriptors.ConnectorDescriptor; import org.apache.flink.table.functions.ScalarFunction; +import org.apache.flink.table.module.Module; +import org.apache.flink.table.module.exceptions.ModuleAlreadyExistException; +import org.apache.flink.table.module.exceptions.ModuleNotFoundException; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.table.sources.TableSource; @@ -103,6 +106,23 @@ static TableEnvironment create(EnvironmentSettings settings) { */ Optional getCatalog(String catalogName); + /** + * Loads a {@link Module} under a unique name. Modules will be kept in the loaded order. + * + * @param moduleName name of the {@link Module} + * @param module the module instance + * @throws ModuleAlreadyExistException thrown when there is already a module with the same name + */ + void loadModule(String moduleName, Module module) throws ModuleAlreadyExistException; + + /** + * Unloads a {@link Module} with given name. + * + * @param moduleName name of the {@link Module} + * @throws ModuleNotFoundException thrown when there is no module with the given name + */ + void unloadModule(String moduleName) throws ModuleNotFoundException; + /** * Registers a {@link ScalarFunction} under a unique name. Replaces already existing * user-defined functions under this name. @@ -237,6 +257,13 @@ static TableEnvironment create(EnvironmentSettings settings) { */ String[] listCatalogs(); + /** + * Gets an array of names of all modules in this environment in the loaded order. + * + * @return A list of the names of all modules in the loaded order. + */ + String[] listModules(); + /** * Gets the names of all databases registered in the current catalog. * 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 12e874ece3b8..d13dea35562b 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 @@ -48,6 +48,10 @@ import org.apache.flink.table.expressions.TableReferenceExpression; import org.apache.flink.table.factories.ComponentFactoryService; import org.apache.flink.table.functions.ScalarFunction; +import org.apache.flink.table.module.Module; +import org.apache.flink.table.module.ModuleManager; +import org.apache.flink.table.module.exceptions.ModuleAlreadyExistException; +import org.apache.flink.table.module.exceptions.ModuleNotFoundException; import org.apache.flink.table.operations.CatalogQueryOperation; import org.apache.flink.table.operations.CatalogSinkModifyOperation; import org.apache.flink.table.operations.ModifyOperation; @@ -80,6 +84,7 @@ public class TableEnvironmentImpl implements TableEnvironment { // and this should always be true. This avoids too many hard code. private static final boolean IS_STREAM_TABLE = true; private final CatalogManager catalogManager; + private final ModuleManager moduleManager; private final OperationTreeBuilder operationTreeBuilder; private final List bufferedModifyOperations = new ArrayList<>(); @@ -90,12 +95,14 @@ public class TableEnvironmentImpl implements TableEnvironment { protected TableEnvironmentImpl( CatalogManager catalogManager, + ModuleManager moduleManager, TableConfig tableConfig, Executor executor, FunctionCatalog functionCatalog, Planner planner, boolean isStreamingMode) { this.catalogManager = catalogManager; + this.moduleManager = moduleManager; this.execEnv = executor; this.tableConfig = tableConfig; @@ -118,7 +125,8 @@ public static TableEnvironmentImpl create(EnvironmentSettings settings) { settings.getBuiltInCatalogName(), new GenericInMemoryCatalog(settings.getBuiltInCatalogName(), settings.getBuiltInDatabaseName())); - FunctionCatalog functionCatalog = new FunctionCatalog(catalogManager); + ModuleManager moduleManager = new ModuleManager(); + FunctionCatalog functionCatalog = new FunctionCatalog(catalogManager, moduleManager); Map executorProperties = settings.toExecutorProperties(); Executor executor = ComponentFactoryService.find(ExecutorFactory.class, executorProperties) @@ -131,6 +139,7 @@ public static TableEnvironmentImpl create(EnvironmentSettings settings) { return new TableEnvironmentImpl( catalogManager, + moduleManager, tableConfig, executor, functionCatalog, @@ -161,6 +170,16 @@ public Optional getCatalog(String catalogName) { return catalogManager.getCatalog(catalogName); } + @Override + public void loadModule(String moduleName, Module module) throws ModuleAlreadyExistException { + moduleManager.loadModule(moduleName, module); + } + + @Override + public void unloadModule(String moduleName) throws ModuleNotFoundException { + moduleManager.unloadModule(moduleName); + } + @Override public void registerFunction(String name, ScalarFunction function) { functionCatalog.registerTempSystemScalarFunction( @@ -229,6 +248,11 @@ public String[] listCatalogs() { return catalogManager.getCatalogs().toArray(new String[0]); } + @Override + public String[] listModules() { + return moduleManager.listModules().toArray(new String[0]); + } + @Override public String[] listDatabases() { return catalogManager.getCatalog(catalogManager.getCurrentCatalog()) 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 68082165309d..61f115440e9e 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 @@ -38,6 +38,7 @@ import org.apache.flink.table.functions.TableFunctionDefinition; import org.apache.flink.table.functions.UserDefinedAggregateFunction; import org.apache.flink.table.functions.UserFunctionsTypeHelper; +import org.apache.flink.table.module.ModuleManager; import org.apache.flink.util.Preconditions; import java.util.HashSet; @@ -57,6 +58,7 @@ public class FunctionCatalog implements FunctionLookup { private final CatalogManager catalogManager; + private final ModuleManager moduleManager; private final Map tempSystemFunctions = new LinkedHashMap<>(); private final Map tempCatalogFunctions = new LinkedHashMap<>(); @@ -66,8 +68,9 @@ public class FunctionCatalog implements FunctionLookup { */ private PlannerTypeInferenceUtil plannerTypeInferenceUtil; - public FunctionCatalog(CatalogManager catalogManager) { + public FunctionCatalog(CatalogManager catalogManager, ModuleManager moduleManager) { this.catalogManager = checkNotNull(catalogManager); + this.moduleManager = checkNotNull(moduleManager); } public void setPlannerTypeInferenceUtil(PlannerTypeInferenceUtil plannerTypeInferenceUtil) { diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/module/ModuleManager.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/module/ModuleManager.java new file mode 100644 index 000000000000..9216237cd31a --- /dev/null +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/module/ModuleManager.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.table.module; + +import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.module.exceptions.ModuleAlreadyExistException; +import org.apache.flink.table.module.exceptions.ModuleNotFoundException; +import org.apache.flink.util.StringUtils; + +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Responsible for loading/unloading modules, managing their life cycles, and resolving module objects. + */ +public class ModuleManager { + private LinkedHashMap modules; + + public ModuleManager() { + this.modules = new LinkedHashMap<>(); + + // TODO: Add Core module to modules + } + + /** + * Load a module under a unique name. Modules will be kept in the loaded order, and new module + * will be added to the end. + * + * @param name name of the module + * @param module the module instance + * @throws ModuleAlreadyExistException thrown when there is already a module with the same name + */ + public void loadModule(String name, Module module) throws ModuleAlreadyExistException { + checkArgument(!StringUtils.isNullOrWhitespaceOnly(name), "name cannot be null or empty string"); + checkNotNull(module, "module cannot be null"); + + if (!modules.containsKey(name)) { + modules.put(name, module); + } else { + throw new ModuleAlreadyExistException(name); + } + } + + /** + * Unload a module with given name. + * + * @param name name of the module + * @throws ModuleNotFoundException thrown when there is no module with the given name + */ + public void unloadModule(String name) throws ModuleNotFoundException { + if (modules.containsKey(name)) { + modules.remove(name); + } else { + throw new ModuleNotFoundException(name); + } + } + + /** + * Get names of all modules loaded. + * + * @return a list of names of modules loaded + */ + public List listModules() { + return new ArrayList<>(modules.keySet()); + } + + /** + * Get names of all functions from all modules. + * + * @return a set of names of registered modules. + */ + public Set listFunctions() { + return modules.values().stream() + .map(m -> m.listFunctions()) + .flatMap(n -> n.stream()) + .collect(Collectors.toSet()); + } + + /** + * Get an optional of {@link FunctionDefinition} by a given name. + * Function will be resolved to modules in the loaded order, and the first match will be returned. + * If no match is found in all modules, return an optional. + * + * @param name name of the function + * @return an optional of {@link FunctionDefinition} + */ + public Optional getFunctionDefinition(String name) { + Optional module = modules.values().stream() + .filter(p -> p.listFunctions().stream().anyMatch(e -> e.equals(name))) + .findFirst(); + + return module.isPresent() ? module.get().getFunctionDefinition(name) : Optional.empty(); + } + +} diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java index fb11cbc5e8d8..5f79dd5e0a35 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java @@ -19,6 +19,7 @@ package org.apache.flink.table.catalog; import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.module.ModuleManager; import org.junit.Test; @@ -37,7 +38,8 @@ public class FunctionCatalogTest { @Test public void testGetBuiltInFunctions() { FunctionCatalog functionCatalog = new FunctionCatalog( - new CatalogManager("test", new GenericInMemoryCatalog("test"))); + new CatalogManager("test", new GenericInMemoryCatalog("test")), + new ModuleManager()); Set actual = new HashSet<>(); Collections.addAll(actual, functionCatalog.getFunctions()); diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/TableEnvironmentMock.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/TableEnvironmentMock.java index dc8b99290dcd..fa2c58c46438 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/TableEnvironmentMock.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/TableEnvironmentMock.java @@ -25,6 +25,7 @@ import org.apache.flink.table.catalog.CatalogManager; import org.apache.flink.table.catalog.FunctionCatalog; import org.apache.flink.table.catalog.GenericInMemoryCatalog; +import org.apache.flink.table.module.ModuleManager; /** * Mocking {@link TableEnvironment} for tests. @@ -41,12 +42,13 @@ public class TableEnvironmentMock extends TableEnvironmentImpl { protected TableEnvironmentMock( CatalogManager catalogManager, + ModuleManager moduleManager, TableConfig tableConfig, ExecutorMock executor, FunctionCatalog functionCatalog, PlannerMock planner, boolean isStreamingMode) { - super(catalogManager, tableConfig, executor, functionCatalog, planner, isStreamingMode); + super(catalogManager, moduleManager, tableConfig, executor, functionCatalog, planner, isStreamingMode); this.catalogManager = catalogManager; this.executor = executor; @@ -64,11 +66,13 @@ public static TableEnvironmentMock getBatchInstance() { private static TableEnvironmentMock getInstance(boolean isStreamingMode) { final CatalogManager catalogManager = createCatalogManager(); + final ModuleManager moduleManager = new ModuleManager(); return new TableEnvironmentMock( catalogManager, + moduleManager, createTableConfig(), createExecutor(), - createFunctionCatalog(catalogManager), + createFunctionCatalog(catalogManager, moduleManager), createPlanner(), isStreamingMode); } @@ -89,8 +93,8 @@ private static ExecutorMock createExecutor() { return new ExecutorMock(); } - private static FunctionCatalog createFunctionCatalog(CatalogManager catalogManager) { - return new FunctionCatalog(catalogManager); + private static FunctionCatalog createFunctionCatalog(CatalogManager catalogManager, ModuleManager moduleManager) { + return new FunctionCatalog(catalogManager, moduleManager); } private static PlannerMock createPlanner() { diff --git a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/BatchTableEnvironment.scala b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/BatchTableEnvironment.scala index 516e6b17013b..c57b0351e990 100644 --- a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/BatchTableEnvironment.scala +++ b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/BatchTableEnvironment.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.catalog.{CatalogManager, GenericInMemoryCatalog} import org.apache.flink.table.descriptors.{BatchTableDescriptor, ConnectorDescriptor} import org.apache.flink.table.expressions.Expression import org.apache.flink.table.functions.{AggregateFunction, TableFunction} +import org.apache.flink.table.module.ModuleManager import org.apache.flink.table.sinks.TableSink /** @@ -294,7 +295,8 @@ object BatchTableEnvironment { .getConstructor( classOf[ExecutionEnvironment], classOf[TableConfig], - classOf[CatalogManager]) + classOf[CatalogManager], + classOf[ModuleManager]) val builtInCatalog = "default_catalog" val catalogManager = new CatalogManager( "default_catalog", @@ -302,7 +304,8 @@ object BatchTableEnvironment { builtInCatalog, "default_database") ) - const.newInstance(executionEnvironment, tableConfig, catalogManager) + val moduleManager = new ModuleManager + const.newInstance(executionEnvironment, tableConfig, catalogManager, moduleManager) .asInstanceOf[BatchTableEnvironment] } catch { case t: Throwable => throw new TableException("Create BatchTableEnvironment failed.", t) 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 1384be09e5a4..97da79868eef 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 @@ -35,14 +35,15 @@ import org.apache.flink.table.descriptors.{ConnectorDescriptor, StreamTableDescr import org.apache.flink.table.expressions.Expression import org.apache.flink.table.factories.ComponentFactoryService import org.apache.flink.table.functions.{AggregateFunction, TableAggregateFunction, TableFunction, UserFunctionsTypeHelper} +import org.apache.flink.table.module.ModuleManager import org.apache.flink.table.operations.{OutputConversionModifyOperation, ScalaDataStreamQueryOperation} import org.apache.flink.table.sources.{TableSource, TableSourceValidation} import org.apache.flink.table.types.utils.TypeConversions import org.apache.flink.table.typeutils.FieldInfoUtils - import java.util import java.util.{Collections, List => JList, Map => JMap} + import _root_.scala.collection.JavaConverters._ /** @@ -52,6 +53,7 @@ import _root_.scala.collection.JavaConverters._ @Internal class StreamTableEnvironmentImpl ( catalogManager: CatalogManager, + moduleManager: ModuleManager, functionCatalog: FunctionCatalog, config: TableConfig, scalaExecutionEnvironment: StreamExecutionEnvironment, @@ -60,6 +62,7 @@ class StreamTableEnvironmentImpl ( isStreaming: Boolean) extends TableEnvironmentImpl( catalogManager, + moduleManager, config, executor, functionCatalog, @@ -278,7 +281,8 @@ object StreamTableEnvironmentImpl { settings.getBuiltInCatalogName, new GenericInMemoryCatalog(settings.getBuiltInCatalogName, settings.getBuiltInDatabaseName)) - val functionCatalog = new FunctionCatalog(catalogManager) + val moduleManager = new ModuleManager + val functionCatalog = new FunctionCatalog(catalogManager, moduleManager) val executorProperties = settings.toExecutorProperties val executor = lookupExecutor(executorProperties, executionEnvironment) @@ -294,6 +298,7 @@ object StreamTableEnvironmentImpl { new StreamTableEnvironmentImpl( catalogManager, + moduleManager, functionCatalog, tableConfig, executionEnvironment, diff --git a/flink-table/flink-table-api-scala-bridge/src/test/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImplTest.scala b/flink-table/flink-table-api-scala-bridge/src/test/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImplTest.scala index 91b37d0ce3ea..cf36ed29db08 100644 --- a/flink-table/flink-table-api-scala-bridge/src/test/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImplTest.scala +++ b/flink-table/flink-table-api-scala-bridge/src/test/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImplTest.scala @@ -26,13 +26,13 @@ import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog, GenericI import org.apache.flink.table.delegation.{Executor, Planner} import org.apache.flink.table.operations.{ModifyOperation, Operation} import org.apache.flink.types.Row - import org.hamcrest.CoreMatchers.equalTo import org.junit.Assert.assertThat import org.junit.Test - import java.util.{Collections, List => JList} +import org.apache.flink.table.module.ModuleManager + /** * Tests for [[StreamTableEnvironmentImpl]]. */ @@ -83,9 +83,11 @@ class StreamTableEnvironmentImplTest { val catalogManager = new CatalogManager( "cat", new GenericInMemoryCatalog("cat", "db")) + val moduleManager = new ModuleManager new StreamTableEnvironmentImpl( catalogManager, - new FunctionCatalog(catalogManager), + moduleManager, + new FunctionCatalog(catalogManager, moduleManager), new TableConfig, env, new TestPlanner(elements.javaStream.getTransformation), diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/Module.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/Module.java new file mode 100644 index 000000000000..eccbe703c282 --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/Module.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.module; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.table.functions.FunctionDefinition; + +import java.util.Collections; +import java.util.Optional; +import java.util.Set; + +/** + * Modules define a set of metadata, including functions, user defined types, operators, rules, etc. + * Metadata from modules are regarded as built-in or system metadata that users can take advantages of. + */ +@PublicEvolving +public interface Module { + + /** + * List names of all functions in this module. + * + * @return a set of function names + */ + default Set listFunctions() { + return Collections.emptySet(); + } + + /** + * Get an optional of {@link FunctionDefinition} by a give name. + * + * @param name name of the {@link FunctionDefinition}. + * @return an optional function definition + */ + default Optional getFunctionDefinition(String name) { + return Optional.empty(); + } + + // user defined types, operators, rules, etc +} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/exceptions/ModuleAlreadyExistException.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/exceptions/ModuleAlreadyExistException.java new file mode 100644 index 000000000000..1f27ba6a4f48 --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/exceptions/ModuleAlreadyExistException.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.table.module.exceptions; + +/** + * Exception for trying to load a module that already exists. + */ +public class ModuleAlreadyExistException extends Exception { + public ModuleAlreadyExistException(String moduleName) { + super(String.format("Module %s already exists.", moduleName)); + } +} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/exceptions/ModuleNotFoundException.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/exceptions/ModuleNotFoundException.java new file mode 100644 index 000000000000..bf575cd63071 --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/exceptions/ModuleNotFoundException.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.table.module.exceptions; + +/** + * Exception for operating on a module that does not exists. + */ +public class ModuleNotFoundException extends Exception { + public ModuleNotFoundException(String moduleName) { + super(String.format("Module %s does not exist.", moduleName)); + } +} diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java index 1acdd88c24e3..9f872752b066 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java @@ -33,6 +33,7 @@ import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.module.ModuleManager; import org.apache.flink.table.operations.CatalogSinkModifyOperation; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.ddl.CreateTableOperation; @@ -71,7 +72,8 @@ public class SqlToOperationConverterTest { "default"); private final CatalogManager catalogManager = new CatalogManager("builtin", catalog); - private final FunctionCatalog functionCatalog = new FunctionCatalog(catalogManager); + private final ModuleManager moduleManager = new ModuleManager(); + private final FunctionCatalog functionCatalog = new FunctionCatalog(catalogManager, moduleManager); private final PlannerContext plannerContext = new PlannerContext(tableConfig, functionCatalog, diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/AggCallSelectivityEstimatorTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/AggCallSelectivityEstimatorTest.scala index 92905e71dec9..554d51b4ceb0 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/AggCallSelectivityEstimatorTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/AggCallSelectivityEstimatorTest.scala @@ -26,7 +26,6 @@ import org.apache.flink.table.planner.plan.schema._ import org.apache.flink.table.planner.plan.stats.FlinkStatistic import org.apache.flink.table.planner.{JDouble, JLong} import org.apache.flink.util.Preconditions - import com.google.common.collect.ImmutableList import org.apache.calcite.plan.{AbstractRelOptPlanner, RelOptCluster} import org.apache.calcite.rel.`type`.RelDataType @@ -46,9 +45,10 @@ import org.junit.{Before, BeforeClass, Test} import org.powermock.api.mockito.PowerMockito._ import org.powermock.core.classloader.annotations.PrepareForTest import org.powermock.modules.junit4.PowerMockRunner - import java.math.BigDecimal +import org.apache.flink.table.module.ModuleManager + import scala.collection.JavaConversions._ /** @@ -83,7 +83,8 @@ class AggCallSelectivityEstimatorTest { val cluster = mock(classOf[RelOptCluster]) val planner = mock(classOf[AbstractRelOptPlanner]) val catalogManager = mock(classOf[CatalogManager]) - val functionCatalog = new FunctionCatalog(catalogManager) + val moduleManager = mock(classOf[ModuleManager]) + val functionCatalog = new FunctionCatalog(catalogManager, moduleManager) val context = new FlinkContextImpl(new TableConfig, functionCatalog) when(tableScan, "getCluster").thenReturn(cluster) when(cluster, "getRexBuilder").thenReturn(rexBuilder) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala index dd6d64d23a9e..3802d44f3e99 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala @@ -67,10 +67,11 @@ import org.apache.calcite.sql.fun.{SqlCountAggFunction, SqlStdOperatorTable} import org.apache.calcite.sql.parser.SqlParserPos import org.apache.calcite.util.{DateString, ImmutableBitSet, ImmutableIntList, TimeString, TimestampString} import org.junit.{Before, BeforeClass} - import java.math.BigDecimal import java.util +import org.apache.flink.table.module.ModuleManager + import scala.collection.JavaConversions._ class FlinkRelMdHandlerTestBase { @@ -82,13 +83,14 @@ class FlinkRelMdHandlerTestBase { val builtinDatabase = "default_database" val catalogManager = new CatalogManager( builtinCatalog, new GenericInMemoryCatalog(builtinCatalog, builtinDatabase)) + val moduleManager = new ModuleManager // TODO batch RelNode and stream RelNode should have different PlannerContext // and RelOptCluster due to they have different trait definitions. val plannerContext: PlannerContext = new PlannerContext( tableConfig, - new FunctionCatalog(catalogManager), + new FunctionCatalog(catalogManager, moduleManager), CalciteSchema.from(rootSchema), util.Arrays.asList( ConventionTraitDef.INSTANCE, diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimatorTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimatorTest.scala index 99de5a72c75f..f16ae27adecb 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimatorTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimatorTest.scala @@ -26,7 +26,6 @@ import org.apache.flink.table.planner.plan.schema._ import org.apache.flink.table.planner.plan.stats.FlinkStatistic import org.apache.flink.table.planner.{JDouble, JLong} import org.apache.flink.util.Preconditions - import org.apache.calcite.plan.{AbstractRelOptPlanner, RelOptCluster} import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.TableScan @@ -43,9 +42,10 @@ import org.junit.{Before, BeforeClass, Test} import org.powermock.api.mockito.PowerMockito._ import org.powermock.core.classloader.annotations.PrepareForTest import org.powermock.modules.junit4.PowerMockRunner - import java.math.BigDecimal +import org.apache.flink.table.module.ModuleManager + import scala.collection.JavaConverters._ /** @@ -85,7 +85,8 @@ class SelectivityEstimatorTest { val cluster = mock(classOf[RelOptCluster]) val planner = mock(classOf[AbstractRelOptPlanner]) val catalogManager = mock(classOf[CatalogManager]) - val functionCatalog = new FunctionCatalog(catalogManager) + val moduleManager = mock(classOf[ModuleManager]) + val functionCatalog = new FunctionCatalog(catalogManager, moduleManager) val context: FlinkContext = new FlinkContextImpl(tableConfig, functionCatalog) when(tableScan, "getCluster").thenReturn(cluster) when(cluster, "getRexBuilder").thenReturn(rexBuilder) 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 f0289e4b1801..efe2f6e0998d 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 @@ -31,7 +31,6 @@ import org.apache.flink.table.planner.functions.sql.FlinkSqlOperatorTable import org.apache.flink.table.planner.functions.utils.ScalarSqlFunction import org.apache.flink.table.planner.plan.utils.InputTypeBuilder.inputOf import org.apache.flink.table.planner.utils.{DateTimeTestUtil, IntSumAggFunction} - import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rex.{RexBuilder, RexNode} import org.apache.calcite.sql.SqlPostfixOperator @@ -42,11 +41,12 @@ import org.apache.calcite.util.{DateString, TimeString, TimestampString} import org.hamcrest.CoreMatchers.is import org.junit.Assert.{assertArrayEquals, assertEquals, assertThat, assertTrue} import org.junit.Test - import java.math.BigDecimal import java.sql.Timestamp import java.util.{TimeZone, List => JList} +import org.apache.flink.table.module.ModuleManager + import scala.collection.JavaConverters._ /** @@ -56,8 +56,8 @@ class RexNodeExtractorTest extends RexNodeTestBase { val defaultCatalog = "default_catalog" val catalogManager = new CatalogManager( defaultCatalog, new GenericInMemoryCatalog(defaultCatalog, "default_database")) - - private val functionCatalog = new FunctionCatalog(catalogManager) + val moduleManager = new ModuleManager + private val functionCatalog = new FunctionCatalog(catalogManager, moduleManager) private val expressionBridge: ExpressionBridge[PlannerExpression] = new ExpressionBridge[PlannerExpression]( 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 a452717e0f64..4ded0fc0eb4c 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 @@ -57,16 +57,16 @@ import org.apache.flink.table.types.logical.LogicalType import org.apache.flink.table.types.utils.TypeConversions import org.apache.flink.table.typeutils.FieldInfoUtils import org.apache.flink.types.Row - import org.apache.calcite.rel.RelNode import org.apache.calcite.sql.SqlExplainLevel import org.apache.commons.lang3.SystemUtils import org.junit.Assert.{assertEquals, assertTrue} import org.junit.Rule import org.junit.rules.{ExpectedException, TestName} - import _root_.java.util +import org.apache.flink.table.module.ModuleManager + import _root_.scala.collection.JavaConversions._ import _root_.scala.io.Source @@ -887,6 +887,7 @@ class TestTableSource(override val isBounded: Boolean, schema: TableSchema) class TestingTableEnvironment private( catalogManager: CatalogManager, + moduleManager: ModuleManager, tableConfig: TableConfig, executor: Executor, functionCatalog: FunctionCatalog, @@ -894,6 +895,7 @@ class TestingTableEnvironment private( isStreamingMode: Boolean) extends TableEnvironmentImpl( catalogManager, + moduleManager, tableConfig, executor, functionCatalog, @@ -1027,7 +1029,8 @@ object TestingTableEnvironment { new GenericInMemoryCatalog( settings.getBuiltInCatalogName, settings.getBuiltInDatabaseName)) } - val functionCatalog = new FunctionCatalog(catalogMgr) + val moduleManager = new ModuleManager + val functionCatalog = new FunctionCatalog(catalogMgr, moduleManager) val plannerProperties = settings.toPlannerProperties val executorProperties = settings.toExecutorProperties val executor = ComponentFactoryService.find(classOf[ExecutorFactory], @@ -1036,7 +1039,13 @@ object TestingTableEnvironment { .create(plannerProperties, executor, tableConfig, functionCatalog, catalogMgr) .asInstanceOf[PlannerBase] new TestingTableEnvironment( - catalogMgr, tableConfig, executor, functionCatalog, planner, settings.isStreamingMode) + catalogMgr, + moduleManager, + tableConfig, + executor, + functionCatalog, + planner, + settings.isStreamingMode) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala index a4ed52381a75..5ec2bbaefc9b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala @@ -34,6 +34,7 @@ import org.apache.flink.table.explain.PlanJsonParser import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor import org.apache.flink.table.expressions.{Expression, UnresolvedCallExpression} import org.apache.flink.table.functions.BuiltInFunctionDefinitions.TIME_ATTRIBUTES +import org.apache.flink.table.module.ModuleManager import org.apache.flink.table.operations.DataSetQueryOperation import org.apache.flink.table.plan.BatchOptimizer import org.apache.flink.table.plan.nodes.dataset.DataSetRel @@ -59,8 +60,9 @@ import _root_.scala.collection.JavaConversions._ abstract class BatchTableEnvImpl( private[flink] val execEnv: ExecutionEnvironment, config: TableConfig, - catalogManager: CatalogManager) - extends TableEnvImpl(config, catalogManager) { + catalogManager: CatalogManager, + moduleManager: ModuleManager) + extends TableEnvImpl(config, catalogManager, moduleManager) { private[flink] val optimizer = new BatchOptimizer( () => config.getPlannerConfig.unwrap(classOf[CalciteConfig]).orElse(CalciteConfig.DEFAULT), 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 a00ea3526b22..e02803259bc9 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 @@ -29,6 +29,7 @@ import org.apache.flink.table.expressions._ import org.apache.flink.table.expressions.resolver.lookups.TableReferenceLookup import org.apache.flink.table.factories.{TableFactoryService, TableFactoryUtil, TableSinkFactory} import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, TableFunction, UserDefinedAggregateFunction, _} +import org.apache.flink.table.module.{Module, ModuleManager} import org.apache.flink.table.operations.ddl.CreateTableOperation import org.apache.flink.table.operations.utils.OperationTreeBuilder import org.apache.flink.table.operations.{CatalogQueryOperation, PlannerQueryOperation, TableSourceQueryOperation, _} @@ -37,12 +38,10 @@ import org.apache.flink.table.sinks.{OverwritableTableSink, PartitionableTableSi import org.apache.flink.table.sources.TableSource import org.apache.flink.table.sqlexec.SqlToOperationConverter import org.apache.flink.table.util.JavaScalaConversionUtil - import org.apache.calcite.jdbc.CalciteSchemaBuilder.asRootSchema import org.apache.calcite.sql._ import org.apache.calcite.sql.parser.SqlParser import org.apache.calcite.tools.FrameworkConfig - import _root_.java.util.{Optional, HashMap => JHashMap, Map => JMap} import _root_.scala.collection.JavaConversions._ @@ -55,11 +54,13 @@ import _root_.scala.collection.JavaConverters._ */ abstract class TableEnvImpl( val config: TableConfig, - private val catalogManager: CatalogManager) + private val catalogManager: CatalogManager, + private val moduleManager: ModuleManager) extends TableEnvironment { // Table API/SQL function catalog - private[flink] val functionCatalog: FunctionCatalog = new FunctionCatalog(catalogManager) + private[flink] val functionCatalog: FunctionCatalog = + new FunctionCatalog(catalogManager, moduleManager) // temporary utility until we don't use planner expressions anymore functionCatalog.setPlannerTypeInferenceUtil(PlannerTypeInferenceUtilImpl.INSTANCE) @@ -156,6 +157,14 @@ abstract class TableEnvImpl( catalogManager.getCatalog(catalogName) } + override def loadModule(moduleName: String, module: Module): Unit = { + moduleManager.loadModule(moduleName, module) + } + + override def unloadModule(moduleName: String): Unit = { + moduleManager.unloadModule(moduleName) + } + override def getCurrentCatalog: String = { catalogManager.getCurrentCatalog } @@ -329,6 +338,10 @@ abstract class TableEnvImpl( .map(t => new CatalogQueryOperation(objectIdentifier, t.getSchema)) } + override def listModules(): Array[String] = { + moduleManager.listModules().asScala.toArray + } + override def listCatalogs(): Array[String] = { catalogManager.getCatalogs.asScala.toArray } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/java/internal/BatchTableEnvironmentImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/java/internal/BatchTableEnvironmentImpl.scala index 735872160870..971ba3718ac1 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/java/internal/BatchTableEnvironmentImpl.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/java/internal/BatchTableEnvironmentImpl.scala @@ -26,6 +26,7 @@ import org.apache.flink.table.api.java.BatchTableEnvironment import org.apache.flink.table.catalog.CatalogManager import org.apache.flink.table.expressions.ExpressionParser import org.apache.flink.table.functions.{AggregateFunction, TableFunction} +import org.apache.flink.table.module.ModuleManager import _root_.scala.collection.JavaConverters._ @@ -39,11 +40,13 @@ import _root_.scala.collection.JavaConverters._ class BatchTableEnvironmentImpl( execEnv: ExecutionEnvironment, config: TableConfig, - catalogManager: CatalogManager) + catalogManager: CatalogManager, + moduleManager: ModuleManager) extends BatchTableEnvImpl( execEnv, config, - catalogManager) + catalogManager, + moduleManager) with org.apache.flink.table.api.java.BatchTableEnvironment { override def fromDataSet[T](dataSet: DataSet[T]): Table = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/scala/internal/BatchTableEnvironmentImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/scala/internal/BatchTableEnvironmentImpl.scala index 8e134268cdf7..7324a4bbccd3 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/scala/internal/BatchTableEnvironmentImpl.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/scala/internal/BatchTableEnvironmentImpl.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.api.scala.BatchTableEnvironment import org.apache.flink.table.catalog.CatalogManager import org.apache.flink.table.expressions.Expression import org.apache.flink.table.functions.{AggregateFunction, TableFunction} +import org.apache.flink.table.module.ModuleManager import _root_.scala.reflect.ClassTag @@ -38,11 +39,13 @@ import _root_.scala.reflect.ClassTag class BatchTableEnvironmentImpl( execEnv: ExecutionEnvironment, config: TableConfig, - catalogManager: CatalogManager) + catalogManager: CatalogManager, + moduleManager: ModuleManager) extends BatchTableEnvImpl( execEnv.getJavaEnv, config, - catalogManager) + catalogManager, + moduleManager) with org.apache.flink.table.api.scala.BatchTableEnvironment { override def fromDataSet[T](dataSet: DataSet[T]): Table = { 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 de1d0e00dd04..334fcb34a116 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 @@ -26,6 +26,7 @@ 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.module.ModuleManager import org.apache.flink.table.plan.nodes.logical.{FlinkLogicalCalc, FlinkLogicalTableSourceScan} import org.apache.flink.table.plan.util.RexProgramExtractor import org.apache.flink.table.sources.FilterableTableSource @@ -73,7 +74,7 @@ class PushFilterIntoTableSourceScanRule extends RelOptRule( RexProgramExtractor.extractConjunctiveConditions( program, call.builder().getRexBuilder, - new FunctionCatalog(catalogManager)) + new FunctionCatalog(catalogManager, new ModuleManager)) if (predicates.isEmpty) { // no condition can be translated to expression return diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java index 0424109e9a28..79f5470d557a 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java @@ -40,6 +40,7 @@ import org.apache.flink.table.catalog.exceptions.TableNotExistException; import org.apache.flink.table.expressions.ExpressionBridge; import org.apache.flink.table.expressions.PlannerExpressionConverter; +import org.apache.flink.table.module.ModuleManager; import org.apache.flink.table.operations.CatalogSinkModifyOperation; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.ddl.CreateTableOperation; @@ -74,7 +75,8 @@ public class SqlToOperationConverterTest { "default"); private final CatalogManager catalogManager = new CatalogManager("builtin", catalog); - private final FunctionCatalog functionCatalog = new FunctionCatalog(catalogManager); + private final ModuleManager moduleManager = new ModuleManager(); + private final FunctionCatalog functionCatalog = new FunctionCatalog(catalogManager, moduleManager); private final PlanningConfigurationBuilder planningConfigurationBuilder = new PlanningConfigurationBuilder(tableConfig, functionCatalog, diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/StreamTableEnvironmentTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/StreamTableEnvironmentTest.scala index 11e43cf88af8..a3450f4a549a 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/StreamTableEnvironmentTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/StreamTableEnvironmentTest.scala @@ -36,12 +36,12 @@ import org.apache.flink.table.runtime.utils.StreamTestData import org.apache.flink.table.utils.TableTestBase import org.apache.flink.table.utils.TableTestUtil.{binaryNode, streamTableNode, term, unaryNode} import org.apache.flink.types.Row - import org.junit.Test import org.mockito.Mockito.{mock, when} - import java.lang.{Integer => JInt, Long => JLong} +import org.apache.flink.table.module.ModuleManager + class StreamTableEnvironmentTest extends TableTestBase { @Test @@ -206,11 +206,13 @@ class StreamTableEnvironmentTest extends TableTestBase { val manager: CatalogManager = new CatalogManager( "default_catalog", new GenericInMemoryCatalog("default_catalog", "default_database")) + val moduleManager: ModuleManager = new ModuleManager val executor: StreamExecutor = new StreamExecutor(jStreamExecEnv) - val functionCatalog = new FunctionCatalog(manager) + val functionCatalog = new FunctionCatalog(manager, moduleManager) val streamPlanner = new StreamPlanner(executor, config, functionCatalog, manager) val jTEnv = new JStreamTableEnvironmentImpl( manager, + moduleManager, functionCatalog, config, jStreamExecEnv, diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/AggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/AggregateTest.scala index c91726748c69..1bd89f1cbcc7 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/AggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/AggregateTest.scala @@ -29,6 +29,7 @@ import org.apache.flink.table.api.{TableConfig, Types} import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog, GenericInMemoryCatalog} import org.apache.flink.table.delegation.{Executor, Planner} import org.apache.flink.table.functions.{AggregateFunction, AggregateFunctionDefinition} +import org.apache.flink.table.module.ModuleManager import org.apache.flink.table.utils.TableTestUtil.{streamTableNode, term, unaryNode} import org.apache.flink.table.utils.{StreamTableTestUtil, TableTestBase} import org.apache.flink.types.Row @@ -69,10 +70,11 @@ class AggregateTest extends TableTestBase { val defaultCatalog = "default_catalog" val catalogManager = new CatalogManager( defaultCatalog, new GenericInMemoryCatalog(defaultCatalog, "default_database")) - - val functionCatalog = new FunctionCatalog(catalogManager) + val moduleManager = new ModuleManager + val functionCatalog = new FunctionCatalog(catalogManager, moduleManager) val tablEnv = new StreamTableEnvironmentImpl( catalogManager, + moduleManager, functionCatalog, new TableConfig, Mockito.mock(classOf[StreamExecutionEnvironment]), diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/RexProgramExtractorTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/RexProgramExtractorTest.scala index b752b76a4c40..d1097267b9e9 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/RexProgramExtractorTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/RexProgramExtractorTest.scala @@ -30,6 +30,7 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable import org.apache.calcite.util.{DateString, TimeString, TimestampString} import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog, GenericInMemoryCatalog} import org.apache.flink.table.expressions._ +import org.apache.flink.table.module.ModuleManager import org.apache.flink.table.plan.util.{RexNodeToExpressionConverter, RexProgramExtractor} import org.apache.flink.table.utils.InputTypeBuilder.inputOf import org.hamcrest.CoreMatchers.is @@ -42,7 +43,9 @@ import scala.collection.mutable class RexProgramExtractorTest extends RexProgramTestBase { private val functionCatalog: FunctionCatalog = new FunctionCatalog( - new CatalogManager("default_catalog", new GenericInMemoryCatalog("default_catalog"))) + new CatalogManager("default_catalog", new GenericInMemoryCatalog("default_catalog")), + new ModuleManager + ) private val expressionBridge: ExpressionBridge[PlannerExpression] = new ExpressionBridge[PlannerExpression]( functionCatalog, diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala index 6104724c4f52..b0f11f3b6079 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala @@ -26,9 +26,10 @@ import org.apache.flink.table.descriptors.{ConnectTableDescriptor, ConnectorDesc import org.apache.flink.table.functions.ScalarFunction import org.apache.flink.table.sinks.TableSink import org.apache.flink.table.sources.TableSource - import java.util.Optional +import org.apache.flink.table.module.Module + class MockTableEnvironment extends TableEnvironment { override def fromTableSource(source: TableSource[_]): Table = ??? @@ -52,6 +53,8 @@ class MockTableEnvironment extends TableEnvironment { override def listCatalogs(): Array[String] = ??? + override def listModules(): Array[String] = ??? + override def listDatabases(): Array[String] = ??? override def listTables(): Array[String] = ??? @@ -94,4 +97,8 @@ class MockTableEnvironment extends TableEnvironment { sinkPathContinued: String*): Unit = ??? override def execute(jobName: String): JobExecutionResult = ??? + + override def loadModule(moduleName: String, module: Module): Unit = ??? + + override def unloadModule(moduleName: String): Unit = ??? } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala index 6e0c3a3f43af..5ebc79e51297 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala @@ -37,9 +37,9 @@ import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, Tabl import org.apache.flink.table.operations.{DataSetQueryOperation, JavaDataStreamQueryOperation, ScalaDataStreamQueryOperation} import org.apache.flink.table.planner.StreamPlanner import org.apache.flink.table.utils.TableTestUtil.createCatalogManager - import org.apache.calcite.plan.RelOptUtil import org.apache.calcite.rel.RelNode +import org.apache.flink.table.module.ModuleManager import org.junit.Assert.assertEquals import org.junit.rules.ExpectedException import org.junit.{ComparisonFailure, Rule} @@ -231,12 +231,14 @@ case class BatchTableTestUtil( val javaTableEnv = new JavaBatchTableEnvironmentImpl( javaEnv, new TableConfig, - catalogManager.getOrElse(createCatalogManager())) + catalogManager.getOrElse(createCatalogManager()), + new ModuleManager) val env = new ExecutionEnvironment(javaEnv) val tableEnv = new ScalaBatchTableEnvironmentImpl( env, new TableConfig, - catalogManager.getOrElse(createCatalogManager())) + catalogManager.getOrElse(createCatalogManager()), + new ModuleManager) def addTable[T: TypeInformation]( name: String, @@ -328,12 +330,14 @@ case class StreamTableTestUtil( private val tableConfig = new TableConfig private val manager: CatalogManager = catalogManager.getOrElse(createCatalogManager()) + private val moduleManager: ModuleManager = new ModuleManager private val executor: StreamExecutor = new StreamExecutor(javaEnv) - private val functionCatalog = new FunctionCatalog(manager) + private val functionCatalog = new FunctionCatalog(manager, moduleManager) private val streamPlanner = new StreamPlanner(executor, tableConfig, functionCatalog, manager) val javaTableEnv = new JavaStreamTableEnvironmentImpl( manager, + moduleManager, functionCatalog, tableConfig, javaEnv, @@ -344,6 +348,7 @@ case class StreamTableTestUtil( val env = new StreamExecutionEnvironment(javaEnv) val tableEnv = new ScalaStreamTableEnvironmentImpl( manager, + moduleManager, functionCatalog, tableConfig, env, From 76cd40e5c9bf9cb8cc2b4a3a7afd7168363f2e7a Mon Sep 17 00:00:00 2001 From: ouyangwulin Date: Thu, 24 Oct 2019 16:28:42 +0800 Subject: [PATCH 204/746] [FLINK-13873][rocksdb][metrics] Optionally expose column_family as variable --- .../rocks_db_native_metric_configuration.html | 5 ++++ .../state/RocksDBNativeMetricMonitor.java | 8 +++++- .../state/RocksDBNativeMetricOptions.java | 28 +++++++++++++++++++ 3 files changed, 40 insertions(+), 1 deletion(-) diff --git a/docs/_includes/generated/rocks_db_native_metric_configuration.html b/docs/_includes/generated/rocks_db_native_metric_configuration.html index 78fc523b35bf..420e7c333890 100644 --- a/docs/_includes/generated/rocks_db_native_metric_configuration.html +++ b/docs/_includes/generated/rocks_db_native_metric_configuration.html @@ -17,6 +17,11 @@ false Monitor the number of background errors in RocksDB. + +
state.backend.rocksdb.metrics.column-family-as-variable
+ false + Whether to expose the column family as a variable. +
state.backend.rocksdb.metrics.compaction-pending
false diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.java index a42ef5763ad2..356c496671be 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.java @@ -50,6 +50,8 @@ public class RocksDBNativeMetricMonitor implements Closeable { private final Object lock; + static final String COLUMN_FAMILY_KEY = "column_family"; + @GuardedBy("lock") private RocksDB rocksDB; @@ -71,7 +73,11 @@ public RocksDBNativeMetricMonitor( * @param handle native handle to the column family */ void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle handle) { - MetricGroup group = metricGroup.addGroup(columnFamilyName); + + boolean columnFamilyAsVariable = options.isColumnFaminlyAsVariable(); + MetricGroup group = columnFamilyAsVariable + ? metricGroup.addGroup(COLUMN_FAMILY_KEY, columnFamilyName) + : metricGroup.addGroup(columnFamilyName); for (String property : options.getProperties()) { RocksDBNativeMetricView gauge = new RocksDBNativeMetricView(handle, property); diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricOptions.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricOptions.java index f647e4c153f2..2d9ecaf0a70e 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricOptions.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricOptions.java @@ -38,6 +38,9 @@ */ public class RocksDBNativeMetricOptions implements Serializable { + public static final String METRICS_COLUMN_FAMILY_AS_VARIABLE_KEY = "state.backend.rocksdb.metrics" + + ".column-family-as-variable"; + public static final ConfigOption MONITOR_NUM_IMMUTABLE_MEM_TABLES = ConfigOptions .key(RocksDBProperty.NumImmutableMemTable.getConfigKey()) .defaultValue(false) @@ -149,6 +152,12 @@ public class RocksDBNativeMetricOptions implements Serializable { .key(RocksDBProperty.ActualDelayedWriteRate.getConfigKey()) .defaultValue(false) .withDescription("Monitor the current actual delayed write rate. 0 means no delay."); + + public static final ConfigOption COLUMN_FAMILY_AS_VARIABLE = ConfigOptions + .key(METRICS_COLUMN_FAMILY_AS_VARIABLE_KEY) + .defaultValue(false) + .withDescription("Whether to expose the column family as a variable."); + /** * Creates a {@link RocksDBNativeMetricOptions} based on an * external configuration. @@ -239,10 +248,13 @@ public static RocksDBNativeMetricOptions fromConfig(Configuration config) { options.enableActualDelayedWriteRate(); } + options.setColumnFamilyAsVariable(config.getBoolean(COLUMN_FAMILY_AS_VARIABLE)); + return options; } private Set properties; + private boolean columnFamilyAsVariable = COLUMN_FAMILY_AS_VARIABLE.defaultValue(); public RocksDBNativeMetricOptions() { this.properties = new HashSet<>(); @@ -401,6 +413,13 @@ public void enableActualDelayedWriteRate() { this.properties.add(RocksDBProperty.ActualDelayedWriteRate.getRocksDBProperty()); } + /** + * Returns the column family as variable. + */ + public void setColumnFamilyAsVariable(boolean columnFamilyAsVariable) { + this.columnFamilyAsVariable = columnFamilyAsVariable; + } + /** * @return the enabled RocksDB metrics */ @@ -416,4 +435,13 @@ public Collection getProperties() { public boolean isEnabled() { return !properties.isEmpty(); } + + /** + * {{@link RocksDBNativeMetricMonitor}} Whether to expose the column family as a variable.. + * + * @return true is column family to expose variable, false otherwise. + */ + public boolean isColumnFaminlyAsVariable() { + return this.columnFamilyAsVariable; + } } From c83f7244e1fdb940efb0df8716f49ae681199312 Mon Sep 17 00:00:00 2001 From: Aleksey Pak Date: Thu, 5 Sep 2019 11:50:56 +0200 Subject: [PATCH 205/746] [hotfix][runtime] Refactoring: remove ProcessingTimeService::shutdownAndAwaitPending method --- .../api/runtime/NeverFireProcessingTimeService.java | 8 -------- .../runtime/tasks/ProcessingTimeService.java | 12 ------------ .../runtime/tasks/SystemProcessingTimeService.java | 13 +++++++++++-- .../runtime/tasks/TestProcessingTimeService.java | 6 ------ 4 files changed, 11 insertions(+), 28 deletions(-) diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/NeverFireProcessingTimeService.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/NeverFireProcessingTimeService.java index b2a00a949eff..2939bd4d178f 100644 --- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/NeverFireProcessingTimeService.java +++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/NeverFireProcessingTimeService.java @@ -23,7 +23,6 @@ import org.apache.flink.util.concurrent.NeverCompleteFuture; import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; /** @@ -76,11 +75,4 @@ public boolean shutdownServiceUninterruptible(long timeoutMs) { shutdown.set(true); return shutdown.get(); } - - @Override - public boolean shutdownAndAwaitPending(long time, TimeUnit timeUnit) throws InterruptedException { - shutdown.set(true); - return shutdown.get(); - } } - diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/ProcessingTimeService.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/ProcessingTimeService.java index 4515ce298e5d..17ed0898bd67 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/ProcessingTimeService.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/ProcessingTimeService.java @@ -18,7 +18,6 @@ package org.apache.flink.streaming.runtime.tasks; import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.TimeUnit; /** * Defines the current processing time and handles all related actions, @@ -105,15 +104,4 @@ public abstract class ProcessingTimeService { * @return returns true iff the shutdown was completed. */ public abstract boolean shutdownServiceUninterruptible(long timeoutMs); - - /** - * Shuts down and clean up the timer service provider hard and immediately. This does wait - * for all timers to complete or until the time limit is exceeded. Any call to - * {@link #registerTimer(long, ProcessingTimeCallback)} will result in a hard exception after calling this method. - * @param time time to wait for termination. - * @param timeUnit time unit of parameter time. - * @return {@code true} if this timer service and all pending timers are terminated and - * {@code false} if the timeout elapsed before this happened. - */ - public abstract boolean shutdownAndAwaitPending(long time, TimeUnit timeUnit) throws InterruptedException; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeService.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeService.java index 22e61c774f2a..fdda09ad471b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeService.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeService.java @@ -190,8 +190,17 @@ public void shutdownService() { } } - @Override - public boolean shutdownAndAwaitPending(long time, TimeUnit timeUnit) throws InterruptedException { + /** + * Shuts down and clean up the timer service provider hard and immediately. This does wait + * for all timers to complete or until the time limit is exceeded. Any call to + * {@link #registerTimer(long, ProcessingTimeCallback)} will result in a hard exception after calling this method. + * @param time time to wait for termination. + * @param timeUnit time unit of parameter time. + * @return {@code true} if this timer service and all pending timers are terminated and + * {@code false} if the timeout elapsed before this happened. + */ + @VisibleForTesting + boolean shutdownAndAwaitPending(long time, TimeUnit timeUnit) throws InterruptedException { shutdownService(); return timerService.awaitTermination(time, timeUnit); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestProcessingTimeService.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestProcessingTimeService.java index f4a5f37bded7..3d1e624835e5 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestProcessingTimeService.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestProcessingTimeService.java @@ -140,12 +140,6 @@ public boolean shutdownServiceUninterruptible(long timeoutMs) { return true; } - @Override - public boolean shutdownAndAwaitPending(long time, TimeUnit timeUnit) throws InterruptedException { - shutdownService(); - return true; - } - public int getNumActiveTimers() { int count = 0; From 8665c389ea2214aebdc5f9a8e9f7df1b34b53e1b Mon Sep 17 00:00:00 2001 From: Aleksey Pak Date: Thu, 5 Sep 2019 12:12:38 +0200 Subject: [PATCH 206/746] [hotfix][runtime] Refactoring: move timer service lifecycle methods into a separate interface --- .../NeverFireProcessingTimeService.java | 4 +- .../runtime/tasks/OneInputStreamTask.java | 4 +- .../runtime/tasks/ProcessingTimeService.java | 59 +------------- .../streaming/runtime/tasks/StreamTask.java | 20 ++--- .../tasks/SystemProcessingTimeService.java | 4 +- .../tasks/TestProcessingTimeService.java | 6 +- .../streaming/runtime/tasks/TimerService.java | 78 +++++++++++++++++++ 7 files changed, 101 insertions(+), 74 deletions(-) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TimerService.java diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/NeverFireProcessingTimeService.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/NeverFireProcessingTimeService.java index 2939bd4d178f..26381519b338 100644 --- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/NeverFireProcessingTimeService.java +++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/NeverFireProcessingTimeService.java @@ -19,7 +19,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeCallback; -import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; +import org.apache.flink.streaming.runtime.tasks.TimerService; import org.apache.flink.util.concurrent.NeverCompleteFuture; import java.util.concurrent.ScheduledFuture; @@ -29,7 +29,7 @@ * A processing time service whose timers never fire so all timers are included in savepoints. */ @Internal -public final class NeverFireProcessingTimeService extends ProcessingTimeService { +public final class NeverFireProcessingTimeService implements TimerService { private static final NeverCompleteFuture FUTURE = new NeverCompleteFuture(Long.MAX_VALUE); private AtomicBoolean shutdown = new AtomicBoolean(true); 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 2610661d6c76..2ca611a96c6f 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 @@ -69,7 +69,7 @@ public OneInputStreamTask(Environment env) { /** * Constructor for initialization, possibly with initial state (recovery / savepoint / etc). * - *

This constructor accepts a special {@link ProcessingTimeService}. By default (and if + *

This constructor accepts a special {@link TimerService}. By default (and if * null is passes for the time provider) a {@link SystemProcessingTimeService DefaultTimerService} * will be used. * @@ -79,7 +79,7 @@ public OneInputStreamTask(Environment env) { @VisibleForTesting public OneInputStreamTask( Environment env, - @Nullable ProcessingTimeService timeProvider) { + @Nullable TimerService timeProvider) { super(env, timeProvider); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/ProcessingTimeService.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/ProcessingTimeService.java index 17ed0898bd67..7f239415efbd 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/ProcessingTimeService.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/ProcessingTimeService.java @@ -25,24 +25,13 @@ * *

The access to the time via {@link #getCurrentProcessingTime()} is always available, regardless of * whether the timer service has been shut down. - * - *

The registration of timers follows a life cycle of three phases: - *

    - *
  1. In the initial state, it accepts timer registrations and triggers when the time is reached.
  2. - *
  3. After calling {@link #quiesce()}, further calls to - * {@link #registerTimer(long, ProcessingTimeCallback)} will not register any further timers, and will - * return a "dummy" future as a result. This is used for clean shutdown, where currently firing - * timers are waited for and no future timers can be scheduled, without causing hard exceptions.
  4. - *
  5. After a call to {@link #shutdownService()}, all calls to {@link #registerTimer(long, ProcessingTimeCallback)} - * will result in a hard exception.
  6. - *
*/ -public abstract class ProcessingTimeService { +public interface ProcessingTimeService { /** * Returns the current processing time. */ - public abstract long getCurrentProcessingTime(); + long getCurrentProcessingTime(); /** * Registers a task to be executed when (processing) time is {@code timestamp}. @@ -53,7 +42,7 @@ public abstract class ProcessingTimeService { * @return The future that represents the scheduled task. This always returns some future, * even if the timer was shut down */ - public abstract ScheduledFuture registerTimer(long timestamp, ProcessingTimeCallback target); + ScheduledFuture registerTimer(long timestamp, ProcessingTimeCallback target); /** * Registers a task to be executed repeatedly at a fixed rate. @@ -63,45 +52,5 @@ public abstract class ProcessingTimeService { * @param period after the initial delay after which the callback is executed * @return Scheduled future representing the task to be executed repeatedly */ - public abstract ScheduledFuture scheduleAtFixedRate(ProcessingTimeCallback callback, long initialDelay, long period); - - /** - * Returns true if the service has been shut down, false otherwise. - */ - public abstract boolean isTerminated(); - - /** - * This method puts the service into a state where it does not register new timers, but - * returns for each call to {@link #registerTimer(long, ProcessingTimeCallback)} only a "mock" future. - * Furthermore, the method clears all not yet started timers. - * - *

This method can be used to cleanly shut down the timer service. The using components - * will not notice that the service is shut down (as for example via exceptions when registering - * a new timer), but the service will simply not fire any timer any more. - */ - public abstract void quiesce() throws InterruptedException; - - /** - * This method can be used after calling {@link #quiesce()}, and awaits the completion - * of currently executing timers. - */ - public abstract void awaitPendingAfterQuiesce() throws InterruptedException; - - /** - * Shuts down and clean up the timer service provider hard and immediately. This does not wait - * for any timer to complete. Any further call to {@link #registerTimer(long, ProcessingTimeCallback)} - * will result in a hard exception. - */ - public abstract void shutdownService(); - - /** - * Shuts down and clean up the timer service provider hard and immediately. This does not wait - * for any timer to complete. Any further call to {@link #registerTimer(long, ProcessingTimeCallback)} - * will result in a hard exception. This call cannot be interrupted and will block until the shutdown is completed - * or the timeout is exceeded. - * - * @param timeoutMs timeout for blocking on the service shutdown in milliseconds. - * @return returns true iff the shutdown was completed. - */ - public abstract boolean shutdownServiceUninterruptible(long timeoutMs); + ScheduledFuture scheduleAtFixedRate(ProcessingTimeCallback callback, long initialDelay, long period); } 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 07c0a59d0137..8e9fd68961cd 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 @@ -172,11 +172,11 @@ public abstract class StreamTask> private CheckpointStorageWorkerView checkpointStorage; /** - * The internal {@link ProcessingTimeService} used to define the current + * The internal {@link TimerService} used to define the current * processing time (default = {@code System.currentTimeMillis()}) and * register timers for tasks to be executed in the future. */ - protected ProcessingTimeService timerService; + protected TimerService timerService; private final Thread.UncaughtExceptionHandler uncaughtExceptionHandler; @@ -221,31 +221,31 @@ protected StreamTask(Environment env) { * Constructor for initialization, possibly with initial state (recovery / savepoint / etc). * * @param env The task environment for this task. - * @param timeProvider Optionally, a specific time provider to use. + * @param timerService Optionally, a specific timer service to use. */ - protected StreamTask(Environment env, @Nullable ProcessingTimeService timeProvider) { - this(env, timeProvider, FatalExitExceptionHandler.INSTANCE); + protected StreamTask(Environment env, @Nullable TimerService timerService) { + this(env, timerService, FatalExitExceptionHandler.INSTANCE); } /** * Constructor for initialization, possibly with initial state (recovery / savepoint / etc). * - *

This constructor accepts a special {@link ProcessingTimeService}. By default (and if - * null is passes for the time provider) a {@link SystemProcessingTimeService DefaultTimerService} + *

This constructor accepts a special {@link TimerService}. By default (and if + * null is passes for the timer service) a {@link SystemProcessingTimeService DefaultTimerService} * will be used. * * @param environment The task environment for this task. - * @param timeProvider Optionally, a specific time provider to use. + * @param timerService Optionally, a specific timer service to use. * @param uncaughtExceptionHandler to handle uncaught exceptions in the async operations thread pool */ protected StreamTask( Environment environment, - @Nullable ProcessingTimeService timeProvider, + @Nullable TimerService timerService, Thread.UncaughtExceptionHandler uncaughtExceptionHandler) { super(environment); - this.timerService = timeProvider; + this.timerService = timerService; this.uncaughtExceptionHandler = Preconditions.checkNotNull(uncaughtExceptionHandler); this.configuration = new StreamConfig(getTaskConfiguration()); this.accumulatorMap = getEnvironment().getAccumulatorRegistry().getUserMap(); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeService.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeService.java index fdda09ad471b..1be8cd42154f 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeService.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeService.java @@ -38,11 +38,11 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** - * A {@link ProcessingTimeService} which assigns as current processing time the result of calling + * A {@link TimerService} which assigns as current processing time the result of calling * {@link System#currentTimeMillis()} and registers timers using a {@link ScheduledThreadPoolExecutor}. */ @Internal -public class SystemProcessingTimeService extends ProcessingTimeService { +public class SystemProcessingTimeService implements TimerService { private static final Logger LOG = LoggerFactory.getLogger(SystemProcessingTimeService.class); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestProcessingTimeService.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestProcessingTimeService.java index 3d1e624835e5..e8d377029db5 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestProcessingTimeService.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestProcessingTimeService.java @@ -32,10 +32,10 @@ import java.util.concurrent.atomic.AtomicReference; /** - * This is a {@link ProcessingTimeService} used strictly for testing the + * This is a {@link TimerService} used strictly for testing the * processing time functionality. - * */ -public class TestProcessingTimeService extends ProcessingTimeService { + */ +public class TestProcessingTimeService implements TimerService { private volatile long currentTime = Long.MIN_VALUE; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TimerService.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TimerService.java new file mode 100644 index 000000000000..51982d53ad14 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TimerService.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.streaming.runtime.tasks; + +import org.apache.flink.annotation.Internal; + +/** + * A common timer service interface with life cycle methods. + * + *

The registration of timers follows a life cycle of three phases: + *

    + *
  1. In the initial state, it accepts timer registrations and triggers when the time is reached.
  2. + *
  3. After calling {@link #quiesce()}, further calls to + * {@link #registerTimer(long, ProcessingTimeCallback)} will not register any further timers, and will + * return a "dummy" future as a result. This is used for clean shutdown, where currently firing + * timers are waited for and no future timers can be scheduled, without causing hard exceptions.
  4. + *
  5. After a call to {@link #shutdownService()}, all calls to {@link #registerTimer(long, ProcessingTimeCallback)} + * will result in a hard exception.
  6. + *
+ */ +@Internal +public interface TimerService extends ProcessingTimeService { + + /** + * Returns true if the service has been shut down, false otherwise. + */ + boolean isTerminated(); + + /** + * This method puts the service into a state where it does not register new timers, but + * returns for each call to {@link #registerTimer(long, ProcessingTimeCallback)} only a "mock" future. + * Furthermore, the method clears all not yet started timers. + * + *

This method can be used to cleanly shut down the timer service. The using components + * will not notice that the service is shut down (as for example via exceptions when registering + * a new timer), but the service will simply not fire any timer any more. + */ + void quiesce() throws InterruptedException; + + /** + * This method can be used after calling {@link #quiesce()}, and awaits the completion + * of currently executing timers. + */ + void awaitPendingAfterQuiesce() throws InterruptedException; + + /** + * Shuts down and clean up the timer service provider hard and immediately. This does not wait + * for any timer to complete. Any further call to {@link #registerTimer(long, ProcessingTimeCallback)} + * will result in a hard exception. + */ + void shutdownService(); + + /** + * Shuts down and clean up the timer service provider hard and immediately. This does not wait + * for any timer to complete. Any further call to {@link #registerTimer(long, ProcessingTimeCallback)} + * will result in a hard exception. This call cannot be interrupted and will block until the shutdown is completed + * or the timeout is exceeded. + * + * @param timeoutMs timeout for blocking on the service shutdown in milliseconds. + * @return returns true iff the shutdown was completed. + */ + boolean shutdownServiceUninterruptible(long timeoutMs); +} From 9ca6e157f4ffe8b4b46e2893a3c2e2be805c46da Mon Sep 17 00:00:00 2001 From: Aleksey Pak Date: Tue, 17 Sep 2019 12:21:37 +0200 Subject: [PATCH 207/746] [hotfix][tests] Reformat StreamSourceOperatorWatermarksTest according to code style (method argument on new lines) --- .../StreamSourceOperatorWatermarksTest.java | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorWatermarksTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorWatermarksTest.java index b35d3bc2c42f..74aba703614f 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorWatermarksTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorWatermarksTest.java @@ -155,7 +155,8 @@ public void testAutomaticWatermarkContext() throws Exception { final List output = new ArrayList<>(); - StreamSourceContexts.getSourceContext(TimeCharacteristic.IngestionTime, + StreamSourceContexts.getSourceContext( + TimeCharacteristic.IngestionTime, operator.getContainingTask().getProcessingTimeService(), operator.getContainingTask().getCheckpointLock(), operator.getContainingTask().getStreamStatusMaintainer(), @@ -184,17 +185,19 @@ public void testAutomaticWatermarkContext() throws Exception { // ------------------------------------------------------------------------ @SuppressWarnings("unchecked") - private static void setupSourceOperator(StreamSource operator, + private static void setupSourceOperator( + StreamSource operator, TimeCharacteristic timeChar, long watermarkInterval) throws Exception { setupSourceOperator(operator, timeChar, watermarkInterval, new TestProcessingTimeService()); } @SuppressWarnings("unchecked") - private static void setupSourceOperator(StreamSource operator, - TimeCharacteristic timeChar, - long watermarkInterval, - final ProcessingTimeService timeProvider) throws Exception { + private static void setupSourceOperator( + StreamSource operator, + TimeCharacteristic timeChar, + long watermarkInterval, + final ProcessingTimeService timeProvider) throws Exception { ExecutionConfig executionConfig = new ExecutionConfig(); executionConfig.setAutoWatermarkInterval(watermarkInterval); From 92eb7b15eaaf3d14ca6155c3cf438417cbe5aa95 Mon Sep 17 00:00:00 2001 From: Aleksey Pak Date: Thu, 5 Sep 2019 15:53:57 +0200 Subject: [PATCH 208/746] [FLINK-14156][runtime] Refactoring: move ProcessingTimeService access request at operator level --- .../flink/state/api/input/KeyedStateInputFormat.java | 4 ++-- .../api/operators/AbstractStreamOperator.java | 7 +++++-- .../api/operators/StreamTaskStateInitializer.java | 3 +++ .../api/operators/StreamTaskStateInitializerImpl.java | 11 ++++------- .../flink/streaming/runtime/tasks/StreamTask.java | 3 +-- .../operators/StateInitializationContextImplTest.java | 5 +++-- .../operators/StreamTaskStateInitializerImplTest.java | 11 +++++------ .../operators/StreamSourceOperatorWatermarksTest.java | 2 +- .../streaming/runtime/tasks/OperatorChainTest.java | 4 ++-- .../flink/streaming/runtime/tasks/StreamTaskTest.java | 7 ++++--- .../util/AbstractStreamOperatorTestHarness.java | 3 +-- .../flink/streaming/util/MockStreamTaskBuilder.java | 2 +- .../restore/StreamOperatorSnapshotRestoreTest.java | 3 ++- 13 files changed, 34 insertions(+), 31 deletions(-) diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/input/KeyedStateInputFormat.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/input/KeyedStateInputFormat.java index f0bf44fb6cb8..454b74cf770e 100644 --- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/input/KeyedStateInputFormat.java +++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/input/KeyedStateInputFormat.java @@ -216,13 +216,13 @@ private Iterator getKeyIterator(SavepointRuntimeContext ctx) throws IOExcepti private StreamOperatorStateContext getStreamOperatorStateContext(Environment environment) throws IOException { StreamTaskStateInitializer initializer = new StreamTaskStateInitializerImpl( environment, - stateBackend, - new NeverFireProcessingTimeService()); + stateBackend); try { return initializer.streamOperatorStateContext( operatorState.getOperatorID(), operatorState.getOperatorID().toString(), + new NeverFireProcessingTimeService(), this, keySerializer, registry, diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java index 2e4310a77dd0..ffc33024de73 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java @@ -155,6 +155,7 @@ public abstract class AbstractStreamOperator // ---------------- time handler ------------------ + private transient ProcessingTimeService processingTimeService; protected transient InternalTimeServiceManager timeServiceManager; // ---------------- two-input operator watermarks ------------------ @@ -173,6 +174,7 @@ public abstract class AbstractStreamOperator public void setup(StreamTask containingTask, StreamConfig config, Output> output) { final Environment environment = containingTask.getEnvironment(); this.container = containingTask; + this.processingTimeService = containingTask.getProcessingTimeService(); this.config = config; try { OperatorMetricGroup operatorMetricGroup = environment.getMetricGroup().getOrAddOperator(config.getOperatorID(), config.getOperatorName()); @@ -253,6 +255,7 @@ public final void initializeState() throws Exception { streamTaskStateManager.streamOperatorStateContext( getOperatorID(), getClass().getSimpleName(), + getProcessingTimeService(), this, keySerializer, streamTaskCloseableRegistry, @@ -552,11 +555,11 @@ public OperatorStateBackend getOperatorStateBackend() { } /** - * Returns the {@link ProcessingTimeService} responsible for getting the current + * Returns the {@link ProcessingTimeService} responsible for getting the current * processing time and registering timers. */ protected ProcessingTimeService getProcessingTimeService() { - return container.getProcessingTimeService(); + return processingTimeService; } /** diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializer.java index 60e81828c54f..35ded9ecf070 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializer.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializer.java @@ -22,6 +22,7 @@ import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -39,6 +40,7 @@ public interface StreamTaskStateInitializer { * * @param operatorID the id of the operator for which the context is created. Cannot be null. * @param operatorClassName the classname of the operator instance for which the context is created. Cannot be null. + * @param processingTimeService * @param keyContext the key context of the operator instance for which the context is created Cannot be null. * @param keySerializer the key-serializer for the operator. Can be null. * @param streamTaskCloseableRegistry the closeable registry to which created closeable objects will be registered. @@ -49,6 +51,7 @@ public interface StreamTaskStateInitializer { StreamOperatorStateContext streamOperatorStateContext( @Nonnull OperatorID operatorID, @Nonnull String operatorClassName, + @Nonnull ProcessingTimeService processingTimeService, @Nonnull KeyContext keyContext, @Nullable TypeSerializer keySerializer, @Nonnull CloseableRegistry streamTaskCloseableRegistry, diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java index 5264e50a10a8..e33ce6116e21 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java @@ -79,9 +79,6 @@ public class StreamTaskStateInitializerImpl implements StreamTaskStateInitialize */ private final Environment environment; - /** This processing time service is required to construct an internal timer service manager. */ - private final ProcessingTimeService processingTimeService; - /** The state manager of the tasks provides the information used to restore potential previous state. */ private final TaskStateManager taskStateManager; @@ -90,13 +87,11 @@ public class StreamTaskStateInitializerImpl implements StreamTaskStateInitialize public StreamTaskStateInitializerImpl( Environment environment, - StateBackend stateBackend, - ProcessingTimeService processingTimeService) { + StateBackend stateBackend) { this.environment = environment; this.taskStateManager = Preconditions.checkNotNull(environment.getTaskStateManager()); this.stateBackend = Preconditions.checkNotNull(stateBackend); - this.processingTimeService = processingTimeService; } // ----------------------------------------------------------------------------------------------------------------- @@ -105,6 +100,7 @@ public StreamTaskStateInitializerImpl( public StreamOperatorStateContext streamOperatorStateContext( @Nonnull OperatorID operatorID, @Nonnull String operatorClassName, + @Nonnull ProcessingTimeService processingTimeService, @Nonnull KeyContext keyContext, @Nullable TypeSerializer keySerializer, @Nonnull CloseableRegistry streamTaskCloseableRegistry, @@ -155,7 +151,7 @@ public StreamOperatorStateContext streamOperatorStateContext( streamTaskCloseableRegistry.registerCloseable(rawOperatorStateInputs); // -------------- Internal Timer Service Manager -------------- - timeServiceManager = internalTimeServiceManager(keyedStatedBackend, keyContext, rawKeyedStateInputs); + timeServiceManager = internalTimeServiceManager(keyedStatedBackend, keyContext, processingTimeService, rawKeyedStateInputs); // -------------- Preparing return value -------------- @@ -199,6 +195,7 @@ public StreamOperatorStateContext streamOperatorStateContext( protected InternalTimeServiceManager internalTimeServiceManager( AbstractKeyedStateBackend keyedStatedBackend, KeyContext keyContext, //the operator + ProcessingTimeService processingTimeService, Iterable rawKeyedStates) throws Exception { if (keyedStatedBackend == null) { 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 8e9fd68961cd..727b087a9881 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 @@ -349,8 +349,7 @@ protected void finishTask() throws Exception { public StreamTaskStateInitializer createStreamTaskStateInitializer() { return new StreamTaskStateInitializerImpl( getEnvironment(), - stateBackend, - timerService); + stateBackend); } protected Counter setupNumRecordsInCounter(StreamOperator streamOperator) { diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StateInitializationContextImplTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StateInitializationContextImplTest.java index bcfe195f3406..a8c8da9a6199 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StateInitializationContextImplTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StateInitializationContextImplTest.java @@ -171,13 +171,13 @@ public void setUp() throws Exception { StateBackend stateBackend = new MemoryStateBackend(1024); StreamTaskStateInitializer streamTaskStateManager = new StreamTaskStateInitializerImpl( environment, - stateBackend, - mock(ProcessingTimeService.class)) { + stateBackend) { @Override protected InternalTimeServiceManager internalTimeServiceManager( AbstractKeyedStateBackend keyedStatedBackend, KeyContext keyContext, + ProcessingTimeService processingTimeService, Iterable rawKeyedStates) throws Exception { // We do not initialize a timer service manager here, because it would already consume the raw keyed @@ -193,6 +193,7 @@ protected InternalTimeServiceManager internalTimeServiceManager( StreamOperatorStateContext stateContext = streamTaskStateManager.streamOperatorStateContext( operatorID, "TestOperatorClass", + mock(ProcessingTimeService.class), mockOperator, // notice that this essentially disables the previous test of the keyed stream because it was and is always // consumed by the timer service. diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImplTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImplTest.java index 430c0576896a..004bdfc2d7cc 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImplTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImplTest.java @@ -93,6 +93,7 @@ public void testNoRestore() throws Exception { StreamOperatorStateContext stateContext = streamTaskStateManager.streamOperatorStateContext( streamOperator.getOperatorID(), streamOperator.getClass().getSimpleName(), + new TestProcessingTimeService(), streamOperator, typeSerializer, closeableRegistry, @@ -201,6 +202,7 @@ public OperatorStateBackend createOperatorStateBackend( StreamOperatorStateContext stateContext = streamTaskStateManager.streamOperatorStateContext( streamOperator.getOperatorID(), streamOperator.getClass().getSimpleName(), + new TestProcessingTimeService(), streamOperator, typeSerializer, closeableRegistry, @@ -268,22 +270,19 @@ private StreamTaskStateInitializer streamTaskStateManager( DummyEnvironment dummyEnvironment = new DummyEnvironment("test-task", 1, 0); dummyEnvironment.setTaskStateManager(taskStateManager); - ProcessingTimeService processingTimeService = new TestProcessingTimeService(); - if (createTimerServiceManager) { return new StreamTaskStateInitializerImpl( dummyEnvironment, - stateBackend, - processingTimeService); + stateBackend); } else { return new StreamTaskStateInitializerImpl( dummyEnvironment, - stateBackend, - processingTimeService) { + stateBackend) { @Override protected InternalTimeServiceManager internalTimeServiceManager( AbstractKeyedStateBackend keyedStatedBackend, KeyContext keyContext, + ProcessingTimeService processingTimeService, Iterable rawKeyedStates) throws Exception { return null; } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorWatermarksTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorWatermarksTest.java index 74aba703614f..4960af751692 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorWatermarksTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorWatermarksTest.java @@ -157,7 +157,7 @@ public void testAutomaticWatermarkContext() throws Exception { StreamSourceContexts.getSourceContext( TimeCharacteristic.IngestionTime, - operator.getContainingTask().getProcessingTimeService(), + processingTimeService, operator.getContainingTask().getCheckpointLock(), operator.getContainingTask().getStreamStatusMaintainer(), new CollectorOutput(output), diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OperatorChainTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OperatorChainTest.java index e7bf45a69086..e6efa45e22b7 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OperatorChainTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OperatorChainTest.java @@ -34,6 +34,7 @@ import org.apache.flink.streaming.runtime.tasks.OperatorChain.BroadcastingOutputCollector; import org.apache.flink.streaming.runtime.tasks.OperatorChain.ChainingOutput; import org.apache.flink.streaming.runtime.tasks.OperatorChain.WatermarkGaugeExposingOutput; +import org.apache.flink.streaming.util.MockStreamTaskBuilder; import org.junit.Test; @@ -78,8 +79,7 @@ private static > OperatorChain setupOpera checkArgument(operators.length > 0); try (MockEnvironment env = MockEnvironment.builder().build()) { - - final StreamTask containingTask = new OneInputStreamTask>(env); + final StreamTask containingTask = new MockStreamTaskBuilder(env).build(); final StreamStatusProvider statusProvider = mock(StreamStatusProvider.class); final StreamConfig cfg = new StreamConfig(new Configuration()); 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 61953b43201f..39c0f7e03cab 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 @@ -975,7 +975,7 @@ void waitForTaskCompletion(boolean cancelled) throws Exception { public static class NoOpStreamTask> extends StreamTask { public NoOpStreamTask(Environment environment) { - super(environment, null); + super(environment); } @Override @@ -1257,11 +1257,12 @@ protected void cleanup() throws Exception {} @Override public StreamTaskStateInitializer createStreamTaskStateInitializer() { final StreamTaskStateInitializer streamTaskStateManager = super.createStreamTaskStateInitializer(); - return (operatorID, operatorClassName, keyContext, keySerializer, closeableRegistry, metricGroup) -> { + return (operatorID, operatorClassName, processingTimeService, keyContext, keySerializer, closeableRegistry, metricGroup) -> { final StreamOperatorStateContext context = streamTaskStateManager.streamOperatorStateContext( operatorID, operatorClassName, + processingTimeService, keyContext, keySerializer, closeableRegistry, @@ -1425,7 +1426,7 @@ private static class ThreadInspectingTask extends StreamTask InternalTimeServiceManager internalTimeServiceManager( AbstractKeyedStateBackend keyedStatedBackend, KeyContext keyContext, + ProcessingTimeService processingTimeService, Iterable rawKeyedStates) throws Exception { return null; From e4c0042ad290eabacc2df54948ec69888ddf1c59 Mon Sep 17 00:00:00 2001 From: Aleksey Pak Date: Tue, 17 Sep 2019 11:28:38 +0200 Subject: [PATCH 209/746] [FLINK-14156][runtime] Instantiate ProcessingTimeService per operator (with timer execution by operator precedence) --- .../api/operators/AbstractStreamOperator.java | 2 +- .../operators/InternalTimerServiceImpl.java | 12 +- .../tasks/ProcessingTimeServiceImpl.java | 49 ++++++ .../streaming/runtime/tasks/StreamTask.java | 57 +++---- .../tasks/SystemProcessingTimeService.java | 33 ++-- .../tasks/TestProcessingTimeService.java | 2 +- ...treamSourceOperatorLatencyMetricsTest.java | 6 +- .../StreamSourceOperatorWatermarksTest.java | 6 +- .../StreamTaskOperatorTimerTest.java | 148 ++++++++++++++++++ .../operators/StreamTaskTimerTest.java | 2 +- .../TestProcessingTimeServiceTest.java | 13 +- .../runtime/tasks/OneInputStreamTaskTest.java | 4 +- .../runtime/tasks/StreamConfigChainer.java | 26 ++- .../runtime/tasks/StreamTaskTest.java | 7 +- .../runtime/tasks/StreamTaskTestHarness.java | 14 +- .../SystemProcessingTimeServiceTest.java | 44 +----- .../AbstractStreamOperatorTestHarness.java | 7 +- .../flink/streaming/util/MockStreamTask.java | 24 +-- .../streaming/util/MockStreamTaskBuilder.java | 10 +- 19 files changed, 310 insertions(+), 156 deletions(-) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/ProcessingTimeServiceImpl.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskOperatorTimerTest.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java index ffc33024de73..cd7b004b8899 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java @@ -174,7 +174,7 @@ public abstract class AbstractStreamOperator public void setup(StreamTask containingTask, StreamConfig config, Output> output) { final Environment environment = containingTask.getEnvironment(); this.container = containingTask; - this.processingTimeService = containingTask.getProcessingTimeService(); + this.processingTimeService = containingTask.getProcessingTimeService(config.getChainIndex()); this.config = config; try { OperatorMetricGroup operatorMetricGroup = environment.getMetricGroup().getOrAddOperator(config.getOperatorID(), config.getOperatorName()); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImpl.java index 08f3a1d06c17..76cf8ca1ba61 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImpl.java @@ -24,7 +24,6 @@ import org.apache.flink.runtime.state.InternalPriorityQueue; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyGroupedInternalPriorityQueue; -import org.apache.flink.streaming.runtime.tasks.ProcessingTimeCallback; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.FlinkRuntimeException; @@ -43,7 +42,7 @@ /** * {@link InternalTimerService} that stores timers on the Java heap. */ -public class InternalTimerServiceImpl implements InternalTimerService, ProcessingTimeCallback { +public class InternalTimerServiceImpl implements InternalTimerService { private final ProcessingTimeService processingTimeService; @@ -175,7 +174,7 @@ public void startTimerService( // re-register the restored timers (if any) final InternalTimer headTimer = processingTimeTimersQueue.peek(); if (headTimer != null) { - nextTimer = processingTimeService.registerTimer(headTimer.getTimestamp(), this); + nextTimer = processingTimeService.registerTimer(headTimer.getTimestamp(), this::onProcessingTime); } this.isInitialized = true; } else { @@ -206,7 +205,7 @@ public void registerProcessingTimeTimer(N namespace, long time) { if (nextTimer != null) { nextTimer.cancel(false); } - nextTimer = processingTimeService.registerTimer(time, this); + nextTimer = processingTimeService.registerTimer(time, this::onProcessingTime); } } } @@ -246,8 +245,7 @@ private void foreachTimer(BiConsumerWithException consumer, } } - @Override - public void onProcessingTime(long time) throws Exception { + private void onProcessingTime(long time) throws Exception { // null out the timer in case the Triggerable calls registerProcessingTimeTimer() // inside the callback. nextTimer = null; @@ -261,7 +259,7 @@ public void onProcessingTime(long time) throws Exception { } if (timer != null && nextTimer == null) { - nextTimer = processingTimeService.registerTimer(timer.getTimestamp(), this); + nextTimer = processingTimeService.registerTimer(timer.getTimestamp(), this::onProcessingTime); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/ProcessingTimeServiceImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/ProcessingTimeServiceImpl.java new file mode 100644 index 000000000000..1dbe2303b1a4 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/ProcessingTimeServiceImpl.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.runtime.tasks; + +import java.util.concurrent.ScheduledFuture; +import java.util.function.Function; + +class ProcessingTimeServiceImpl implements ProcessingTimeService { + private final TimerService timerService; + private final Function processingTimeCallbackWrapper; + + ProcessingTimeServiceImpl( + TimerService timerService, + Function processingTimeCallbackWrapper) { + this.timerService = timerService; + this.processingTimeCallbackWrapper = processingTimeCallbackWrapper; + } + + @Override + public long getCurrentProcessingTime() { + return timerService.getCurrentProcessingTime(); + } + + @Override + public ScheduledFuture registerTimer(long timestamp, ProcessingTimeCallback target) { + return timerService.registerTimer(timestamp, processingTimeCallbackWrapper.apply(target)); + } + + @Override + public ScheduledFuture scheduleAtFixedRate(ProcessingTimeCallback callback, long initialDelay, long period) { + return timerService.scheduleAtFixedRate(processingTimeCallbackWrapper.apply(callback), initialDelay, period); + } +} 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 727b087a9881..4469da26ee54 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 @@ -379,7 +379,9 @@ public final void invoke() throws Exception { ThreadFactory timerThreadFactory = new DispatcherThreadFactory(TRIGGER_THREAD_GROUP, "Time Trigger for " + getName()); - timerService = new SystemProcessingTimeService(new TimerInvocationContext(), timerThreadFactory); + timerService = new SystemProcessingTimeService( + this::handleTimerException, + timerThreadFactory); } operatorChain = new OperatorChain<>(this, recordWriters); @@ -980,16 +982,20 @@ private StateBackend createStateBackend() throws Exception { } /** - * Returns the {@link ProcessingTimeService} responsible for telling the current - * processing time and registering timers. + * Returns the {@link TimerService} responsible for telling the current processing time and registering actual timers. */ - public ProcessingTimeService getProcessingTimeService() { - if (timerService == null) { - throw new IllegalStateException("The timer service has not been initialized."); - } + @VisibleForTesting + TimerService getTimerService() { + Preconditions.checkState(timerService != null, "The timer service has not been initialized."); return timerService; } + public ProcessingTimeService getProcessingTimeService(int operatorIndex) { + Preconditions.checkState(timerService != null, "The timer service has not been initialized."); + MailboxExecutor mailboxExecutor = mailboxProcessor.getMailboxExecutor(operatorIndex); + return new ProcessingTimeServiceImpl(timerService, callback -> deferCallbackToMailbox(mailboxExecutor, callback)); + } + /** * Handles an exception thrown by another thread (e.g. a TriggerTask), * other than the one executing the main task by failing the task entirely. @@ -1436,30 +1442,27 @@ private static RecordWriter>> crea return output; } - private class TimerInvocationContext implements SystemProcessingTimeService.ScheduledCallbackExecutionContext { - @Override - public void invoke(ProcessingTimeCallback callback, long timestamp) { + private void handleTimerException(Exception ex) { + handleAsyncException("Caught exception while processing timer.", new TimerException(ex)); + } + + private ProcessingTimeCallback deferCallbackToMailbox(MailboxExecutor mailboxExecutor, ProcessingTimeCallback callback) { + return timestamp -> { + mailboxExecutor.execute( + () -> invokeProcessingTimeCallback(callback, timestamp), + "Timer callback for %s @ %d", + callback, + timestamp); + }; + } + + private void invokeProcessingTimeCallback(ProcessingTimeCallback callback, long timestamp) { + synchronized (getCheckpointLock()) { try { - mailboxProcessor.getMailboxExecutor(TaskMailbox.MAX_PRIORITY).execute( - () -> invokeProcessingTimeCallback(callback, timestamp), - "Timer callback for %s @ %d", - callback, - timestamp); + callback.onProcessingTime(timestamp); } catch (Throwable t) { handleAsyncException("Caught exception while processing timer.", new TimerException(t)); } } - - private void invokeProcessingTimeCallback(ProcessingTimeCallback callback, long timestamp) { - synchronized (getCheckpointLock()) { - try { - callback.onProcessingTime(timestamp); - } catch (Throwable t) { - handleAsyncException( - "Caught exception while processing timer.", - new TimerException(t)); - } - } - } } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeService.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeService.java index 1be8cd42154f..82ccc59d1791 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeService.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SystemProcessingTimeService.java @@ -55,17 +55,17 @@ public class SystemProcessingTimeService implements TimerService { /** The executor service that schedules and calls the triggers of this task. */ private final ScheduledThreadPoolExecutor timerService; - private final ScheduledCallbackExecutionContext callbackExecutionContext; + private final ExceptionHandler exceptionHandler; private final AtomicInteger status; @VisibleForTesting - SystemProcessingTimeService(ScheduledCallbackExecutionContext callbackExecutionContext) { - this(callbackExecutionContext, null); + SystemProcessingTimeService(ExceptionHandler exceptionHandler) { + this(exceptionHandler, null); } - SystemProcessingTimeService(ScheduledCallbackExecutionContext callbackExecutionContext, ThreadFactory threadFactory) { + SystemProcessingTimeService(ExceptionHandler exceptionHandler, ThreadFactory threadFactory) { - this.callbackExecutionContext = checkNotNull(callbackExecutionContext); + this.exceptionHandler = checkNotNull(exceptionHandler); this.status = new AtomicInteger(STATUS_ALIVE); if (threadFactory == null) { @@ -250,24 +250,23 @@ int getNumTasksScheduled() { // ------------------------------------------------------------------------ /** - * A context to which {@link ProcessingTimeCallback} would be passed to be invoked when a timer is up. + * An exception handler, called when {@link ProcessingTimeCallback} throws an exception. */ - interface ScheduledCallbackExecutionContext { - - void invoke(ProcessingTimeCallback callback, long timestamp); + interface ExceptionHandler { + void handleException(Exception ex); } private Runnable wrapOnTimerCallback(ProcessingTimeCallback callback, long timestamp) { - return new ScheduledTask(status, callbackExecutionContext, callback, timestamp, 0); + return new ScheduledTask(status, exceptionHandler, callback, timestamp, 0); } private Runnable wrapOnTimerCallback(ProcessingTimeCallback callback, long nextTimestamp, long period) { - return new ScheduledTask(status, callbackExecutionContext, callback, nextTimestamp, period); + return new ScheduledTask(status, exceptionHandler, callback, nextTimestamp, period); } private static final class ScheduledTask implements Runnable { private final AtomicInteger serviceStatus; - private final ScheduledCallbackExecutionContext callbackExecutionContext; + private final ExceptionHandler exceptionHandler; private final ProcessingTimeCallback callback; private long nextTimestamp; @@ -275,12 +274,12 @@ private static final class ScheduledTask implements Runnable { ScheduledTask( AtomicInteger serviceStatus, - ScheduledCallbackExecutionContext callbackExecutionContext, + ExceptionHandler exceptionHandler, ProcessingTimeCallback callback, long timestamp, long period) { this.serviceStatus = serviceStatus; - this.callbackExecutionContext = callbackExecutionContext; + this.exceptionHandler = exceptionHandler; this.callback = callback; this.nextTimestamp = timestamp; this.period = period; @@ -291,7 +290,11 @@ public void run() { if (serviceStatus.get() != STATUS_ALIVE) { return; } - callbackExecutionContext.invoke(callback, nextTimestamp); + try { + callback.onProcessingTime(nextTimestamp); + } catch (Exception ex) { + exceptionHandler.handleException(ex); + } nextTimestamp += period; } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestProcessingTimeService.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestProcessingTimeService.java index e8d377029db5..204e7e9ed28d 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestProcessingTimeService.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/TestProcessingTimeService.java @@ -32,7 +32,7 @@ import java.util.concurrent.atomic.AtomicReference; /** - * This is a {@link TimerService} used strictly for testing the + * This is a {@link TimerService} and {@link ProcessingTimeService} used strictly for testing the * processing time functionality. */ public class TestProcessingTimeService implements TimerService { diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorLatencyMetricsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorLatencyMetricsTest.java index 8e7fdbdc2de0..c996c96f5e0f 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorLatencyMetricsTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorLatencyMetricsTest.java @@ -36,9 +36,9 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; import org.apache.flink.streaming.runtime.tasks.OperatorChain; -import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; +import org.apache.flink.streaming.runtime.tasks.TimerService; import org.apache.flink.streaming.util.CollectorOutput; import org.apache.flink.streaming.util.MockStreamTask; import org.apache.flink.streaming.util.MockStreamTaskBuilder; @@ -210,7 +210,7 @@ private static void setupSourceOperator( StreamSource operator, ExecutionConfig executionConfig, Environment env, - ProcessingTimeService timeProvider) { + TimerService timerService) { StreamConfig cfg = new StreamConfig(new Configuration()); cfg.setStateBackend(new MemoryStateBackend()); @@ -222,7 +222,7 @@ private static void setupSourceOperator( MockStreamTask mockTask = new MockStreamTaskBuilder(env) .setConfig(cfg) .setExecutionConfig(executionConfig) - .setProcessingTimeService(timeProvider) + .setTimerService(timerService) .build(); operator.setup(mockTask, cfg, (Output>) mock(Output.class)); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorWatermarksTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorWatermarksTest.java index 4960af751692..fd814bdc75ea 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorWatermarksTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorWatermarksTest.java @@ -38,9 +38,9 @@ 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.ProcessingTimeService; import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; +import org.apache.flink.streaming.runtime.tasks.TimerService; import org.apache.flink.streaming.util.CollectorOutput; import org.apache.flink.streaming.util.MockStreamTask; import org.apache.flink.streaming.util.MockStreamTaskBuilder; @@ -197,7 +197,7 @@ private static void setupSourceOperator( StreamSource operator, TimeCharacteristic timeChar, long watermarkInterval, - final ProcessingTimeService timeProvider) throws Exception { + final TimerService timeProvider) throws Exception { ExecutionConfig executionConfig = new ExecutionConfig(); executionConfig.setAutoWatermarkInterval(watermarkInterval); @@ -217,7 +217,7 @@ private static void setupSourceOperator( .setConfig(cfg) .setExecutionConfig(executionConfig) .setStreamStatusMaintainer(streamStatusMaintainer) - .setProcessingTimeService(timeProvider) + .setTimerService(timeProvider) .build(); operator.setup(mockTask, cfg, (Output>) mock(Output.class)); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskOperatorTimerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskOperatorTimerTest.java new file mode 100644 index 000000000000..e3017ebde2af --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskOperatorTimerTest.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.operators; + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.runtime.jobgraph.OperatorID; +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.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.YieldingOperatorFactory; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask; +import org.apache.flink.streaming.runtime.tasks.OneInputStreamTaskTestHarness; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; +import org.apache.flink.streaming.runtime.tasks.StreamTask; +import org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxExecutor; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; + +/** + * Test to verify that timer triggers are run according to operator precedence (combined with yield() at operator level). + */ +public class StreamTaskOperatorTimerTest extends TestLogger { + private static List events; + + @Test + public void testOperatorYieldExecutesSelectedTimers() throws Exception { + events = new ArrayList<>(); + final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness<>( + OneInputStreamTask::new, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO); + + testHarness.setupOperatorChain(new OperatorID(), new TestOperatorFactory<>()) + .chain(new OperatorID(), new TestOperatorFactory<>(), IntSerializer.INSTANCE) + .finish(); + + testHarness.invoke(); + testHarness.waitForTaskRunning(); + + testHarness.processElement(new StreamRecord<>(42)); + + testHarness.endInput(); + testHarness.waitForTaskCompletion(); + + assertThat(events, is(Arrays.asList("Timer:1:0", "Timer:0:0"))); + } + + private static class TestOperatorFactory implements OneInputStreamOperatorFactory, YieldingOperatorFactory { + private MailboxExecutor mailboxExecutor; + + @Override + public void setMailboxExecutor(MailboxExecutor mailboxExecutor) { + this.mailboxExecutor = mailboxExecutor; + } + + @Override + public > Operator createStreamOperator( + StreamTask containingTask, + StreamConfig config, + Output> output) { + TestOperator operator = new TestOperator<>(config.getChainIndex(), mailboxExecutor); + operator.setup(containingTask, config, output); + return (Operator) operator; + } + + @Override + public void setChainingStrategy(ChainingStrategy strategy) { + } + + @Override + public ChainingStrategy getChainingStrategy() { + return ChainingStrategy.ALWAYS; + } + + @Override + public Class getStreamOperatorClass(ClassLoader classLoader) { + return TestOperator.class; + } + } + + private static class TestOperator + extends AbstractStreamOperator + implements OneInputStreamOperator { + + private final transient MailboxExecutor mailboxExecutor; + private final int chainIndex; + private transient int count; + + TestOperator(int chainIndex, MailboxExecutor mailboxExecutor) { + this.chainIndex = chainIndex; + this.mailboxExecutor = mailboxExecutor; + } + + @Override + public void processElement(StreamRecord element) throws Exception { + // The test operator creates a one-time timer (per input element) and passes the input element further + // (to the next operator or to the output). + // The execution is yielded until the operator's timer trigger is confirmed. + + int index = count; + ProcessingTimeService processingTimeService = getProcessingTimeService(); + processingTimeService + .registerTimer( + processingTimeService.getCurrentProcessingTime() + 1000L, + timestamp -> { + events.add("Timer:" + chainIndex + ":" + index); + --count; + }); + + ++count; + output.collect(element); + + while (count > 0) { + mailboxExecutor.yield(); + } + } + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskTimerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskTimerTest.java index 946ec7d379a8..df7503671810 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskTimerTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskTimerTest.java @@ -57,7 +57,7 @@ public class StreamTaskTimerTest extends TestLogger { @Before public void setup() throws Exception { testHarness = startTestHarness(); - timeService = testHarness.getProcessingTimeService(); + timeService = testHarness.getTask().getProcessingTimeService(0); } @After diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TestProcessingTimeServiceTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TestProcessingTimeServiceTest.java index 7d327bcdf209..ed5fac3ddcb4 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TestProcessingTimeServiceTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TestProcessingTimeServiceTest.java @@ -25,6 +25,7 @@ import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask; import org.apache.flink.streaming.runtime.tasks.OneInputStreamTaskTestHarness; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeCallback; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; import org.junit.Test; @@ -55,26 +56,26 @@ public void testCustomTimeServiceProvider() throws Throwable { testHarness.invoke(); - final OneInputStreamTask mapTask = testHarness.getTask(); + ProcessingTimeService processingTimeService = testHarness.getTask().getProcessingTimeService(0); - assertEquals(Long.MIN_VALUE, testHarness.getProcessingTimeService().getCurrentProcessingTime()); + assertEquals(Long.MIN_VALUE, processingTimeService.getCurrentProcessingTime()); tp.setCurrentTime(11); - assertEquals(testHarness.getProcessingTimeService().getCurrentProcessingTime(), 11); + assertEquals(processingTimeService.getCurrentProcessingTime(), 11); tp.setCurrentTime(15); tp.setCurrentTime(16); - assertEquals(testHarness.getProcessingTimeService().getCurrentProcessingTime(), 16); + assertEquals(processingTimeService.getCurrentProcessingTime(), 16); // register 2 tasks - mapTask.getProcessingTimeService().registerTimer(30, new ProcessingTimeCallback() { + processingTimeService.registerTimer(30, new ProcessingTimeCallback() { @Override public void onProcessingTime(long timestamp) { } }); - mapTask.getProcessingTimeService().registerTimer(40, new ProcessingTimeCallback() { + processingTimeService.registerTimer(40, new ProcessingTimeCallback() { @Override public void onProcessingTime(long timestamp) { diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java index 8d34f89e2b3b..9765abacbc26 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java @@ -587,7 +587,7 @@ public void testQuiesceTimerServiceAfterOpClose() throws Exception { testHarness.waitForTaskRunning(); SystemProcessingTimeService timeService = (SystemProcessingTimeService) - testHarness.getTask().getProcessingTimeService(); + testHarness.getTimerService(); // verify that the timer service is running Assert.assertTrue(timeService.isAlive()); @@ -647,7 +647,7 @@ public void close() throws Exception { // verify that the timer service is still running Assert.assertTrue( - ((SystemProcessingTimeService) getContainingTask().getProcessingTimeService()) + ((SystemProcessingTimeService) getContainingTask().getTimerService()) .isAlive()); super.close(); } 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 0efb278946e9..56d8a298b177 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,6 +25,7 @@ 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.OneInputStreamOperatorFactory; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.StreamOperatorFactory; @@ -49,20 +50,15 @@ public class StreamConfigChainer { 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) { + StreamConfigChainer(OperatorID headOperatorID, StreamConfig headConfig) { this.headConfig = checkNotNull(headConfig); this.tailConfig = checkNotNull(headConfig); this.bufferTimeout = headConfig.getBufferTimeout(); - head(headOperatorID, headOperatorFactory); + head(headOperatorID); } - private void head(OperatorID headOperatorID, StreamOperatorFactory headOperatorFactory) { - headConfig.setStreamOperatorFactory(headOperatorFactory); + private void head(OperatorID headOperatorID) { headConfig.setOperatorID(headOperatorID); headConfig.setChainStart(); headConfig.setChainIndex(chainIndex); @@ -81,18 +77,17 @@ public StreamConfigChainer chain( OperatorID operatorID, OneInputStreamOperator operator, TypeSerializer typeSerializer) { - return chain(operatorID, operator, typeSerializer, typeSerializer); + return chain(operatorID, operator, typeSerializer, typeSerializer, false); } - public StreamConfigChainer chain( + public StreamConfigChainer chain( OperatorID operatorID, - OneInputStreamOperator operator, - TypeSerializer inputSerializer, - TypeSerializer outputSerializer) { - return chain(operatorID, operator, inputSerializer, outputSerializer, false); + OneInputStreamOperatorFactory operatorFactory, + TypeSerializer typeSerializer) { + return chain(operatorID, operatorFactory, typeSerializer, typeSerializer, false); } - public StreamConfigChainer chain( + private StreamConfigChainer chain( OperatorID operatorID, OneInputStreamOperator operator, TypeSerializer inputSerializer, @@ -112,6 +107,7 @@ public StreamConfigChainer chain( TypeSerializer inputSerializer, TypeSerializer outputSerializer, boolean createKeyedStateBackend) { + chainIndex++; tailConfig.setChainedOutputs(Collections.singletonList( 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 39c0f7e03cab..9eb220c37c2b 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 @@ -849,9 +849,10 @@ public void testOperatorNotSupportedByNonNetworkCreditMode() throws Exception { public void testRecordWriterClosedOnStreamOperatorFactoryDeserializationError() throws Exception { Configuration taskConfiguration = new Configuration(); StreamConfig streamConfig = new StreamConfig(taskConfiguration); + streamConfig.setStreamOperatorFactory(new UnusedOperatorFactory()); // 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); + StreamConfigChainer cfg = new StreamConfigChainer(new OperatorID(42, 42), streamConfig); cfg.chain( new OperatorID(44, 44), new UnusedOperatorFactory(), @@ -1442,11 +1443,11 @@ protected void init() throws Exception { checkTaskThreadInfo(); // Create a time trigger to validate that it would also be invoked in the task's thread. - getProcessingTimeService().registerTimer(0, new ProcessingTimeCallback() { + getProcessingTimeService(0).registerTimer(0, new ProcessingTimeCallback() { @Override public void onProcessingTime(long timestamp) throws Exception { - hasTimerTriggered = true; checkTaskThreadInfo(); + hasTimerTriggered = true; } }); } 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 bc26c70e041a..39f6b6b77335 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 @@ -42,7 +42,9 @@ import org.apache.flink.streaming.api.graph.StreamEdge; import org.apache.flink.streaming.api.graph.StreamNode; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +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 org.apache.flink.streaming.runtime.streamrecord.StreamElement; import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer; @@ -148,8 +150,8 @@ public StreamMockEnvironment getEnvironment() { return mockEnv; } - public ProcessingTimeService getProcessingTimeService() { - return taskThread.task.getProcessingTimeService(); + public TimerService getTimerService() { + return taskThread.task.getTimerService(); } /** @@ -418,9 +420,15 @@ public void endInput(int gateIndex, int channelIndex) { } public StreamConfigChainer setupOperatorChain(OperatorID headOperatorId, StreamOperator headOperator) { + return setupOperatorChain(headOperatorId, SimpleOperatorFactory.of(headOperator)); + } + + public StreamConfigChainer setupOperatorChain(OperatorID headOperatorId, StreamOperatorFactory headOperatorFactory) { Preconditions.checkState(!setupCalled, "This harness was already setup."); setupCalled = true; - return new StreamConfigChainer(headOperatorId, headOperator, getStreamConfig()); + StreamConfig streamConfig = getStreamConfig(); + streamConfig.setStreamOperatorFactory(headOperatorFactory); + return new StreamConfigChainer(headOperatorId, streamConfig); } // ------------------------------------------------------------------------ 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 493dbf3636f1..18220b7c408a 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 @@ -33,7 +33,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReentrantLock; -import java.util.function.Consumer; import static org.hamcrest.Matchers.instanceOf; import static org.junit.Assert.assertEquals; @@ -363,15 +362,13 @@ public void testShutdownServiceUninterruptible() { private static SystemProcessingTimeService createSystemProcessingTimeService(CompletableFuture errorFuture) { Preconditions.checkArgument(!errorFuture.isDone()); - return new SystemProcessingTimeService(new TestOnTimerCompletablyCallbackContext(errorFuture)); + return new SystemProcessingTimeService(errorFuture::complete); } private static SystemProcessingTimeService createSystemProcessingTimeService(AtomicReference errorRef) { - Preconditions.checkArgument(errorRef.get() == null); - return new SystemProcessingTimeService( - new TestOnTimerCallbackContext(ex -> errorRef.compareAndSet(null, ex))); + return new SystemProcessingTimeService(ex -> errorRef.compareAndSet(null, ex)); } private static SystemProcessingTimeService createBlockingSystemProcessingTimeService( @@ -382,8 +379,7 @@ private static SystemProcessingTimeService createBlockingSystemProcessingTimeSer Preconditions.checkState(!check.get()); - final SystemProcessingTimeService timeService = new SystemProcessingTimeService( - new TestOnTimerCallbackContext(exception -> {})); + final SystemProcessingTimeService timeService = new SystemProcessingTimeService(exception -> {}); timeService.scheduleAtFixedRate( timestamp -> { @@ -413,38 +409,4 @@ private static SystemProcessingTimeService createBlockingSystemProcessingTimeSer return timeService; } - - private static class TestOnTimerCallbackContext implements SystemProcessingTimeService.ScheduledCallbackExecutionContext { - private final Consumer exceptionHandler; - - TestOnTimerCallbackContext(Consumer exceptionHandler) { - this.exceptionHandler = exceptionHandler; - } - - @Override - public void invoke(ProcessingTimeCallback callback, long timestamp) { - try { - callback.onProcessingTime(timestamp); - } catch (Throwable t) { - exceptionHandler.accept(t); - } - } - } - - 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); - } - } - } } 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 f49c49a879fb..0d6edbd293e8 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 @@ -66,7 +66,6 @@ import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; -import org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxExecutorFactory; import org.apache.flink.util.OutputTag; import org.apache.flink.util.Preconditions; @@ -255,7 +254,7 @@ private AbstractStreamOperatorTestHarness( .setStreamTaskStateInitializer(streamTaskStateInitializer) .setClosableRegistry(closableRegistry) .setCheckpointStorage(checkpointStorage) - .setProcessingTimeService(processingTimeService) + .setTimerService(processingTimeService) .setHandleAsyncException(handleAsyncException) .build(); } @@ -344,10 +343,6 @@ public void setup(TypeSerializer outputSerializer) { } } - private MailboxExecutorFactory getMailboxExecutorFactory() { - return mockTask.getMailboxExecutorFactory(); - } - /** * Calls {@link org.apache.flink.streaming.api.operators.StreamOperator#initializeState()}. * Calls {@link org.apache.flink.streaming.api.operators.SetupableStreamOperator#setup(StreamTask, StreamConfig, Output)} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTask.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTask.java index 259292b94d25..9685ee17e9d6 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTask.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTask.java @@ -29,8 +29,8 @@ import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.streaming.runtime.tasks.StreamTask; +import org.apache.flink.streaming.runtime.tasks.TimerService; import org.apache.flink.streaming.runtime.tasks.mailbox.execution.DefaultActionContext; -import org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxExecutor; import java.util.Map; import java.util.function.BiConsumer; @@ -62,11 +62,11 @@ public MockStreamTask( CloseableRegistry closableRegistry, StreamStatusMaintainer streamStatusMaintainer, CheckpointStorageWorkerView checkpointStorage, - ProcessingTimeService processingTimeService, + TimerService timerService, BiConsumer handleAsyncException, Map> accumulatorMap ) { - super(environment); + super(environment, timerService); this.name = name; this.checkpointLock = checkpointLock; this.config = config; @@ -75,7 +75,7 @@ public MockStreamTask( this.closableRegistry = closableRegistry; this.streamStatusMaintainer = streamStatusMaintainer; this.checkpointStorage = checkpointStorage; - this.processingTimeService = processingTimeService; + this.processingTimeService = timerService; this.handleAsyncException = handleAsyncException; this.accumulatorMap = accumulatorMap; } @@ -143,11 +143,6 @@ public CheckpointStorageWorkerView getCheckpointStorage() { return checkpointStorage; } - @Override - public ProcessingTimeService getProcessingTimeService() { - return processingTimeService; - } - @Override public void handleAsyncException(String message, Throwable exception) { handleAsyncException.accept(message, exception); @@ -158,13 +153,8 @@ public void handleAsyncException(String message, Throwable exception) { return accumulatorMap; } - /** - * Creates the mailbox executor for the operator with the given configuration. - * - * @param config the config of the operator. - * @return the mailbox executor of the operator. - */ - public MailboxExecutor getMailboxExecutor(StreamConfig config) { - return getMailboxExecutorFactory().createExecutor(config.getChainIndex()); + @Override + public ProcessingTimeService getProcessingTimeService(int operatorIndex) { + return processingTimeService; } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTaskBuilder.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTaskBuilder.java index ddf053bf27d0..967c4df5e964 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTaskBuilder.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTaskBuilder.java @@ -33,8 +33,8 @@ import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer; import org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl; import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; -import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; +import org.apache.flink.streaming.runtime.tasks.TimerService; import java.util.Collections; import java.util.Map; @@ -52,7 +52,7 @@ public class MockStreamTaskBuilder { private CloseableRegistry closableRegistry = new CloseableRegistry(); private StreamStatusMaintainer streamStatusMaintainer = new MockStreamStatusMaintainer(); private CheckpointStorageWorkerView checkpointStorage; - private ProcessingTimeService processingTimeService = new TestProcessingTimeService(); + private TimerService timerService = new TestProcessingTimeService(); private StreamTaskStateInitializer streamTaskStateInitializer; private BiConsumer handleAsyncException = (message, throwable) -> { }; private Map> accumulatorMap = Collections.emptyMap(); @@ -105,8 +105,8 @@ public MockStreamTaskBuilder setCheckpointStorage(CheckpointStorage checkpointSt return this; } - public MockStreamTaskBuilder setProcessingTimeService(ProcessingTimeService processingTimeService) { - this.processingTimeService = processingTimeService; + public MockStreamTaskBuilder setTimerService(TimerService timerService) { + this.timerService = timerService; return this; } @@ -126,7 +126,7 @@ public MockStreamTask build() { closableRegistry, streamStatusMaintainer, checkpointStorage, - processingTimeService, + timerService, handleAsyncException, accumulatorMap); } From b93e2201a5dc6406634161d6418672c6f9a8992f Mon Sep 17 00:00:00 2001 From: Aleksey Pak Date: Fri, 27 Sep 2019 15:01:06 +0200 Subject: [PATCH 210/746] [hotfix][tests] RocksDBAsyncSnapshotTest: use test harness helper method to wait until task is running --- .../state/RocksDBAsyncSnapshotTest.java | 24 ++----------------- 1 file changed, 2 insertions(+), 22 deletions(-) diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java index 362224895051..8ab52280f6b2 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java @@ -69,7 +69,6 @@ import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask; import org.apache.flink.streaming.runtime.tasks.OneInputStreamTaskTestHarness; import org.apache.flink.streaming.runtime.tasks.StreamMockEnvironment; -import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.util.IOUtils; import org.apache.flink.util.TestLogger; @@ -85,7 +84,6 @@ import java.io.File; import java.io.IOException; -import java.lang.reflect.Field; import java.util.Arrays; import java.util.Collections; import java.util.Map; @@ -216,19 +214,10 @@ public void declineCheckpoint( AtomicReference errorRef = new AtomicReference<>(); mockEnv.setExternalExceptionHandler(errorRef::set); testHarness.invoke(mockEnv); + testHarness.waitForTaskRunning(); final OneInputStreamTask task = testHarness.getTask(); - // wait for the task to be running - for (Field field: StreamTask.class.getDeclaredFields()) { - if (field.getName().equals("isRunning")) { - field.setAccessible(true); - while (!field.getBoolean(task)) { - Thread.sleep(10); - } - } - } - task.triggerCheckpointAsync(new CheckpointMetaData(42, 17), CheckpointOptions.forCheckpointWithDefaultLocation(), false) .get(); @@ -331,19 +320,10 @@ public CheckpointStateOutputStream createCheckpointStateOutputStream(Checkpointe blockerCheckpointStreamFactory.setWaiterLatch(new OneShotLatch()); testHarness.invoke(mockEnv); + testHarness.waitForTaskRunning(); final OneInputStreamTask task = testHarness.getTask(); - // wait for the task to be running - for (Field field: StreamTask.class.getDeclaredFields()) { - if (field.getName().equals("isRunning")) { - field.setAccessible(true); - while (!field.getBoolean(task)) { - Thread.sleep(10); - } - } - } - task.triggerCheckpointAsync( new CheckpointMetaData(42, 17), CheckpointOptions.forCheckpointWithDefaultLocation(), From 432efb3e7b12aa71a56464c7651181c28b1c652d Mon Sep 17 00:00:00 2001 From: Aleksey Pak Date: Wed, 23 Oct 2019 15:17:49 +0200 Subject: [PATCH 211/746] [FLINK-14156][tests] Rewrite StreamTaskOperatorTimerTest to observe flow via output (to avoid using mutable static fields) --- .../StreamTaskOperatorTimerTest.java | 58 ++++++++++++------- 1 file changed, 37 insertions(+), 21 deletions(-) diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskOperatorTimerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskOperatorTimerTest.java index e3017ebde2af..12e0c347bc58 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskOperatorTimerTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskOperatorTimerTest.java @@ -19,7 +19,7 @@ package org.apache.flink.streaming.runtime.operators; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; @@ -50,32 +50,35 @@ * Test to verify that timer triggers are run according to operator precedence (combined with yield() at operator level). */ public class StreamTaskOperatorTimerTest extends TestLogger { - private static List events; + private static final String TRIGGER_PREFIX = "trigger:"; + private static final String RESULT_PREFIX = "timer:"; @Test public void testOperatorYieldExecutesSelectedTimers() throws Exception { - events = new ArrayList<>(); - final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness<>( - OneInputStreamTask::new, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO); + final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness<>( + OneInputStreamTask::new, + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO); - testHarness.setupOperatorChain(new OperatorID(), new TestOperatorFactory<>()) - .chain(new OperatorID(), new TestOperatorFactory<>(), IntSerializer.INSTANCE) - .finish(); + testHarness.setupOperatorChain(new OperatorID(), new TestOperatorFactory()) + .chain(new OperatorID(), new TestOperatorFactory(), StringSerializer.INSTANCE) + .finish(); testHarness.invoke(); testHarness.waitForTaskRunning(); - testHarness.processElement(new StreamRecord<>(42)); + final String trigger = TRIGGER_PREFIX + 42; + testHarness.processElement(new StreamRecord<>(trigger)); testHarness.endInput(); testHarness.waitForTaskCompletion(); - assertThat(events, is(Arrays.asList("Timer:1:0", "Timer:0:0"))); + List events = new ArrayList<>(); + testHarness.getOutput().forEach(element -> events.add(((StreamRecord) element).getValue())); + assertThat(events, is(Arrays.asList(trigger, RESULT_PREFIX + "1:0", RESULT_PREFIX + "0:0"))); } - private static class TestOperatorFactory implements OneInputStreamOperatorFactory, YieldingOperatorFactory { + private static class TestOperatorFactory implements OneInputStreamOperatorFactory, YieldingOperatorFactory { private MailboxExecutor mailboxExecutor; @Override @@ -84,11 +87,11 @@ public void setMailboxExecutor(MailboxExecutor mailboxExecutor) { } @Override - public > Operator createStreamOperator( + public > Operator createStreamOperator( StreamTask containingTask, StreamConfig config, - Output> output) { - TestOperator operator = new TestOperator<>(config.getChainIndex(), mailboxExecutor); + Output> output) { + TestOperator operator = new TestOperator(config.getChainIndex(), mailboxExecutor); operator.setup(containingTask, config, output); return (Operator) operator; } @@ -108,9 +111,9 @@ public Class getStreamOperatorClass(ClassLoader classL } } - private static class TestOperator - extends AbstractStreamOperator - implements OneInputStreamOperator { + private static class TestOperator + extends AbstractStreamOperator + implements OneInputStreamOperator { private final transient MailboxExecutor mailboxExecutor; private final int chainIndex; @@ -122,7 +125,13 @@ private static class TestOperator } @Override - public void processElement(StreamRecord element) throws Exception { + public void processElement(StreamRecord element) throws Exception { + if (!isTriggerEvent(element)) { + // Pass through entries that are not triggers as is, so that the test can observe them. + output.collect(element); + return; + } + // The test operator creates a one-time timer (per input element) and passes the input element further // (to the next operator or to the output). // The execution is yielded until the operator's timer trigger is confirmed. @@ -133,7 +142,7 @@ public void processElement(StreamRecord element) throws Exception { .registerTimer( processingTimeService.getCurrentProcessingTime() + 1000L, timestamp -> { - events.add("Timer:" + chainIndex + ":" + index); + output.collect(new StreamRecord<>(RESULT_PREFIX + chainIndex + ":" + index)); --count; }); @@ -144,5 +153,12 @@ public void processElement(StreamRecord element) throws Exception { mailboxExecutor.yield(); } } + + private static boolean isTriggerEvent(StreamRecord element) { + if (element.isRecord()) { + return element.getValue().startsWith(TRIGGER_PREFIX); + } + return false; + } } } From 97d732c86ee80e67d5f37643c861eed4908d62a1 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Tue, 15 Oct 2019 16:23:39 +0800 Subject: [PATCH 212/746] [hotfix][tests] Clean up ExecutionFailureHandlerTest --- .../flip1/ExecutionFailureHandlerTest.java | 77 +++++++++---------- 1 file changed, 36 insertions(+), 41 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandlerTest.java index c757ab70e540..5b7ad7504d39 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandlerTest.java @@ -23,12 +23,12 @@ import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; import org.apache.flink.util.TestLogger; +import org.junit.Before; import org.junit.Test; -import java.util.HashSet; +import java.util.Collections; import java.util.Set; -import static org.apache.flink.util.Preconditions.checkNotNull; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -40,24 +40,32 @@ */ public class ExecutionFailureHandlerTest extends TestLogger { + private static final long restartDelayMs = 1234L; + + private TestFailoverStrategy failoverStrategy; + + private TestRestartBackoffTimeStrategy restartStrategy; + + private ExecutionFailureHandler executionFailureHandler; + + @Before + public void setUp() { + failoverStrategy = new TestFailoverStrategy(); + restartStrategy = new TestRestartBackoffTimeStrategy(true, restartDelayMs); + executionFailureHandler = new ExecutionFailureHandler(failoverStrategy, restartStrategy); + } + /** * Tests the case that task restarting is accepted. */ @Test public void testNormalFailureHandling() { - // failover strategy which always suggests restarting the given tasks - Set tasksToRestart = new HashSet<>(); - tasksToRestart.add(new ExecutionVertexID(new JobVertexID(), 0)); - FailoverStrategy failoverStrategy = new TestFailoverStrategy(tasksToRestart); - - // restart strategy which accepts restarting - boolean canRestart = true; - long restartDelayMs = 1234; - RestartBackoffTimeStrategy restartStrategy = new TestRestartBackoffTimeStrategy(canRestart, restartDelayMs); - ExecutionFailureHandler executionFailureHandler = new ExecutionFailureHandler(failoverStrategy, restartStrategy); + final Set tasksToRestart = Collections.singleton( + new ExecutionVertexID(new JobVertexID(), 0)); + failoverStrategy.setTasksToRestart(tasksToRestart); // trigger a task failure - FailureHandlingResult result = executionFailureHandler.getFailureHandlingResult( + final FailureHandlingResult result = executionFailureHandler.getFailureHandlingResult( new ExecutionVertexID(new JobVertexID(), 0), new Exception("test failure")); @@ -78,19 +86,11 @@ public void testNormalFailureHandling() { */ @Test public void testRestartingSuppressedFailureHandlingResult() { - // failover strategy which always suggests restarting the given tasks - Set tasksToRestart = new HashSet<>(); - tasksToRestart.add(new ExecutionVertexID(new JobVertexID(), 0)); - FailoverStrategy failoverStrategy = new TestFailoverStrategy(tasksToRestart); - - // restart strategy which suppresses restarting - boolean canRestart = false; - long restartDelayMs = 1234; - RestartBackoffTimeStrategy restartStrategy = new TestRestartBackoffTimeStrategy(canRestart, restartDelayMs); - ExecutionFailureHandler executionFailureHandler = new ExecutionFailureHandler(failoverStrategy, restartStrategy); + // restart strategy suppresses restarting + restartStrategy.setCanRestart(false); // trigger a task failure - FailureHandlingResult result = executionFailureHandler.getFailureHandlingResult( + final FailureHandlingResult result = executionFailureHandler.getFailureHandlingResult( new ExecutionVertexID(new JobVertexID(), 0), new Exception("test failure")); @@ -117,19 +117,8 @@ public void testRestartingSuppressedFailureHandlingResult() { */ @Test public void testNonRecoverableFailureHandlingResult() { - // failover strategy which always suggests restarting the given tasks - Set tasksToRestart = new HashSet<>(); - tasksToRestart.add(new ExecutionVertexID(new JobVertexID(), 0)); - FailoverStrategy failoverStrategy = new TestFailoverStrategy(tasksToRestart); - - // restart strategy which accepts restarting - boolean canRestart = true; - long restartDelayMs = 1234; - RestartBackoffTimeStrategy restartStrategy = new TestRestartBackoffTimeStrategy(canRestart, restartDelayMs); - ExecutionFailureHandler executionFailureHandler = new ExecutionFailureHandler(failoverStrategy, restartStrategy); - // trigger an unrecoverable task failure - FailureHandlingResult result = executionFailureHandler.getFailureHandlingResult( + final FailureHandlingResult result = executionFailureHandler.getFailureHandlingResult( new ExecutionVertexID(new JobVertexID(), 0), new Exception(new SuppressRestartsException(new Exception("test failure")))); @@ -172,18 +161,24 @@ public void testUnrecoverableErrorCheck() { // ------------------------------------------------------------------------ /** - * A FailoverStrategy implementation for tests. It always suggest restarting the given task set on construction. + * A FailoverStrategy implementation for tests. It always suggests restarting the given tasks to restart. */ private class TestFailoverStrategy implements FailoverStrategy { - private final Set tasksToRestart; + private Set tasksToRestart; - public TestFailoverStrategy(Set tasksToRestart) { - this.tasksToRestart = checkNotNull(tasksToRestart); + public TestFailoverStrategy() { + } + + public void setTasksToRestart(final Set tasksToRestart) { + this.tasksToRestart = tasksToRestart; } @Override - public Set getTasksNeedingRestart(ExecutionVertexID executionVertexId, Throwable cause) { + public Set getTasksNeedingRestart( + final ExecutionVertexID executionVertexId, + final Throwable cause) { + return tasksToRestart; } } From b0e5464beb604f5a6ed3fb9653b32e5c10de7704 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Tue, 15 Oct 2019 16:50:40 +0800 Subject: [PATCH 213/746] [FLINK-14232][runtime] Support global failure handling in DefaultScheduler - Enable ExecutionFailureHandler for global failure handling - Introduce handleGlobalFailure interface to SchedulerNG - Refactor InternalTaskFailuresListener to also support global failure listening - Notify the InternalFailuresListener about global failure for NG scheduler in ExecutionGraph#failGlobal This closes #9904. --- .../executiongraph/ExecutionGraph.java | 13 ++++--- .../flip1/ExecutionFailureHandler.java | 35 +++++++++++++++++-- .../runtime/scheduler/DefaultScheduler.java | 12 ++++++- ...ner.java => InternalFailuresListener.java} | 7 ++-- .../runtime/scheduler/LegacyScheduler.java | 5 +++ .../runtime/scheduler/SchedulerBase.java | 5 ++- .../flink/runtime/scheduler/SchedulerNG.java | 2 ++ ...chedulerNgOnInternalFailuresListener.java} | 12 +++++-- .../flip1/ExecutionFailureHandlerTest.java | 34 ++++++++++++++---- .../scheduler/DefaultSchedulerTest.java | 20 +++++++++++ 10 files changed, 121 insertions(+), 24 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/{InternalTaskFailuresListener.java => InternalFailuresListener.java} (84%) rename flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/{UpdateSchedulerNgOnInternalTaskFailuresListener.java => UpdateSchedulerNgOnInternalFailuresListener.java} (79%) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index ddd48fd5bc27..e61296613ee4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -69,7 +69,7 @@ import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; import org.apache.flink.runtime.query.KvStateLocationRegistry; -import org.apache.flink.runtime.scheduler.InternalTaskFailuresListener; +import org.apache.flink.runtime.scheduler.InternalFailuresListener; import org.apache.flink.runtime.scheduler.adapter.ExecutionGraphToSchedulingTopologyAdapter; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; import org.apache.flink.runtime.scheduler.strategy.SchedulingExecutionVertex; @@ -264,7 +264,7 @@ public class ExecutionGraph implements AccessExecutionGraph { private SchedulingTopology schedulingTopology; @Nullable - private InternalTaskFailuresListener internalTaskFailuresListener; + private InternalFailuresListener internalTaskFailuresListener; /** Counts all restarts. Used by other Gauges/Meters and does not register to metric group. */ private final Counter numberOfRestartsCounter = new SimpleCounter(); @@ -885,7 +885,7 @@ public StringifiedAccumulatorResult[] getAccumulatorResultsStringified() { return StringifiedAccumulatorResult.stringifyAccumulatorResults(accumulatorMap); } - public void enableNgScheduling(final InternalTaskFailuresListener internalTaskFailuresListener) { + public void enableNgScheduling(final InternalFailuresListener internalTaskFailuresListener) { checkNotNull(internalTaskFailuresListener); checkState(this.internalTaskFailuresListener == null, "enableNgScheduling can be only called once"); this.internalTaskFailuresListener = internalTaskFailuresListener; @@ -1156,9 +1156,8 @@ void failGlobalIfExecutionIsStillRunning(Throwable cause, ExecutionAttemptID fai */ public void failGlobal(Throwable t) { if (!isLegacyScheduling()) { - // Implementation does not work for new generation scheduler. - // Will be fixed with FLINK-14232. - ExceptionUtils.rethrow(t); + internalTaskFailuresListener.notifyGlobalFailure(t); + return; } assertRunningInJobMasterMainThread(); @@ -1823,7 +1822,7 @@ void assertRunningInJobMasterMainThread() { void notifySchedulerNgAboutInternalTaskFailure(final ExecutionAttemptID attemptId, final Throwable t) { if (internalTaskFailuresListener != null) { - internalTaskFailuresListener.notifyFailed(attemptId, t); + internalTaskFailuresListener.notifyTaskFailure(attemptId, t); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandler.java index 322599238b33..882fc13e9a06 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandler.java @@ -24,6 +24,9 @@ import org.apache.flink.runtime.throwable.ThrowableType; import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -33,6 +36,8 @@ */ public class ExecutionFailureHandler { + private final FailoverTopology failoverTopology; + /** Strategy to judge which tasks should be restarted. */ private final FailoverStrategy failoverStrategy; @@ -42,13 +47,16 @@ public class ExecutionFailureHandler { /** * Creates the handler to deal with task failures. * + * @param failoverTopology contains the topology info for failover * @param failoverStrategy helps to decide tasks to restart on task failures * @param restartBackoffTimeStrategy helps to decide whether to restart failed tasks and the restarting delay */ public ExecutionFailureHandler( + FailoverTopology failoverTopology, FailoverStrategy failoverStrategy, RestartBackoffTimeStrategy restartBackoffTimeStrategy) { + this.failoverTopology = checkNotNull(failoverTopology); this.failoverStrategy = checkNotNull(failoverStrategy); this.restartBackoffTimeStrategy = checkNotNull(restartBackoffTimeStrategy); } @@ -62,6 +70,29 @@ public ExecutionFailureHandler( * @return result of the failure handling */ public FailureHandlingResult getFailureHandlingResult(ExecutionVertexID failedTask, Throwable cause) { + return handleFailure(cause, failoverStrategy.getTasksNeedingRestart(failedTask, cause)); + } + + /** + * Return result of failure handling on a global failure. Can be a set of task vertices to restart + * and a delay of the restarting. Or that the failure is not recoverable and the reason for it. + * + * @param cause of the task failure + * @return result of the failure handling + */ + public FailureHandlingResult getGlobalFailureHandlingResult(final Throwable cause) { + return handleFailure( + cause, + StreamSupport + .stream(failoverTopology.getFailoverVertices().spliterator(), false) + .map(FailoverVertex::getExecutionVertexID) + .collect(Collectors.toSet())); + } + + private FailureHandlingResult handleFailure( + final Throwable cause, + final Set verticesToRestart) { + if (isUnrecoverableError(cause)) { return FailureHandlingResult.unrecoverable(new JobException("The failure is not recoverable", cause)); } @@ -69,11 +100,11 @@ public FailureHandlingResult getFailureHandlingResult(ExecutionVertexID failedTa restartBackoffTimeStrategy.notifyFailure(cause); if (restartBackoffTimeStrategy.canRestart()) { return FailureHandlingResult.restartable( - failoverStrategy.getTasksNeedingRestart(failedTask, cause), + verticesToRestart, restartBackoffTimeStrategy.getBackoffTime()); } else { return FailureHandlingResult.unrecoverable( - new JobException("Failed task restarting is suppressed by " + restartBackoffTimeStrategy, cause)); + new JobException("Recovery is suppressed by " + restartBackoffTimeStrategy, cause)); } } 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 638a633c6337..a249b8dcf76d 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 @@ -139,7 +139,10 @@ public DefaultScheduler( this.executionVertexOperations = checkNotNull(executionVertexOperations); this.executionVertexVersioner = checkNotNull(executionVertexVersioner); - this.executionFailureHandler = new ExecutionFailureHandler(failoverStrategyFactory.create(getFailoverTopology()), restartBackoffTimeStrategy); + this.executionFailureHandler = new ExecutionFailureHandler( + getFailoverTopology(), + failoverStrategyFactory.create(getFailoverTopology()), + restartBackoffTimeStrategy); this.schedulingStrategy = schedulingStrategyFactory.createInstance(this, getSchedulingTopology(), getJobGraph()); this.executionSlotAllocator = checkNotNull(executionSlotAllocatorFactory).createInstance(getInputsLocationsRetriever()); } @@ -173,6 +176,13 @@ private void handleTaskFailure(final ExecutionVertexID executionVertexId, final maybeRestartTasks(failureHandlingResult); } + @Override + public void handleGlobalFailure(final Throwable error) { + log.info("Trying to recover from a global failure.", error); + final FailureHandlingResult failureHandlingResult = executionFailureHandler.getGlobalFailureHandlingResult(error); + maybeRestartTasks(failureHandlingResult); + } + private void maybeRestartTasks(final FailureHandlingResult failureHandlingResult) { if (failureHandlingResult.canRestart()) { restartTasksWithDelay(failureHandlingResult); 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/InternalFailuresListener.java similarity index 84% rename from flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/InternalTaskFailuresListener.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/InternalFailuresListener.java index 6dbda3bfb106..ad3bdcfad845 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/InternalTaskFailuresListener.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/InternalFailuresListener.java @@ -25,13 +25,14 @@ import org.apache.flink.runtime.taskmanager.TaskExecutionState; /** - * This interface enables subscribing to Task failures that are detected from the JobMaster side + * This interface enables subscribing to 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 { +public interface InternalFailuresListener { - void notifyFailed(ExecutionAttemptID attemptId, Throwable t); + void notifyTaskFailure(ExecutionAttemptID attemptId, Throwable t); + void notifyGlobalFailure(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 bc5c45091f73..7be0d9d24787 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 @@ -91,4 +91,9 @@ protected void startSchedulingInternal() { executionGraph.failGlobal(t); } } + + @Override + public void handleGlobalFailure(Throwable cause) { + throw new IllegalStateException("Unexpected handleGlobalFailure(...) call"); + } } 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 1b4f62edcf1c..65fcecf09daf 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 @@ -310,7 +310,7 @@ protected final InputsLocationsRetriever getInputsLocationsRetriever() { } protected final void prepareExecutionGraphForNgScheduling() { - executionGraph.enableNgScheduling(new UpdateSchedulerNgOnInternalTaskFailuresListener(this, jobGraph.getJobID())); + executionGraph.enableNgScheduling(new UpdateSchedulerNgOnInternalFailuresListener(this, jobGraph.getJobID())); executionGraph.transitionToRunning(); } @@ -376,6 +376,9 @@ public CompletableFuture getTerminationFuture() { return executionGraph.getTerminationFuture().thenApply(FunctionUtils.nullFn()); } + @Override + public abstract void handleGlobalFailure(final Throwable cause); + @Override public final boolean updateTaskExecutionState(final TaskExecutionState taskExecutionState) { final Optional executionVertexId = getExecutionVertexId(taskExecutionState.getID()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerNG.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerNG.java index 0c8c2c9537a7..f45e2387dc28 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerNG.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerNG.java @@ -77,6 +77,8 @@ public interface SchedulerNG { CompletableFuture getTerminationFuture(); + void handleGlobalFailure(Throwable cause); + boolean updateTaskExecutionState(TaskExecutionState taskExecutionState); SerializedInputSplit requestNextInputSplit(JobVertexID vertexID, ExecutionAttemptID executionAttempt) throws IOException; 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/UpdateSchedulerNgOnInternalFailuresListener.java similarity index 79% rename from flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/UpdateSchedulerNgOnInternalTaskFailuresListener.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/UpdateSchedulerNgOnInternalFailuresListener.java index f37e4ecddb19..b930a30a95c2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/UpdateSchedulerNgOnInternalTaskFailuresListener.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/UpdateSchedulerNgOnInternalFailuresListener.java @@ -28,14 +28,15 @@ /** * Calls {@link SchedulerNG#updateTaskExecutionState(TaskExecutionState)} on task failure. + * Calls {@link SchedulerNG#handleGlobalFailure(Throwable)} on global failures. */ -class UpdateSchedulerNgOnInternalTaskFailuresListener implements InternalTaskFailuresListener { +class UpdateSchedulerNgOnInternalFailuresListener implements InternalFailuresListener { private final SchedulerNG schedulerNg; private final JobID jobId; - public UpdateSchedulerNgOnInternalTaskFailuresListener( + public UpdateSchedulerNgOnInternalFailuresListener( final SchedulerNG schedulerNg, final JobID jobId) { @@ -44,11 +45,16 @@ public UpdateSchedulerNgOnInternalTaskFailuresListener( } @Override - public void notifyFailed(final ExecutionAttemptID attemptId, final Throwable t) { + public void notifyTaskFailure(final ExecutionAttemptID attemptId, final Throwable t) { schedulerNg.updateTaskExecutionState(new TaskExecutionState( jobId, attemptId, ExecutionState.FAILED, t)); } + + @Override + public void notifyGlobalFailure(Throwable t) { + schedulerNg.handleGlobalFailure(t); + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandlerTest.java index 5b7ad7504d39..56d0c188a401 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandlerTest.java @@ -28,6 +28,8 @@ import java.util.Collections; import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -40,19 +42,25 @@ */ public class ExecutionFailureHandlerTest extends TestLogger { - private static final long restartDelayMs = 1234L; + private static final long RESTART_DELAY_MS = 1234L; + + private FailoverTopology failoverTopology; private TestFailoverStrategy failoverStrategy; - private TestRestartBackoffTimeStrategy restartStrategy; + private TestRestartBackoffTimeStrategy backoffTimeStrategy; private ExecutionFailureHandler executionFailureHandler; @Before public void setUp() { + TestFailoverTopology.Builder topologyBuilder = new TestFailoverTopology.Builder(); + topologyBuilder.newVertex(); + failoverTopology = topologyBuilder.build(); + failoverStrategy = new TestFailoverStrategy(); - restartStrategy = new TestRestartBackoffTimeStrategy(true, restartDelayMs); - executionFailureHandler = new ExecutionFailureHandler(failoverStrategy, restartStrategy); + backoffTimeStrategy = new TestRestartBackoffTimeStrategy(true, RESTART_DELAY_MS); + executionFailureHandler = new ExecutionFailureHandler(failoverTopology, failoverStrategy, backoffTimeStrategy); } /** @@ -71,7 +79,7 @@ public void testNormalFailureHandling() { // verify results assertTrue(result.canRestart()); - assertEquals(restartDelayMs, result.getRestartDelayMS()); + assertEquals(RESTART_DELAY_MS, result.getRestartDelayMS()); assertEquals(tasksToRestart, result.getVerticesToRestart()); try { result.getError(); @@ -87,7 +95,7 @@ public void testNormalFailureHandling() { @Test public void testRestartingSuppressedFailureHandlingResult() { // restart strategy suppresses restarting - restartStrategy.setCanRestart(false); + backoffTimeStrategy.setCanRestart(false); // trigger a task failure final FailureHandlingResult result = executionFailureHandler.getFailureHandlingResult( @@ -156,6 +164,18 @@ public void testUnrecoverableErrorCheck() { new Exception(new SuppressRestartsException(new Exception())))); } + @Test + public void testGlobalFailureHandling() { + final FailureHandlingResult result = executionFailureHandler.getGlobalFailureHandlingResult( + new Exception("test failure")); + + assertEquals( + StreamSupport.stream(failoverTopology.getFailoverVertices().spliterator(), false) + .map(FailoverVertex::getExecutionVertexID) + .collect(Collectors.toSet()), + result.getVerticesToRestart()); + } + // ------------------------------------------------------------------------ // utilities // ------------------------------------------------------------------------ @@ -163,7 +183,7 @@ public void testUnrecoverableErrorCheck() { /** * A FailoverStrategy implementation for tests. It always suggests restarting the given tasks to restart. */ - private class TestFailoverStrategy implements FailoverStrategy { + private static class TestFailoverStrategy implements FailoverStrategy { private Set tasksToRestart; 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 5c0b60c92cc8..5d6be10d0e2d 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 @@ -335,6 +335,26 @@ public void scheduleOnlyIfVertexIsCreated() throws Exception { } } + @Test + public void handleGlobalFailure() { + final JobGraph jobGraph = singleNonParallelJobVertexJobGraph(); + final JobVertex onlyJobVertex = getOnlyJobVertex(jobGraph); + + final DefaultScheduler scheduler = createSchedulerAndStartScheduling(jobGraph); + + scheduler.handleGlobalFailure(new Exception("forced failure")); + + final ArchivedExecutionVertex onlyExecutionVertex = Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices()); + final ExecutionAttemptID attemptId = onlyExecutionVertex.getCurrentExecutionAttempt().getAttemptId(); + scheduler.updateTaskExecutionState(new TaskExecutionState(jobGraph.getJobID(), attemptId, ExecutionState.CANCELED)); + + taskRestartExecutor.triggerScheduledTasks(); + + final List deployedExecutionVertices = testExecutionVertexOperations.getDeployedVertices(); + final ExecutionVertexID executionVertexId = new ExecutionVertexID(onlyJobVertex.getID(), 0); + assertThat(deployedExecutionVertices, contains(executionVertexId, executionVertexId)); + } + private void waitForTermination(final DefaultScheduler scheduler) throws Exception { scheduler.getTerminationFuture().get(TIMEOUT_MS, TimeUnit.MILLISECONDS); } From d89d3d959c464d32d126ba61582ecab3cefd28de Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Thu, 24 Oct 2019 19:40:04 +0800 Subject: [PATCH 214/746] [FLINK-14265][table-planner-blink] Don't use ContinuousFileReaderOperator to support multiple paths in batch (#9803) --- .../physical/PhysicalTableSourceScan.scala | 18 ++++++++---- .../batch/BatchExecTableSourceScan.scala | 17 +++++++++++ .../stream/StreamExecTableSourceScan.scala | 13 +++++++++ .../runtime/batch/sql/TableSourceITCase.scala | 29 ++++++++++++++++++- .../planner/utils/testTableSources.scala | 17 ++++++++++- 5 files changed, 86 insertions(+), 8 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/PhysicalTableSourceScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/PhysicalTableSourceScan.scala index 50fa428039ea..63aa63a8492e 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/PhysicalTableSourceScan.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/PhysicalTableSourceScan.scala @@ -61,18 +61,24 @@ abstract class PhysicalTableSourceScan( super.explainTerms(pw).item("fields", getRowType.getFieldNames.asScala.mkString(", ")) } - def getSourceTransformation( - streamEnv: StreamExecutionEnvironment): Transformation[_] = { + def createInput[IN]( + env: StreamExecutionEnvironment, + format: InputFormat[IN, _ <: InputSplit], + t: TypeInformation[IN]): Transformation[IN] + + def getSourceTransformation(env: StreamExecutionEnvironment): Transformation[_] = { if (sourceTransform == null) { sourceTransform = tableSource match { case format: InputFormatTableSource[_] => // we don't use InputFormatTableSource.getDataStream, because in here we use planner // type conversion to support precision of Varchar and something else. - streamEnv.createInput( + val typeInfo = fromDataTypeToTypeInfo(format.getProducedDataType) + .asInstanceOf[TypeInformation[Any]] + createInput( + env, format.getInputFormat.asInstanceOf[InputFormat[Any, _ <: InputSplit]], - fromDataTypeToTypeInfo(format.getProducedDataType).asInstanceOf[TypeInformation[Any]] - ).name(format.explainSource()).getTransformation - case s: StreamTableSource[_] => s.getDataStream(streamEnv).getTransformation + typeInfo.asInstanceOf[TypeInformation[Any]]) + case s: StreamTableSource[_] => s.getDataStream(env).getTransformation } } sourceTransform diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecTableSourceScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecTableSourceScan.scala index 5ab23d376953..4b97b0d1510c 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecTableSourceScan.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecTableSourceScan.scala @@ -18,8 +18,13 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch +import org.apache.flink.api.common.io.InputFormat +import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.dag.Transformation +import org.apache.flink.core.io.InputSplit import org.apache.flink.runtime.operators.DamBehavior +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment +import org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction import org.apache.flink.table.api.TableException import org.apache.flink.table.dataformat.BaseRow import org.apache.flink.table.planner.codegen.CodeGeneratorContext @@ -136,4 +141,16 @@ class BatchExecTableSourceScan( def getEstimatedRowCount: lang.Double = { getCluster.getMetadataQuery.getRowCount(this) } + + override def createInput[IN]( + env: StreamExecutionEnvironment, + format: InputFormat[IN, _ <: InputSplit], + t: TypeInformation[IN]): Transformation[IN] = { + // env.createInput will use ContinuousFileReaderOperator, but it do not support multiple + // paths. If read partitioned source, after partition pruning, we need let InputFormat + // to read multiple partitions which are multiple paths. + // We can use InputFormatSourceFunction directly to support InputFormat. + val func = new InputFormatSourceFunction[IN](format, t) + env.addSource(func, tableSource.explainSource(), t).getTransformation + } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecTableSourceScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecTableSourceScan.scala index 746aad7bddf8..f44c8daf9260 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecTableSourceScan.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecTableSourceScan.scala @@ -18,8 +18,12 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream +import org.apache.flink.api.common.io.InputFormat +import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.dag.Transformation +import org.apache.flink.core.io.InputSplit import org.apache.flink.streaming.api.datastream.DataStream +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import org.apache.flink.streaming.api.functions.{AssignerWithPeriodicWatermarks, AssignerWithPunctuatedWatermarks} import org.apache.flink.streaming.api.watermark.Watermark import org.apache.flink.table.api.{DataTypes, TableException} @@ -183,6 +187,15 @@ class StreamExecTableSourceScan( ScanUtil.hasTimeAttributeField(fieldIndexes) || ScanUtil.needsConversion(tableSource.getProducedDataType) } + + override def createInput[IN]( + env: StreamExecutionEnvironment, + format: InputFormat[IN, _ <: InputSplit], + t: TypeInformation[IN]): Transformation[IN] = { + // See StreamExecutionEnvironment.createInput, it is better to deal with checkpoint. + // The disadvantage is that streaming not support multi-paths. + env.createInput(format, t).name(tableSource.explainSource()).getTransformation + } } /** diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/TableSourceITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/TableSourceITCase.scala index dcff996b3f83..6c788a59b1ab 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/TableSourceITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/TableSourceITCase.scala @@ -21,14 +21,16 @@ package org.apache.flink.table.planner.runtime.batch.sql import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.table.api.{DataTypes, TableSchema, Types} +import org.apache.flink.table.planner.runtime.utils.BatchAbstractTestBase.TEMPORARY_FOLDER import org.apache.flink.table.planner.runtime.utils.BatchTestBase.row import org.apache.flink.table.planner.runtime.utils.{BatchTestBase, TestData} -import org.apache.flink.table.planner.utils.{TestDataTypeTableSource, TestFilterableTableSource, TestInputFormatTableSource, TestNestedProjectableTableSource, TestPartitionableTableSource, TestProjectableTableSource, TestTableSources} +import org.apache.flink.table.planner.utils.{TestDataTypeTableSource, TestFileInputFormatTableSource, TestFilterableTableSource, TestInputFormatTableSource, TestNestedProjectableTableSource, TestPartitionableTableSource, TestProjectableTableSource, TestTableSources} import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter import org.apache.flink.types.Row import org.junit.{Before, Test} +import java.io.{File, FileWriter} import java.lang.{Boolean => JBool, Integer => JInt, Long => JLong} class TableSourceITCase extends BatchTestBase { @@ -247,4 +249,29 @@ class TableSourceITCase extends BatchTestBase { row(3, "7.10", "123", "123")) ) } + + @Test + def testMultiPaths(): Unit = { + val tmpFile1 = TEMPORARY_FOLDER.newFile("tmpFile1.tmp") + new FileWriter(tmpFile1).append("t1\n").append("t2\n").close() + + val tmpFile2 = TEMPORARY_FOLDER.newFile("tmpFile2.tmp") + new FileWriter(tmpFile2).append("t3\n").append("t4\n").close() + + val schema = new TableSchema(Array("a"), Array(Types.STRING)) + + val tableSource = new TestFileInputFormatTableSource( + Array(tmpFile1.getPath, tmpFile2.getPath), schema) + tEnv.registerTableSource("MyMultiPathTable", tableSource) + + checkResult( + "select * from MyMultiPathTable", + Seq( + row("t1"), + row("t2"), + row("t3"), + row("t4") + ) + ) + } } 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 0416218eb837..147fde5a9bf1 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 @@ -22,7 +22,7 @@ import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.io.InputFormat import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} import org.apache.flink.api.common.typeutils.TypeSerializer -import org.apache.flink.api.java.io.CollectionInputFormat +import org.apache.flink.api.java.io.{CollectionInputFormat, RowCsvInputFormat} import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.core.io.InputSplit import org.apache.flink.streaming.api.datastream.DataStream @@ -656,3 +656,18 @@ class TestStreamTableSource( override def getTableSchema: TableSchema = tableSchema } + +class TestFileInputFormatTableSource( + paths: Array[String], + tableSchema: TableSchema) extends InputFormatTableSource[Row] { + + override def getInputFormat: InputFormat[Row, _ <: InputSplit] = { + val format = new RowCsvInputFormat(null, tableSchema.getFieldTypes) + format.setFilePaths(paths: _*) + format + } + + override def getReturnType: TypeInformation[Row] = tableSchema.toRowType + + override def getTableSchema: TableSchema = tableSchema +} From d0f462d90245375fda1c7e83b3f51cf56bdf99e5 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 24 Oct 2019 13:51:31 +0200 Subject: [PATCH 215/746] [hotfix][coordination] Update field name PartitionTable was at some point only tracking per job but was later generalized to work with arbitrary keys. The field name was updated accordingly however. --- .../runtime/taskexecutor/partition/PartitionTable.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/partition/PartitionTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/partition/PartitionTable.java index d214e09881aa..ac032ddc2246 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/partition/PartitionTable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/partition/PartitionTable.java @@ -35,13 +35,13 @@ @ThreadSafe public class PartitionTable { - private final Map> trackedPartitionsPerJob = new ConcurrentHashMap<>(8); + private final Map> trackedPartitionsPerKey = new ConcurrentHashMap<>(8); /** * Returns whether any partitions are being tracked for the given key. */ public boolean hasTrackedPartitions(K key) { - return trackedPartitionsPerJob.containsKey(key); + return trackedPartitionsPerKey.containsKey(key); } /** @@ -55,7 +55,7 @@ public void startTrackingPartitions(K key, Collection newPart return; } - trackedPartitionsPerJob.compute(key, (ignored, partitionIds) -> { + trackedPartitionsPerKey.compute(key, (ignored, partitionIds) -> { if (partitionIds == null) { partitionIds = new HashSet<>(8); } @@ -70,7 +70,7 @@ public void startTrackingPartitions(K key, Collection newPart public Collection stopTrackingPartitions(K key) { Preconditions.checkNotNull(key); - Set storedPartitions = trackedPartitionsPerJob.remove(key); + Set storedPartitions = trackedPartitionsPerKey.remove(key); return storedPartitions == null ? Collections.emptyList() : storedPartitions; @@ -84,7 +84,7 @@ public void stopTrackingPartitions(K key, Collection partitio Preconditions.checkNotNull(partitionIds); // If the key is unknown we do not fail here, in line with ShuffleEnvironment#releaseFinishedPartitions - trackedPartitionsPerJob.computeIfPresent( + trackedPartitionsPerKey.computeIfPresent( key, (ignored, resultPartitionIDS) -> { resultPartitionIDS.removeAll(partitionIds); From 37f67b7d394ccb3355ebc995af1c3fee04ce060f Mon Sep 17 00:00:00 2001 From: liuyongvs Date: Tue, 22 Oct 2019 20:13:46 +0800 Subject: [PATCH 216/746] [FLINK-14053][table-planner-blink] Fix DenseRankAggFunction first row bug. We should consider the possibility that first row's order by key is equal to the initial last value. This closes #9966 --- .../aggfunctions/DenseRankAggFunction.java | 7 ++++-- .../runtime/batch/sql/OverWindowITCase.scala | 22 +++++++++++++++++++ 2 files changed, 27 insertions(+), 2 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/DenseRankAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/DenseRankAggFunction.java index 7d9e18d186f8..f1db0bb37c75 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/DenseRankAggFunction.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/DenseRankAggFunction.java @@ -27,8 +27,11 @@ import java.util.Arrays; +import static org.apache.flink.table.planner.expressions.ExpressionBuilder.and; +import static org.apache.flink.table.planner.expressions.ExpressionBuilder.equalTo; import static org.apache.flink.table.planner.expressions.ExpressionBuilder.ifThenElse; import static org.apache.flink.table.planner.expressions.ExpressionBuilder.literal; +import static org.apache.flink.table.planner.expressions.ExpressionBuilder.not; import static org.apache.flink.table.planner.expressions.ExpressionBuilder.plus; /** @@ -73,8 +76,8 @@ public Expression[] initialValuesExpressions() { @Override public Expression[] accumulateExpressions() { Expression[] accExpressions = new Expression[1 + operands().length]; - // sequence = if (lastValues equalTo orderKeys) sequence else sequence + 1 - accExpressions[0] = ifThenElse(orderKeyEqualsExpression(), sequence, plus(sequence, literal(1L))); + // sequence = if (lastValues equalTo orderKeys and sequence != 0) sequence else sequence + 1 + accExpressions[0] = ifThenElse(and(orderKeyEqualsExpression(), not(equalTo(sequence, literal(0L)))), sequence, plus(sequence, literal(1L))); Expression[] operands = operands(); System.arraycopy(operands, 0, accExpressions, 1, operands.length); return accExpressions; diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/OverWindowITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/OverWindowITCase.scala index bd73238333f7..955a4c6cae7f 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/OverWindowITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/OverWindowITCase.scala @@ -386,6 +386,28 @@ class OverWindowITCase extends BatchTestBase { row(5, 5) ) ) + + // deal with input with 0 as the first row's rank field + checkResult( + "SELECT f, dense_rank() over (order by f) FROM Table5", + Seq( + row(0, 1), + row(1, 2), + row(2, 3), + row(3, 4), + row(4, 5), + row(5, 6), + row(6, 7), + row(7, 8), + row(8, 9), + row(9, 10), + row(10, 11), + row(11, 12), + row(12, 13), + row(13, 14), + row(14, 15) + ) + ) } @Test From 12996bd8ef4f83f1c31df06a13aec6512c46999b Mon Sep 17 00:00:00 2001 From: whlwanghailong Date: Wed, 16 Oct 2019 23:04:56 +0800 Subject: [PATCH 217/746] [FLINK-14408][table-planner] Fix open() is not called if UDF is reduced during optimization This closes #9916 --- .../planner/codegen/ExpressionReducer.scala | 6 +- .../flink/table/codegen/CodeGenerator.scala | 9 ++- .../table/codegen/ExpressionReducer.scala | 76 +++++++++++++++++-- .../plan/ExpressionReductionRulesTest.scala | 36 +++++++++ 4 files changed, 113 insertions(+), 14 deletions(-) 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 d12701193a4a..3aeb8755e9ff 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 @@ -112,11 +112,7 @@ class ExpressionReducer( case _ => throw new TableException("RichMapFunction[GenericRow, GenericRow] required here") } - val parameters = if (config.getConfiguration != null) { - config.getConfiguration - } else { - new Configuration() - } + val parameters = config.getConfiguration val reduced = try { richMapFunction.open(parameters) // execute diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala index 2a7bb6781e05..c6f7d762650b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala @@ -1617,7 +1617,10 @@ abstract class CodeGenerator( * @param contextTerm [[RuntimeContext]] term to access the [[RuntimeContext]] * @return member variable term */ - def addReusableFunction(function: UserDefinedFunction, contextTerm: String = null): String = { + def addReusableFunction( + function: UserDefinedFunction, + contextTerm: String = null, + functionContextClass: Class[_ <: FunctionContext] = classOf[FunctionContext]): String = { val classQualifier = function.getClass.getCanonicalName val functionSerializedData = EncodingUtils.encodeObjectToString(function) val fieldTerm = s"function_${function.functionIdentifier}" @@ -1640,11 +1643,11 @@ abstract class CodeGenerator( val openFunction = if (contextTerm != null) { s""" - |$fieldTerm.open(new ${classOf[FunctionContext].getCanonicalName}($contextTerm)); + |$fieldTerm.open(new ${functionContextClass.getCanonicalName}($contextTerm)); """.stripMargin } else { s""" - |$fieldTerm.open(new ${classOf[FunctionContext].getCanonicalName}(getRuntimeContext())); + |$fieldTerm.open(new ${functionContextClass.getCanonicalName}(getRuntimeContext())); """.stripMargin } reusableOpenStatements.add(openFunction) 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 e3e44ae8a1d4..b4cbc22d4b9a 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 @@ -18,17 +18,21 @@ package org.apache.flink.table.codegen +import java.io.File import java.util import org.apache.calcite.plan.RelOptPlanner import org.apache.calcite.rex.{RexBuilder, RexNode} import org.apache.calcite.sql.`type`.SqlTypeName +import org.apache.flink.api.common.functions.util.FunctionUtils import org.apache.flink.api.common.functions.MapFunction -import org.apache.flink.api.common.typeinfo.BasicTypeInfo +import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.configuration.Configuration +import org.apache.flink.metrics.MetricGroup 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.functions.{FunctionContext, FunctionLanguage, UserDefinedFunction} import org.apache.flink.table.plan.util.PythonUtil import org.apache.flink.types.Row @@ -36,7 +40,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.ListBuffer /** - * Evaluates constant expressions using Flink's [[FunctionCodeGenerator]]. + * Evaluates constant expressions using Flink's [[ConstantFunctionCodeGenerator]]. */ class ExpressionReducer(config: TableConfig) extends RelOptPlanner.Executor with Compiler[MapFunction[Row, Row]] { @@ -94,8 +98,9 @@ class ExpressionReducer(config: TableConfig) val literalTypes = literals.map(e => FlinkTypeFactory.toTypeInfo(e.getType)) val resultType = new RowTypeInfo(literalTypes: _*) + val parameters = config.getConfiguration // generate MapFunction - val generator = new FunctionCodeGenerator(config, false, EMPTY_ROW_INFO) + val generator = new ConstantFunctionCodeGenerator(config, false, EMPTY_ROW_INFO) val result = generator.generateResultExpression( resultType, @@ -117,8 +122,12 @@ class ExpressionReducer(config: TableConfig) generatedFunction.code) val function = clazz.newInstance() - // execute - val reduced = function.map(EMPTY_ROW) + val reduced = try { + FunctionUtils.openFunction(function, parameters) + function.map(EMPTY_ROW) + } finally { + FunctionUtils.closeFunction(function) + } // add the reduced results or keep them unreduced var i = 0 @@ -164,3 +173,58 @@ class ExpressionReducer(config: TableConfig) } } } + +/** + * A [[ConstantFunctionContext]] allows to obtain user-defined configuration information set + * in [[TableConfig]]. + * + * @param parameters User-defined configuration set in [[TableConfig]]. + */ +class ConstantFunctionContext(parameters: Configuration) extends FunctionContext(null) { + + override def getMetricGroup: MetricGroup = { + throw new UnsupportedOperationException("getMetricGroup is not supported when optimizing") + } + + override def getCachedFile(name: String): File = { + throw new UnsupportedOperationException("getCachedFile is not supported when optimizing") + } + + /** + * Gets the user-defined configuration value associated with the given key as a string. + * + * @param key key pointing to the associated value + * @param defaultValue default value which is returned in case user-defined configuration + * value is null or there is no value associated with the given key + * @return (default) value associated with the given key + */ + override def getJobParameter(key: String, defaultValue: String): String = { + parameters.getString(key, defaultValue) + } +} + +/** + * A [[ConstantFunctionCodeGenerator]] used for constant expression code generator + * @param config configuration that determines runtime behavior + * @param nullableInput input(s) can be null. + * @param input1 type information about the first input of the Function + */ +class ConstantFunctionCodeGenerator( + config: TableConfig, + nullableInput: Boolean, + input1: TypeInformation[_ <: Any]) + extends FunctionCodeGenerator(config, nullableInput, input1) { + /** + * Adds a reusable [[UserDefinedFunction]] to the member area of the generated [[Function]]. + * + * @param function [[UserDefinedFunction]] object to be instantiated during runtime + * @param contextTerm term to access the Context + * @return member variable term + */ + override def addReusableFunction( + function: UserDefinedFunction, + contextTerm: String = null, + functionContextClass: Class[_ <: FunctionContext] = classOf[FunctionContext]): String = { + super.addReusableFunction(function, "parameters", classOf[ConstantFunctionContext]) + } +} 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 5209bdf6cd2a..97bb3e291df6 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,6 +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.expressions.utils.{Func1, RichFunc1} import org.apache.flink.table.functions.{FunctionLanguage, ScalarFunction} import org.apache.flink.table.utils.TableTestBase import org.apache.flink.table.utils.TableTestUtil._ @@ -520,6 +521,41 @@ class ExpressionReductionRulesTest extends TableTestBase { util.verifyTable(result, expected) } + + @Test + def testExpressionReductionWithUDF(): Unit = { + val util = streamTestUtil() + val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c) + + util.addFunction("MyUdf", Func1) + + val expected = unaryNode( + "DataStreamCalc", + streamTableNode(table), + term("select", "CAST(2) AS constantValue") + ) + + util.verifySql("SELECT MyUdf(1) as constantValue FROM MyTable", expected) + + } + + @Test + def testExpressionReductionWithRichUDF(): Unit = { + val util = streamTestUtil() + val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c) + + util.addFunction("MyUdf", new RichFunc1) + util.tableEnv.getConfig.getConfiguration.setString("int.value", "10") + + val expected = unaryNode( + "DataStreamCalc", + streamTableNode(table), + term("select", "CAST(11) AS constantValue") + ) + + util.verifySql("SELECT MyUdf(1) as constantValue FROM MyTable", expected) + } + } object NonDeterministicNullFunc extends ScalarFunction { From 1cefbb8b5d0ba0e687635441c04d1790e33ef76c Mon Sep 17 00:00:00 2001 From: Fawad Halim Date: Fri, 25 Oct 2019 01:40:22 -0500 Subject: [PATCH 218/746] [FLINK-14524] [flink-jdbc] Fix syntax for PostgreSQL dialect "upsert" statement (#9990) --- .../org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java index 12e70dbbaf93..9334ec27c716 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java @@ -129,7 +129,7 @@ public Optional getUpsertStatement(String tableName, String[] fieldNames .map(f -> quoteIdentifier(f) + "=EXCLUDED." + quoteIdentifier(f)) .collect(Collectors.joining(", ")); return Optional.of(getInsertIntoStatement(tableName, fieldNames) + - " ON CONFLICT (" + uniqueColumns + + " ON CONFLICT (" + uniqueColumns + ")" + " DO UPDATE SET " + updateClause ); } From bccc4404ec9e73cc03e36631e6b3cf38a8c0e571 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Mon, 21 Oct 2019 17:34:56 +0800 Subject: [PATCH 219/746] [FLINK-14321][sql-parser] Support to parse watermark statement in SQL DDL This closes #9952 --- .../src/main/codegen/data/Parser.tdd | 10 +- .../src/main/codegen/includes/parserImpls.ftl | 36 ++- .../flink/sql/parser/ddl/SqlCreateTable.java | 110 +++++++-- .../flink/sql/parser/ddl/SqlWatermark.java | 78 ++++++ .../sql/parser/utils/ParserResource.java | 44 ++++ .../ParserResource.properties | 22 ++ .../sql/parser/FlinkSqlParserImplTest.java | 223 ++++++------------ .../operations/SqlToOperationConverter.java | 5 + .../planner/catalog/CatalogTableITCase.scala | 22 +- .../sqlexec/SqlToOperationConverter.java | 5 + .../table/catalog/CatalogTableITCase.scala | 22 +- 11 files changed, 371 insertions(+), 206 deletions(-) create mode 100644 flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlWatermark.java create mode 100644 flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/utils/ParserResource.java create mode 100644 flink-table/flink-sql-parser/src/main/resources/org.apache.flink.sql.parser.utils/ParserResource.properties diff --git a/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd b/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd index f60b3ed85725..20ed012a5a70 100644 --- a/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd +++ b/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd @@ -31,11 +31,13 @@ "org.apache.flink.sql.parser.ddl.SqlDropView", "org.apache.flink.sql.parser.ddl.SqlTableColumn", "org.apache.flink.sql.parser.ddl.SqlTableOption", + "org.apache.flink.sql.parser.ddl.SqlWatermark", "org.apache.flink.sql.parser.dml.RichSqlInsert", "org.apache.flink.sql.parser.dml.RichSqlInsertKeyword", "org.apache.flink.sql.parser.type.ExtendedSqlBasicTypeNameSpec", "org.apache.flink.sql.parser.type.ExtendedSqlCollectionTypeNameSpec", "org.apache.flink.sql.parser.utils.SqlTimeUnit", + "org.apache.flink.sql.parser.utils.ParserResource", "org.apache.flink.sql.parser.validate.FlinkSqlConformance", "org.apache.flink.sql.parser.SqlProperty", "org.apache.calcite.sql.SqlDrop", @@ -51,10 +53,6 @@ "PARTITIONED", "IF", "WATERMARK", - "ASCENDING", - "FROM_SOURCE", - "BOUNDED", - "DELAY", "OVERWRITE", "STRING", "BYTES" @@ -368,10 +366,6 @@ # not in core, added in Flink "PARTITIONED", "IF", - "ASCENDING", - "FROM_SOURCE", - "BOUNDED", - "DELAY", "OVERWRITE" ] 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 6c86e06dba77..f752a55248e3 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 @@ -27,9 +27,31 @@ void TableColumn(TableCreationContext context) : UniqueKey(context.uniqueKeysList) | ComputedColumn(context) + | + Watermark(context) ) } +void Watermark(TableCreationContext context) : +{ + SqlIdentifier eventTimeColumnName; + SqlParserPos pos; + SqlNode watermarkStrategy; +} +{ + {pos = getPos();} + eventTimeColumnName = CompoundIdentifier() + + watermarkStrategy = Expression(ExprContext.ACCEPT_NON_QUERY) { + if (context.watermark != null) { + throw SqlUtil.newContextException(pos, + ParserResource.RESOURCE.multipleWatermarksUnsupported()); + } else { + context.watermark = new SqlWatermark(pos, eventTimeColumnName, watermarkStrategy); + } + } +} + void ComputedColumn(TableCreationContext context) : { SqlNode identifier; @@ -40,7 +62,7 @@ void ComputedColumn(TableCreationContext context) : { identifier = SimpleIdentifier() {pos = getPos();} - expr = Expression(ExprContext.ACCEPT_SUB_QUERY) { + expr = Expression(ExprContext.ACCEPT_NON_QUERY) { expr = SqlStdOperatorTable.AS.createCall(Span.of(identifier, expr).pos(), expr, identifier); context.columnList.add(expr); } @@ -175,6 +197,7 @@ SqlCreate SqlCreateTable(Span s, boolean replace) : SqlIdentifier tableName; SqlNodeList primaryKeyList = SqlNodeList.EMPTY; List uniqueKeysList = new ArrayList(); + SqlWatermark watermark = null; SqlNodeList columnList = SqlNodeList.EMPTY; SqlCharStringLiteral comment = null; @@ -197,6 +220,7 @@ SqlCreate SqlCreateTable(Span s, boolean replace) : columnList = new SqlNodeList(ctx.columnList, pos); primaryKeyList = ctx.primaryKeyList; uniqueKeysList = ctx.uniqueKeysList; + watermark = ctx.watermark; } ] @@ -220,6 +244,7 @@ SqlCreate SqlCreateTable(Span s, boolean replace) : uniqueKeysList, propertyList, partitionColumns, + watermark, comment); } } @@ -272,9 +297,11 @@ SqlNode RichSqlInsert() : | { if (!((FlinkSqlConformance) this.conformance).allowInsertOverwrite()) { - throw new ParseException("OVERWRITE expression is only allowed for HIVE dialect"); + throw SqlUtil.newContextException(getPos(), + ParserResource.RESOURCE.overwriteIsOnlyAllowedForHive()); } else if (RichSqlInsert.isUpsert(keywords)) { - throw new ParseException("OVERWRITE expression is only used with INSERT mode"); + throw SqlUtil.newContextException(getPos(), + ParserResource.RESOURCE.overwriteIsOnlyUsedWithInsert()); } extendedKeywords.add(RichSqlInsertKeyword.OVERWRITE.symbol(getPos())); } @@ -307,7 +334,8 @@ SqlNode RichSqlInsert() : [ PartitionSpecCommaList(partitionList) { if (!((FlinkSqlConformance) this.conformance).allowInsertIntoPartition()) { - throw new ParseException("PARTITION expression is only allowed for HIVE dialect"); + throw SqlUtil.newContextException(getPos(), + ParserResource.RESOURCE.partitionIsOnlyAllowedForHive()); } } ] 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 ba0763938d62..f542af1f92ab 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 @@ -21,16 +21,19 @@ import org.apache.flink.sql.parser.ExtendedSqlNode; import org.apache.flink.sql.parser.error.SqlValidateException; +import org.apache.calcite.sql.ExtendedSqlRowTypeNameSpec; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlCall; import org.apache.calcite.sql.SqlCharStringLiteral; import org.apache.calcite.sql.SqlCreate; +import org.apache.calcite.sql.SqlDataTypeSpec; import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlNodeList; import org.apache.calcite.sql.SqlOperator; import org.apache.calcite.sql.SqlSpecialOperator; +import org.apache.calcite.sql.SqlTypeNameSpec; import org.apache.calcite.sql.SqlWriter; import org.apache.calcite.sql.dialect.AnsiSqlDialect; import org.apache.calcite.sql.parser.SqlParserPos; @@ -66,6 +69,9 @@ public class SqlCreateTable extends SqlCreate implements ExtendedSqlNode { private final SqlNodeList partitionKeyList; + @Nullable + private final SqlWatermark watermark; + @Nullable private final SqlCharStringLiteral comment; @@ -77,6 +83,7 @@ public SqlCreateTable( List uniqueKeysList, SqlNodeList propertyList, SqlNodeList partitionKeyList, + SqlWatermark watermark, SqlCharStringLiteral comment) { super(OPERATOR, pos, false, false); this.tableName = requireNonNull(tableName, "tableName should not be null"); @@ -85,6 +92,7 @@ public SqlCreateTable( 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.watermark = watermark; this.comment = comment; } @@ -96,7 +104,7 @@ public SqlOperator getOperator() { @Override public List getOperandList() { return ImmutableNullableList.of(tableName, columnList, primaryKeyList, - propertyList, partitionKeyList, comment); + propertyList, partitionKeyList, watermark, comment); } public SqlIdentifier getTableName() { @@ -123,6 +131,10 @@ public List getUniqueKeysList() { return uniqueKeysList; } + public Optional getWatermark() { + return Optional.ofNullable(watermark); + } + public Optional getComment() { return Optional.ofNullable(comment); } @@ -132,28 +144,14 @@ public boolean isIfNotExists() { } public void validate() throws SqlValidateException { - Set columnNames = new HashSet<>(); + ColumnValidator validator = new ColumnValidator(); 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()); - } + validator.addColumn(column); } for (SqlNode primaryKeyNode : this.primaryKeyList) { String primaryKey = ((SqlIdentifier) primaryKeyNode).getSimple(); - if (!columnNames.contains(primaryKey)) { + if (!validator.contains(primaryKey)) { throw new SqlValidateException( primaryKeyNode.getParserPosition(), "Primary key [" + primaryKey + "] not defined in columns, at " + @@ -164,7 +162,7 @@ public void validate() throws SqlValidateException { for (SqlNodeList uniqueKeys: this.uniqueKeysList) { for (SqlNode uniqueKeyNode : uniqueKeys) { String uniqueKey = ((SqlIdentifier) uniqueKeyNode).getSimple(); - if (!columnNames.contains(uniqueKey)) { + if (!validator.contains(uniqueKey)) { throw new SqlValidateException( uniqueKeyNode.getParserPosition(), "Unique key [" + uniqueKey + "] not defined in columns, at " + uniqueKeyNode.getParserPosition()); @@ -174,7 +172,7 @@ public void validate() throws SqlValidateException { for (SqlNode partitionKeyNode : this.partitionKeyList.getList()) { String partitionKey = ((SqlIdentifier) partitionKeyNode).getSimple(); - if (!columnNames.contains(partitionKey)) { + if (!validator.contains(partitionKey)) { throw new SqlValidateException( partitionKeyNode.getParserPosition(), "Partition column [" + partitionKey + "] not defined in columns, at " @@ -182,6 +180,16 @@ public void validate() throws SqlValidateException { } } + if (this.watermark != null) { + // SqlIdentifier.toString() returns a qualified identifier string using "." separator + String rowtimeField = watermark.getEventTimeColumnName().toString(); + if (!validator.contains(rowtimeField)) { + throw new SqlValidateException( + watermark.getEventTimeColumnName().getParserPosition(), + "The rowtime attribute field \"" + rowtimeField + "\" is not defined in columns, at " + + watermark.getEventTimeColumnName().getParserPosition()); + } + } } public boolean containsComputedColumn() { @@ -266,6 +274,10 @@ public void unparse( writer.endList(keyFrame); } } + if (watermark != null) { + printIndent(writer); + watermark.unparse(writer, leftPrec, rightPrec); + } writer.newlineAndIndent(); writer.endList(frame); @@ -310,9 +322,67 @@ public static class TableCreationContext { public List columnList = new ArrayList<>(); public SqlNodeList primaryKeyList = SqlNodeList.EMPTY; public List uniqueKeysList = new ArrayList<>(); + @Nullable public SqlWatermark watermark; } public String[] fullTableName() { return tableName.names.toArray(new String[0]); } + + // ------------------------------------------------------------------------------------- + + private static final class ColumnValidator { + + private final Set allColumnNames = new HashSet<>(); + + /** + * Adds column name to the registered column set. This will add nested column names recursive. + * Nested column names are qualified using "." separator. + */ + public void addColumn(SqlNode column) throws SqlValidateException { + String columnName; + if (column instanceof SqlTableColumn) { + SqlTableColumn tableColumn = (SqlTableColumn) column; + columnName = tableColumn.getName().getSimple(); + addNestedColumn(columnName, tableColumn.getType()); + } else if (column instanceof SqlBasicCall) { + SqlBasicCall tableColumn = (SqlBasicCall) column; + columnName = tableColumn.getOperands()[1].toString(); + } else { + throw new UnsupportedOperationException("Unsupported column:" + column); + } + + addColumnName(columnName, column.getParserPosition()); + } + + /** + * Returns true if the column name is existed in the registered column set. + * This supports qualified column name using "." separator. + */ + public boolean contains(String columnName) { + return allColumnNames.contains(columnName); + } + + private void addNestedColumn(String columnName, SqlDataTypeSpec columnType) throws SqlValidateException { + SqlTypeNameSpec typeName = columnType.getTypeNameSpec(); + // validate composite type + if (typeName instanceof ExtendedSqlRowTypeNameSpec) { + ExtendedSqlRowTypeNameSpec rowType = (ExtendedSqlRowTypeNameSpec) typeName; + for (int i = 0; i < rowType.getFieldNames().size(); i++) { + SqlIdentifier fieldName = rowType.getFieldNames().get(i); + String fullName = columnName + "." + fieldName; + addColumnName(fullName, fieldName.getParserPosition()); + SqlDataTypeSpec fieldType = rowType.getFieldTypes().get(i); + addNestedColumn(fullName, fieldType); + } + } + } + + private void addColumnName(String columnName, SqlParserPos pos) throws SqlValidateException { + if (!allColumnNames.add(columnName)) { + throw new SqlValidateException(pos, + "Duplicate column name [" + columnName + "], at " + pos); + } + } + } } diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlWatermark.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlWatermark.java new file mode 100644 index 000000000000..ebc639746b37 --- /dev/null +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlWatermark.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.sql.parser.ddl; + +import org.apache.calcite.sql.SqlCall; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.SqlSpecialOperator; +import org.apache.calcite.sql.SqlWriter; +import org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.calcite.util.ImmutableNullableList; + +import java.util.List; + +import static java.util.Objects.requireNonNull; + + +/** + * Watermark statement in CREATE TABLE DDL, e.g. {@code WATERMARK FOR ts AS ts - INTERVAL '5' SECOND}. + */ +public class SqlWatermark extends SqlCall { + + private static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("WATERMARK", SqlKind.OTHER); + + private final SqlIdentifier eventTimeColumnName; + private final SqlNode watermarkStrategy; + + public SqlWatermark(SqlParserPos pos, SqlIdentifier eventTimeColumnName, SqlNode watermarkStrategy) { + super(pos); + this.eventTimeColumnName = requireNonNull(eventTimeColumnName); + this.watermarkStrategy = requireNonNull(watermarkStrategy); + } + + @Override + public SqlOperator getOperator() { + return OPERATOR; + } + + @Override + public List getOperandList() { + return ImmutableNullableList.of(eventTimeColumnName, watermarkStrategy); + } + + @Override + public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { + writer.keyword("WATERMARK"); + writer.keyword("FOR"); + eventTimeColumnName.unparse(writer, leftPrec, rightPrec); + writer.keyword("AS"); + watermarkStrategy.unparse(writer, leftPrec, rightPrec); + } + + public SqlIdentifier getEventTimeColumnName() { + return eventTimeColumnName; + } + + public SqlNode getWatermarkStrategy() { + return watermarkStrategy; + } +} diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/utils/ParserResource.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/utils/ParserResource.java new file mode 100644 index 000000000000..8d7d59d5db7c --- /dev/null +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/utils/ParserResource.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.sql.parser.utils; + +import org.apache.flink.sql.parser.impl.ParseException; + +import org.apache.calcite.runtime.Resources; + +/** + * Compiler-checked resources for the Flink SQL parser. + */ +public interface ParserResource { + + /** Resources. */ + ParserResource RESOURCE = Resources.create(ParserResource.class); + + @Resources.BaseMessage("Multiple WATERMARK statements is not supported yet.") + Resources.ExInst multipleWatermarksUnsupported(); + + @Resources.BaseMessage("OVERWRITE expression is only allowed for HIVE dialect.") + Resources.ExInst overwriteIsOnlyAllowedForHive(); + + @Resources.BaseMessage("OVERWRITE expression is only used with INSERT statement.") + Resources.ExInst overwriteIsOnlyUsedWithInsert(); + + @Resources.BaseMessage("PARTITION expression is only allowed for HIVE dialect.") + Resources.ExInst partitionIsOnlyAllowedForHive(); +} diff --git a/flink-table/flink-sql-parser/src/main/resources/org.apache.flink.sql.parser.utils/ParserResource.properties b/flink-table/flink-sql-parser/src/main/resources/org.apache.flink.sql.parser.utils/ParserResource.properties new file mode 100644 index 000000000000..a48c4da3865b --- /dev/null +++ b/flink-table/flink-sql-parser/src/main/resources/org.apache.flink.sql.parser.utils/ParserResource.properties @@ -0,0 +1,22 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to you under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# Resources for the Apache Calcite project. +# See wrapper class org.apache.calcite.runtime.CalciteResource. +# +MultipleWatermarksUnsupported=Multiple WATERMARK statements is not supported yet. +OverwriteIsOnlyAllowedForHive=OVERWRITE expression is only allowed for HIVE dialect. +OverwriteIsOnlyUsedWithInsert=OVERWRITE expression is only used with INSERT statement. +PartitionIsOnlyAllowedForHive=PARTITION expression is only allowed for HIVE dialect. 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 c4d1dc496078..7cb9014849bf 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 @@ -33,7 +33,6 @@ import org.hamcrest.BaseMatcher; import org.hamcrest.Description; import org.junit.Before; -import org.junit.Ignore; import org.junit.Test; import java.io.Reader; @@ -174,191 +173,115 @@ public void testCreateTableWithPrimaryKeyAndUniqueKey() { ")"); } - @Ignore // need to implement @Test - public void testCreateTableWithoutWatermarkFieldName() { - check("CREATE TABLE tbl1 (\n" + - " a bigint,\n" + - " b varchar, \n" + - " c as 2 * (a + 1), \n" + - " WATERMARK FOR a AS BOUNDED WITH DELAY 1000 MILLISECOND\n" + - ")\n" + - " with (\n" + - " 'connector' = 'kafka', \n" + - " 'kafka.topic' = 'log.test'\n" + - ")\n", - "CREATE TABLE `TBL1` (\n" + - " `A` BIGINT,\n" + - " `B` VARCHAR,\n" + - " `C` AS (2 * (`A` + 1)),\n" + - " WATERMARK FOR `A` AS BOUNDED WITH DELAY 1000 MILLISECOND\n" + - ") WITH (\n" + - " 'connector' = 'kafka',\n" + - " 'kafka.topic' = 'log.test'\n" + - ")"); - } - - @Ignore // need to implement - @Test - public void testCreateTableWithWatermarkBoundedDelay() { - check("CREATE TABLE tbl1 (\n" + - " a bigint,\n" + - " b varchar, \n" + - " c as 2 * (a + 1), \n" + - " WATERMARK wk FOR a AS BOUNDED WITH DELAY 1000 DAY\n" + - ")\n" + - " with (\n" + - " 'connector' = 'kafka', \n" + - " 'kafka.topic' = 'log.test'\n" + - ")\n", + public void testCreateTableWithWatermark() { + String sql = "CREATE TABLE tbl1 (\n" + + " ts timestamp(3),\n" + + " id varchar, \n" + + " watermark FOR ts AS ts - interval '3' second\n" + + ")\n" + + " with (\n" + + " 'connector' = 'kafka', \n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + check(sql, "CREATE TABLE `TBL1` (\n" + - " `A` BIGINT,\n" + - " `B` VARCHAR,\n" + - " `C` AS (2 * (`A` + 1)),\n" + - " WATERMARK `WK` FOR `A` AS BOUNDED WITH DELAY 1000 DAY\n" + + " `TS` TIMESTAMP(3),\n" + + " `ID` VARCHAR,\n" + + " WATERMARK FOR `TS` AS (`TS` - INTERVAL '3' SECOND)\n" + ") WITH (\n" + " 'connector' = 'kafka',\n" + " 'kafka.topic' = 'log.test'\n" + ")"); } - @Ignore // need to implement @Test - public void testCreateTableWithWatermarkBoundedDelay1() { - check("CREATE TABLE tbl1 (\n" + - " a bigint,\n" + - " b varchar, \n" + - " c as 2 * (a + 1), \n" + - " WATERMARK wk FOR a AS BOUNDED WITH DELAY 1000 HOUR\n" + - ")\n" + - " with (\n" + - " 'connector' = 'kafka', \n" + - " 'kafka.topic' = 'log.test'\n" + - ")\n", + public void testCreateTableWithWatermarkOnComputedColumn() { + String sql = "CREATE TABLE tbl1 (\n" + + " log_ts varchar,\n" + + " ts as to_timestamp(log_ts), \n" + + " WATERMARK FOR ts AS ts + interval '1' second\n" + + ")\n" + + " with (\n" + + " 'connector' = 'kafka', \n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + check(sql, "CREATE TABLE `TBL1` (\n" + - " `A` BIGINT,\n" + - " `B` VARCHAR,\n" + - " `C` AS (2 * (`A` + 1)),\n" + - " WATERMARK `WK` FOR `A` AS BOUNDED WITH DELAY 1000 HOUR\n" + + " `LOG_TS` VARCHAR,\n" + + " `TS` AS `TO_TIMESTAMP`(`LOG_TS`),\n" + + " WATERMARK FOR `TS` AS (`TS` + INTERVAL '1' SECOND)\n" + ") WITH (\n" + " 'connector' = 'kafka',\n" + " 'kafka.topic' = 'log.test'\n" + ")"); } - @Ignore // need to implement @Test - public void testCreateTableWithWatermarkBoundedDelay2() { + public void testCreateTableWithWatermarkOnNestedField() { check("CREATE TABLE tbl1 (\n" + - " a bigint,\n" + - " b varchar, \n" + - " c as 2 * (a + 1), \n" + - " WATERMARK wk FOR a AS BOUNDED WITH DELAY 1000 MINUTE\n" + + " f1 row, q3 boolean>,\n" + + " WATERMARK FOR f1.q2.t1 AS NOW()\n" + ")\n" + " with (\n" + " 'connector' = 'kafka', \n" + " 'kafka.topic' = 'log.test'\n" + ")\n", "CREATE TABLE `TBL1` (\n" + - " `A` BIGINT,\n" + - " `B` VARCHAR,\n" + - " `C` AS (2 * (`A` + 1)),\n" + - " WATERMARK `WK` FOR `A` AS BOUNDED WITH DELAY 1000 MINUTE\n" + + " `F1` ROW< `Q1` BIGINT, `Q2` ROW< `T1` TIMESTAMP, `T2` VARCHAR >, `Q3` BOOLEAN >,\n" + + " WATERMARK FOR `F1`.`Q2`.`T1` AS `NOW`()\n" + ") WITH (\n" + " 'connector' = 'kafka',\n" + " 'kafka.topic' = 'log.test'\n" + ")"); } - @Ignore // need to implement @Test - public void testCreateTableWithWatermarkBoundedDelay3() { - check("CREATE TABLE tbl1 (\n" + - " a bigint,\n" + - " b varchar, \n" + - " c as 2 * (a + 1), \n" + - " WATERMARK wk FOR a AS BOUNDED WITH DELAY 1000 SECOND\n" + - ")\n" + - " with (\n" + - " 'connector' = 'kafka', \n" + - " 'kafka.topic' = 'log.test'\n" + - ")\n", - "CREATE TABLE `TBL1` (\n" + - " `A` BIGINT,\n" + - " `B` VARCHAR,\n" + - " `C` AS (2 * (`A` + 1)),\n" + - " WATERMARK `WK` FOR `A` AS BOUNDED WITH DELAY 1000 SECOND\n" + - ") WITH (\n" + - " 'connector' = 'kafka',\n" + - " 'kafka.topic' = 'log.test'\n" + - ")"); - } + public void testCreateTableWithInvalidWatermark() { + String sql = "CREATE TABLE tbl1 (\n" + + " f1 row,\n" + + " WATERMARK FOR f1.q0 AS NOW()\n" + + ")\n" + + " with (\n" + + " 'connector' = 'kafka', \n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + sql(sql).node(new ValidationMatcher() + .fails("The rowtime attribute field \"F1.Q0\" is not defined in columns, at line 3, column 17")); - @Ignore // need to implement - @Test - public void testCreateTableWithNegativeWatermarkOffsetDelay() { - checkFails("CREATE TABLE tbl1 (\n" + - " a bigint,\n" + - " b varchar, \n" + - " c as 2 * (a + 1), \n" + - " WATERMARK wk FOR a AS BOUNDED WITH DELAY ^-^1000 SECOND\n" + - ")\n" + - " with (\n" + - " 'connector' = 'kafka', \n" + - " 'kafka.topic' = 'log.test'\n" + - ")\n", - "(?s).*Encountered \"-\" at line 5, column 44.\n" + - "Was expecting:\n" + - " ...\n" + - ".*"); } - @Ignore // need to implement @Test - public void testCreateTableWithWatermarkStrategyAscending() { - check("CREATE TABLE tbl1 (\n" + - " a bigint,\n" + - " b varchar, \n" + - " c as 2 * (a + 1), \n" + - " WATERMARK wk FOR a AS ASCENDING\n" + - ")\n" + - " with (\n" + - " 'connector' = 'kafka', \n" + - " 'kafka.topic' = 'log.test'\n" + - ")\n", - "CREATE TABLE `TBL1` (\n" + - " `A` BIGINT,\n" + - " `B` VARCHAR,\n" + - " `C` AS (2 * (`A` + 1)),\n" + - " WATERMARK `WK` FOR `A` AS ASCENDING\n" + - ") WITH (\n" + - " 'connector' = 'kafka',\n" + - " 'kafka.topic' = 'log.test'\n" + - ")"); + public void testCreateTableWithMultipleWatermark() { + String sql = "CREATE TABLE tbl1 (\n" + + " f0 bigint,\n" + + " f1 varchar,\n" + + " f2 boolean,\n" + + " WATERMARK FOR f0 AS NOW(),\n" + + " ^WATERMARK^ FOR f1 AS NOW()\n" + + ")\n" + + " with (\n" + + " 'connector' = 'kafka', \n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + sql(sql) + .fails("Multiple WATERMARK statements is not supported yet."); } - @Ignore // need to implement @Test - public void testCreateTableWithWatermarkStrategyFromSource() { - check("CREATE TABLE tbl1 (\n" + - " a bigint,\n" + - " b varchar, \n" + - " c as 2 * (a + 1), \n" + - " WATERMARK wk FOR a AS FROM_SOURCE\n" + - ")\n" + - " with (\n" + - " 'connector' = 'kafka', \n" + - " 'kafka.topic' = 'log.test'\n" + - ")\n", - "CREATE TABLE `TBL1` (\n" + - " `A` BIGINT,\n" + - " `B` VARCHAR,\n" + - " `C` AS (2 * (`A` + 1)),\n" + - " WATERMARK `WK` FOR `A` AS FROM_SOURCE\n" + - ") WITH (\n" + - " 'connector' = 'kafka',\n" + - " 'kafka.topic' = 'log.test'\n" + - ")"); + public void testCreateTableWithQueryWatermarkExpression() { + String sql = "CREATE TABLE tbl1 (\n" + + " f0 bigint,\n" + + " f1 varchar,\n" + + " f2 boolean,\n" + + " WATERMARK FOR f0 AS ^(^SELECT f1 FROM tbl1)\n" + + ")\n" + + " with (\n" + + " 'connector' = 'kafka', \n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + sql(sql) + .fails("Query expression encountered in illegal context"); } @Test @@ -620,8 +543,8 @@ public void testInsertOverwrite() { @Test public void testInvalidUpsertOverwrite() { conformance0 = FlinkSqlConformance.HIVE; - checkFails("UPSERT OVERWRITE myDB.myTbl SELECT * FROM src", - "OVERWRITE expression is only used with INSERT mode"); + sql("UPSERT ^OVERWRITE^ myDB.myTbl SELECT * FROM src") + .fails("OVERWRITE expression is only used with INSERT statement."); } @Test 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 3a1296536794..d19c725aa590 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 @@ -103,6 +103,11 @@ private Operation convertCreateTable(SqlCreateTable sqlCreateTable) { throw new SqlConversionException("Primary key and unique key are not supported yet."); } + if (sqlCreateTable.getWatermark().isPresent()) { + // TODO: FLINK-14320 + throw new SqlConversionException("Watermark statement is not supported yet."); + } + // set with properties Map properties = new HashMap<>(); sqlCreateTable.getPropertyList().getList().forEach(p -> diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala index ae30db40f6e4..6a1aabb1bd50 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala @@ -306,7 +306,7 @@ class CatalogTableITCase(isStreamingMode: Boolean) { assertEquals(TestCollectionTableFactory.RESULT.sorted, sourceData.sorted) } - @Test @Ignore // need to implement + @Test @Ignore("FLINK-14320") // need to implement def testStreamSourceTableWithRowtime(): Unit = { val sourceData = List( toRow(1, 1000), @@ -317,10 +317,9 @@ class CatalogTableITCase(isStreamingMode: Boolean) { val sourceDDL = """ |create table t1( - | a bigint, + | a timestamp(3), | b bigint, - | primary key(a), - | WATERMARK wm FOR a AS BOUNDED WITH DELAY 1000 MILLISECOND + | WATERMARK FOR a AS a - interval '1' SECOND |) with ( | 'connector' = 'COLLECTION' |) @@ -328,7 +327,7 @@ class CatalogTableITCase(isStreamingMode: Boolean) { val sinkDDL = """ |create table t2( - | a bigint, + | a timestamp(3), | b bigint |) with ( | 'connector' = 'COLLECTION' @@ -337,7 +336,7 @@ class CatalogTableITCase(isStreamingMode: Boolean) { val query = """ |insert into t2 - |select sum(a), sum(b) from t1 group by TUMBLE(wm, INTERVAL '1' SECOND) + |select a, sum(b) from t1 group by TUMBLE(a, INTERVAL '1' SECOND) """.stripMargin tableEnv.sqlUpdate(sourceDDL) @@ -388,7 +387,7 @@ class CatalogTableITCase(isStreamingMode: Boolean) { assertEquals(TestCollectionTableFactory.RESULT.sorted, sourceData.sorted) } - @Test @Ignore // need to implement + @Test @Ignore("FLINK-14320") // need to implement def testBatchTableWithRowtime(): Unit = { val sourceData = List( toRow(1, 1000), @@ -399,10 +398,9 @@ class CatalogTableITCase(isStreamingMode: Boolean) { val sourceDDL = """ |create table t1( - | a bigint, + | a timestamp(3), | b bigint, - | primary key(a), - | WATERMARK wm FOR a AS BOUNDED WITH DELAY 1000 MILLISECOND + | WATERMARK FOR a AS a - interval '1' SECOND |) with ( | 'connector' = 'COLLECTION' |) @@ -410,7 +408,7 @@ class CatalogTableITCase(isStreamingMode: Boolean) { val sinkDDL = """ |create table t2( - | a bigint, + | a timestamp(3), | b bigint |) with ( | 'connector' = 'COLLECTION' @@ -419,7 +417,7 @@ class CatalogTableITCase(isStreamingMode: Boolean) { val query = """ |insert into t2 - |select sum(a), sum(b) from t1 group by TUMBLE(wm, INTERVAL '1' SECOND) + |select a, sum(b) from t1 group by TUMBLE(a, INTERVAL '1' SECOND) """.stripMargin tableEnv.sqlUpdate(sourceDDL) 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 c7566602f737..db8221254bd9 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 @@ -104,6 +104,11 @@ private Operation convertCreateTable(SqlCreateTable sqlCreateTable) { throw new SqlConversionException("Primary key and unique key are not supported yet."); } + if (sqlCreateTable.getWatermark().isPresent()) { + throw new SqlConversionException( + "Watermark statement is not supported in Old Planner, please use Blink Planner instead."); + } + // set with properties Map properties = new HashMap<>(); sqlCreateTable.getPropertyList().getList().forEach(p -> diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/catalog/CatalogTableITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/catalog/CatalogTableITCase.scala index e53819606705..9d948497c00c 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/catalog/CatalogTableITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/catalog/CatalogTableITCase.scala @@ -318,7 +318,7 @@ class CatalogTableITCase(isStreaming: Boolean) { assertEquals(TestCollectionTableFactory.RESULT.sorted, sourceData.sorted) } - @Test @Ignore // need to implement + @Test @Ignore("FLINK-14320") // need to implement def testStreamSourceTableWithRowtime(): Unit = { val sourceData = List( toRow(1, 1000), @@ -329,10 +329,9 @@ class CatalogTableITCase(isStreaming: Boolean) { val sourceDDL = """ |create table t1( - | a bigint, + | a timestamp(3), | b bigint, - | primary key(a), - | WATERMARK wm FOR a AS BOUNDED WITH DELAY 1000 MILLISECOND + | WATERMARK FOR a AS a - interval '1' SECOND |) with ( | 'connector' = 'COLLECTION' |) @@ -340,7 +339,7 @@ class CatalogTableITCase(isStreaming: Boolean) { val sinkDDL = """ |create table t2( - | a bigint, + | a timestamp(3), | b bigint |) with ( | 'connector' = 'COLLECTION' @@ -349,7 +348,7 @@ class CatalogTableITCase(isStreaming: Boolean) { val query = """ |insert into t2 - |select sum(a), sum(b) from t1 group by TUMBLE(wm, INTERVAL '1' SECOND) + |select a, sum(b) from t1 group by TUMBLE(a, INTERVAL '1' SECOND) """.stripMargin tableEnv.sqlUpdate(sourceDDL) @@ -400,7 +399,7 @@ class CatalogTableITCase(isStreaming: Boolean) { assertEquals(TestCollectionTableFactory.RESULT.sorted, sourceData.sorted) } - @Test @Ignore // need to implement + @Test @Ignore("FLINK-14320") // need to implement def testBatchTableWithRowtime(): Unit = { val sourceData = List( toRow(1, 1000), @@ -411,10 +410,9 @@ class CatalogTableITCase(isStreaming: Boolean) { val sourceDDL = """ |create table t1( - | a bigint, + | a timestamp(3), | b bigint, - | primary key(a), - | WATERMARK wm FOR a AS BOUNDED WITH DELAY 1000 MILLISECOND + | WATERMARK FOR a AS a - interval '1' SECOND |) with ( | 'connector' = 'COLLECTION' |) @@ -422,7 +420,7 @@ class CatalogTableITCase(isStreaming: Boolean) { val sinkDDL = """ |create table t2( - | a bigint, + | a timestamp(3), | b bigint |) with ( | 'connector' = 'COLLECTION' @@ -431,7 +429,7 @@ class CatalogTableITCase(isStreaming: Boolean) { val query = """ |insert into t2 - |select sum(a), sum(b) from t1 group by TUMBLE(wm, INTERVAL '1' SECOND) + |select a, sum(b) from t1 group by TUMBLE(a, INTERVAL '1' SECOND) """.stripMargin tableEnv.sqlUpdate(sourceDDL) From fbd5b639538070d4d83526cb97c54e9a38ee7220 Mon Sep 17 00:00:00 2001 From: Wei Zhong Date: Mon, 14 Oct 2019 14:49:12 +0800 Subject: [PATCH 220/746] [FLINK-14342][table][python] Remove method FunctionDefinition#getLanguage. Do some cleanup in this commit. We don't need to add getLanguage() in FunctionDefinition as we can use the interface PythonFunction directly to judge whether it is a Python ScalarFunction. This closes #9894. --- .../table/functions/FunctionDefinition.java | 7 -- .../table/functions/FunctionLanguage.java | 32 --------- .../planner/codegen/ExpressionReducer.scala | 6 +- .../codegen/PythonFunctionCodeGenerator.scala | 7 +- .../plan/nodes/common/CommonPythonCalc.scala | 66 +++++++++--------- .../rules/logical/PythonCalcSplitRule.scala | 28 ++------ .../physical/batch/BatchExecCalcRule.scala | 5 +- .../batch/BatchExecPythonCalcRule.scala | 5 +- .../physical/stream/StreamExecCalcRule.scala | 5 +- .../stream/StreamExecPythonCalcRule.scala | 5 +- .../table/planner/plan/utils/PythonUtil.scala | 68 ++++++++++++------- .../utils/JavaUserDefinedScalarFunctions.java | 33 ++++++--- .../ExpressionReductionRulesTest.scala | 12 ++-- .../table/codegen/ExpressionReducer.scala | 6 +- .../codegen/PythonFunctionCodeGenerator.scala | 7 +- .../table/plan/nodes/CommonPythonCalc.scala | 64 ++++++++--------- .../rules/datastream/DataStreamCalcRule.scala | 5 +- .../datastream/DataStreamPythonCalcRule.scala | 5 +- .../rules/logical/PythonCalcSplitRule.scala | 28 ++------ .../flink/table/plan/util/PythonUtil.scala | 67 +++++++++++------- .../plan/ExpressionReductionRulesTest.scala | 11 +-- .../table/plan/PythonCalcSplitRuleTest.scala | 19 ++++-- 22 files changed, 232 insertions(+), 259 deletions(-) delete mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/FunctionLanguage.java 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 e43e84121c32..16379285f2e2 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,13 +40,6 @@ 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 deleted file mode 100644 index 7b36b132a471..000000000000 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/FunctionLanguage.java +++ /dev/null @@ -1,32 +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.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-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 3aeb8755e9ff..a50a5f41751e 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,10 +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, FunctionLanguage, UserDefinedFunction} +import org.apache.flink.table.functions.{FunctionContext, 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.planner.plan.utils.PythonUtil.containsPythonCall import org.apache.flink.table.runtime.functions.SqlDateTimeUtils import org.apache.flink.table.types.logical.RowType @@ -68,7 +68,7 @@ class ExpressionReducer( // 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) => + case (_, e) if containsPythonCall(e) => pythonUDFExprs += e None diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/PythonFunctionCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/PythonFunctionCodeGenerator.scala index 68069bd26def..496030dd7438 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/PythonFunctionCodeGenerator.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/PythonFunctionCodeGenerator.scala @@ -19,7 +19,7 @@ package org.apache.flink.table.planner.codegen import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.functions.python.{PythonEnv, PythonFunction} -import org.apache.flink.table.functions.{FunctionLanguage, ScalarFunction, UserDefinedFunction} +import org.apache.flink.table.functions.{ScalarFunction, UserDefinedFunction} import org.apache.flink.table.planner.codegen.CodeGenUtils.{newName, primitiveDefaultValue, primitiveTypeTermForType} import org.apache.flink.table.planner.codegen.Indenter.toISC import org.apache.flink.table.runtime.generated.GeneratedFunction @@ -100,11 +100,6 @@ object PythonFunctionCodeGenerator { | } | | @Override - | public ${classOf[FunctionLanguage].getCanonicalName} getLanguage() { - | return ${classOf[FunctionLanguage].getCanonicalName}.PYTHON; - | } - | - | @Override | public byte[] getSerializedPythonFunction() { | return $serializedScalarFunctionNameTerm; | } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonCalc.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonCalc.scala index a29ca11ebfb8..cf59d177b4d6 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonCalc.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonCalc.scala @@ -24,7 +24,6 @@ import org.apache.flink.api.dag.Transformation import org.apache.flink.streaming.api.operators.OneInputStreamOperator import org.apache.flink.streaming.api.transformations.OneInputTransformation import org.apache.flink.table.dataformat.BaseRow -import org.apache.flink.table.functions.FunctionLanguage import org.apache.flink.table.functions.python.{PythonFunction, PythonFunctionInfo, SimplePythonFunction} import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.functions.utils.ScalarSqlFunction @@ -55,38 +54,39 @@ trait CommonPythonCalc { } private def createPythonScalarFunctionInfo( - rexCall: RexCall, - inputNodes: mutable.Map[RexNode, Integer]): PythonFunctionInfo = rexCall.getOperator match { - case sfc: ScalarSqlFunction if sfc.scalarFunction.getLanguage == FunctionLanguage.PYTHON => - val inputs = new mutable.ArrayBuffer[AnyRef]() - rexCall.getOperands.foreach { - case pythonRexCall: RexCall if pythonRexCall.getOperator.asInstanceOf[ScalarSqlFunction] - .scalarFunction.getLanguage == FunctionLanguage.PYTHON => - // Continuous Python UDFs can be chained together - 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 { - 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.scalarFunction.asInstanceOf[PythonFunction].getSerializedPythonFunction, - sfc.scalarFunction.asInstanceOf[PythonFunction].getPythonEnv) - new PythonFunctionInfo(pythonFunction, inputs.toArray) + pythonRexCall: RexCall, + inputNodes: mutable.Map[RexNode, Integer]): PythonFunctionInfo = { + pythonRexCall.getOperator match { + case sfc: ScalarSqlFunction => + val inputs = new mutable.ArrayBuffer[AnyRef]() + pythonRexCall.getOperands.foreach { + case pythonRexCall: RexCall => + // Continuous Python UDFs can be chained together + 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 { + 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.scalarFunction.asInstanceOf[PythonFunction].getSerializedPythonFunction, + sfc.scalarFunction.asInstanceOf[PythonFunction].getPythonEnv) + new PythonFunctionInfo(pythonFunction, inputs.toArray) + } } private def getPythonScalarFunctionOperator( diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRule.scala index 874952511139..e0478a85e42c 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PythonCalcSplitRule.scala @@ -22,11 +22,9 @@ 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.planner.functions.utils.ScalarSqlFunction import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCalc -import org.apache.flink.table.planner.plan.utils.PythonUtil.containsFunctionOf +import org.apache.flink.table.planner.plan.utils.PythonUtil.{containsPythonCall, containsNonPythonCall, isPythonCall, isNonPythonCall} import org.apache.flink.table.planner.plan.utils.{InputRefVisitor, RexDefaultVisitor} import scala.collection.JavaConverters._ @@ -140,8 +138,7 @@ object PythonCalcSplitConditionRule extends PythonCalcSplitRuleBase( // matches if it contains Python functions in condition Option(calc.getProgram.getCondition) - .map(calc.getProgram.expandLocalRef) - .exists(containsFunctionOf(_, FunctionLanguage.PYTHON)) + .map(calc.getProgram.expandLocalRef).exists(containsPythonCall) } override def isConvertPythonFunction(program: RexProgram): Boolean = true @@ -166,14 +163,11 @@ object PythonCalcSplitProjectionRule extends PythonCalcSplitRuleBase( val projects = calc.getProgram.getProjectList.map(calc.getProgram.expandLocalRef) // matches if it contains both Python functions and Java functions in the projection - projects.exists(containsFunctionOf(_, FunctionLanguage.PYTHON)) && - projects.exists(containsFunctionOf(_, FunctionLanguage.JVM)) + projects.exists(containsPythonCall) && projects.exists(containsNonPythonCall) } override def isConvertPythonFunction(program: RexProgram): Boolean = { - program.getProjectList - .map(program.expandLocalRef) - .exists(containsFunctionOf(_, FunctionLanguage.JVM, recursive = false)) + program.getProjectList.map(program.expandLocalRef).exists(isNonPythonCall) } override def split(program: RexProgram, splitter: ScalarFunctionSplitter) @@ -197,8 +191,7 @@ object PythonCalcPushConditionRule extends PythonCalcSplitRuleBase( // matches if all the following conditions hold true: // 1) the condition is not null // 2) it contains Python functions in the projection - calc.getProgram.getCondition != null && - projects.exists(containsFunctionOf(_, FunctionLanguage.PYTHON)) + calc.getProgram.getCondition != null && projects.exists(containsPythonCall) } override def isConvertPythonFunction(program: RexProgram): Boolean = false @@ -228,7 +221,7 @@ object PythonCalcRewriteProjectionRule extends PythonCalcSplitRuleBase( // 1) it contains Python functions in the projection // 2) it contains RexNodes besides RexInputRef and RexCall or // not all the RexCalls lying at the end of the project list - projects.exists(containsFunctionOf(_, FunctionLanguage.PYTHON)) && + projects.exists(containsPythonCall) && (projects.exists(expr => !expr.isInstanceOf[RexCall] && !expr.isInstanceOf[RexInputRef]) || projects.indexWhere(_.isInstanceOf[RexCall]) < projects.lastIndexWhere(_.isInstanceOf[RexInputRef])) @@ -249,14 +242,7 @@ private class ScalarFunctionSplitter( extends RexDefaultVisitor[RexNode] { override def visitCall(call: RexCall): RexNode = { - call.getOperator match { - case sfc: ScalarSqlFunction if sfc.scalarFunction.getLanguage == - FunctionLanguage.PYTHON => - visit(convertPythonFunction, call) - - case _ => - visit(!convertPythonFunction, call) - } + visit(if (isPythonCall(call)) convertPythonFunction else !convertPythonFunction, call) } override def visitNode(rexNode: RexNode): RexNode = rexNode diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecCalcRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecCalcRule.scala index c6c2e93fc2eb..9163de31a02f 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecCalcRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecCalcRule.scala @@ -24,8 +24,7 @@ import org.apache.flink.table.planner.plan.nodes.physical.batch.BatchExecCalc import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall} 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.planner.plan.utils.PythonUtil.containsFunctionOf +import org.apache.flink.table.planner.plan.utils.PythonUtil.containsPythonCall import scala.collection.JavaConverters._ @@ -42,7 +41,7 @@ class BatchExecCalcRule 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)) + !program.getExprList.asScala.exists(containsPythonCall) } def convert(rel: RelNode): RelNode = { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecPythonCalcRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecPythonCalcRule.scala index 1a16626a267e..874d44b3e3fe 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecPythonCalcRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecPythonCalcRule.scala @@ -21,11 +21,10 @@ package org.apache.flink.table.planner.plan.rules.physical.batch import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall} 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.planner.plan.nodes.FlinkConventions import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCalc import org.apache.flink.table.planner.plan.nodes.physical.batch.BatchExecPythonCalc -import org.apache.flink.table.planner.plan.utils.PythonUtil.containsFunctionOf +import org.apache.flink.table.planner.plan.utils.PythonUtil.containsPythonCall import scala.collection.JavaConverters._ @@ -42,7 +41,7 @@ class BatchExecPythonCalcRule 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)) + program.getExprList.asScala.exists(containsPythonCall) } def convert(rel: RelNode): RelNode = { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecCalcRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecCalcRule.scala index 1626e2c95581..ed9dac733fcf 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecCalcRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecCalcRule.scala @@ -21,11 +21,10 @@ package org.apache.flink.table.planner.plan.rules.physical.stream import org.apache.flink.table.planner.plan.nodes.FlinkConventions import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCalc import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamExecCalc -import org.apache.flink.table.planner.plan.utils.PythonUtil.containsFunctionOf +import org.apache.flink.table.planner.plan.utils.PythonUtil.containsPythonCall 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 scala.collection.JavaConverters._ @@ -42,7 +41,7 @@ class StreamExecCalcRule 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)) + !program.getExprList.asScala.exists(containsPythonCall) } def convert(rel: RelNode): RelNode = { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecPythonCalcRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecPythonCalcRule.scala index 858f4d2109d0..bae9b99cf5d3 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecPythonCalcRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecPythonCalcRule.scala @@ -21,11 +21,10 @@ package org.apache.flink.table.planner.plan.rules.physical.stream 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.planner.plan.nodes.FlinkConventions import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalCalc import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamExecPythonCalc -import org.apache.flink.table.planner.plan.utils.PythonUtil.containsFunctionOf +import org.apache.flink.table.planner.plan.utils.PythonUtil.containsPythonCall import scala.collection.JavaConverters._ @@ -42,7 +41,7 @@ class StreamExecPythonCalcRule 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)) + program.getExprList.asScala.exists(containsPythonCall) } def convert(rel: RelNode): RelNode = { 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 index 141d1dc3cb44..c8e059051ab5 100644 --- 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 @@ -19,50 +19,70 @@ 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.functions.python.PythonFunction 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. + * Checks whether it contains Python function call 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. + * @return true if it contains the Python function call in the specified node. */ - def containsFunctionOf( - node: RexNode, - language: FunctionLanguage, - recursive: Boolean = true): Boolean = { - node.accept(new FunctionFinder(language, recursive)) - } + def containsPythonCall(node: RexNode): Boolean = node.accept(new FunctionFinder(true, true)) + + /** + * Checks whether it contains non-Python function call in the specified node. + * + * @param node the RexNode to check + * @return true if it contains the non-Python function call in the specified node. + */ + def containsNonPythonCall(node: RexNode): Boolean = node.accept(new FunctionFinder(false, true)) + + /** + * Checks whether the specified node is a Python function call. + * + * @param node the RexNode to check + * @return true if the specified node is a Python function call. + */ + def isPythonCall(node: RexNode): Boolean = node.accept(new FunctionFinder(true, false)) + + /** + * Checks whether the specified node is a non-Python function call. + * + * @param node the RexNode to check + * @return true if the specified node is a non-Python function call. + */ + def isNonPythonCall(node: RexNode): Boolean = node.accept(new FunctionFinder(false, false)) /** * Checks whether it contains the specified kind of function in a RexNode. * - * @param expectedLanguage the expected kind of function to find + * @param findPythonFunction true to find python function, false to find non-python function * @param recursive whether check the inputs */ - private class FunctionFinder(expectedLanguage: FunctionLanguage, recursive: Boolean) + private class FunctionFinder(findPythonFunction: Boolean, recursive: Boolean) extends RexDefaultVisitor[Boolean] { + /** + * Checks whether the specified rexCall is python function call. + * + * @param rexCall the RexCall to check. + * @return true if it is python function call. + */ + private def isPythonRexCall(rexCall: RexCall): Boolean = rexCall.getOperator match { + case sfc: ScalarSqlFunction => sfc.scalarFunction.isInstanceOf[PythonFunction] + case _ => false + } + 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) - } + findPythonFunction == isPythonRexCall(call) || + (recursive && call.getOperands.exists(_.accept(this))) } 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/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedScalarFunctions.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedScalarFunctions.java index afb992b3e825..93481d09dc0d 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedScalarFunctions.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedScalarFunctions.java @@ -23,8 +23,9 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.table.functions.AggregateFunction; import org.apache.flink.table.functions.FunctionContext; -import org.apache.flink.table.functions.FunctionLanguage; import org.apache.flink.table.functions.ScalarFunction; +import org.apache.flink.table.functions.python.PythonEnv; +import org.apache.flink.table.functions.python.PythonFunction; import java.util.Arrays; import java.util.Random; @@ -160,7 +161,7 @@ public boolean isDeterministic() { /** * Test for Python Scalar Function. */ - public static class PythonScalarFunction extends ScalarFunction { + public static class PythonScalarFunction extends ScalarFunction implements PythonFunction { private final String name; public PythonScalarFunction(String name) { @@ -177,20 +178,25 @@ public TypeInformation getResultType(Class[] signature) { } @Override - public FunctionLanguage getLanguage() { - return FunctionLanguage.PYTHON; + public String toString() { + return name; } @Override - public String toString() { - return name; + public byte[] getSerializedPythonFunction() { + return new byte[0]; + } + + @Override + public PythonEnv getPythonEnv() { + return null; } } /** * Test for Python Scalar Function. */ - public static class BooleanPythonScalarFunction extends ScalarFunction { + public static class BooleanPythonScalarFunction extends ScalarFunction implements PythonFunction { private final String name; public BooleanPythonScalarFunction(String name) { @@ -207,13 +213,18 @@ public TypeInformation getResultType(Class[] signature) { } @Override - public FunctionLanguage getLanguage() { - return FunctionLanguage.PYTHON; + public String toString() { + return name; } @Override - public String toString() { - return name; + public byte[] getSerializedPythonFunction() { + return new byte[0]; + } + + @Override + public PythonEnv getPythonEnv() { + return null; } } } 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 54541ba3fad1..13cf8991824d 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,10 +20,10 @@ 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.functions.python.{PythonEnv, PythonFunction} +import org.apache.flink.table.functions.ScalarFunction import org.apache.flink.table.planner.expressions.utils.{Func1, RichFunc1} import org.apache.flink.table.planner.utils.TableTestBase - import org.junit.Test /** @@ -55,11 +55,11 @@ class ExpressionReductionRulesTest extends TableTestBase { } } -object DeterministicPythonFunc extends ScalarFunction { - - override def getLanguage: FunctionLanguage = FunctionLanguage.PYTHON +object DeterministicPythonFunc extends ScalarFunction with PythonFunction { def eval(): Long = 1 - override def isDeterministic = true + override def getSerializedPythonFunction: Array[Byte] = null + + override def getPythonEnv: PythonEnv = null } 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 b4cbc22d4b9a..3b01bd96a021 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 @@ -32,8 +32,8 @@ import org.apache.flink.configuration.Configuration import org.apache.flink.metrics.MetricGroup import org.apache.flink.table.api.TableConfig import org.apache.flink.table.calcite.FlinkTypeFactory -import org.apache.flink.table.functions.{FunctionContext, FunctionLanguage, UserDefinedFunction} -import org.apache.flink.table.plan.util.PythonUtil +import org.apache.flink.table.functions.{FunctionContext, UserDefinedFunction} +import org.apache.flink.table.plan.util.PythonUtil.containsPythonCall import org.apache.flink.types.Row import scala.collection.JavaConverters._ @@ -61,7 +61,7 @@ class ExpressionReducer(config: TableConfig) // 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) => + case (_, e) if containsPythonCall(e) => pythonUDFExprs += e None diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/PythonFunctionCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/PythonFunctionCodeGenerator.scala index 332d5730b368..bd87b29fbbd3 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/PythonFunctionCodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/PythonFunctionCodeGenerator.scala @@ -20,7 +20,7 @@ package org.apache.flink.table.codegen import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.codegen.CodeGenUtils.{primitiveDefaultValue, primitiveTypeTermForTypeInfo, newName} import org.apache.flink.table.codegen.Indenter.toISC -import org.apache.flink.table.functions.{UserDefinedFunction, FunctionLanguage, ScalarFunction} +import org.apache.flink.table.functions.{UserDefinedFunction, ScalarFunction} import org.apache.flink.table.functions.python.{PythonEnv, PythonFunction} import org.apache.flink.table.utils.EncodingUtils @@ -92,11 +92,6 @@ object PythonFunctionCodeGenerator extends Compiler[UserDefinedFunction] { | } | | @Override - | public ${classOf[FunctionLanguage].getCanonicalName} getLanguage() { - | return ${classOf[FunctionLanguage].getCanonicalName}.PYTHON; - | } - | - | @Override | public byte[] getSerializedPythonFunction() { | return $encodingUtilsTypeTerm.decodeBase64ToBytes("$encodedScalarFunction"); | } 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 6784c212dacd..2acf2dc25eeb 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 @@ -19,7 +19,6 @@ 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 @@ -34,10 +33,10 @@ trait CommonPythonCalc { } private[flink] def extractPythonScalarFunctionInfos( - rexCalls: Array[RexCall]): (Array[Int], Array[PythonFunctionInfo]) = { + pythonRexCalls: 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 pythonFunctionInfos = pythonRexCalls.map(createPythonScalarFunctionInfo(_, inputNodes)) val udfInputOffsets = inputNodes.toArray .map(_._1) @@ -46,37 +45,38 @@ trait CommonPythonCalc { } 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) + pythonRexCall: RexCall, + inputNodes: mutable.Map[RexNode, Integer]): PythonFunctionInfo = { + pythonRexCall.getOperator match { + case sfc: ScalarSqlFunction => + val inputs = new mutable.ArrayBuffer[AnyRef]() + pythonRexCall.getOperands.foreach { + case pythonRexCall: RexCall => + // Continuous Python UDFs can be chained together + val argPythonInfo = createPythonScalarFunctionInfo(pythonRexCall, inputNodes) + inputs.append(argPythonInfo) - case literal: RexLiteral => - inputs.append( - convertLiteralToPython.invoke(null, literal, literal.getType.getSqlTypeName)) + 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 { - case Some(existing) => inputs.append(existing) - case None => - val inputOffset = Integer.valueOf(inputNodes.size) - inputs.append(inputOffset) - inputNodes.put(argNode, inputOffset) - } - } + 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) + // 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/rules/datastream/DataStreamCalcRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCalcRule.scala index 4ca8913e862c..30e5ef712871 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 @@ -21,12 +21,11 @@ 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.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 org.apache.flink.table.plan.util.PythonUtil.containsPythonCall import scala.collection.JavaConverters._ @@ -40,7 +39,7 @@ class 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)) + !program.getExprList.asScala.exists(containsPythonCall) } def convert(rel: RelNode): RelNode = { 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 7c618dceb70e..6e2bee168724 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 @@ -21,12 +21,11 @@ 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 org.apache.flink.table.plan.util.PythonUtil.containsPythonCall import scala.collection.JavaConverters._ @@ -40,7 +39,7 @@ class 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)) + program.getExprList.asScala.exists(containsPythonCall) } def convert(rel: RelNode): RelNode = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/PythonCalcSplitRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/PythonCalcSplitRule.scala index 8a55ec131b69..4c2fad07bed8 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/PythonCalcSplitRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/PythonCalcSplitRule.scala @@ -22,11 +22,9 @@ 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.PythonUtil.{containsPythonCall, containsNonPythonCall, isPythonCall, isNonPythonCall} import org.apache.flink.table.plan.util.{InputRefVisitor, RexDefaultVisitor} import scala.collection.JavaConverters._ @@ -140,8 +138,7 @@ object PythonCalcSplitConditionRule extends PythonCalcSplitRuleBase( // matches if it contains Python functions in condition Option(calc.getProgram.getCondition) - .map(calc.getProgram.expandLocalRef) - .exists(containsFunctionOf(_, FunctionLanguage.PYTHON)) + .map(calc.getProgram.expandLocalRef).exists(containsPythonCall) } override def isConvertPythonFunction(program: RexProgram): Boolean = true @@ -166,14 +163,11 @@ object PythonCalcSplitProjectionRule extends PythonCalcSplitRuleBase( val projects = calc.getProgram.getProjectList.map(calc.getProgram.expandLocalRef) // matches if it contains both Python functions and Java functions in the projection - projects.exists(containsFunctionOf(_, FunctionLanguage.PYTHON)) && - projects.exists(containsFunctionOf(_, FunctionLanguage.JVM)) + projects.exists(containsPythonCall) && projects.exists(containsNonPythonCall) } override def isConvertPythonFunction(program: RexProgram): Boolean = { - program.getProjectList - .map(program.expandLocalRef) - .exists(containsFunctionOf(_, FunctionLanguage.JVM, recursive = false)) + program.getProjectList.map(program.expandLocalRef).exists(isNonPythonCall) } override def split(program: RexProgram, splitter: ScalarFunctionSplitter) @@ -197,8 +191,7 @@ object PythonCalcPushConditionRule extends PythonCalcSplitRuleBase( // matches if all the following conditions hold true: // 1) the condition is not null // 2) it contains Python functions in the projection - calc.getProgram.getCondition != null && - projects.exists(containsFunctionOf(_, FunctionLanguage.PYTHON)) + calc.getProgram.getCondition != null && projects.exists(containsPythonCall) } override def isConvertPythonFunction(program: RexProgram): Boolean = false @@ -228,7 +221,7 @@ object PythonCalcRewriteProjectionRule extends PythonCalcSplitRuleBase( // 1) it contains Python functions in the projection // 2) it contains RexNodes besides RexInputRef and RexCall or // not all the RexCalls lying at the end of the project list - projects.exists(containsFunctionOf(_, FunctionLanguage.PYTHON)) && + projects.exists(containsPythonCall) && (projects.exists(expr => !expr.isInstanceOf[RexCall] && !expr.isInstanceOf[RexInputRef]) || projects.indexWhere(_.isInstanceOf[RexCall]) < projects.lastIndexWhere(_.isInstanceOf[RexInputRef])) @@ -249,14 +242,7 @@ private class ScalarFunctionSplitter( 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) - } + visit(if (isPythonCall(call)) convertPythonFunction else !convertPythonFunction, call) } override def visitNode(rexNode: RexNode): RexNode = rexNode 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 2e0073cc12ff..f6221b1b4258 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 @@ -18,7 +18,7 @@ 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.python.PythonFunction import org.apache.flink.table.functions.utils.ScalarSqlFunction import scala.collection.JavaConversions._ @@ -26,43 +26,62 @@ import scala.collection.JavaConversions._ object PythonUtil { /** - * Checks whether it contains the specified kind of function in the specified node. + * Checks whether it contains Python function call 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. + * @return true if it contains the Python function call in the specified node. */ - def containsFunctionOf( - node: RexNode, - language: FunctionLanguage, - recursive: Boolean = true): Boolean = { - node.accept(new FunctionFinder(language, recursive)) - } + def containsPythonCall(node: RexNode): Boolean = node.accept(new FunctionFinder(true, true)) + + /** + * Checks whether it contains non-Python function call in the specified node. + * + * @param node the RexNode to check + * @return true if it contains the non-Python function call in the specified node. + */ + def containsNonPythonCall(node: RexNode): Boolean = node.accept(new FunctionFinder(false, true)) + + /** + * Checks whether the specified node is a Python function call. + * + * @param node the RexNode to check + * @return true if the specified node is a Python function call. + */ + def isPythonCall(node: RexNode): Boolean = node.accept(new FunctionFinder(true, false)) + + /** + * Checks whether the specified node is a non-Python function call. + * + * @param node the RexNode to check + * @return true if the specified node is a non-Python function call. + */ + def isNonPythonCall(node: RexNode): Boolean = node.accept(new FunctionFinder(false, false)) /** * Checks whether it contains the specified kind of function in a RexNode. * - * @param expectedLanguage the expected kind of function to find + * @param findPythonFunction true to find python function, false to find non-python function * @param recursive whether check the inputs */ - private class FunctionFinder(expectedLanguage: FunctionLanguage, recursive: Boolean) + private class FunctionFinder(findPythonFunction: Boolean, recursive: Boolean) extends RexDefaultVisitor[Boolean] { + /** + * Checks whether the specified rexCall is python function call. + * + * @param rexCall the RexCall to check. + * @return true if it is python function call. + */ + private def isPythonRexCall(rexCall: RexCall): Boolean = rexCall.getOperator match { + case sfc: ScalarSqlFunction => sfc.getScalarFunction.isInstanceOf[PythonFunction] + case _ => false + } + 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) - } + findPythonFunction == isPythonRexCall(call) || + (recursive && call.getOperands.exists(_.accept(this))) } 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/plan/ExpressionReductionRulesTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/ExpressionReductionRulesTest.scala index 97bb3e291df6..b9dd590e93e2 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 @@ -22,7 +22,8 @@ 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.expressions.utils.{Func1, RichFunc1} -import org.apache.flink.table.functions.{FunctionLanguage, ScalarFunction} +import org.apache.flink.table.functions.python.{PythonEnv, PythonFunction} +import org.apache.flink.table.functions.ScalarFunction import org.apache.flink.table.utils.TableTestBase import org.apache.flink.table.utils.TableTestUtil._ import org.junit.{Ignore, Test} @@ -568,11 +569,11 @@ object DeterministicNullFunc extends ScalarFunction { override def isDeterministic = true } -object DeterministicPythonFunc extends ScalarFunction { - - override def getLanguage: FunctionLanguage = FunctionLanguage.PYTHON +object DeterministicPythonFunc extends ScalarFunction with PythonFunction { def eval(): Long = 1L - override def isDeterministic = true + override def getSerializedPythonFunction: Array[Byte] = null + + override def getPythonEnv: PythonEnv = null } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/PythonCalcSplitRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/PythonCalcSplitRuleTest.scala index 5d59e3767cad..0567a49362c6 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/PythonCalcSplitRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/PythonCalcSplitRuleTest.scala @@ -21,7 +21,8 @@ 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.functions.python.{PythonEnv, PythonFunction} +import org.apache.flink.table.functions.ScalarFunction import org.apache.flink.table.utils.TableTestUtil._ import org.apache.flink.table.utils.TableTestBase import org.junit.Test @@ -269,24 +270,28 @@ class PythonCalcSplitRuleTest extends TableTestBase { } } -class PythonScalarFunction(name: String) extends ScalarFunction { +class PythonScalarFunction(name: String) extends ScalarFunction with PythonFunction { 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 + + override def getSerializedPythonFunction: Array[Byte] = null + + override def getPythonEnv: PythonEnv = null } -class BooleanPythonScalarFunction(name: String) extends ScalarFunction { +class BooleanPythonScalarFunction(name: String) extends ScalarFunction with PythonFunction { 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 + + override def getSerializedPythonFunction: Array[Byte] = null + + override def getPythonEnv: PythonEnv = null } From 6df53ea9558d0dd01392ee4ae9622240574628c6 Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Fri, 25 Oct 2019 11:27:01 +0200 Subject: [PATCH 221/746] [FLINK-14522] Revert FLINK-13985 (sun.misc.Cleaner is not available in Java 9+) --- .../org/apache/flink/core/memory/MemorySegmentFactory.java | 5 ++--- .../main/java/org/apache/flink/core/memory/MemoryUtils.java | 3 ++- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java index c297a2648047..3a82f5cd5d69 100644 --- a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java +++ b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java @@ -110,9 +110,8 @@ public static MemorySegment allocateUnpooledOffHeapMemory(int size, Object owner * @return A new memory segment, backed by off-heap unsafe memory. */ public static MemorySegment allocateOffHeapUnsafeMemory(int size, Object owner) { - long address = MemoryUtils.allocateUnsafe(size); - ByteBuffer offHeapBuffer = MemoryUtils.wrapUnsafeMemoryWithByteBuffer(address, size); - return new HybridMemorySegment(offHeapBuffer, owner, MemoryUtils.createMemoryGcCleaner(offHeapBuffer, address)); + // TODO: rollbacked to direct memory because of FLINK-13985 + return allocateUnpooledOffHeapMemory(size, owner); } /** diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java b/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java index 76a295d8ef2d..ce10b06d2a46 100644 --- a/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java +++ b/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java @@ -110,12 +110,13 @@ static long allocateUnsafe(long size) { */ @SuppressWarnings("UseOfSunClasses") static Runnable createMemoryGcCleaner(Object owner, long address) { + // TODO: rollbacked to direct memory because of FLINK-13985 // The release call is wrapped with the sun.misc.Cleaner // which will schedule it before GC is run for the owner object (not reachable in user code). // but only if sun.misc.Cleaner::clean has not been already called explicitly by user before. // If sun.misc.Cleaner::clean is called after GC it will not call the release. // This way we guarantee that there will always be a release at some point but only once. - return sun.misc.Cleaner.create(owner, () -> releaseUnsafe(address))::clean; + return null; // sun.misc.Cleaner.create(owner, () -> releaseUnsafe(address))::clean; } private static void releaseUnsafe(long address) { From bde12512c3ebb27e81a8f54f45fabb7eb02db55d Mon Sep 17 00:00:00 2001 From: hequn8128 Date: Fri, 25 Oct 2019 19:42:50 +0800 Subject: [PATCH 222/746] [hotfix][table-planner] Remove unused import in CalcITCase --- .../org/apache/flink/table/runtime/stream/table/CalcITCase.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/table/CalcITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/table/CalcITCase.scala index 292bc47f3851..521b1396ee6f 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/table/CalcITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/table/CalcITCase.scala @@ -21,7 +21,6 @@ package org.apache.flink.table.runtime.stream.table import org.apache.flink.api.scala._ import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment import org.apache.flink.table.api.scala._ -import org.apache.flink.table.expressions.Literal import org.apache.flink.table.expressions.utils._ import org.apache.flink.table.runtime.utils.{StreamITCase, StreamTestData, UserDefinedFunctionTestUtils} import org.apache.flink.test.util.AbstractTestBase From 7c1182260663d1d692c4dbbd32fc2eab418ca8d4 Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Mon, 14 Oct 2019 13:05:37 +0800 Subject: [PATCH 223/746] [FLINK-14381][table-planner] Remove dynamic partition shuffle support in legacy planner --- .../api/internal/BatchTableEnvImpl.scala | 26 +--- .../table/api/internal/TableEnvImpl.scala | 4 +- .../flink/table/planner/StreamPlanner.scala | 32 +--- .../flink/table/sinks/TableSinkUtils.scala | 28 +--- .../batch/sql/PartitionableSinkITCase.scala | 137 +++--------------- 5 files changed, 31 insertions(+), 196 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala index 5ec2bbaefc9b..789948a66f30 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala @@ -49,7 +49,6 @@ import org.apache.flink.table.utils.TableConnectorUtils import org.apache.flink.types.Row import _root_.scala.collection.JavaConverters._ -import _root_.scala.collection.JavaConversions._ /** * The abstract base class for the implementation of batch TableEnvironments. @@ -118,13 +117,12 @@ abstract class BatchTableEnvImpl( // translate the Table into a DataSet and provide the type that the TableSink expects. val result: DataSet[T] = translate(table)(outputType) // Give the DataSet to the TableSink to emit it. - batchSink.emitDataSet(shuffleByPartitionFieldsIfNeeded(batchSink, result)) + batchSink.emitDataSet(result) case boundedSink: OutputFormatTableSink[T] => val outputType = fromDataTypeToLegacyInfo(sink.getConsumedDataType) .asInstanceOf[TypeInformation[T]] // translate the Table into a DataSet and provide the type that the TableSink expects. - val translated: DataSet[T] = translate(table)(outputType) - val result = shuffleByPartitionFieldsIfNeeded(boundedSink, translated) + val result: DataSet[T] = translate(table)(outputType) // use the OutputFormat to consume the DataSet. val dataSink = result.output(boundedSink.getOutputFormat) dataSink.name( @@ -137,26 +135,6 @@ abstract class BatchTableEnvImpl( } } - /** - * Key by the partition fields if the sink is a [[PartitionableTableSink]]. - * @param sink the table sink - * @param dataSet the data set - * @tparam R the data set record type - * @return a data set that maybe keyed by. - */ - private def shuffleByPartitionFieldsIfNeeded[R]( - sink: TableSink[_], - dataSet: DataSet[R]): DataSet[R] = { - sink match { - case partitionableSink: PartitionableTableSink - if partitionableSink.getPartitionFieldNames.nonEmpty => - val fieldNames = sink.getTableSchema.getFieldNames - val indices = partitionableSink.getPartitionFieldNames.map(fieldNames.indexOf(_)) - dataSet.partitionByHash(indices:_*) - case _ => dataSet - } - } - /** * Creates a final converter that maps the internal row type to external type. * 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 e02803259bc9..ad14eca3c48d 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 @@ -488,9 +488,7 @@ abstract class TableEnvImpl( tableSink) // set static partitions if it is a partitioned table sink tableSink match { - case partitionableSink: PartitionableTableSink - if partitionableSink.getPartitionFieldNames != null - && partitionableSink.getPartitionFieldNames.nonEmpty => + case partitionableSink: PartitionableTableSink => partitionableSink.setStaticPartition(insertOptions.staticPartitions) case _ => } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala index 10a04dee47ae..c50d32b61595 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala @@ -161,9 +161,7 @@ class StreamPlanner( sink) // set static partitions if it is a partitioned sink sink match { - case partitionableSink: PartitionableTableSink - if partitionableSink.getPartitionFieldNames != null - && partitionableSink.getPartitionFieldNames.nonEmpty => + case partitionableSink: PartitionableTableSink => partitionableSink.setStaticPartition(catalogSink.getStaticPartitions) case _ => } @@ -273,8 +271,7 @@ class StreamPlanner( val resultSink = sink match { case retractSink: RetractStreamTableSink[T] => retractSink match { - case partitionableSink: PartitionableTableSink - if partitionableSink.getPartitionFieldNames.nonEmpty => + case _: PartitionableTableSink => throw new TableException("Partitionable sink in retract stream mode " + "is not supported yet!") case _ => @@ -283,8 +280,7 @@ class StreamPlanner( case upsertSink: UpsertStreamTableSink[T] => upsertSink match { - case partitionableSink: PartitionableTableSink - if partitionableSink.getPartitionFieldNames.nonEmpty => + case _: PartitionableTableSink => throw new TableException("Partitionable sink in upsert stream mode " + "is not supported yet!") case _ => @@ -351,7 +347,7 @@ class StreamPlanner( streamQueryConfig, withChangeFlag = false) // Give the DataStream to the TableSink to emit it. - sink.consumeDataStream(shuffleByPartitionFieldsIfNeeded(sink, result)) + sink.consumeDataStream(result) } private def writeToUpsertSink[T]( @@ -389,26 +385,6 @@ class StreamPlanner( sink.consumeDataStream(result) } - /** - * Key by the partition fields if the sink is a [[PartitionableTableSink]]. - * @param sink the table sink - * @param dataStream the data stream - * @tparam R the data stream record type - * @return a data stream that maybe keyed by. - */ - private def shuffleByPartitionFieldsIfNeeded[R]( - sink: TableSink[_], - dataStream: DataStream[R]): DataStream[R] = { - sink match { - case partitionableSink: PartitionableTableSink - if partitionableSink.getPartitionFieldNames.nonEmpty => - val fieldNames = sink.getTableSchema.getFieldNames - val indices = partitionableSink.getPartitionFieldNames.map(fieldNames.indexOf(_)) - dataStream.keyBy(indices:_*) - case _ => dataStream - } - } - private def translateToType[A]( table: QueryOperation, queryConfig: StreamQueryConfig, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/sinks/TableSinkUtils.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/sinks/TableSinkUtils.scala index ad61727deefb..17641315d81a 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/sinks/TableSinkUtils.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/sinks/TableSinkUtils.scala @@ -24,8 +24,6 @@ import org.apache.flink.table.operations.QueryOperation import java.util.{Map => JMap} -import scala.collection.JavaConversions._ - object TableSinkUtils { /** @@ -72,30 +70,10 @@ object TableSinkUtils { // check partitions are valid if (staticPartitions != null && !staticPartitions.isEmpty) { val invalidMsg = "Can't insert static partitions into a non-partitioned table sink. " + - "A partitioned sink should implement 'PartitionableTableSink' and return partition " + - "field names via 'getPartitionFieldNames()' method." + "A partitioned sink should implement 'PartitionableTableSink'." sink match { - case pts: PartitionableTableSink => - val partitionFields = pts.getPartitionFieldNames - if (partitionFields == null || partitionFields.isEmpty) { - throw new ValidationException(invalidMsg) - } - staticPartitions.map(_._1) foreach { p => - if (!partitionFields.contains(p)) { - throw new ValidationException(s"Static partition column $p " + - s"should be in the partition fields list $partitionFields.") - } - } - staticPartitions.map(_._1).zip(partitionFields).foreach { - case (p1, p2) => - if (p1 != p2) { - throw new ValidationException(s"Static partition column $p1 " + - s"should appear before dynamic partition $p2.") - } - } - case _ => - throw new ValidationException(invalidMsg) - + case _: PartitionableTableSink => + case _ => throw new ValidationException(invalidMsg) } } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala index cb17bab6826f..8aaf570a8c33 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala @@ -28,9 +28,9 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.api.scala.ExecutionEnvironment import org.apache.flink.configuration.Configuration import org.apache.flink.table.api.scala.BatchTableEnvironment -import org.apache.flink.table.api.{DataTypes, SqlDialect, TableSchema, ValidationException} +import org.apache.flink.table.api.{DataTypes, SqlDialect, TableSchema} import org.apache.flink.table.factories.utils.TestCollectionTableFactory.TestCollectionInputFormat -import org.apache.flink.table.runtime.batch.sql.PartitionableSinkITCase.{RESULT1, RESULT2, RESULT3, _} +import org.apache.flink.table.runtime.batch.sql.PartitionableSinkITCase._ import org.apache.flink.table.sinks.{BatchTableSink, PartitionableTableSink, TableSink} import org.apache.flink.table.sources.BatchTableSource import org.apache.flink.table.types.logical.{BigIntType, IntType, VarCharType} @@ -40,7 +40,7 @@ import org.junit.Assert.assertEquals import org.junit.rules.ExpectedException import org.junit.{Before, Rule, Test} -import java.util.{ArrayList => JArrayList, LinkedList => JLinkedList, List => JList, Map => JMap} +import java.util.{LinkedList => JLinkedList, List => JList, Map => JMap} import org.apache.flink.api.java import scala.collection.JavaConversions._ @@ -59,7 +59,7 @@ class PartitionableSinkITCase { @Before def before(): Unit = { - batchExec.setParallelism(3) + batchExec.setParallelism(1) tEnv = BatchTableEnvironment.create(batchExec) tEnv.getConfig.setSqlDialect(SqlDialect.HIVE) registerTableSource("nonSortTable", testData.toList) @@ -76,52 +76,6 @@ class PartitionableSinkITCase { tEnv.registerTableSource(name, new CollectionTableSource(data, 100, tableSchema)) } - @Test - def testInsertWithOutPartitionGrouping(): Unit = { - registerTableSink(grouping = false) - tEnv.sqlUpdate("insert into sinkTable select a, max(b), c" - + " from nonSortTable group by a, c") - tEnv.execute("testJob") - assertEquals(List("1,5,Hi", - "1,5,Hi01", - "1,5,Hi02"), - RESULT1.sorted) - assert(RESULT2.isEmpty) - assertEquals(List("2,1,Hello world01", - "2,1,Hello world02", - "2,1,Hello world03", - "2,1,Hello world04", - "2,2,Hello world, how are you?", - "3,1,Hello world", - "3,2,Hello", - "3,2,Hello01", - "3,2,Hello02", - "3,2,Hello03", - "3,2,Hello04"), - RESULT3.sorted) - } - - @Test - def testInsertWithPartitionGrouping(): Unit = { - registerTableSink() - tEnv.sqlUpdate("insert into sinkTable select a, b, c from sortTable") - tEnv.execute("testJob") - assertEquals(List("1,1,Hello world", - "1,1,Hello world, how are you?"), - RESULT1.toList) - assertEquals(List("4,4,你好,陌生人", - "4,4,你好,陌生人,我是", - "4,4,你好,陌生人,我是中国人", - "4,4,你好,陌生人,我是中国人,你来自哪里?"), - RESULT2.toList) - assertEquals(List("2,2,Hi", - "2,2,Hello", - "3,3,I'm fine, thank", - "3,3,I'm fine, thank you", - "3,3,I'm fine, thank you, and you?"), - RESULT3.toList) - } - @Test def testInsertWithStaticPartitions(): Unit = { val testSink = registerTableSink() @@ -140,49 +94,12 @@ class PartitionableSinkITCase { "1,4,你好,陌生人,我是", "1,4,你好,陌生人,我是中国人", "1,4,你好,陌生人,我是中国人,你来自哪里?"), - RESULT1.toList) - assert(RESULT2.isEmpty) - assert(RESULT3.isEmpty) - } - - @Test - def testInsertWithStaticAndDynamicPartitions(): Unit = { - val testSink = registerTableSink(partitionColumns = Array("a", "b")) - tEnv.sqlUpdate("insert into sinkTable partition(a=1) select b, c from sortTable") - tEnv.execute("testJob") - // this sink should have been set up with static partitions - assertEquals(testSink.getStaticPartitions.toMap, Map("a" -> "1")) - assertEquals(List("1,3,I'm fine, thank", - "1,3,I'm fine, thank you", - "1,3,I'm fine, thank you, and you?"), - RESULT1.toList) - assertEquals(List("1,2,Hi", - "1,2,Hello"), - RESULT2.toList) - assertEquals(List("1,1,Hello world", - "1,1,Hello world, how are you?", - "1,4,你好,陌生人", - "1,4,你好,陌生人,我是", - "1,4,你好,陌生人,我是中国人", - "1,4,你好,陌生人,我是中国人,你来自哪里?"), - RESULT3.toList) - } - - @Test - def testDynamicPartitionInFrontOfStaticPartition(): Unit = { - expectedEx.expect(classOf[ValidationException]) - expectedEx.expectMessage("Static partition column b " - + "should appear before dynamic partition a") - registerTableSink(partitionColumns = Array("a", "b")) - tEnv.sqlUpdate("insert into sinkTable partition(b=1) select a, c from sortTable") - tEnv.execute("testJob") + RESULT.toList) } @Test def testStaticPartitionNotInPartitionFields(): Unit = { - expectedEx.expect(classOf[ValidationException]) - expectedEx.expectMessage("Static partition column c " + - "should be in the partition fields list [a, b].") + expectedEx.expect(classOf[RuntimeException]) registerTableSink(tableName = "sinkTable2", rowType = type4, partitionColumns = Array("a", "b")) tEnv.sqlUpdate("insert into sinkTable2 partition(c=1) select a, b from sinkTable2") @@ -191,9 +108,7 @@ class PartitionableSinkITCase { @Test def testInsertStaticPartitionOnNonPartitionedSink(): Unit = { - expectedEx.expect(classOf[ValidationException]) - expectedEx.expectMessage( - "Can't insert static partitions into a non-partitioned table sink.") + expectedEx.expect(classOf[RuntimeException]) registerTableSink(tableName = "sinkTable2", rowType = type4, partitionColumns = Array()) tEnv.sqlUpdate("insert into sinkTable2 partition(c=1) select a, b from sinkTable2") tEnv.execute("testJob") @@ -202,30 +117,33 @@ class PartitionableSinkITCase { private def registerTableSink( tableName: String = "sinkTable", rowType: RowTypeInfo = type3, - grouping: Boolean = true, partitionColumns: Array[String] = Array[String]("a")): TestSink = { - val testSink = new TestSink(rowType, grouping, partitionColumns) + val testSink = new TestSink(rowType, partitionColumns) tEnv.registerTableSink(tableName, testSink) testSink } - private class TestSink(rowType: RowTypeInfo, - supportsGrouping: Boolean, - partitionColumns: Array[String]) + private class TestSink(rowType: RowTypeInfo, partitionColumns: Array[String]) extends BatchTableSink[Row] with PartitionableTableSink { private var staticPartitions: JMap[String, String] = _ override def getPartitionFieldNames: JList[String] = partitionColumns.toList - override def setStaticPartition(partitions: JMap[String, String]): Unit = + override def setStaticPartition(partitions: JMap[String, String]): Unit = { + partitions.foreach { case (part, v) => + if (!partitionColumns.contains(part)) { + throw new RuntimeException + } + } this.staticPartitions = partitions + } override def configure(fieldNames: Array[String], fieldTypes: Array[TypeInformation[_]]): TableSink[Row] = this override def configurePartitionGrouping(s: Boolean): Boolean = { - supportsGrouping + false } override def getTableSchema: TableSchema = { @@ -271,33 +189,20 @@ class PartitionableSinkITCase { } object PartitionableSinkITCase { - val RESULT1 = new JLinkedList[String]() - val RESULT2 = new JLinkedList[String]() - val RESULT3 = new JLinkedList[String]() - val RESULT_QUEUE: JList[JLinkedList[String]] = new JArrayList[JLinkedList[String]]() + val RESULT = new JLinkedList[String]() def init(): Unit = { - RESULT1.clear() - RESULT2.clear() - RESULT3.clear() - RESULT_QUEUE.clear() - RESULT_QUEUE.add(RESULT1) - RESULT_QUEUE.add(RESULT2) - RESULT_QUEUE.add(RESULT3) + RESULT.clear() } /** OutputFormat that writes data to a collection. **/ class CollectionOutputFormat extends RichOutputFormat[String] { - private var resultSet: JLinkedList[String] = _ - override def configure(parameters: Configuration): Unit = {} - override def open(taskNumber: Int, numTasks: Int): Unit = { - resultSet = RESULT_QUEUE.get(taskNumber) - } + override def open(taskNumber: Int, numTasks: Int): Unit = {} override def writeRecord(record: String): Unit = { - resultSet.add(record) + RESULT.add(record) } override def close(): Unit = {} From fcc4b7fc49dbf33bfa2dbc29a752ccc99666138c Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Mon, 14 Oct 2019 15:24:36 +0800 Subject: [PATCH 224/746] [FLINK-14381][table-planner-blink] User table factory and catalog table to test partition source and sink --- ....apache.flink.table.factories.TableFactory | 2 + .../plan/batch/sql/TableSourceTest.scala | 4 +- ...PartitionIntoTableSourceScanRuleTest.scala | 4 +- .../plan/stream/sql/TableSourceTest.scala | 4 +- .../batch/sql/PartitionableSinkITCase.scala | 174 ++++++++++++------ .../runtime/batch/sql/TableSourceITCase.scala | 6 +- .../stream/sql/TableSourceITCase.scala | 4 +- .../planner/utils/testTableSources.scala | 81 +++++++- 8 files changed, 210 insertions(+), 69 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink-table/flink-table-planner-blink/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory index 43387ecc5a82..54d5e7e256fd 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory +++ b/flink-table/flink-table-planner-blink/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory @@ -14,3 +14,5 @@ # limitations under the License. org.apache.flink.table.planner.factories.utils.TestCollectionTableFactory +org.apache.flink.table.planner.runtime.batch.sql.TestPartitionableSinkFactory +org.apache.flink.table.planner.utils.TestPartitionableSourceFactory diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.scala index ba5191e1d2c2..bdf960ed398b 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.scala @@ -22,7 +22,7 @@ import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, LocalTimeTypeInfo, T import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.table.api.{DataTypes, TableSchema, Types, ValidationException} import org.apache.flink.table.planner.expressions.utils.Func1 -import org.apache.flink.table.planner.utils.{DateTimeTestUtil, TableTestBase, TestFilterableTableSource, TestNestedProjectableTableSource, TestPartitionableTableSource, TestProjectableTableSource, TestTableSource} +import org.apache.flink.table.planner.utils.{DateTimeTestUtil, TableTestBase, TestFilterableTableSource, TestNestedProjectableTableSource, TestPartitionableSourceFactory, TestProjectableTableSource, TestTableSource} import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter import org.apache.flink.table.sources.TableSource import org.apache.flink.table.types.DataType @@ -48,7 +48,7 @@ class TableSourceTest extends TableTestBase { Seq.empty[Row]) ) util.tableEnv.registerTableSource("FilterableTable", TestFilterableTableSource(true)) - util.tableEnv.registerTableSource("PartitionableTable", new TestPartitionableTableSource(true)) + TestPartitionableSourceFactory.registerTableSource(util.tableEnv, "PartitionableTable", true) } @Test diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRuleTest.scala index 6af963c9aec5..03344bd53a1c 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRuleTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRuleTest.scala @@ -19,7 +19,7 @@ package org.apache.flink.table.planner.plan.rules.logical import org.apache.flink.table.planner.expressions.utils.Func1 import org.apache.flink.table.planner.plan.optimize.program.{FlinkBatchProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE} -import org.apache.flink.table.planner.utils.{TableConfigUtils, TableTestBase, TestPartitionableTableSource} +import org.apache.flink.table.planner.utils.{TableConfigUtils, TableTestBase, TestPartitionableSourceFactory} import org.apache.calcite.plan.hep.HepMatchOrder import org.apache.calcite.tools.RuleSets @@ -44,7 +44,7 @@ class PushPartitionIntoTableSourceScanRuleTest extends TableTestBase { .build() ) - util.tableEnv.registerTableSource("MyTable", new TestPartitionableTableSource(true)) + TestPartitionableSourceFactory.registerTableSource(util.tableEnv, "MyTable", true) } @Test diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.scala index e6168361388a..8a108c9a773d 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.scala @@ -22,7 +22,7 @@ import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, SqlTimeTypeInfo, Typ import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.table.api.{DataTypes, TableSchema, Types, ValidationException} import org.apache.flink.table.planner.expressions.utils.Func1 -import org.apache.flink.table.planner.utils.{DateTimeTestUtil, TableTestBase, TestFilterableTableSource, TestNestedProjectableTableSource, TestPartitionableTableSource, TestProjectableTableSource, TestTableSource, TestTableSourceWithTime} +import org.apache.flink.table.planner.utils.{DateTimeTestUtil, TableTestBase, TestFilterableTableSource, TestNestedProjectableTableSource, TestPartitionableSourceFactory, TestProjectableTableSource, TestTableSource, TestTableSourceWithTime} import org.apache.flink.table.sources.TableSource import org.apache.flink.table.types.DataType import org.apache.flink.types.Row @@ -40,7 +40,7 @@ class TableSourceTest extends TableTestBase { @Before def setup(): Unit = { util.tableEnv.registerTableSource("FilterableTable", TestFilterableTableSource(false)) - util.tableEnv.registerTableSource("PartitionableTable", new TestPartitionableTableSource(false)) + TestPartitionableSourceFactory.registerTableSource(util.tableEnv, "PartitionableTable", false) } @Test diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala index 80b2b4e4603c..9a94928088b9 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala @@ -23,14 +23,21 @@ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.datastream.{DataStream, DataStreamSink} +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import org.apache.flink.streaming.api.functions.sink.RichSinkFunction import org.apache.flink.table.api.config.ExecutionConfigOptions -import org.apache.flink.table.api.{SqlDialect, TableSchema, ValidationException} +import org.apache.flink.table.api.{SqlDialect, TableException, TableSchema, ValidationException} +import org.apache.flink.table.catalog.{CatalogTableImpl, ObjectPath} +import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE +import org.apache.flink.table.descriptors.DescriptorProperties +import org.apache.flink.table.descriptors.Schema.SCHEMA +import org.apache.flink.table.factories.{TableSinkFactory, TableSourceFactory} import org.apache.flink.table.planner.runtime.batch.sql.PartitionableSinkITCase._ import org.apache.flink.table.planner.runtime.utils.BatchTestBase import org.apache.flink.table.planner.runtime.utils.BatchTestBase.row import org.apache.flink.table.planner.runtime.utils.TestData._ import org.apache.flink.table.sinks.{PartitionableTableSink, StreamTableSink, TableSink} +import org.apache.flink.table.sources.{StreamTableSource, TableSource} import org.apache.flink.table.types.logical.{BigIntType, IntType, VarCharType} import org.apache.flink.types.Row @@ -38,9 +45,11 @@ import org.junit.Assert._ import org.junit.rules.ExpectedException import org.junit.{Before, Rule, Test} -import java.util.{ArrayList => JArrayList, LinkedList => JLinkedList, List => JList, Map => JMap} +import java.util +import java.util.{function, ArrayList => JArrayList, LinkedList => JLinkedList, List => JList, Map => JMap} import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.Seq /** @@ -115,11 +124,9 @@ class PartitionableSinkITCase extends BatchTestBase { @Test def testInsertWithStaticPartitions(): Unit = { - val testSink = registerTableSink() + registerTableSink() tEnv.sqlUpdate("insert into sinkTable partition(a=1) select b, c from sortTable") tEnv.execute("testJob") - // this sink should have been set up with static partitions - assertEquals(testSink.getStaticPartitions.toMap, Map("a" -> "1")) assertEquals(List("1,2,Hi", "1,1,Hello world", "1,2,Hello", @@ -138,11 +145,9 @@ class PartitionableSinkITCase extends BatchTestBase { @Test def testInsertWithStaticAndDynamicPartitions(): Unit = { - val testSink = registerTableSink(partitionColumns = Array("a", "b")) + registerTableSink(partitionColumns = Array("a", "b")) tEnv.sqlUpdate("insert into sinkTable partition(a=1) select b, c from sortTable") tEnv.execute("testJob") - // this sink should have been set up with static partitions - assertEquals(testSink.getStaticPartitions.toMap, Map("a" -> "1")) assertEquals(List("1,3,I'm fine, thank", "1,3,I'm fine, thank you", "1,3,I'm fine, thank you, and you?"), @@ -172,8 +177,6 @@ class PartitionableSinkITCase extends BatchTestBase { @Test def testStaticPartitionNotInPartitionFields(): Unit = { expectedEx.expect(classOf[ValidationException]) - expectedEx.expectMessage("Static partition column c " + - "should be in the partition fields list [a, b].") registerTableSink(tableName = "sinkTable2", rowType = type4, partitionColumns = Array("a", "b")) tEnv.sqlUpdate("insert into sinkTable2 partition(c=1) select a, b from sinkTable2") @@ -182,9 +185,7 @@ class PartitionableSinkITCase extends BatchTestBase { @Test def testInsertStaticPartitionOnNonPartitionedSink(): Unit = { - expectedEx.expect(classOf[ValidationException]) - expectedEx.expectMessage( - "Can't insert static partitions into a non-partitioned table sink.") + expectedEx.expect(classOf[TableException]) registerTableSink(tableName = "sinkTable2", rowType = type4, partitionColumns = Array()) tEnv.sqlUpdate("insert into sinkTable2 partition(c=1) select a, b from sinkTable2") tEnv.execute("testJob") @@ -194,51 +195,22 @@ class PartitionableSinkITCase extends BatchTestBase { tableName: String = "sinkTable", rowType: RowTypeInfo = type3, grouping: Boolean = true, - partitionColumns: Array[String] = Array[String]("a")): TestSink = { - val testSink = new TestSink(rowType, grouping, partitionColumns) - tEnv.registerTableSink(tableName, testSink) - testSink - } - - private class TestSink( - rowType: RowTypeInfo, - supportsGrouping: Boolean, - partitionColumns: Array[String]) - extends StreamTableSink[Row] - with PartitionableTableSink { - private var staticPartitions: JMap[String, String] = _ - - override def getPartitionFieldNames: JList[String] = partitionColumns.toList - - override def setStaticPartition(partitions: JMap[String, String]): Unit = - this.staticPartitions = partitions - - override def configure(fieldNames: Array[String], - fieldTypes: Array[TypeInformation[_]]): TableSink[Row] = this - - override def configurePartitionGrouping(s: Boolean): Boolean = { - supportsGrouping - } - - override def getTableSchema: TableSchema = { - new TableSchema(Array("a", "b", "c"), type3.getFieldTypes) - } - - override def getOutputType: RowTypeInfo = type3 - - override def emitDataStream(dataStream: DataStream[Row]): Unit = { - dataStream.addSink(new UnsafeMemorySinkFunction(type3)) - .setParallelism(dataStream.getParallelism) + partitionColumns: Array[String] = Array[String]("a")): Unit = { + val properties = new DescriptorProperties() + properties.putString("supports-grouping", grouping.toString) + properties.putString(CONNECTOR_TYPE, "TestPartitionableSink") + partitionColumns.zipWithIndex.foreach { case (part, i) => + properties.putString("partition-column." + i, part) } - override def consumeDataStream(dataStream: DataStream[Row]): DataStreamSink[_] = { - dataStream.addSink(new UnsafeMemorySinkFunction(type3)) - .setParallelism(dataStream.getParallelism) - } - - def getStaticPartitions: JMap[String, String] = { - staticPartitions - } + val table = new CatalogTableImpl( + new TableSchema(Array("a", "b", "c"), rowType.getFieldTypes), + util.Arrays.asList[String](partitionColumns: _*), + properties.asMap(), + "" + ) + tEnv.getCatalog(tEnv.getCurrentCatalog).get() + .createTable(new ObjectPath(tEnv.getCurrentDatabase, tableName), table, false) } } @@ -311,3 +283,93 @@ object PartitionableSinkITCase { row(4, 4L, "你好,陌生人,我是中国人,你来自哪里?") ) } + +private class TestSink( + rowType: RowTypeInfo, + supportsGrouping: Boolean, + partitionColumns: Array[String]) + extends StreamTableSink[Row] + with PartitionableTableSink { + private var staticPartitions: JMap[String, String] = _ + + override def getPartitionFieldNames: JList[String] = partitionColumns.toList + + override def setStaticPartition(partitions: JMap[String, String]): Unit = + this.staticPartitions = partitions + + override def configure(fieldNames: Array[String], + fieldTypes: Array[TypeInformation[_]]): TableSink[Row] = this + + override def configurePartitionGrouping(s: Boolean): Boolean = { + supportsGrouping + } + + override def getTableSchema: TableSchema = { + new TableSchema(Array("a", "b", "c"), type3.getFieldTypes) + } + + override def getOutputType: RowTypeInfo = type3 + + override def emitDataStream(dataStream: DataStream[Row]): Unit = { + dataStream.addSink(new UnsafeMemorySinkFunction(type3)) + .setParallelism(dataStream.getParallelism) + } + + override def consumeDataStream(dataStream: DataStream[Row]): DataStreamSink[_] = { + dataStream.addSink(new UnsafeMemorySinkFunction(type3)) + .setParallelism(dataStream.getParallelism) + } + + def getStaticPartitions: JMap[String, String] = { + staticPartitions + } +} + +class TestPartitionableSinkFactory extends TableSinkFactory[Row] with TableSourceFactory[Row] { + + override def requiredContext(): util.Map[String, String] = { + val context = new util.HashMap[String, String]() + context.put(CONNECTOR_TYPE, "TestPartitionableSink") + context + } + + override def supportedProperties(): util.List[String] = { + val supported = new util.ArrayList[String]() + supported.add("*") + supported + } + + override def createTableSink(properties: util.Map[String, String]): TableSink[Row] = { + val dp = new DescriptorProperties() + dp.putProperties(properties) + + val schema = dp.getTableSchema(SCHEMA) + val supportsGrouping = dp.getBoolean("supports-grouping") + val partitionColumns = dp.getArray("partition-column", new function.Function[String, String] { + override def apply(t: String): String = dp.getString(t) + }) + new TestSink( + schema.toRowType.asInstanceOf[RowTypeInfo], + supportsGrouping, + partitionColumns.asScala.toArray[String]) + } + + /** + * Remove it after FLINK-14387. + */ + override def createTableSource(properties: JMap[String, String]): TableSource[Row] = { + val dp = new DescriptorProperties() + dp.putProperties(properties) + + new StreamTableSource[Row] { + override def getTableSchema: TableSchema = { + dp.getTableSchema(SCHEMA) + } + + override def getDataStream(execEnv: StreamExecutionEnvironment): DataStream[Row] = { + throw new RuntimeException + } + } + } +} + diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/TableSourceITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/TableSourceITCase.scala index 6c788a59b1ab..f3b7e6dbe410 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/TableSourceITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/TableSourceITCase.scala @@ -24,13 +24,13 @@ import org.apache.flink.table.api.{DataTypes, TableSchema, Types} import org.apache.flink.table.planner.runtime.utils.BatchAbstractTestBase.TEMPORARY_FOLDER import org.apache.flink.table.planner.runtime.utils.BatchTestBase.row import org.apache.flink.table.planner.runtime.utils.{BatchTestBase, TestData} -import org.apache.flink.table.planner.utils.{TestDataTypeTableSource, TestFileInputFormatTableSource, TestFilterableTableSource, TestInputFormatTableSource, TestNestedProjectableTableSource, TestPartitionableTableSource, TestProjectableTableSource, TestTableSources} +import org.apache.flink.table.planner.utils.{TestDataTypeTableSource, TestFileInputFormatTableSource, TestFilterableTableSource, TestInputFormatTableSource, TestNestedProjectableTableSource, TestPartitionableSourceFactory, TestProjectableTableSource, TestTableSources} import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter import org.apache.flink.types.Row import org.junit.{Before, Test} -import java.io.{File, FileWriter} +import java.io.FileWriter import java.lang.{Boolean => JBool, Integer => JInt, Long => JLong} class TableSourceITCase extends BatchTestBase { @@ -154,7 +154,7 @@ class TableSourceITCase extends BatchTestBase { @Test def testTableSourceWithPartitionable(): Unit = { - tEnv.registerTableSource("PartitionableTable", new TestPartitionableTableSource(true)) + TestPartitionableSourceFactory.registerTableSource(tEnv, "PartitionableTable", true) checkResult( "SELECT * FROM PartitionableTable WHERE part2 > 1 and id > 2 AND part1 = 'A'", Seq(row(3, "John", "A", 2), row(4, "nosharp", "A", 2)) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSourceITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSourceITCase.scala index 4cf4969a3898..3e522a3fc7ca 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSourceITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSourceITCase.scala @@ -25,7 +25,7 @@ import org.apache.flink.table.api.scala._ import org.apache.flink.table.api.{DataTypes, TableSchema, Types} import org.apache.flink.table.planner.runtime.utils.BatchTestBase.row import org.apache.flink.table.planner.runtime.utils.{StreamingTestBase, TestData, TestingAppendSink} -import org.apache.flink.table.planner.utils.{TestDataTypeTableSource, TestFilterableTableSource, TestInputFormatTableSource, TestNestedProjectableTableSource, TestPartitionableTableSource, TestProjectableTableSource, TestStreamTableSource, TestTableSources} +import org.apache.flink.table.planner.utils.{TestDataTypeTableSource, TestFilterableTableSource, TestInputFormatTableSource, TestNestedProjectableTableSource, TestPartitionableSourceFactory, TestProjectableTableSource, TestStreamTableSource, TestTableSources} import org.apache.flink.types.Row import org.junit.Assert._ @@ -320,7 +320,7 @@ class TableSourceITCase extends StreamingTestBase { @Test def testTableSourceWithPartitionable(): Unit = { - tEnv.registerTableSource("PartitionableTable", new TestPartitionableTableSource(true)) + TestPartitionableSourceFactory.registerTableSource(tEnv, "PartitionableTable", true) val sqlQuery = "SELECT * FROM PartitionableTable WHERE part2 > 1 and id > 2 AND part1 = 'A'" val result = tEnv.sqlQuery(sqlQuery).toAppendStream[Row] 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 147fde5a9bf1..aa55442ddee5 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 @@ -27,9 +27,13 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.core.io.InputSplit import org.apache.flink.streaming.api.datastream.DataStream import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment -import org.apache.flink.table.api.{TableSchema, Types} +import org.apache.flink.table.api.{TableEnvironment, TableSchema, Types} +import org.apache.flink.table.catalog.{CatalogTableImpl, ObjectPath} +import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE +import org.apache.flink.table.descriptors.DescriptorProperties import org.apache.flink.table.expressions.utils.ApiExpressionUtils.unresolvedCall import org.apache.flink.table.expressions.{CallExpression, Expression, FieldReferenceExpression, ValueLiteralExpression} +import org.apache.flink.table.factories.TableSourceFactory import org.apache.flink.table.functions.BuiltInFunctionDefinitions import org.apache.flink.table.functions.BuiltInFunctionDefinitions.AND import org.apache.flink.table.planner.runtime.utils.BatchTestBase.row @@ -44,7 +48,7 @@ import org.apache.flink.types.Row import java.io.{File, FileOutputStream, OutputStreamWriter} import java.util -import java.util.{Collections, List => JList, Map => JMap} +import java.util.{Collections, function, List => JList, Map => JMap} import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ @@ -671,3 +675,76 @@ class TestFileInputFormatTableSource( override def getTableSchema: TableSchema = tableSchema } + +class TestPartitionableSourceFactory extends TableSourceFactory[Row] { + + override def requiredContext(): util.Map[String, String] = { + val context = new util.HashMap[String, String]() + context.put(CONNECTOR_TYPE, "TestPartitionableSource") + context + } + + override def supportedProperties(): util.List[String] = { + val supported = new util.ArrayList[String]() + supported.add("*") + supported + } + + override def createTableSource(properties: util.Map[String, String]): TableSource[Row] = { + val dp = new DescriptorProperties() + dp.putProperties(properties) + + val isBounded = dp.getBoolean("is-bounded") + val remainingPartitions = dp.getOptionalArray("remaining-partition", + new function.Function[String, util.Map[String, String]] { + override def apply(t: String): util.Map[String, String] = { + dp.getString(t).split(",") + .map(kv => kv.split(":")) + .map(a => (a(0), a(1))) + .toMap[String, String] + } + }) + new TestPartitionableTableSource( + isBounded, + remainingPartitions.orElse(null)) + } +} + +object TestPartitionableSourceFactory { + + def registerTableSource( + tEnv: TableEnvironment, + tableName: String, + isBounded: Boolean, + remainingPartitions: JList[JMap[String, String]] = null): Unit = { + val properties = new DescriptorProperties() + properties.putString("is-bounded", isBounded.toString) + properties.putString(CONNECTOR_TYPE, "TestPartitionableSource") + if (remainingPartitions != null) { + remainingPartitions.zipWithIndex.foreach { case (part, i) => + properties.putString( + "remaining-partition." + i, + part.map {case (k, v) => s"$k:$v"}.reduce {(kv1, kv2) => + s"$kv1,:$kv2" + } + ) + } + } + + val fieldTypes: Array[TypeInformation[_]] = Array( + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO) + val fieldNames = Array("id", "name", "part1", "part2") + + val table = new CatalogTableImpl( + new TableSchema(fieldNames, fieldTypes), + util.Arrays.asList[String]("part1", "part2"), + properties.asMap(), + "" + ) + tEnv.getCatalog(tEnv.getCurrentCatalog).get() + .createTable(new ObjectPath(tEnv.getCurrentDatabase, tableName), table, false) + } +} From ad4fd63fc917641963e507ad030ce3806baeb0f0 Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Wed, 16 Oct 2019 16:01:12 +0800 Subject: [PATCH 225/746] [FLINK-14381][table-planner-blink] Get partition keys from catalog table for PartitionableTableSink --- .../calcite/RelTimeIndicatorConverter.scala | 3 +- .../planner/delegation/PlannerBase.scala | 39 +++++++++-------- .../plan/nodes/calcite/LogicalSink.scala | 13 ++++-- .../plan/nodes/logical/FlinkLogicalSink.scala | 16 ++++--- .../physical/batch/BatchExecSinkRule.scala | 43 ++++++++----------- .../physical/stream/StreamExecSinkRule.scala | 41 ++++++++---------- .../table/planner/sinks/TableSinkUtils.scala | 42 +++++++----------- .../batch/sql/PartitionableSinkITCase.scala | 10 ----- 8 files changed, 94 insertions(+), 113 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.scala index fb582d29dbd4..9354543ddc8b 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.scala @@ -184,7 +184,8 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle { sink.getTraitSet, newInput, sink.sink, - sink.sinkName) + sink.sinkName, + sink.catalogTable) case _ => throw new TableException(s"Unsupported logical operator: ${other.getClass.getSimpleName}") diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala index 90cdab9e4520..ac9670e76b21 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala @@ -175,12 +175,10 @@ abstract class PlannerBase( case catalogSink: CatalogSinkModifyOperation => val input = getRelBuilder.queryOperation(modifyOperation.getChild).build() val identifier = catalogManager.qualifyIdentifier(catalogSink.getTablePath: _*) - getTableSink(identifier).map(sink => { - TableSinkUtils.validateSink(catalogSink, identifier, sink) + getTableSink(identifier).map { case (table, sink) => + TableSinkUtils.validateSink(catalogSink, identifier, sink, table.getPartitionKeys) sink match { - case partitionableSink: PartitionableTableSink - if partitionableSink.getPartitionFieldNames != null - && partitionableSink.getPartitionFieldNames.nonEmpty => + case partitionableSink: PartitionableTableSink => partitionableSink.setStaticPartition(catalogSink.getStaticPartitions) case _ => } @@ -192,8 +190,8 @@ abstract class PlannerBase( s"${classOf[OverwritableTableSink].getSimpleName} but actually got " + sink.getClass.getName) } - LogicalSink.create(input, sink, catalogSink.getTablePath.mkString(".")) - }) match { + LogicalSink.create(input, sink, catalogSink.getTablePath.mkString("."), table) + } match { case Some(sinkRel) => sinkRel case None => throw new TableException(s"Sink ${catalogSink.getTablePath} does not exists") } @@ -254,28 +252,31 @@ abstract class PlannerBase( */ protected def translateToPlan(execNodes: util.List[ExecNode[_, _]]): util.List[Transformation[_]] - private def getTableSink(objectIdentifier: ObjectIdentifier): Option[TableSink[_]] = { - JavaScalaConversionUtil.toScala(catalogManager.getTable(objectIdentifier)) match { + private def getTableSink( + tableIdentifier: ObjectIdentifier): Option[(CatalogTable, TableSink[_])] = { + JavaScalaConversionUtil.toScala(catalogManager.getTable(tableIdentifier)) match { case Some(s) if s.isInstanceOf[ConnectorCatalogTable[_, _]] => - JavaScalaConversionUtil - .toScala(s.asInstanceOf[ConnectorCatalogTable[_, _]].getTableSink) + val table = s.asInstanceOf[ConnectorCatalogTable[_, _]] + JavaScalaConversionUtil.toScala(table.getTableSink) match { + case Some(sink) => Some(table, sink) + case None => None + } case Some(s) if s.isInstanceOf[CatalogTable] => - - val catalog = catalogManager.getCatalog(objectIdentifier.getCatalogName) - val catalogTable = s.asInstanceOf[CatalogTable] + val catalog = catalogManager.getCatalog(tableIdentifier.getCatalogName) + val table = s.asInstanceOf[CatalogTable] if (catalog.isPresent && catalog.get().getTableFactory.isPresent) { - val objectPath = objectIdentifier.toObjectPath + val objectPath = tableIdentifier.toObjectPath val sink = TableFactoryUtil.createTableSinkForCatalogTable( catalog.get(), - catalogTable, + table, objectPath) if (sink.isPresent) { - return Option(sink.get()) + return Option(table, sink.get()) } } - val sinkProperties = catalogTable.toProperties - Option(TableFactoryService.find(classOf[TableSinkFactory[_]], sinkProperties) + val sinkProperties = table.toProperties + Option(table, TableFactoryService.find(classOf[TableSinkFactory[_]], sinkProperties) .createTableSink(sinkProperties)) case _ => None diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalSink.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalSink.scala index e94399c40319..cbd64ba991eb 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalSink.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalSink.scala @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.calcite +import org.apache.flink.table.catalog.CatalogTable import org.apache.flink.table.sinks.TableSink import org.apache.calcite.plan.{Convention, RelOptCluster, RelTraitSet} @@ -37,11 +38,13 @@ final class LogicalSink( traitSet: RelTraitSet, input: RelNode, sink: TableSink[_], - sinkName: String) + sinkName: String, + val catalogTable: CatalogTable) extends Sink(cluster, traitSet, input, sink, sinkName) { override def copy(traitSet: RelTraitSet, inputs: util.List[RelNode]): RelNode = { - new LogicalSink(cluster, traitSet, inputs.head, sink, sinkName) + new LogicalSink( + cluster, traitSet, inputs.head, sink, sinkName, catalogTable) } } @@ -50,8 +53,10 @@ object LogicalSink { def create(input: RelNode, sink: TableSink[_], - sinkName: String): LogicalSink = { + sinkName: String, + catalogTable: CatalogTable = null): LogicalSink = { val traits = input.getCluster.traitSetOf(Convention.NONE) - new LogicalSink(input.getCluster, traits, input, sink, sinkName) + new LogicalSink( + input.getCluster, traits, input, sink, sinkName, catalogTable) } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalSink.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalSink.scala index 6ebb58091bbb..a1c96b7673c1 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalSink.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalSink.scala @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.logical +import org.apache.flink.table.catalog.CatalogTable import org.apache.flink.table.planner.plan.nodes.FlinkConventions import org.apache.flink.table.planner.plan.nodes.calcite.{LogicalSink, Sink} import org.apache.flink.table.sinks.TableSink @@ -39,12 +40,14 @@ class FlinkLogicalSink( traitSet: RelTraitSet, input: RelNode, sink: TableSink[_], - sinkName: String) + sinkName: String, + val catalogTable: CatalogTable) extends Sink(cluster, traitSet, input, sink, sinkName) with FlinkLogicalRel { override def copy(traitSet: RelTraitSet, inputs: util.List[RelNode]): RelNode = { - new FlinkLogicalSink(cluster, traitSet, inputs.head, sink, sinkName) + new FlinkLogicalSink( + cluster, traitSet, inputs.head, sink, sinkName, catalogTable) } } @@ -62,7 +65,8 @@ private class FlinkLogicalSinkConverter FlinkLogicalSink.create( newInput, sink.sink, - sink.sinkName) + sink.sinkName, + sink.catalogTable) } } @@ -72,9 +76,11 @@ object FlinkLogicalSink { def create( input: RelNode, sink: TableSink[_], - sinkName: String): FlinkLogicalSink = { + sinkName: String, + catalogTable: CatalogTable = null): FlinkLogicalSink = { val cluster = input.getCluster val traitSet = cluster.traitSetOf(FlinkConventions.LOGICAL).simplify() - new FlinkLogicalSink(cluster, traitSet, input, sink, sinkName) + new FlinkLogicalSink( + cluster, traitSet, input, sink, sinkName, catalogTable) } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSinkRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSinkRule.scala index d78e8755a547..beb52a6e60ab 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSinkRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSinkRule.scala @@ -31,8 +31,6 @@ import org.apache.calcite.rel.convert.ConverterRule import org.apache.calcite.rel.{RelCollations, RelNode} import scala.collection.JavaConversions._ -import org.apache.calcite.rel.RelNode -import org.apache.calcite.rel.convert.ConverterRule class BatchExecSinkRule extends ConverterRule( classOf[FlinkLogicalSink], @@ -44,32 +42,27 @@ class BatchExecSinkRule extends ConverterRule( val sinkNode = rel.asInstanceOf[FlinkLogicalSink] val newTrait = rel.getTraitSet.replace(FlinkConventions.BATCH_PHYSICAL) var requiredTraitSet = sinkNode.getInput.getTraitSet.replace(FlinkConventions.BATCH_PHYSICAL) - sinkNode.sink match { - case partitionSink: PartitionableTableSink - if partitionSink.getPartitionFieldNames != null && - partitionSink.getPartitionFieldNames.nonEmpty => - val partitionFields = partitionSink.getPartitionFieldNames - val partitionIndices = partitionFields - .map(partitionSink.getTableSchema.getFieldNames.indexOf(_)) - // validate - partitionIndices.foreach { idx => - if (idx < 0) { - throw new TableException(s"Partitionable sink ${sinkNode.sinkName} field " + - s"${partitionFields.get(idx)} must be in the schema.") - } - } + if (sinkNode.catalogTable != null && sinkNode.catalogTable.isPartitioned) { + sinkNode.sink match { + case partitionSink: PartitionableTableSink => + val partKeys = sinkNode.catalogTable.getPartitionKeys + val partitionIndices = + partKeys.map(partitionSink.getTableSchema.getFieldNames.indexOf(_)) - requiredTraitSet = requiredTraitSet.plus( - FlinkRelDistribution.hash(partitionIndices - .map(Integer.valueOf), requireStrict = false)) + requiredTraitSet = requiredTraitSet.plus( + FlinkRelDistribution.hash(partitionIndices + .map(Integer.valueOf), requireStrict = false)) - if (partitionSink.configurePartitionGrouping(true)) { - // default to asc. - val fieldCollations = partitionIndices.map(FlinkRelOptUtil.ofRelFieldCollation) - requiredTraitSet = requiredTraitSet.plus(RelCollations.of(fieldCollations: _*)) - } - case _ => + if (partitionSink.configurePartitionGrouping(true)) { + // default to asc. + val fieldCollations = partitionIndices.map(FlinkRelOptUtil.ofRelFieldCollation) + requiredTraitSet = requiredTraitSet.plus(RelCollations.of(fieldCollations: _*)) + } + case _ => throw new TableException("We need PartitionableTableSink to write data to" + + s" partitioned table: ${sinkNode.sinkName}") + } } + val newInput = RelOptRule.convert(sinkNode.getInput, requiredTraitSet) new BatchExecSink( diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecSinkRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecSinkRule.scala index dbeb936ddb98..7280e4822375 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecSinkRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecSinkRule.scala @@ -42,32 +42,27 @@ class StreamExecSinkRule extends ConverterRule( val sinkNode = rel.asInstanceOf[FlinkLogicalSink] val newTrait = rel.getTraitSet.replace(FlinkConventions.STREAM_PHYSICAL) var requiredTraitSet = sinkNode.getInput.getTraitSet.replace(FlinkConventions.STREAM_PHYSICAL) - sinkNode.sink match { - case partitionSink: PartitionableTableSink - if partitionSink.getPartitionFieldNames != null && - partitionSink.getPartitionFieldNames.nonEmpty => - val partitionFields = partitionSink.getPartitionFieldNames - val partitionIndices = partitionFields - .map(partitionSink.getTableSchema.getFieldNames.indexOf(_)) - // validate - partitionIndices.foreach { idx => - if (idx < 0) { - throw new TableException(s"Partitionable sink ${sinkNode.sinkName} field " + - s"${partitionFields.get(idx)} must be in the schema.") - } - } + if (sinkNode.catalogTable != null && sinkNode.catalogTable.isPartitioned) { + sinkNode.sink match { + case partitionSink: PartitionableTableSink => + val partKeys = sinkNode.catalogTable.getPartitionKeys + val partitionIndices = partKeys + .map(partitionSink.getTableSchema.getFieldNames.indexOf(_)) - if (partitionSink.configurePartitionGrouping(false)) { - throw new TableException("Partition grouping in stream mode is not supported yet!") - } + if (partitionSink.configurePartitionGrouping(false)) { + throw new TableException("Partition grouping in stream mode is not supported yet!") + } - if (!partitionSink.isInstanceOf[DataStreamTableSink[_]]) { - requiredTraitSet = requiredTraitSet.plus( - FlinkRelDistribution.hash(partitionIndices - .map(Integer.valueOf), requireStrict = false)) - } - case _ => + if (!partitionSink.isInstanceOf[DataStreamTableSink[_]]) { + requiredTraitSet = requiredTraitSet.plus( + FlinkRelDistribution.hash(partitionIndices + .map(Integer.valueOf), requireStrict = false)) + } + case _ => throw new TableException("We need PartitionableTableSink to write data to" + + s" partitioned table: ${sinkNode.sinkName}") + } } + val newInput = RelOptRule.convert(sinkNode.getInput, requiredTraitSet) new StreamExecSink( diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/TableSinkUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/TableSinkUtils.scala index 8cce143b5c04..07e59ae446e8 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/TableSinkUtils.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/TableSinkUtils.scala @@ -38,11 +38,13 @@ object TableSinkUtils { * @param sinkIdentifier Tha path of the sink. It is needed just for logging. It does not * participate in the validation. * @param sink The sink that we want to write to. + * @param partitionKeys The partition keys of this table. */ def validateSink( sinkOperation: CatalogSinkModifyOperation, sinkIdentifier: ObjectIdentifier, - sink: TableSink[_]): Unit = { + sink: TableSink[_], + partitionKeys: Seq[String]): Unit = { val query = sinkOperation.getChild // validate schema of source table and table sink val srcFieldTypes = query.getTableSchema.getFieldDataTypes @@ -73,33 +75,21 @@ object TableSinkUtils { } // check partitions are valid - val staticPartitions = sinkOperation.getStaticPartitions - if (staticPartitions != null && !staticPartitions.isEmpty) { - val invalidMsg = "Can't insert static partitions into a non-partitioned table sink. " + - "A partitioned sink should implement 'PartitionableTableSink' and return partition " + - "field names via 'getPartitionFieldNames()' method." + if (partitionKeys.nonEmpty) { sink match { - case pts: PartitionableTableSink => - val partitionFields = pts.getPartitionFieldNames - if (partitionFields == null || partitionFields.isEmpty) { - throw new ValidationException(invalidMsg) - } - staticPartitions.map(_._1) foreach { p => - if (!partitionFields.contains(p)) { - throw new ValidationException(s"Static partition column $p " + - s"should be in the partition fields list $partitionFields.") - } - } - staticPartitions.map(_._1).zip(partitionFields).foreach { - case (p1, p2) => - if (p1 != p2) { - throw new ValidationException(s"Static partition column $p1 " + - s"should appear before dynamic partition $p2.") - } - } - case _ => - throw new ValidationException(invalidMsg) + case _: PartitionableTableSink => + case _ => throw new ValidationException("We need PartitionableTableSink to write data to" + + s" partitioned table: $sinkIdentifier") + } + } + val staticPartitions = sinkOperation.getStaticPartitions + if (staticPartitions != null && !staticPartitions.isEmpty) { + staticPartitions.map(_._1) foreach { p => + if (!partitionKeys.contains(p)) { + throw new ValidationException(s"Static partition column $p should be in the partition" + + s" fields list $partitionKeys for Table($sinkIdentifier).") + } } } } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala index 9a94928088b9..1cca77f0c24a 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala @@ -164,16 +164,6 @@ class PartitionableSinkITCase extends BatchTestBase { RESULT3.toList) } - @Test - def testDynamicPartitionInFrontOfStaticPartition(): Unit = { - expectedEx.expect(classOf[ValidationException]) - expectedEx.expectMessage("Static partition column b " - + "should appear before dynamic partition a") - registerTableSink(partitionColumns = Array("a", "b")) - tEnv.sqlUpdate("insert into sinkTable partition(b=1) select a, c from sortTable") - tEnv.execute("testJob") - } - @Test def testStaticPartitionNotInPartitionFields(): Unit = { expectedEx.expect(classOf[ValidationException]) From 8e0d8ace3c191dbbc93e15c13e7dafa258b0a01d Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Wed, 16 Oct 2019 16:31:04 +0800 Subject: [PATCH 226/746] [FLINK-14381][table-planner-blink] Get partition keys from catalog table for PartitionableTableSource --- .../catalog/DatabaseCalciteSchema.java | 6 ++-- .../planner/plan/QueryOperationConverter.java | 3 +- .../PushFilterIntoTableSourceScanRule.scala | 5 +++- ...PushPartitionIntoTableSourceScanRule.scala | 30 ++++++++----------- .../PushProjectIntoTableSourceScanRule.scala | 3 +- .../plan/schema/TableSourceTable.scala | 17 +++++++---- 6 files changed, 37 insertions(+), 27 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/DatabaseCalciteSchema.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/DatabaseCalciteSchema.java index 1bb664438855..ade4b46d6848 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/DatabaseCalciteSchema.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/DatabaseCalciteSchema.java @@ -143,7 +143,8 @@ private Table convertConnectorTable( return new TableSourceTable<>( tableSource, isStreamingMode, - FlinkStatistic.builder().tableStats(tableStats).build()); + FlinkStatistic.builder().tableStats(tableStats).build(), + null); } else { Optional tableSinkTable = table.getTableSink() .map(tableSink -> new TableSinkTable<>( @@ -180,7 +181,8 @@ private Table convertCatalogTable(ObjectPath tablePath, CatalogTable table) { return new TableSourceTable<>( tableSource, !((StreamTableSource) tableSource).isBounded(), - FlinkStatistic.UNKNOWN() + FlinkStatistic.UNKNOWN(), + table ); } diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java index 7a4da1fe6bbe..a6720eed8079 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java @@ -354,7 +354,8 @@ public RelNode visit(TableSourceQueryOperation tableSourceOperation) { names = Collections.singletonList(refId); } - TableSourceTable tableSourceTable = new TableSourceTable<>(tableSource, !isBatch, statistic); + TableSourceTable tableSourceTable = new TableSourceTable<>( + tableSource, !isBatch, statistic, null); FlinkRelOptTable table = FlinkRelOptTable.create( relBuilder.getRelOptSchema(), tableSourceTable.getRowType(relBuilder.getTypeFactory()), 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 a70c3ca8e7b7..e63dc218b10c 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 @@ -151,7 +151,10 @@ class PushFilterIntoTableSourceScanRule extends RelOptRule( FlinkStatistic.builder().statistic(statistic).tableStats(null).build() } val newTableSourceTable = new TableSourceTable( - newTableSource, tableSourceTable.isStreamingMode, newStatistic) + newTableSource, + tableSourceTable.isStreamingMode, + newStatistic, + tableSourceTable.catalogTable) relOptTable.copy(newTableSourceTable, tableSourceTable.getRowType(typeFactory)) } 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 c752647bbf32..a39ab6e98b45 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 @@ -18,17 +18,18 @@ package org.apache.flink.table.planner.plan.rules.logical +import org.apache.flink.table.api.TableException import org.apache.flink.table.planner.calcite.{FlinkContext, FlinkTypeFactory} import org.apache.flink.table.planner.plan.schema.{FlinkRelOptTable, TableSourceTable} 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._ @@ -49,11 +50,9 @@ class PushPartitionIntoTableSourceScanRule extends RelOptRule( val scan: LogicalTableScan = call.rel(1) scan.getTable.unwrap(classOf[TableSourceTable[_]]) match { - case table: TableSourceTable[_] => - table.tableSource match { - case p: PartitionableTableSource => p.getPartitionFieldNames.nonEmpty - case _ => false - } + case table: TableSourceTable[_] => table.catalogTable != null && + table.catalogTable.isPartitioned && + table.tableSource.isInstanceOf[PartitionableTableSource] case _ => false } } @@ -62,18 +61,12 @@ class PushPartitionIntoTableSourceScanRule extends RelOptRule( val filter: Filter = call.rel(0) val scan: LogicalTableScan = call.rel(1) val table: FlinkRelOptTable = scan.getTable.asInstanceOf[FlinkRelOptTable] - pushPartitionIntoScan(call, filter, scan, table) - } - private def pushPartitionIntoScan( - call: RelOptRuleCall, - filter: Filter, - scan: LogicalTableScan, - relOptTable: FlinkRelOptTable): Unit = { + val tableSourceTable = table.unwrap(classOf[TableSourceTable[_]]) + + val partitionFieldNames = tableSourceTable.catalogTable.getPartitionKeys.toSeq.toArray[String] - val tableSourceTable = relOptTable.unwrap(classOf[TableSourceTable[_]]) val tableSource = tableSourceTable.tableSource.asInstanceOf[PartitionableTableSource] - val partitionFieldNames = tableSource.getPartitionFieldNames.toList.toArray val inputFieldType = filter.getInput.getRowType val relBuilder = call.builder() @@ -131,8 +124,11 @@ class PushPartitionIntoTableSourceScanRule extends RelOptRule( FlinkStatistic.builder().statistic(statistic).tableStats(null).build() } val newTableSourceTable = new TableSourceTable( - newTableSource, tableSourceTable.isStreamingMode, newStatistic) - val newRelOptTable = relOptTable.copy(newTableSourceTable, relOptTable.getRowType) + newTableSource, + tableSourceTable.isStreamingMode, + newStatistic, + tableSourceTable.catalogTable) + val newRelOptTable = table.copy(newTableSourceTable, table.getRowType) val newScan = new LogicalTableScan(scan.getCluster, scan.getTraitSet, newRelOptTable) // check whether framework still need to do a filter 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 e4973886ef50..99aebfed8a93 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 @@ -98,7 +98,8 @@ class PushProjectIntoTableSourceScanRule extends RelOptRule( newTableSource, tableSourceTable.isStreamingMode, tableSourceTable.statistic, - Option(usedFields)) + Option(usedFields), + tableSourceTable.catalogTable) // row type is changed after project push down val newRowType = newTableSourceTable.getRowType(scan.getCluster.getTypeFactory) val newRelOptTable = relOptTable.copy(newTableSourceTable, newRowType) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TableSourceTable.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TableSourceTable.scala index ab118238e265..9444107b1c28 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TableSourceTable.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TableSourceTable.scala @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.schema +import org.apache.flink.table.catalog.CatalogTable import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.stats.FlinkStatistic import org.apache.flink.table.planner.sources.TableSourceUtil @@ -37,11 +38,16 @@ class TableSourceTable[T]( val tableSource: TableSource[T], val isStreamingMode: Boolean, val statistic: FlinkStatistic, - val selectedFields: Option[Array[Int]]) + val selectedFields: Option[Array[Int]], + val catalogTable: CatalogTable) extends FlinkTable { - def this(tableSource: TableSource[T], isStreamingMode: Boolean, statistic: FlinkStatistic) { - this(tableSource, isStreamingMode, statistic, None) + def this( + tableSource: TableSource[T], + isStreamingMode: Boolean, + statistic: FlinkStatistic, + catalogTable: CatalogTable) { + this(tableSource, isStreamingMode, statistic, None, catalogTable) } // TODO implements this @@ -62,7 +68,7 @@ class TableSourceTable[T]( * @return Copy of this table, substituting statistic. */ override def copy(statistic: FlinkStatistic): TableSourceTable[T] = { - new TableSourceTable(tableSource, isStreamingMode, statistic) + new TableSourceTable(tableSource, isStreamingMode, statistic, catalogTable) } /** @@ -77,6 +83,7 @@ class TableSourceTable[T]( * @return new TableSourceTable */ def replaceTableSource(tableSource: TableSource[T]): TableSourceTable[T] = { - new TableSourceTable[T](tableSource, isStreamingMode, statistic) + new TableSourceTable[T]( + tableSource, isStreamingMode, statistic, catalogTable) } } From 335c4f733cd8e52e849f56b95de047c5f9aaf908 Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Wed, 16 Oct 2019 16:35:13 +0800 Subject: [PATCH 227/746] [FLINK-14381][table-planner-blink] Remove getPartitionFieldNames in PartitionableTableSource and PartitionableTableSink This closes #9909 --- .../flink/connectors/hive/HiveTableSink.java | 3 +-- .../flink/connectors/hive/HiveTableSource.java | 5 ----- .../table/sinks/PartitionableTableSink.java | 16 +--------------- .../table/sources/PartitionableTableSource.java | 14 +------------- .../batch/sql/PartitionableSinkITCase.scala | 2 -- .../table/planner/utils/testTableSources.scala | 2 -- .../batch/sql/PartitionableSinkITCase.scala | 2 -- 7 files changed, 3 insertions(+), 41 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSink.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSink.java index d23a3ed53dae..34639a71faa9 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSink.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSink.java @@ -163,8 +163,7 @@ private String toStagingDir(String finalDir, Configuration conf) throws IOExcept return res; } - @Override - public List getPartitionFieldNames() { + private List getPartitionFieldNames() { return catalogTable.getPartitionKeys(); } 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 cfa1e629fec6..c50cb792f506 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 @@ -141,11 +141,6 @@ public List> getPartitions() { return partitionList; } - @Override - public List getPartitionFieldNames() { - return catalogTable.getPartitionKeys(); - } - @Override public TableSource applyPartitionPruning(List> remainingPartitions) { if (catalogTable.getPartitionKeys() == null || catalogTable.getPartitionKeys().size() == 0) { diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sinks/PartitionableTableSink.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sinks/PartitionableTableSink.java index 5e7c26c5662a..9c804cf08fa5 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sinks/PartitionableTableSink.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sinks/PartitionableTableSink.java @@ -20,16 +20,13 @@ import org.apache.flink.annotation.Experimental; -import java.util.List; import java.util.Map; /** * An interface for partitionable {@link TableSink}. A partitionable sink can writes * query results to partitions. * - *

Partition columns are defined via {@link #getPartitionFieldNames()} and the field names - * should be sorted in a strict order. And all the partition fields should exist in the - * {@link TableSink#getTableSchema()}. + *

Partition columns are defined via catalog table. * *

For example, a partitioned table named {@code my_table} with a table schema * {@code [a INT, b VARCHAR, c DOUBLE, dt VARCHAR, country VARCHAR]} is partitioned on columns @@ -62,17 +59,6 @@ @Experimental public interface PartitionableTableSink { - /** - * Gets the partition field names of the table. The partition field names should be sorted in - * a strict order, i.e. they have the order as specified in the PARTITION statement in DDL. - * This should be an empty set if the table is not partitioned. - * - *

All the partition fields should exist in the {@link TableSink#getTableSchema()}. - * - * @return partition field names of the table, empty if the table is not partitioned. - */ - List getPartitionFieldNames(); - /** * Sets the static partition into the {@link TableSink}. The static partition may be partial * of all partition columns. See the class Javadoc for more details. diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/PartitionableTableSource.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/PartitionableTableSource.java index 591ec50a7f0d..cb155cf5ba9c 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/PartitionableTableSource.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/PartitionableTableSource.java @@ -19,7 +19,6 @@ package org.apache.flink.table.sources; import org.apache.flink.annotation.Experimental; -import org.apache.flink.table.sinks.TableSink; import java.util.List; import java.util.Map; @@ -33,7 +32,7 @@ * *

A partition is represented as a {@code Map} which maps from partition * field name to partition value. Since the map is NOT ordered, the correct order of partition - * fields should be obtained via {@link #getPartitionFieldNames()}. + * fields should be obtained via partition keys of catalog table. */ @Experimental public interface PartitionableTableSource { @@ -43,17 +42,6 @@ public interface PartitionableTableSource { */ List> getPartitions(); - /** - * Gets the partition field names of the table. The partition field names should be sorted in - * a strict order, i.e. they have the order as specified in the PARTITION statement in DDL. - * This should be an empty set if the table is not partitioned. - * - *

All the partition fields should exist in the {@link TableSink#getTableSchema()}. - * - * @return partition field names of the table, empty if the table is not partitioned. - */ - List getPartitionFieldNames(); - /** * Applies the remaining partitions to the table source. The {@code remainingPartitions} is * the remaining partitions of {@link #getPartitions()} after partition pruning applied. diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala index 1cca77f0c24a..dacc0fd7f38c 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala @@ -282,8 +282,6 @@ private class TestSink( with PartitionableTableSink { private var staticPartitions: JMap[String, String] = _ - override def getPartitionFieldNames: JList[String] = partitionColumns.toList - override def setStaticPartition(partitions: JMap[String, String]): Unit = this.staticPartitions = partitions 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 aa55442ddee5..89fb02158e6a 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 @@ -578,8 +578,6 @@ class TestPartitionableTableSource( Map("part1"->"C", "part2"->"1").asJava ).asJava - override def getPartitionFieldNames: JList[String] = List("part1", "part2").asJava - override def applyPartitionPruning( remainingPartitions: JList[JMap[String, String]]): TableSource[_] = { new TestPartitionableTableSource(isBounded, remainingPartitions) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala index 8aaf570a8c33..c7fe1be25f1e 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala @@ -128,8 +128,6 @@ class PartitionableSinkITCase { with PartitionableTableSink { private var staticPartitions: JMap[String, String] = _ - override def getPartitionFieldNames: JList[String] = partitionColumns.toList - override def setStaticPartition(partitions: JMap[String, String]): Unit = { partitions.foreach { case (part, v) => if (!partitionColumns.contains(part)) { From 7d065eb20eec83d8cf2933438230f34682c48dcf Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Fri, 25 Oct 2019 16:13:44 +0800 Subject: [PATCH 228/746] [FLINK-14363][runtime] Prevent vertex from being affected by outdated deployment This closes #9902. --- .../runtime/scheduler/DefaultScheduler.java | 1 - .../scheduler/DefaultSchedulerTest.java | 38 +++++++++++++++++++ 2 files changed, 38 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 a249b8dcf76d..b8e730a87bcf 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 @@ -423,7 +423,6 @@ private BiFunction deployOrHandleError(final Deployment } 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 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 5d6be10d0e2d..97693dd367c2 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 @@ -66,6 +66,7 @@ import org.junit.Test; import java.util.Collections; +import java.util.Iterator; import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -355,15 +356,52 @@ public void handleGlobalFailure() { assertThat(deployedExecutionVertices, contains(executionVertexId, executionVertexId)); } + @Test + public void vertexIsNotAffectedByOutdatedDeployment() { + final JobGraph jobGraph = singleJobVertexJobGraph(2); + + testExecutionSlotAllocator.disableAutoCompletePendingRequests(); + final DefaultScheduler scheduler = createSchedulerAndStartScheduling(jobGraph); + + final Iterator vertexIterator = scheduler.requestJob().getAllExecutionVertices().iterator(); + final ArchivedExecutionVertex v1 = vertexIterator.next(); + final ArchivedExecutionVertex v2 = vertexIterator.next(); + + final SchedulingExecutionVertex sv1 = scheduler.getSchedulingTopology().getVertices().iterator().next(); + + // fail v1 and let it recover to SCHEDULED + // the initial deployment of v1 will be outdated + scheduler.updateTaskExecutionState(new TaskExecutionState( + jobGraph.getJobID(), + v1.getCurrentExecutionAttempt().getAttemptId(), + ExecutionState.FAILED)); + taskRestartExecutor.triggerScheduledTasks(); + + // fail v2 to get all pending slot requests in the initial deployments to be done + // this triggers the outdated deployment of v1 + scheduler.updateTaskExecutionState(new TaskExecutionState( + jobGraph.getJobID(), + v2.getCurrentExecutionAttempt().getAttemptId(), + ExecutionState.FAILED)); + + // v1 should not be affected + assertThat(sv1.getState(), is(equalTo(ExecutionState.SCHEDULED))); + } + private void waitForTermination(final DefaultScheduler scheduler) throws Exception { scheduler.getTerminationFuture().get(TIMEOUT_MS, TimeUnit.MILLISECONDS); } private static JobGraph singleNonParallelJobVertexJobGraph() { + return singleJobVertexJobGraph(1); + } + + private static JobGraph singleJobVertexJobGraph(final int parallelism) { final JobGraph jobGraph = new JobGraph(TEST_JOB_ID, "Testjob"); jobGraph.setScheduleMode(ScheduleMode.EAGER); final JobVertex vertex = new JobVertex("source"); vertex.setInvokableClass(NoOpInvokable.class); + vertex.setParallelism(parallelism); jobGraph.addVertex(vertex); return jobGraph; } From 76b6dd7dc28f253c60a7f7bfcb7de6634e511cbe Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Fri, 25 Oct 2019 16:17:04 +0800 Subject: [PATCH 229/746] [hotfix][tests] TestingLogicalSlot should only respect the first slot release request Otherwise it may return itself to the slotOwner several times, which may lead to errors or more available slots than expected. --- .../flink/runtime/jobmaster/TestingLogicalSlot.java | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 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 ff680e36c7cb..35b81227fab7 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 @@ -56,6 +56,8 @@ public class TestingLogicalSlot implements LogicalSlot { @Nullable private final SlotSharingGroupId slotSharingGroupId; + private boolean released; + TestingLogicalSlot( TaskManagerLocation taskManagerLocation, TaskManagerGateway taskManagerGateway, @@ -111,10 +113,14 @@ public Payload getPayload() { @Override public CompletableFuture releaseSlot(@Nullable Throwable cause) { - slotOwner.returnLogicalSlot(this); + if (!released) { + released = true; + + slotOwner.returnLogicalSlot(this); - if (automaticallyCompleteReleaseFuture) { - releaseFuture.complete(null); + if (automaticallyCompleteReleaseFuture) { + releaseFuture.complete(null); + } } return releaseFuture; From 766315c6f844a576eba5d8412510e84079394183 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Thu, 24 Oct 2019 22:54:44 +0800 Subject: [PATCH 230/746] [hotfix][runtime] Remove the unnecessary slot release logic for outdated deployment For DefaultScheduler#assignResourceOrHandleError, a normally completed slot request can trigger it only if the vertex is not canceled after this scheduling, which means the deployment is not outdated. For DefaultScheduler#deployOrHandleError, the slot should always be assigned or released. --- .../flink/runtime/scheduler/DefaultScheduler.java | 10 ---------- .../flink/runtime/scheduler/DeploymentHandle.java | 14 -------------- 2 files changed, 24 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 b8e730a87bcf..d95a05a147a3 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 @@ -373,7 +373,6 @@ private BiFunction assignResourceOrHandleError(fin 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; } @@ -409,7 +408,6 @@ private BiFunction deployOrHandleError(final Deployment if (executionVertexVersioner.isModified(requiredVertexVersion)) { log.debug("Refusing to deploy execution vertex {} because this deployment was " + "superseded by another deployment", executionVertexId); - stopDeployment(deploymentHandle); return null; } @@ -422,14 +420,6 @@ private BiFunction deployOrHandleError(final Deployment }; } - private void stopDeployment(final DeploymentHandle deploymentHandle) { - // 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); 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 index 2aed7fb31200..cdf6300312ca 100644 --- 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 @@ -20,13 +20,9 @@ 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}. * @@ -70,14 +66,4 @@ public DeploymentOption 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)); - } } From 1757c86d36e1e8481eea2895d6a1b1f1e1c4f119 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 17 Oct 2019 16:30:19 +0200 Subject: [PATCH 231/746] [FLINK-14449][tests] Use dedicated deadline for each test --- .../utils/SavepointMigrationTestBase.java | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java index bb8f31d69037..5fbfcfb7564b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java @@ -20,6 +20,7 @@ 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.java.tuple.Tuple2; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.CheckpointingOptions; @@ -50,14 +51,12 @@ import java.io.File; import java.net.URI; import java.net.URL; +import java.time.Duration; import java.util.Map; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; -import scala.concurrent.duration.Deadline; -import scala.concurrent.duration.FiniteDuration; - import static junit.framework.Assert.fail; import static org.junit.Assert.assertNotEquals; @@ -79,7 +78,6 @@ public static void before() { public final MiniClusterWithClientResource miniClusterResource; private static final Logger LOG = LoggerFactory.getLogger(SavepointMigrationTestBase.class); - private static final Deadline DEADLINE = new FiniteDuration(5, TimeUnit.MINUTES).fromNow(); protected static final int DEFAULT_PARALLELISM = 4; protected static String getResourceFilename(String filename) { @@ -133,6 +131,8 @@ protected final void executeAndSavepoint( String savepointPath, Tuple2... expectedAccumulators) throws Exception { + final Deadline deadLine = Deadline.fromNow(Duration.ofMinutes(5)); + ClusterClient client = miniClusterResource.getClusterClient(); client.setDetached(true); @@ -144,7 +144,7 @@ protected final void executeAndSavepoint( LOG.info("Submitted job {} and waiting...", jobSubmissionResult.getJobID()); boolean done = false; - while (DEADLINE.hasTimeLeft()) { + while (deadLine.hasTimeLeft()) { Thread.sleep(100); Map> accumulators = client.getAccumulators(jobSubmissionResult.getJobID()); @@ -180,7 +180,7 @@ protected final void executeAndSavepoint( CompletableFuture savepointPathFuture = client.triggerSavepoint(jobSubmissionResult.getJobID(), null); - String jobmanagerSavepointPath = savepointPathFuture.get(DEADLINE.timeLeft().toMillis(), TimeUnit.MILLISECONDS); + String jobmanagerSavepointPath = savepointPathFuture.get(deadLine.timeLeft().toMillis(), TimeUnit.MILLISECONDS); File jobManagerSavepoint = new File(new URI(jobmanagerSavepointPath).getPath()); // savepoints were changed to be directories in Flink 1.3 @@ -197,6 +197,8 @@ protected final void restoreAndExecute( String savepointPath, Tuple2... expectedAccumulators) throws Exception { + final Deadline deadLine = Deadline.fromNow(Duration.ofMinutes(5)); + ClusterClient client = miniClusterResource.getClusterClient(); client.setDetached(true); @@ -208,7 +210,7 @@ protected final void restoreAndExecute( JobSubmissionResult jobSubmissionResult = client.submitJob(jobGraph, SavepointMigrationTestBase.class.getClassLoader()); boolean done = false; - while (DEADLINE.hasTimeLeft()) { + while (deadLine.hasTimeLeft()) { // try and get a job result, this will fail if the job already failed. Use this // to get out of this loop From e136bffed39762face9cf97a798264c83ecd6344 Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Fri, 25 Oct 2019 20:24:27 +0800 Subject: [PATCH 232/746] [FLINK-14497][python] Replace coder with typeserializer --- flink-python/pom.xml | 5 + .../pyflink/fn_execution/coder_impl.py | 11 +- flink-python/pyflink/fn_execution/coders.py | 29 +- .../fn_execution/tests/coders_test_common.py | 37 ++ .../python/AbstractPythonFunctionRunner.java | 40 +- .../AbstractPythonScalarFunctionRunner.java | 4 +- .../BaseRowPythonScalarFunctionRunner.java | 12 +- .../python/PythonScalarFunctionRunner.java | 12 +- .../runtime/typeutils/BeamTypeUtils.java | 408 ------------------ .../runtime/typeutils/PythonTypeUtils.java | 122 ++++++ .../typeutils/coders/BaseRowCoder.java | 146 ------- .../coders/ReusableDataInputView.java | 41 -- .../coders/ReusableDataOutputView.java | 41 -- .../runtime/typeutils/coders/RowCoder.java | 115 ----- .../serializers/python/BaseRowSerializer.java | 232 ++++++++++ ...BaseRowPythonScalarFunctionRunnerTest.java | 82 ++-- .../PythonScalarFunctionRunnerTest.java | 85 ++-- ...tilsTest.java => PythonTypeUtilsTest.java} | 49 ++- .../typeutils/coders/CoderTestBase.java | 105 ----- .../typeutils/coders/RowCoderTest.java | 51 --- .../python/BaseRowSerializerTest.java} | 47 +- 21 files changed, 593 insertions(+), 1081 deletions(-) create mode 100644 flink-python/pyflink/fn_execution/tests/coders_test_common.py delete mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/BeamTypeUtils.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java delete mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/BaseRowCoder.java delete mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/ReusableDataInputView.java delete mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/ReusableDataOutputView.java delete mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/RowCoder.java create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/BaseRowSerializer.java rename flink-python/src/test/java/org/apache/flink/table/functions/python/{BeamTypeUtilsTest.java => PythonTypeUtilsTest.java} (56%) delete mode 100644 flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/coders/CoderTestBase.java delete mode 100644 flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/coders/RowCoderTest.java rename flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/{coders/BaseRowCoderTest.java => serializers/python/BaseRowSerializerTest.java} (71%) diff --git a/flink-python/pom.xml b/flink-python/pom.xml index ed16daa09e3a..3c515efee355 100644 --- a/flink-python/pom.xml +++ b/flink-python/pom.xml @@ -147,6 +147,11 @@ under the License. test + + org.apache.flink + flink-test-utils-junit + + diff --git a/flink-python/pyflink/fn_execution/coder_impl.py b/flink-python/pyflink/fn_execution/coder_impl.py index 6e7b21c99c83..f1fe7d0789a2 100644 --- a/flink-python/pyflink/fn_execution/coder_impl.py +++ b/flink-python/pyflink/fn_execution/coder_impl.py @@ -27,7 +27,8 @@ def __init__(self, field_coders): def encode_to_stream(self, value, out_stream, nested): self.write_null_mask(value, out_stream) for i in range(len(self._field_coders)): - self._field_coders[i].encode_to_stream(value[i], out_stream, nested) + if value[i] is not None: + 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 @@ -74,3 +75,11 @@ def read_null_mask(field_count, in_stream): def __repr__(self): return 'RowCoderImpl[%s]' % ', '.join(str(c) for c in self._field_coders) + + +class BigIntCoderImpl(StreamCoderImpl): + def encode_to_stream(self, value, out_stream, nested): + out_stream.write_bigendian_int64(value) + + def decode_from_stream(self, in_stream, nested): + return in_stream.read_bigendian_int64() diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index 219aa1fdcbcc..43072e31d095 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -16,7 +16,9 @@ # limitations under the License. ################################################################################ -from apache_beam.coders import Coder, VarIntCoder +from abc import ABC + +from apache_beam.coders import Coder from apache_beam.coders.coders import FastCoder from pyflink.fn_execution import coder_impl @@ -25,7 +27,7 @@ FLINK_SCHEMA_CODER_URN = "flink:coder:schema:v1" -__all__ = ['RowCoder'] +__all__ = ['RowCoder', 'BigIntCoder'] class RowCoder(FastCoder): @@ -62,6 +64,27 @@ def __hash__(self): return hash(self._field_coders) +class DeterministicCoder(FastCoder, ABC): + """ + Base Coder for all deterministic Coders. + """ + + def is_deterministic(self): + return True + + +class BigIntCoder(DeterministicCoder): + """ + Coder for 8 bytes long. + """ + + def _create_impl(self): + return coder_impl.BigIntCoderImpl() + + def to_type_hint(self): + return int + + @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]) @@ -75,7 +98,7 @@ def from_proto(field_type): :return: :class:`Coder` """ if field_type.type_name == flink_fn_execution_pb2.Schema.TypeName.BIGINT: - return VarIntCoder() + return BigIntCoder() 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: diff --git a/flink-python/pyflink/fn_execution/tests/coders_test_common.py b/flink-python/pyflink/fn_execution/tests/coders_test_common.py new file mode 100644 index 000000000000..bddf7f0e60bf --- /dev/null +++ b/flink-python/pyflink/fn_execution/tests/coders_test_common.py @@ -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. +################################################################################ +import logging +import unittest + +from pyflink.fn_execution.coders import BigIntCoder + + +class CodersTest(unittest.TestCase): + + def check_coder(self, coder, *values): + for v in values: + self.assertEqual(v, coder.decode(coder.encode(v))) + + def test_bigint_coder(self): + coder = BigIntCoder() + self.check_coder(coder, 1, 100, -100, -1000) + + +if __name__ == '__main__': + logging.getLogger().setLevel(logging.INFO) + unittest.main() 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 53e6ef94fb16..9cd343dfa739 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,9 +20,12 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.core.memory.ByteArrayInputStreamWithPos; import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.apache.flink.table.functions.python.PythonEnv; import org.apache.flink.util.Preconditions; import org.apache.flink.util.StringUtils; @@ -39,7 +42,6 @@ import org.apache.beam.runners.fnexecution.control.StageBundleFactory; import org.apache.beam.runners.fnexecution.provisioning.JobInfo; import org.apache.beam.runners.fnexecution.state.StateRequestHandler; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.PortablePipelineOptions; @@ -122,25 +124,35 @@ public abstract class AbstractPythonFunctionRunner implements PythonFun private transient FnDataReceiver> mainInputReceiver; /** - * The coder for input elements. + * The TypeSerializer for input elements. */ - private transient Coder inputCoder; + private transient TypeSerializer inputTypeSerializer; /** - * The coder for execution results. + * The TypeSerializer for execution results. */ - private transient Coder outputCoder; + private transient TypeSerializer outputTypeSerializer; /** * Reusable InputStream used to holding the execution results to be deserialized. */ private transient ByteArrayInputStreamWithPos bais; + /** + * InputStream Wrapper. + */ + private transient DataInputViewStreamWrapper baisWrapper; + /** * Reusable OutputStream used to holding the serialized input elements. */ private transient ByteArrayOutputStreamWithPos baos; + /** + * OutputStream Wrapper. + */ + private transient DataOutputViewStreamWrapper baosWrapper; + public AbstractPythonFunctionRunner( String taskName, FnDataReceiver resultReceiver, @@ -157,9 +169,11 @@ public AbstractPythonFunctionRunner( @Override public void open() throws Exception { bais = new ByteArrayInputStreamWithPos(); + baisWrapper = new DataInputViewStreamWrapper(bais); baos = new ByteArrayOutputStreamWithPos(); - inputCoder = getInputCoder(); - outputCoder = getOutputCoder(); + baosWrapper = new DataOutputViewStreamWrapper(baos); + inputTypeSerializer = getInputTypeSerializer(); + outputTypeSerializer = getOutputTypeSerializer(); PortablePipelineOptions portableOptions = PipelineOptionsFactory.as(PortablePipelineOptions.class); @@ -202,7 +216,7 @@ public void startBundle() { public FnDataReceiver> create(String pCollectionId) { return input -> { bais.setBuffer(input.getValue(), 0, input.getValue().length); - resultReceiver.accept(outputCoder.decode(bais)); + resultReceiver.accept(outputTypeSerializer.deserialize(baisWrapper)); }; } }; @@ -233,7 +247,7 @@ public void finishBundle() { public void processElement(IN element) { try { baos.reset(); - inputCoder.encode(element, baos); + inputTypeSerializer.serialize(element, baosWrapper); // TODO: support to use ValueOnlyWindowedValueCoder for better performance. // Currently, FullWindowedValueCoder has to be used in Beam's portability framework. mainInputReceiver.accept(WindowedValue.valueInGlobalWindow(baos.toByteArray())); @@ -302,12 +316,12 @@ protected RunnerApi.Environment createPythonExecutionEnvironment() { public abstract ExecutableStage createExecutableStage(); /** - * Returns the coder for input elements. + * Returns the TypeSerializer for input elements. */ - public abstract Coder getInputCoder(); + public abstract TypeSerializer getInputTypeSerializer(); /** - * Returns the coder for execution results. + * Returns the TypeSerializer for execution results. */ - public abstract Coder getOutputCoder(); + public abstract TypeSerializer getOutputTypeSerializer(); } 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 69c178b0ad5d..43c0c535e7a8 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 @@ -26,7 +26,7 @@ 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.runtime.typeutils.PythonTypeUtils; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Preconditions; @@ -213,7 +213,7 @@ private RunnerApi.Coder getRowCoderProto(RowType rowType) { RunnerApi.FunctionSpec.newBuilder() .setUrn(SCHEMA_CODER_URN) .setPayload(org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString.copyFrom( - BeamTypeUtils.toProtoType(rowType).getRowSchema().toByteArray())) + PythonTypeUtils.toProtoType(rowType).getRowSchema().toByteArray())) .build()) .build(); } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/BaseRowPythonScalarFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/BaseRowPythonScalarFunctionRunner.java index 4bb5fc806c03..9c550d2c248c 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/BaseRowPythonScalarFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/BaseRowPythonScalarFunctionRunner.java @@ -24,10 +24,10 @@ 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.runtime.typeutils.BaseRowSerializer; +import org.apache.flink.table.runtime.typeutils.PythonTypeUtils; import org.apache.flink.table.types.logical.RowType; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.fn.data.FnDataReceiver; /** @@ -50,13 +50,13 @@ public BaseRowPythonScalarFunctionRunner( @Override @SuppressWarnings("unchecked") - public Coder getInputCoder() { - return (Coder) BeamTypeUtils.toBlinkCoder(getInputType()); + public BaseRowSerializer getInputTypeSerializer() { + return (BaseRowSerializer) PythonTypeUtils.toBlinkTypeSerializer(getInputType()); } @Override @SuppressWarnings("unchecked") - public Coder getOutputCoder() { - return (Coder) BeamTypeUtils.toBlinkCoder(getOutputType()); + public BaseRowSerializer getOutputTypeSerializer() { + return (BaseRowSerializer) PythonTypeUtils.toBlinkTypeSerializer(getOutputType()); } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/PythonScalarFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/PythonScalarFunctionRunner.java index 8edba3a00f23..469ee7152130 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/PythonScalarFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/PythonScalarFunctionRunner.java @@ -19,15 +19,15 @@ package org.apache.flink.table.runtime.runners.python; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.java.typeutils.runtime.RowSerializer; 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.runtime.typeutils.PythonTypeUtils; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.types.Row; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.fn.data.FnDataReceiver; /** @@ -50,13 +50,13 @@ public PythonScalarFunctionRunner( @Override @SuppressWarnings("unchecked") - public Coder getInputCoder() { - return (Coder) BeamTypeUtils.toCoder(getInputType()); + public RowSerializer getInputTypeSerializer() { + return (RowSerializer) PythonTypeUtils.toFlinkTypeSerializer(getInputType()); } @Override @SuppressWarnings("unchecked") - public Coder getOutputCoder() { - return (Coder) BeamTypeUtils.toCoder(getOutputType()); + public RowSerializer getOutputTypeSerializer() { + return (RowSerializer) PythonTypeUtils.toFlinkTypeSerializer(getOutputType()); } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/BeamTypeUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/BeamTypeUtils.java deleted file mode 100644 index 710529cc775d..000000000000 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/BeamTypeUtils.java +++ /dev/null @@ -1,408 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.runtime.typeutils; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.fnexecution.v1.FlinkFnApi; -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; -import org.apache.flink.table.types.logical.BinaryType; -import org.apache.flink.table.types.logical.BooleanType; -import org.apache.flink.table.types.logical.CharType; -import org.apache.flink.table.types.logical.DateType; -import org.apache.flink.table.types.logical.DayTimeIntervalType; -import org.apache.flink.table.types.logical.DecimalType; -import org.apache.flink.table.types.logical.DistinctType; -import org.apache.flink.table.types.logical.DoubleType; -import org.apache.flink.table.types.logical.FloatType; -import org.apache.flink.table.types.logical.IntType; -import org.apache.flink.table.types.logical.LocalZonedTimestampType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.LogicalTypeVisitor; -import org.apache.flink.table.types.logical.MapType; -import org.apache.flink.table.types.logical.MultisetType; -import org.apache.flink.table.types.logical.NullType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.SmallIntType; -import org.apache.flink.table.types.logical.StructuredType; -import org.apache.flink.table.types.logical.SymbolType; -import org.apache.flink.table.types.logical.TimeType; -import org.apache.flink.table.types.logical.TimestampType; -import org.apache.flink.table.types.logical.TinyIntType; -import org.apache.flink.table.types.logical.VarBinaryType; -import org.apache.flink.table.types.logical.VarCharType; -import org.apache.flink.table.types.logical.YearMonthIntervalType; -import org.apache.flink.table.types.logical.ZonedTimestampType; - -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.VarLongCoder; - -/** - * Utilities for converting Flink data types to Beam data types. - */ -@Internal -public final class BeamTypeUtils { - - private static final String EMPTY_STRING = ""; - - public static Coder toCoder(LogicalType logicalType) { - return logicalType.accept(new LogicalTypeToCoderConverter()); - } - - public static Coder toBlinkCoder(LogicalType logicalType) { - return logicalType.accept(new LogicalTypeToBlinkCoderConverter()); - } - - public static FlinkFnApi.Schema.FieldType toProtoType(LogicalType logicalType) { - return logicalType.accept(new LogicalTypeToProtoTypeConverter()); - } - - private static class LogicalTypeToCoderConverter implements LogicalTypeVisitor { - - @Override - public Coder visit(CharType charType) { - return null; - } - - @Override - public Coder visit(VarCharType varCharType) { - return null; - } - - @Override - public Coder visit(BooleanType booleanType) { - return null; - } - - @Override - public Coder visit(BinaryType binaryType) { - return null; - } - - @Override - public Coder visit(VarBinaryType varBinaryType) { - return null; - } - - @Override - public Coder visit(DecimalType decimalType) { - return null; - } - - @Override - public Coder visit(TinyIntType tinyIntType) { - return null; - } - - @Override - public Coder visit(SmallIntType smallIntType) { - return null; - } - - @Override - public Coder visit(IntType intType) { - return null; - } - - @Override - public Coder visit(BigIntType bigIntType) { - return VarLongCoder.of(); - } - - @Override - public Coder visit(FloatType floatType) { - return null; - } - - @Override - public Coder visit(DoubleType doubleType) { - return null; - } - - @Override - public Coder visit(DateType dateType) { - return null; - } - - @Override - public Coder visit(TimeType timeType) { - return null; - } - - @Override - public Coder visit(TimestampType timestampType) { - return null; - } - - @Override - public Coder visit(ZonedTimestampType zonedTimestampType) { - return null; - } - - @Override - public Coder visit(LocalZonedTimestampType localZonedTimestampType) { - return null; - } - - @Override - public Coder visit(YearMonthIntervalType yearMonthIntervalType) { - return null; - } - - @Override - public Coder visit(DayTimeIntervalType dayTimeIntervalType) { - return null; - } - - @Override - public Coder visit(ArrayType arrayType) { - return null; - } - - @Override - public Coder visit(MultisetType multisetType) { - return null; - } - - @Override - public Coder visit(MapType mapType) { - return null; - } - - @Override - public Coder visit(RowType rowType) { - final Coder[] fieldCoders = rowType.getFields() - .stream() - .map(f -> f.getType().accept(this)) - .toArray(Coder[]::new); - return new RowCoder(fieldCoders); - } - - @Override - public Coder visit(DistinctType distinctType) { - return null; - } - - @Override - public Coder visit(StructuredType structuredType) { - return null; - } - - @Override - public Coder visit(NullType nullType) { - return null; - } - - @Override - public Coder visit(AnyType anyType) { - return null; - } - - @Override - public Coder visit(SymbolType symbolType) { - return null; - } - - @Override - public Coder visit(LogicalType other) { - return null; - } - } - - private static class LogicalTypeToBlinkCoderConverter extends LogicalTypeToCoderConverter { - - @Override - public Coder visit(RowType rowType) { - final Coder[] fieldCoders = rowType.getFields() - .stream() - .map(f -> f.getType().accept(this)) - .toArray(Coder[]::new); - return new BaseRowCoder(fieldCoders, rowType.getChildren().toArray(new LogicalType[0])); - } - } - - private static class LogicalTypeToProtoTypeConverter implements LogicalTypeVisitor { - - @Override - public FlinkFnApi.Schema.FieldType visit(CharType charType) { - return null; - } - - @Override - public FlinkFnApi.Schema.FieldType visit(VarCharType varCharType) { - return null; - } - - @Override - public FlinkFnApi.Schema.FieldType visit(BooleanType booleanType) { - return null; - } - - @Override - public FlinkFnApi.Schema.FieldType visit(BinaryType binaryType) { - return null; - } - - @Override - public FlinkFnApi.Schema.FieldType visit(VarBinaryType varBinaryType) { - return null; - } - - @Override - public FlinkFnApi.Schema.FieldType visit(DecimalType decimalType) { - return null; - } - - @Override - public FlinkFnApi.Schema.FieldType visit(TinyIntType tinyIntType) { - return null; - } - - @Override - public FlinkFnApi.Schema.FieldType visit(SmallIntType smallIntType) { - return null; - } - - @Override - public FlinkFnApi.Schema.FieldType visit(IntType intType) { - return null; - } - - @Override - public FlinkFnApi.Schema.FieldType visit(BigIntType bigIntType) { - return FlinkFnApi.Schema.FieldType.newBuilder() - .setTypeName(FlinkFnApi.Schema.TypeName.BIGINT) - .setNullable(bigIntType.isNullable()) - .build(); - } - - @Override - public FlinkFnApi.Schema.FieldType visit(FloatType floatType) { - return null; - } - - @Override - public FlinkFnApi.Schema.FieldType visit(DoubleType doubleType) { - return null; - } - - @Override - public FlinkFnApi.Schema.FieldType visit(DateType dateType) { - return null; - } - - @Override - public FlinkFnApi.Schema.FieldType visit(TimeType timeType) { - return null; - } - - @Override - public FlinkFnApi.Schema.FieldType visit(TimestampType timestampType) { - return null; - } - - @Override - public FlinkFnApi.Schema.FieldType visit(ZonedTimestampType zonedTimestampType) { - return null; - } - - @Override - public FlinkFnApi.Schema.FieldType visit(LocalZonedTimestampType localZonedTimestampType) { - return null; - } - - @Override - public FlinkFnApi.Schema.FieldType visit(YearMonthIntervalType yearMonthIntervalType) { - return null; - } - - @Override - public FlinkFnApi.Schema.FieldType visit(DayTimeIntervalType dayTimeIntervalType) { - return null; - } - - @Override - public FlinkFnApi.Schema.FieldType visit(ArrayType arrayType) { - return null; - } - - @Override - public FlinkFnApi.Schema.FieldType visit(MultisetType multisetType) { - return null; - } - - @Override - public FlinkFnApi.Schema.FieldType visit(MapType mapType) { - return null; - } - - @Override - public FlinkFnApi.Schema.FieldType visit(RowType rowType) { - FlinkFnApi.Schema.FieldType.Builder builder = - FlinkFnApi.Schema.FieldType.newBuilder() - .setTypeName(FlinkFnApi.Schema.TypeName.ROW) - .setNullable(rowType.isNullable()); - - FlinkFnApi.Schema.Builder schemaBuilder = FlinkFnApi.Schema.newBuilder(); - for (RowType.RowField field : rowType.getFields()) { - schemaBuilder.addFields( - FlinkFnApi.Schema.Field.newBuilder() - .setName(field.getName()) - .setDescription(field.getDescription().orElse(EMPTY_STRING)) - .setType(field.getType().accept(this)) - .build()); - } - builder.setRowSchema(schemaBuilder.build()); - return builder.build(); - } - - @Override - public FlinkFnApi.Schema.FieldType visit(DistinctType distinctType) { - return null; - } - - @Override - public FlinkFnApi.Schema.FieldType visit(StructuredType structuredType) { - return null; - } - - @Override - public FlinkFnApi.Schema.FieldType visit(NullType nullType) { - return null; - } - - @Override - public FlinkFnApi.Schema.FieldType visit(AnyType anyType) { - return null; - } - - @Override - public FlinkFnApi.Schema.FieldType visit(SymbolType symbolType) { - return null; - } - - @Override - public FlinkFnApi.Schema.FieldType visit(LogicalType other) { - return null; - } - } -} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java new file mode 100644 index 000000000000..db4e2f8e9aaa --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.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.table.runtime.typeutils; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.java.typeutils.runtime.RowSerializer; +import org.apache.flink.fnexecution.v1.FlinkFnApi; +import org.apache.flink.table.runtime.typeutils.serializers.python.BaseRowSerializer; +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.utils.LogicalTypeDefaultVisitor; + +/** + * Utilities for converting Flink logical types, such as convert it to the related + * TypeSerializer or ProtoType. + */ +@Internal +public final class PythonTypeUtils { + + private static final String EMPTY_STRING = ""; + + public static TypeSerializer toFlinkTypeSerializer(LogicalType logicalType) { + return logicalType.accept(new LogicalTypeToTypeSerializerConverter()); + } + + public static TypeSerializer toBlinkTypeSerializer(LogicalType logicalType) { + return logicalType.accept(new LogicalTypeToBlinkTypeSerializerConverter()); + } + + public static FlinkFnApi.Schema.FieldType toProtoType(LogicalType logicalType) { + return logicalType.accept(new LogicalTypeToProtoTypeConverter()); + } + + private static class LogicalTypeToTypeSerializerConverter extends LogicalTypeDefaultVisitor { + @Override + public TypeSerializer visit(BigIntType bigIntType) { + return LongSerializer.INSTANCE; + } + + @Override + public TypeSerializer visit(RowType rowType) { + final TypeSerializer[] fieldTypeSerializers = rowType.getFields() + .stream() + .map(f -> f.getType().accept(this)) + .toArray(TypeSerializer[]::new); + return new RowSerializer(fieldTypeSerializers); + } + + @Override + protected TypeSerializer defaultMethod(LogicalType logicalType) { + throw new UnsupportedOperationException(String.format( + "Python UDF doesn't support logical type %s currently.", logicalType.asSummaryString())); + } + } + + private static class LogicalTypeToBlinkTypeSerializerConverter extends LogicalTypeToTypeSerializerConverter { + + @Override + public TypeSerializer visit(RowType rowType) { + final TypeSerializer[] fieldTypeSerializers = rowType.getFields() + .stream() + .map(f -> f.getType().accept(this)) + .toArray(TypeSerializer[]::new); + return new BaseRowSerializer(rowType.getChildren().toArray(new LogicalType[0]), fieldTypeSerializers); + } + } + + private static class LogicalTypeToProtoTypeConverter extends LogicalTypeDefaultVisitor { + @Override + public FlinkFnApi.Schema.FieldType visit(BigIntType bigIntType) { + return FlinkFnApi.Schema.FieldType.newBuilder() + .setTypeName(FlinkFnApi.Schema.TypeName.BIGINT) + .setNullable(bigIntType.isNullable()) + .build(); + } + + @Override + public FlinkFnApi.Schema.FieldType visit(RowType rowType) { + FlinkFnApi.Schema.FieldType.Builder builder = + FlinkFnApi.Schema.FieldType.newBuilder() + .setTypeName(FlinkFnApi.Schema.TypeName.ROW) + .setNullable(rowType.isNullable()); + + FlinkFnApi.Schema.Builder schemaBuilder = FlinkFnApi.Schema.newBuilder(); + for (RowType.RowField field : rowType.getFields()) { + schemaBuilder.addFields( + FlinkFnApi.Schema.Field.newBuilder() + .setName(field.getName()) + .setDescription(field.getDescription().orElse(EMPTY_STRING)) + .setType(field.getType().accept(this)) + .build()); + } + builder.setRowSchema(schemaBuilder.build()); + return builder.build(); + } + + @Override + protected FlinkFnApi.Schema.FieldType defaultMethod(LogicalType logicalType) { + throw new UnsupportedOperationException(String.format( + "Python UDF doesn't support logical type %s currently.", logicalType.asSummaryString())); + } + } +} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/BaseRowCoder.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/BaseRowCoder.java deleted file mode 100644 index 9834c50b24c9..000000000000 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/BaseRowCoder.java +++ /dev/null @@ -1,146 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.runtime.typeutils.coders; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.table.dataformat.BaseRow; -import org.apache.flink.table.dataformat.GenericRow; -import org.apache.flink.table.dataformat.TypeGetterSetters; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.util.Preconditions; - -import org.apache.beam.sdk.coders.Coder; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Collections; -import java.util.List; - -import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.readIntoNullMask; - -/** - * A {@link Coder} for {@link BaseRow}. It should be noted that the header will not be encoded. - */ -@Internal -public class BaseRowCoder extends Coder { - - private static final long serialVersionUID = 1L; - - private final Coder[] fieldCoders; - private final LogicalType[] fieldTypes; - - private transient ReusableDataInputView reusableInputStream; - private transient ReusableDataOutputView reusableOutputStream; - - private transient boolean[] nullMask; - - @SuppressWarnings("unchecked") - public BaseRowCoder(Coder[] fieldCoders, LogicalType[] fieldTypes) { - this.fieldCoders = (Coder[]) Preconditions.checkNotNull(fieldCoders); - this.fieldTypes = Preconditions.checkNotNull(fieldTypes); - this.reusableInputStream = new ReusableDataInputView(); - this.reusableOutputStream = new ReusableDataOutputView(); - this.nullMask = new boolean[fieldCoders.length]; - } - - public Coder[] getFieldCoders() { - return this.fieldCoders; - } - - @Override - public void encode(BaseRow row, OutputStream outStream) throws IOException { - int len = fieldCoders.length; - - if (row.getArity() != len) { - throw new RuntimeException("Row arity of input element does not match coders."); - } - - // write a null mask - reusableOutputStream.reset(outStream); - writeNullMask(len, row, reusableOutputStream); - - for (int i = 0; i < row.getArity(); i++) { - if (!row.isNullAt(i)) { - // TODO: support BaseRow natively in Python, then we can eliminate the redundant serialize/deserialize - fieldCoders[i].encode(TypeGetterSetters.get(row, i, fieldTypes[i]), outStream); - } - } - } - - @Override - public BaseRow decode(InputStream inStream) throws IOException { - int len = fieldCoders.length; - - // read null mask - reusableInputStream.resetInputStream(inStream); - readIntoNullMask(len, reusableInputStream, nullMask); - - GenericRow row = new GenericRow(fieldCoders.length); - for (int i = 0; i < row.getArity(); i++) { - if (nullMask[i]) { - row.setField(i, null); - } else { - row.setField(i, fieldCoders[i].decode(inStream)); - } - } - return row; - } - - @Override - public List> getCoderArguments() { - return Collections.emptyList(); - } - - @Override - public void verifyDeterministic() {} - - @Override - public boolean consistentWithEquals() { - return true; - } - - private static void writeNullMask(int len, BaseRow value, DataOutputView target) throws IOException { - int b = 0x00; - int bytePos = 0; - - int fieldPos = 0; - int numPos = 0; - while (fieldPos < len) { - b = 0x00; - // set bits in byte - bytePos = 0; - numPos = Math.min(8, len - fieldPos); - while (bytePos < numPos) { - b = b << 1; - // set bit if field is null - if (value.isNullAt(fieldPos + bytePos)) { - b |= 0x01; - } - bytePos += 1; - } - fieldPos += numPos; - // shift bits if last byte is not completely filled - b <<= (8 - bytePos); - // write byte - target.writeByte(b); - } - } -} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/ReusableDataInputView.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/ReusableDataInputView.java deleted file mode 100644 index d71cf3bf8fcd..000000000000 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/ReusableDataInputView.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.runtime.typeutils.coders; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataInputViewStreamWrapper; - -import java.io.InputStream; - -/** - * An implementation of {@link DataInputView} that allows the instance - * to be re-used with another underlying input stream. - */ -@Internal -public class ReusableDataInputView extends DataInputViewStreamWrapper { - - public ReusableDataInputView() { - super(null); - } - - public void resetInputStream(InputStream in) { - this.in = in; - } -} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/ReusableDataOutputView.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/ReusableDataOutputView.java deleted file mode 100644 index 612804464819..000000000000 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/ReusableDataOutputView.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.runtime.typeutils.coders; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.core.memory.DataOutputViewStreamWrapper; - -import java.io.OutputStream; - -/** - * An implementation of {@link DataOutputView} that allows the instance - * to be re-used with another underlying output stream. - */ -@Internal -public class ReusableDataOutputView extends DataOutputViewStreamWrapper { - - public ReusableDataOutputView() { - super(null); - } - - public void reset(OutputStream out) { - this.out = out; - } -} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/RowCoder.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/RowCoder.java deleted file mode 100644 index b72d5f117fce..000000000000 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/coders/RowCoder.java +++ /dev/null @@ -1,115 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.runtime.typeutils.coders; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.types.Row; -import org.apache.flink.util.Preconditions; - -import org.apache.beam.sdk.coders.Coder; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Collections; -import java.util.List; - -import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.readIntoNullMask; -import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.writeNullMask; - -/** - * A {@link Coder} for {@link Row}. - */ -@Internal -public class RowCoder extends Coder { - - private static final long serialVersionUID = 1L; - - private final Coder[] fieldCoders; - - private transient ReusableDataInputView reuseInputStream; - private transient ReusableDataOutputView reuseOutputStream; - - private transient boolean[] nullMask; - - @SuppressWarnings("unchecked") - public RowCoder(Coder[] fieldCoders) { - this.fieldCoders = (Coder[]) Preconditions.checkNotNull(fieldCoders); - this.reuseInputStream = new ReusableDataInputView(); - this.reuseOutputStream = new ReusableDataOutputView(); - this.nullMask = new boolean[fieldCoders.length]; - } - - public Coder[] getFieldCoders() { - return this.fieldCoders; - } - - @Override - public void encode(Row row, OutputStream outStream) throws IOException { - int len = fieldCoders.length; - - if (row.getArity() != len) { - throw new RuntimeException("Row arity of input element does not match coders."); - } - - // write a null mask - reuseOutputStream.reset(outStream); - writeNullMask(len, row, reuseOutputStream); - - for (int i = 0; i < row.getArity(); i++) { - Object o = row.getField(i); - if (o != null) { - fieldCoders[i].encode(o, outStream); - } - } - } - - @Override - public Row decode(InputStream inStream) throws IOException { - int len = fieldCoders.length; - - // read null mask - reuseInputStream.resetInputStream(inStream); - readIntoNullMask(len, reuseInputStream, nullMask); - - Row row = new Row(len); - for (int i = 0; i < len; i++) { - if (nullMask[i]) { - row.setField(i, null); - } - else { - row.setField(i, fieldCoders[i].decode(inStream)); - } - } - return row; - } - - @Override - public List> getCoderArguments() { - return Collections.emptyList(); - } - - @Override - public void verifyDeterministic() {} - - @Override - public boolean consistentWithEquals() { - return true; - } -} diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/BaseRowSerializer.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/BaseRowSerializer.java new file mode 100644 index 000000000000..cb422b52cc19 --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/BaseRowSerializer.java @@ -0,0 +1,232 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.typeutils.serializers.python; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerUtil; +import org.apache.flink.api.common.typeutils.NestedSerializersSnapshotDelegate; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.java.typeutils.runtime.DataInputViewStream; +import org.apache.flink.api.java.typeutils.runtime.DataOutputViewStream; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.table.dataformat.GenericRow; +import org.apache.flink.table.dataformat.TypeGetterSetters; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.util.InstantiationUtil; + +import java.io.IOException; +import java.util.Arrays; + +import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.readIntoNullMask; + +/** + * A {@link TypeSerializer} for {@link BaseRow}. It should be noted that the header will not be encoded. + * Currently Python doesn't support BaseRow natively, so we can't use BaseRowSerializer in blink directly. + */ +@Internal +public class BaseRowSerializer extends org.apache.flink.table.runtime.typeutils.BaseRowSerializer { + + private final LogicalType[] fieldTypes; + + private final TypeSerializer[] fieldSerializers; + + private transient boolean[] nullMask; + + public BaseRowSerializer(LogicalType[] types, TypeSerializer[] fieldSerializers) { + super(types, fieldSerializers); + this.fieldTypes = types; + this.fieldSerializers = fieldSerializers; + this.nullMask = new boolean[fieldTypes.length]; + } + + @Override + public void serialize(BaseRow row, DataOutputView target) throws IOException { + int len = fieldSerializers.length; + + if (row.getArity() != len) { + throw new RuntimeException("Row arity of input element does not match serializers."); + } + + // write a null mask + writeNullMask(len, row, target); + + for (int i = 0; i < row.getArity(); i++) { + if (!row.isNullAt(i)) { + // TODO: support BaseRow natively in Python, then we can eliminate the redundant serialize/deserialize + fieldSerializers[i].serialize(TypeGetterSetters.get(row, i, fieldTypes[i]), target); + } + } + } + + @Override + public BaseRow deserialize(DataInputView source) throws IOException { + int len = fieldSerializers.length; + + // read null mask + readIntoNullMask(len, source, nullMask); + + GenericRow row = new GenericRow(fieldSerializers.length); + for (int i = 0; i < row.getArity(); i++) { + if (nullMask[i]) { + row.setField(i, null); + } else { + row.setField(i, fieldSerializers[i].deserialize(source)); + } + } + return row; + } + + @Override + public BaseRow deserialize(BaseRow reuse, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + private static void writeNullMask(int len, BaseRow value, DataOutputView target) throws IOException { + int b = 0x00; + int bytePos = 0; + + int fieldPos = 0; + int numPos = 0; + while (fieldPos < len) { + b = 0x00; + // set bits in byte + bytePos = 0; + numPos = Math.min(8, len - fieldPos); + while (bytePos < numPos) { + b = b << 1; + // set bit if field is null + if (value.isNullAt(fieldPos + bytePos)) { + b |= 0x01; + } + bytePos += 1; + } + fieldPos += numPos; + // shift bits if last byte is not completely filled + b <<= (8 - bytePos); + // write byte + target.writeByte(b); + } + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new BaseRowSerializerSnapshot(fieldTypes, fieldSerializers); + } + + /** + * {@link TypeSerializerSnapshot} for {@link BaseRowSerializer}. + */ + public static final class BaseRowSerializerSnapshot implements TypeSerializerSnapshot { + private static final int CURRENT_VERSION = 3; + + private LogicalType[] previousTypes; + private NestedSerializersSnapshotDelegate nestedSerializersSnapshotDelegate; + + @SuppressWarnings("unused") + public BaseRowSerializerSnapshot() { + // this constructor is used when restoring from a checkpoint/savepoint. + } + + BaseRowSerializerSnapshot(LogicalType[] types, TypeSerializer[] serializers) { + this.previousTypes = types; + this.nestedSerializersSnapshotDelegate = new NestedSerializersSnapshotDelegate( + serializers); + } + + @Override + public int getCurrentVersion() { + return CURRENT_VERSION; + } + + @Override + public void writeSnapshot(DataOutputView out) throws IOException { + out.writeInt(previousTypes.length); + DataOutputViewStream stream = new DataOutputViewStream(out); + for (LogicalType previousType : previousTypes) { + InstantiationUtil.serializeObject(stream, previousType); + } + nestedSerializersSnapshotDelegate.writeNestedSerializerSnapshots(out); + } + + @Override + public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) + throws IOException { + int length = in.readInt(); + DataInputViewStream stream = new DataInputViewStream(in); + previousTypes = new LogicalType[length]; + for (int i = 0; i < length; i++) { + try { + previousTypes[i] = InstantiationUtil.deserializeObject( + stream, + userCodeClassLoader + ); + } catch (ClassNotFoundException e) { + throw new IOException(e); + } + } + this.nestedSerializersSnapshotDelegate = NestedSerializersSnapshotDelegate.readNestedSerializerSnapshots( + in, + userCodeClassLoader + ); + } + + @Override + public BaseRowSerializer restoreSerializer() { + return new BaseRowSerializer( + previousTypes, + nestedSerializersSnapshotDelegate.getRestoredNestedSerializers() + ); + } + + @Override + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility(TypeSerializer newSerializer) { + if (!(newSerializer instanceof BaseRowSerializer)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + + BaseRowSerializer newRowSerializer = (BaseRowSerializer) newSerializer; + if (!Arrays.equals(previousTypes, newRowSerializer.fieldTypes)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + + CompositeTypeSerializerUtil.IntermediateCompatibilityResult intermediateResult = + CompositeTypeSerializerUtil.constructIntermediateCompatibilityResult( + newRowSerializer.fieldSerializers, + nestedSerializersSnapshotDelegate.getNestedSerializerSnapshots() + ); + + if (intermediateResult.isCompatibleWithReconfiguredSerializer()) { + BaseRowSerializer reconfiguredCompositeSerializer = restoreSerializer(); + return TypeSerializerSchemaCompatibility.compatibleWithReconfiguredSerializer( + reconfiguredCompositeSerializer); + } + + return intermediateResult.getFinalResult(); + } + } +} 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 dacd5092b54f..87c5e98db520 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 @@ -18,14 +18,13 @@ package org.apache.flink.table.functions.python; +import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.table.dataformat.BaseRow; 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.runtime.typeutils.BaseRowSerializer; import org.apache.flink.table.types.logical.RowType; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.junit.Test; @@ -42,69 +41,48 @@ public class BaseRowPythonScalarFunctionRunnerTest extends AbstractPythonScalarF public void testInputOutputDataTypeConstructedProperlyForSingleUDF() { final AbstractPythonScalarFunctionRunner runner = createSingleUDFRunner(); - // check input coder - Coder inputCoder = runner.getInputCoder(); - assertTrue(inputCoder instanceof BaseRowCoder); + // check input TypeSerializer + TypeSerializer inputTypeSerializer = runner.getInputTypeSerializer(); + assertTrue(inputTypeSerializer instanceof BaseRowSerializer); - Coder[] inputFieldCoders = ((BaseRowCoder) inputCoder).getFieldCoders(); - assertEquals(1, inputFieldCoders.length); - assertTrue(inputFieldCoders[0] instanceof VarLongCoder); + assertEquals(1, ((BaseRowSerializer) inputTypeSerializer).getArity()); - // check output coder - Coder outputCoder = runner.getOutputCoder(); - assertTrue(outputCoder instanceof BaseRowCoder); - Coder[] outputFieldCoders = ((BaseRowCoder) outputCoder).getFieldCoders(); - assertEquals(1, outputFieldCoders.length); - assertTrue(outputFieldCoders[0] instanceof VarLongCoder); + // check output TypeSerializer + TypeSerializer outputTypeSerializer = runner.getOutputTypeSerializer(); + assertTrue(outputTypeSerializer instanceof BaseRowSerializer); + assertEquals(1, ((BaseRowSerializer) outputTypeSerializer).getArity()); } @Test public void testInputOutputDataTypeConstructedProperlyForMultipleUDFs() { final AbstractPythonScalarFunctionRunner runner = createMultipleUDFRunner(); - // check input coder - Coder inputCoder = runner.getInputCoder(); - assertTrue(inputCoder instanceof BaseRowCoder); - - Coder[] inputFieldCoders = ((BaseRowCoder) inputCoder).getFieldCoders(); - assertEquals(3, inputFieldCoders.length); - assertTrue(inputFieldCoders[0] instanceof VarLongCoder); - assertTrue(inputFieldCoders[1] instanceof VarLongCoder); - assertTrue(inputFieldCoders[2] instanceof VarLongCoder); - - // check output coder - Coder outputCoder = runner.getOutputCoder(); - assertTrue(outputCoder instanceof BaseRowCoder); - Coder[] outputFieldCoders = ((BaseRowCoder) outputCoder).getFieldCoders(); - assertEquals(2, outputFieldCoders.length); - assertTrue(outputFieldCoders[0] instanceof VarLongCoder); - assertTrue(outputFieldCoders[1] instanceof VarLongCoder); + // check input TypeSerializer + TypeSerializer inputTypeSerializer = runner.getInputTypeSerializer(); + assertTrue(inputTypeSerializer instanceof BaseRowSerializer); + + assertEquals(3, ((BaseRowSerializer) inputTypeSerializer).getArity()); + + // check output TypeSerializer + TypeSerializer outputTypeSerializer = runner.getOutputTypeSerializer(); + assertTrue(outputTypeSerializer instanceof BaseRowSerializer); + assertEquals(2, ((BaseRowSerializer) outputTypeSerializer).getArity()); } @Test public void testInputOutputDataTypeConstructedProperlyForChainedUDFs() { final AbstractPythonScalarFunctionRunner runner = createChainedUDFRunner(); - // check input coder - Coder inputCoder = runner.getInputCoder(); - assertTrue(inputCoder instanceof BaseRowCoder); - - Coder[] inputFieldCoders = ((BaseRowCoder) inputCoder).getFieldCoders(); - assertEquals(5, inputFieldCoders.length); - assertTrue(inputFieldCoders[0] instanceof VarLongCoder); - assertTrue(inputFieldCoders[1] instanceof VarLongCoder); - assertTrue(inputFieldCoders[2] instanceof VarLongCoder); - assertTrue(inputFieldCoders[3] instanceof VarLongCoder); - assertTrue(inputFieldCoders[4] instanceof VarLongCoder); - - // check output coder - Coder outputCoder = runner.getOutputCoder(); - assertTrue(outputCoder instanceof BaseRowCoder); - Coder[] outputFieldCoders = ((BaseRowCoder) outputCoder).getFieldCoders(); - assertEquals(3, outputFieldCoders.length); - assertTrue(outputFieldCoders[0] instanceof VarLongCoder); - assertTrue(outputFieldCoders[1] instanceof VarLongCoder); - assertTrue(outputFieldCoders[2] instanceof VarLongCoder); + // check input TypeSerializer + TypeSerializer inputTypeSerializer = runner.getInputTypeSerializer(); + assertTrue(inputTypeSerializer instanceof BaseRowSerializer); + + assertEquals(5, ((BaseRowSerializer) inputTypeSerializer).getArity()); + + // check output TypeSerializer + TypeSerializer outputTypeSerializer = runner.getOutputTypeSerializer(); + assertTrue(outputTypeSerializer instanceof BaseRowSerializer); + assertEquals(3, ((BaseRowSerializer) outputTypeSerializer).getArity()); } @Override 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 212bf2a6e8e9..870dff4320f0 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 @@ -18,10 +18,12 @@ package org.apache.flink.table.functions.python; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.typeutils.runtime.RowSerializer; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.apache.flink.fnexecution.v1.FlinkFnApi; 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; @@ -29,8 +31,6 @@ import org.apache.beam.runners.fnexecution.control.JobBundleFactory; import org.apache.beam.runners.fnexecution.control.RemoteBundle; import org.apache.beam.runners.fnexecution.control.StageBundleFactory; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; @@ -68,69 +68,48 @@ public class PythonScalarFunctionRunnerTest extends AbstractPythonScalarFunction public void testInputOutputDataTypeConstructedProperlyForSingleUDF() { final AbstractPythonScalarFunctionRunner runner = createSingleUDFRunner(); - // check input coder - Coder inputCoder = runner.getInputCoder(); - assertTrue(inputCoder instanceof RowCoder); + // check input TypeSerializer + TypeSerializer inputTypeSerializer = runner.getInputTypeSerializer(); + assertTrue(inputTypeSerializer instanceof RowSerializer); - Coder[] inputFieldCoders = ((RowCoder) inputCoder).getFieldCoders(); - assertEquals(1, inputFieldCoders.length); - assertTrue(inputFieldCoders[0] instanceof VarLongCoder); + assertEquals(1, ((RowSerializer) inputTypeSerializer).getArity()); - // check output coder - Coder outputCoder = runner.getOutputCoder(); - assertTrue(outputCoder instanceof RowCoder); - Coder[] outputFieldCoders = ((RowCoder) outputCoder).getFieldCoders(); - assertEquals(1, outputFieldCoders.length); - assertTrue(outputFieldCoders[0] instanceof VarLongCoder); + // check output TypeSerializer + TypeSerializer outputTypeSerializer = runner.getOutputTypeSerializer(); + assertTrue(outputTypeSerializer instanceof RowSerializer); + assertEquals(1, ((RowSerializer) outputTypeSerializer).getArity()); } @Test public void testInputOutputDataTypeConstructedProperlyForMultipleUDFs() { final AbstractPythonScalarFunctionRunner runner = createMultipleUDFRunner(); - // check input coder - Coder inputCoder = runner.getInputCoder(); - assertTrue(inputCoder instanceof RowCoder); - - Coder[] inputFieldCoders = ((RowCoder) inputCoder).getFieldCoders(); - assertEquals(3, inputFieldCoders.length); - assertTrue(inputFieldCoders[0] instanceof VarLongCoder); - assertTrue(inputFieldCoders[1] instanceof VarLongCoder); - assertTrue(inputFieldCoders[2] instanceof VarLongCoder); - - // check output coder - Coder outputCoder = runner.getOutputCoder(); - assertTrue(outputCoder instanceof RowCoder); - Coder[] outputFieldCoders = ((RowCoder) outputCoder).getFieldCoders(); - assertEquals(2, outputFieldCoders.length); - assertTrue(outputFieldCoders[0] instanceof VarLongCoder); - assertTrue(outputFieldCoders[1] instanceof VarLongCoder); + // check input TypeSerializer + TypeSerializer inputTypeSerializer = runner.getInputTypeSerializer(); + assertTrue(inputTypeSerializer instanceof RowSerializer); + + assertEquals(3, ((RowSerializer) inputTypeSerializer).getArity()); + + // check output TypeSerializer + TypeSerializer outputTypeSerializer = runner.getOutputTypeSerializer(); + assertTrue(outputTypeSerializer instanceof RowSerializer); + assertEquals(2, ((RowSerializer) outputTypeSerializer).getArity()); } @Test public void testInputOutputDataTypeConstructedProperlyForChainedUDFs() { final AbstractPythonScalarFunctionRunner runner = createChainedUDFRunner(); - // check input coder - Coder inputCoder = runner.getInputCoder(); - assertTrue(inputCoder instanceof RowCoder); - - Coder[] inputFieldCoders = ((RowCoder) inputCoder).getFieldCoders(); - assertEquals(5, inputFieldCoders.length); - assertTrue(inputFieldCoders[0] instanceof VarLongCoder); - assertTrue(inputFieldCoders[1] instanceof VarLongCoder); - assertTrue(inputFieldCoders[2] instanceof VarLongCoder); - assertTrue(inputFieldCoders[3] instanceof VarLongCoder); - assertTrue(inputFieldCoders[4] instanceof VarLongCoder); - - // check output coder - Coder outputCoder = runner.getOutputCoder(); - assertTrue(outputCoder instanceof RowCoder); - Coder[] outputFieldCoders = ((RowCoder) outputCoder).getFieldCoders(); - assertEquals(3, outputFieldCoders.length); - assertTrue(outputFieldCoders[0] instanceof VarLongCoder); - assertTrue(outputFieldCoders[1] instanceof VarLongCoder); - assertTrue(outputFieldCoders[2] instanceof VarLongCoder); + // check input TypeSerializer + TypeSerializer inputTypeSerializer = runner.getInputTypeSerializer(); + assertTrue(inputTypeSerializer instanceof RowSerializer); + + assertEquals(5, ((RowSerializer) inputTypeSerializer).getArity()); + + // check output TypeSerializer + TypeSerializer outputTypeSerializer = runner.getOutputTypeSerializer(); + assertTrue(outputTypeSerializer instanceof RowSerializer); + assertEquals(3, ((RowSerializer) outputTypeSerializer).getArity()); } @Test @@ -221,7 +200,7 @@ public void testPythonScalarFunctionRunner() throws Exception { // verify input element is hand over to input receiver runner.processElement(Row.of(1L)); ByteArrayOutputStream baos = new ByteArrayOutputStream(); - runner.getInputCoder().encode(Row.of(1L), baos); + runner.getInputTypeSerializer().serialize(Row.of(1L), new DataOutputViewStreamWrapper(baos)); verify(windowedValueReceiverSpy, times(1)).accept(argThat( windowedValue -> windowedValue.getWindows().equals(Collections.singletonList(GlobalWindow.INSTANCE)) && 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/PythonTypeUtilsTest.java similarity index 56% rename from flink-python/src/test/java/org/apache/flink/table/functions/python/BeamTypeUtilsTest.java rename to flink-python/src/test/java/org/apache/flink/table/functions/python/PythonTypeUtilsTest.java index 0f3f3083e214..85c43de87c6f 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/PythonTypeUtilsTest.java @@ -18,15 +18,17 @@ package org.apache.flink.table.functions.python; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.typeutils.runtime.RowSerializer; import org.apache.flink.fnexecution.v1.FlinkFnApi; -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.runtime.typeutils.BaseRowSerializer; +import org.apache.flink.table.runtime.typeutils.PythonTypeUtils; import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.UnresolvedUserDefinedType; +import org.apache.flink.util.ExceptionUtils; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.VarLongCoder; import org.junit.Test; import java.util.ArrayList; @@ -36,34 +38,30 @@ import static org.junit.Assert.assertTrue; /** - * Tests for {@link BeamTypeUtils}. + * Tests for {@link PythonTypeUtils}. */ -public class BeamTypeUtilsTest { +public class PythonTypeUtilsTest { @Test - public void testLogicalTypeToCoder() { + public void testLogicalTypeToFlinkTypeSerializer() { List rowFields = new ArrayList<>(); rowFields.add(new RowType.RowField("f1", new BigIntType())); RowType rowType = new RowType(rowFields); - Coder coder = BeamTypeUtils.toCoder(rowType); - assertTrue(coder instanceof RowCoder); + TypeSerializer rowSerializer = PythonTypeUtils.toFlinkTypeSerializer(rowType); + assertTrue(rowSerializer instanceof RowSerializer); - Coder[] fieldCoders = ((RowCoder) coder).getFieldCoders(); - assertEquals(1, fieldCoders.length); - assertTrue(fieldCoders[0] instanceof VarLongCoder); + assertEquals(1, ((RowSerializer) rowSerializer).getArity()); } @Test - public void testLogicalTypeToBlinkCoder() { + public void testLogicalTypeToBlinkTypeSerializer() { List rowFields = new ArrayList<>(); rowFields.add(new RowType.RowField("f1", new BigIntType())); RowType rowType = new RowType(rowFields); - Coder coder = BeamTypeUtils.toBlinkCoder(rowType); - assertTrue(coder instanceof BaseRowCoder); + TypeSerializer baseSerializer = PythonTypeUtils.toBlinkTypeSerializer(rowType); + assertTrue(baseSerializer instanceof BaseRowSerializer); - Coder[] fieldCoders = ((BaseRowCoder) coder).getFieldCoders(); - assertEquals(1, fieldCoders.length); - assertTrue(fieldCoders[0] instanceof VarLongCoder); + assertEquals(1, ((BaseRowSerializer) baseSerializer).getArity()); } @Test @@ -71,10 +69,21 @@ public void testLogicalTypeToProto() { List rowFields = new ArrayList<>(); rowFields.add(new RowType.RowField("f1", new BigIntType())); RowType rowType = new RowType(rowFields); - FlinkFnApi.Schema.FieldType protoType = BeamTypeUtils.toProtoType(rowType); + FlinkFnApi.Schema.FieldType protoType = PythonTypeUtils.toProtoType(rowType); FlinkFnApi.Schema schema = protoType.getRowSchema(); assertEquals(1, schema.getFieldsCount()); assertEquals("f1", schema.getFields(0).getName()); assertEquals(FlinkFnApi.Schema.TypeName.BIGINT, schema.getFields(0).getType().getTypeName()); } + + @Test + public void testUnsupportedTypeSerializer() { + LogicalType logicalType = new UnresolvedUserDefinedType("", "", ""); + String expectedTestException = "Python UDF doesn't support logical type ``.``.`` currently."; + try { + PythonTypeUtils.toFlinkTypeSerializer(logicalType); + } catch (Exception e) { + assertTrue(ExceptionUtils.findThrowableWithMessage(e, expectedTestException).isPresent()); + } + } } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/coders/CoderTestBase.java b/flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/coders/CoderTestBase.java deleted file mode 100644 index 308fe5dd7412..000000000000 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/coders/CoderTestBase.java +++ /dev/null @@ -1,105 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.runtime.typeutils.coders; - -import org.apache.flink.testutils.CustomEqualityMatcher; -import org.apache.flink.testutils.DeeplyEqualsChecker; - -import org.apache.beam.sdk.coders.Coder; -import org.junit.Test; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.fail; - -/** - * Abstract test base for coders. - */ -public abstract class CoderTestBase { - - private final DeeplyEqualsChecker checker; - - protected CoderTestBase() { - this.checker = new DeeplyEqualsChecker(); - } - - protected CoderTestBase(DeeplyEqualsChecker checker) { - this.checker = checker; - } - - protected abstract Coder createCoder(); - - protected abstract T[] getTestData(); - - // -------------------------------------------------------------------------------------------- - - @Test - public void testEncodeDecode() { - try { - Coder coder = getCoder(); - T[] testData = getData(); - - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - for (T value : testData) { - coder.encode(value, baos); - } - - ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray()); - int num = 0; - while (bais.available() > 0) { - T deserialized = coder.decode(bais); - - deepEquals("Decoded value if wrong.", testData[num], deserialized); - num++; - } - - assertEquals("Wrong number of elements decoded.", testData.length, num); - } - catch (Exception e) { - System.err.println(e.getMessage()); - e.printStackTrace(); - fail("Exception in test: " + e.getMessage()); - } - } - - // -------------------------------------------------------------------------------------------- - - private void deepEquals(String message, T should, T is) { - assertThat(message, is, CustomEqualityMatcher.deeplyEquals(should).withChecker(checker)); - } - - private Coder getCoder() { - Coder coder = createCoder(); - if (coder == null) { - throw new RuntimeException("Test case corrupt. Returns null as coder."); - } - return coder; - } - - private T[] getData() { - T[] data = getTestData(); - if (data == null) { - throw new RuntimeException("Test case corrupt. Returns null as test data."); - } - return data; - } -} diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/coders/RowCoderTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/coders/RowCoderTest.java deleted file mode 100644 index e8b64830b67e..000000000000 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/coders/RowCoderTest.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.runtime.typeutils.coders; - -import org.apache.flink.types.Row; - -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.VarLongCoder; - -/** - * Tests for {@link RowCoder}. - */ -public class RowCoderTest extends CoderTestBase { - - @Override - protected Coder createCoder() { - Coder[] fieldCoders = { - VarLongCoder.of(), - VarLongCoder.of()}; - return new RowCoder(fieldCoders); - } - - @Override - protected Row[] getTestData() { - Row row1 = new Row(2); - row1.setField(0, 1L); - row1.setField(1, -1L); - - Row row2 = new Row(2); - row2.setField(0, 2L); - row2.setField(1, null); - - return new Row[]{row1, row2}; - } -} diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/coders/BaseRowCoderTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/serializers/python/BaseRowSerializerTest.java similarity index 71% rename from flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/coders/BaseRowCoderTest.java rename to flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/serializers/python/BaseRowSerializerTest.java index 7b5e281a3a5b..7d894b56f86a 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/coders/BaseRowCoderTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/serializers/python/BaseRowSerializerTest.java @@ -16,9 +16,12 @@ * limitations under the License. */ -package org.apache.flink.table.runtime.typeutils.coders; +package org.apache.flink.table.runtime.typeutils.serializers.python; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; import org.apache.flink.table.dataformat.BaseRow; import org.apache.flink.table.dataformat.BinaryRow; import org.apache.flink.table.runtime.typeutils.BaseRowSerializer; @@ -27,21 +30,17 @@ import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.testutils.DeeplyEqualsChecker; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.VarLongCoder; - import java.util.Objects; /** - * Tests for {@link BaseRowCoder}. + * Test for {@link org.apache.flink.table.runtime.typeutils.serializers.python.BaseRowSerializer}. */ -public class BaseRowCoderTest extends CoderTestBase { - - public BaseRowCoderTest() { +public class BaseRowSerializerTest extends SerializerTestBase { + public BaseRowSerializerTest() { super( new DeeplyEqualsChecker() .withCustomCheck( - (o1, o2) -> o1 instanceof BaseRow && o2 instanceof BaseRow, + (o1, o2) -> o1 instanceof BaseRow && o2 instanceof BaseRow, (o1, o2, checker) -> { LogicalType[] fieldTypes = new LogicalType[] { new BigIntType(), @@ -58,24 +57,29 @@ public BaseRowCoderTest() { } @Override - protected Coder createCoder() { - Coder[] fieldCoders = { - VarLongCoder.of(), - VarLongCoder.of() + protected TypeSerializer createSerializer() { + TypeSerializer[] fieldTypeSerializers = { + LongSerializer.INSTANCE, + LongSerializer.INSTANCE }; LogicalType[] fieldTypes = { new BigIntType(), new BigIntType() }; - return new BaseRowCoder(fieldCoders, fieldTypes); + return new org.apache.flink.table.runtime.typeutils.serializers.python.BaseRowSerializer( + fieldTypes, + fieldTypeSerializers); } @Override - protected BaseRow[] getTestData() { - BaseRow row1 = StreamRecordUtils.baserow(null, 1L); - BinaryRow row2 = StreamRecordUtils.binaryrow(1L, null); - return new BaseRow[]{row1, row2}; + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return BaseRow.class; } private static boolean deepEqualsBaseRow( @@ -89,4 +93,11 @@ private static boolean deepEqualsBaseRow( return Objects.equals(row1, row2); } + + @Override + protected BaseRow[] getTestData() { + BaseRow row1 = StreamRecordUtils.baserow(null, 1L); + BinaryRow row2 = StreamRecordUtils.binaryrow(1L, null); + return new BaseRow[]{row1, row2}; + } } From 7e68b187deebfa73a7ceec5499f2e66d632080ad Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Fri, 25 Oct 2019 20:24:29 +0800 Subject: [PATCH 233/746] [FLINK-14497][python] Support TinyIntType for Python UDF --- .../pyflink/fn_execution/coder_impl.py | 9 ++++ flink-python/pyflink/fn_execution/coders.py | 29 +++++++++++-- .../fn_execution/tests/coders_test_common.py | 6 ++- flink-python/pyflink/table/tests/test_udf.py | 43 +++++++++++++++++++ .../runtime/typeutils/PythonTypeUtils.java | 15 +++++++ 5 files changed, 97 insertions(+), 5 deletions(-) diff --git a/flink-python/pyflink/fn_execution/coder_impl.py b/flink-python/pyflink/fn_execution/coder_impl.py index f1fe7d0789a2..f00344436298 100644 --- a/flink-python/pyflink/fn_execution/coder_impl.py +++ b/flink-python/pyflink/fn_execution/coder_impl.py @@ -83,3 +83,12 @@ def encode_to_stream(self, value, out_stream, nested): def decode_from_stream(self, in_stream, nested): return in_stream.read_bigendian_int64() + + +class TinyIntCoderImpl(StreamCoderImpl): + + def encode_to_stream(self, value, out_stream, nested): + out_stream.write_byte(value) + + def decode_from_stream(self, in_stream, nested): + return int(in_stream.read_byte()) diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index 43072e31d095..4579ac584976 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -27,7 +27,7 @@ FLINK_SCHEMA_CODER_URN = "flink:coder:schema:v1" -__all__ = ['RowCoder', 'BigIntCoder'] +__all__ = ['RowCoder', 'BigIntCoder', 'TinyIntCoder'] class RowCoder(FastCoder): @@ -85,11 +85,30 @@ def to_type_hint(self): return int +class TinyIntCoder(DeterministicCoder): + """ + Coder for Byte. + """ + + def _create_impl(self): + return coder_impl.TinyIntCoderImpl() + + def to_type_hint(self): + return int + + @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]) +type_name = flink_fn_execution_pb2.Schema.TypeName +_type_name_mappings = { + type_name.TINYINT: TinyIntCoder(), + type_name.BIGINT: BigIntCoder(), +} + + def from_proto(field_type): """ Creates the corresponding :class:`Coder` given the protocol representation of the field type. @@ -97,9 +116,11 @@ def from_proto(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 BigIntCoder() - elif field_type.type_name == flink_fn_execution_pb2.Schema.TypeName.ROW: + field_type_name = field_type.type_name + coder = _type_name_mappings.get(field_type_name) + if coder is not None: + return coder + if field_type_name == type_name.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/tests/coders_test_common.py b/flink-python/pyflink/fn_execution/tests/coders_test_common.py index bddf7f0e60bf..1a4c66aa7af4 100644 --- a/flink-python/pyflink/fn_execution/tests/coders_test_common.py +++ b/flink-python/pyflink/fn_execution/tests/coders_test_common.py @@ -18,7 +18,7 @@ import logging import unittest -from pyflink.fn_execution.coders import BigIntCoder +from pyflink.fn_execution.coders import BigIntCoder, TinyIntCoder class CodersTest(unittest.TestCase): @@ -31,6 +31,10 @@ def test_bigint_coder(self): coder = BigIntCoder() self.check_coder(coder, 1, 100, -100, -1000) + def test_tinyint_coder(self): + coder = TinyIntCoder() + self.check_coder(coder, 1, 10, 127) + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) diff --git a/flink-python/pyflink/table/tests/test_udf.py b/flink-python/pyflink/table/tests/test_udf.py index 12f4e8c96642..6d77d66ef3fc 100644 --- a/flink-python/pyflink/table/tests/test_udf.py +++ b/flink-python/pyflink/table/tests/test_udf.py @@ -329,6 +329,49 @@ def test_udf_without_arguments(self): actual = source_sink_utils.results() self.assert_equals(actual, ["1,2", "1,2", "1,2"]) + def test_all_data_types(self): + def tinyint_func(tinyint_param): + assert isinstance(tinyint_param, int), 'tinyint_param of wrong type %s !' \ + % type(tinyint_param) + return tinyint_param + + def bigint_func(bigint_param): + assert isinstance(bigint_param, int), 'bigint_param of wrong type %s !' \ + % type(bigint_param) + return bigint_param + + def bigint_func_none(bigint_param): + assert bigint_param is None, 'bigint_param %s should be None!' % bigint_param + return bigint_param + + self.t_env.register_function( + "tinyint_func", udf(tinyint_func, [DataTypes.TINYINT()], DataTypes.TINYINT())) + + self.t_env.register_function( + "bigint_func", udf(bigint_func, [DataTypes.BIGINT()], DataTypes.BIGINT())) + + self.t_env.register_function( + "bigint_func_none", udf(bigint_func_none, [DataTypes.BIGINT()], DataTypes.BIGINT())) + + table_sink = source_sink_utils.TestAppendSink( + ['a', 'b', 'c'], + [DataTypes.BIGINT(), DataTypes.BIGINT(), DataTypes.TINYINT()]) + self.t_env.register_table_sink("Results", table_sink) + + t = self.t_env.from_elements( + [(1, None, 1)], + DataTypes.ROW( + [DataTypes.FIELD("a", DataTypes.BIGINT()), + DataTypes.FIELD("b", DataTypes.BIGINT()), + DataTypes.FIELD("c", DataTypes.TINYINT())])) + + t.select("bigint_func(a), bigint_func_none(b)," + "tinyint_func(c)") \ + .insert_into("Results") + self.t_env.execute("test") + actual = source_sink_utils.results() + self.assert_equals(actual, ["1,null,1"]) + class PyFlinkStreamUserDefinedFunctionTests(UserDefinedFunctionTests, PyFlinkStreamTableTestCase): diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java index db4e2f8e9aaa..36cc57db7791 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.ByteSerializer; import org.apache.flink.api.common.typeutils.base.LongSerializer; import org.apache.flink.api.java.typeutils.runtime.RowSerializer; import org.apache.flink.fnexecution.v1.FlinkFnApi; @@ -27,6 +28,7 @@ import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TinyIntType; import org.apache.flink.table.types.logical.utils.LogicalTypeDefaultVisitor; /** @@ -51,6 +53,11 @@ public static FlinkFnApi.Schema.FieldType toProtoType(LogicalType logicalType) { } private static class LogicalTypeToTypeSerializerConverter extends LogicalTypeDefaultVisitor { + @Override + public TypeSerializer visit(TinyIntType tinyIntType) { + return ByteSerializer.INSTANCE; + } + @Override public TypeSerializer visit(BigIntType bigIntType) { return LongSerializer.INSTANCE; @@ -85,6 +92,14 @@ public TypeSerializer visit(RowType rowType) { } private static class LogicalTypeToProtoTypeConverter extends LogicalTypeDefaultVisitor { + @Override + public FlinkFnApi.Schema.FieldType visit(TinyIntType tinyIntType) { + return FlinkFnApi.Schema.FieldType.newBuilder() + .setTypeName(FlinkFnApi.Schema.TypeName.TINYINT) + .setNullable(tinyIntType.isNullable()) + .build(); + } + @Override public FlinkFnApi.Schema.FieldType visit(BigIntType bigIntType) { return FlinkFnApi.Schema.FieldType.newBuilder() From 3f638db858aa664adf3f4ec727914ccc63e958bd Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Fri, 25 Oct 2019 20:24:30 +0800 Subject: [PATCH 234/746] [FLINK-14497][python] Support BooleanType for Python UDF --- .../pyflink/fn_execution/coder_impl.py | 9 ++++++++ flink-python/pyflink/fn_execution/coders.py | 15 ++++++++++++- .../fn_execution/tests/coders_test_common.py | 8 ++++++- flink-python/pyflink/table/tests/test_udf.py | 22 ++++++++++++++----- .../runtime/typeutils/PythonTypeUtils.java | 15 +++++++++++++ 5 files changed, 61 insertions(+), 8 deletions(-) diff --git a/flink-python/pyflink/fn_execution/coder_impl.py b/flink-python/pyflink/fn_execution/coder_impl.py index f00344436298..8e7232bbb293 100644 --- a/flink-python/pyflink/fn_execution/coder_impl.py +++ b/flink-python/pyflink/fn_execution/coder_impl.py @@ -92,3 +92,12 @@ def encode_to_stream(self, value, out_stream, nested): def decode_from_stream(self, in_stream, nested): return int(in_stream.read_byte()) + + +class BooleanCoderImpl(StreamCoderImpl): + + def encode_to_stream(self, value, out_stream, nested): + out_stream.write_byte(value) + + def decode_from_stream(self, in_stream, nested): + return not not in_stream.read_byte() diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index 4579ac584976..448db5a4b6a3 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -27,7 +27,7 @@ FLINK_SCHEMA_CODER_URN = "flink:coder:schema:v1" -__all__ = ['RowCoder', 'BigIntCoder', 'TinyIntCoder'] +__all__ = ['RowCoder', 'BigIntCoder', 'TinyIntCoder', 'BooleanCoder'] class RowCoder(FastCoder): @@ -97,6 +97,18 @@ def to_type_hint(self): return int +class BooleanCoder(DeterministicCoder): + """ + Coder for Boolean. + """ + + def _create_impl(self): + return coder_impl.BooleanCoderImpl() + + def to_type_hint(self): + return bool + + @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]) @@ -106,6 +118,7 @@ def _pickle_from_runner_api_parameter(schema_proto, unused_components, unused_co _type_name_mappings = { type_name.TINYINT: TinyIntCoder(), type_name.BIGINT: BigIntCoder(), + type_name.BOOLEAN: BooleanCoder(), } diff --git a/flink-python/pyflink/fn_execution/tests/coders_test_common.py b/flink-python/pyflink/fn_execution/tests/coders_test_common.py index 1a4c66aa7af4..e3f1eb231ac6 100644 --- a/flink-python/pyflink/fn_execution/tests/coders_test_common.py +++ b/flink-python/pyflink/fn_execution/tests/coders_test_common.py @@ -15,10 +15,12 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ + +"""Tests common to all coder implementations.""" import logging import unittest -from pyflink.fn_execution.coders import BigIntCoder, TinyIntCoder +from pyflink.fn_execution.coders import BigIntCoder, TinyIntCoder, BooleanCoder class CodersTest(unittest.TestCase): @@ -35,6 +37,10 @@ def test_tinyint_coder(self): coder = TinyIntCoder() self.check_coder(coder, 1, 10, 127) + def test_boolean_coder(self): + coder = BooleanCoder() + self.check_coder(coder, True, False) + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) diff --git a/flink-python/pyflink/table/tests/test_udf.py b/flink-python/pyflink/table/tests/test_udf.py index 6d77d66ef3fc..562b2ed03486 100644 --- a/flink-python/pyflink/table/tests/test_udf.py +++ b/flink-python/pyflink/table/tests/test_udf.py @@ -330,6 +330,11 @@ def test_udf_without_arguments(self): self.assert_equals(actual, ["1,2", "1,2", "1,2"]) def test_all_data_types(self): + def boolean_func(bool_param): + assert isinstance(bool_param, bool), 'bool_param of wrong type %s !' \ + % type(bool_param) + return bool_param + def tinyint_func(tinyint_param): assert isinstance(tinyint_param, int), 'tinyint_param of wrong type %s !' \ % type(tinyint_param) @@ -344,6 +349,9 @@ def bigint_func_none(bigint_param): assert bigint_param is None, 'bigint_param %s should be None!' % bigint_param return bigint_param + self.t_env.register_function( + "boolean_func", udf(boolean_func, [DataTypes.BOOLEAN()], DataTypes.BOOLEAN())) + self.t_env.register_function( "tinyint_func", udf(tinyint_func, [DataTypes.TINYINT()], DataTypes.TINYINT())) @@ -354,23 +362,25 @@ def bigint_func_none(bigint_param): "bigint_func_none", udf(bigint_func_none, [DataTypes.BIGINT()], DataTypes.BIGINT())) table_sink = source_sink_utils.TestAppendSink( - ['a', 'b', 'c'], - [DataTypes.BIGINT(), DataTypes.BIGINT(), DataTypes.TINYINT()]) + ['a', 'b', 'c', 'd'], + [DataTypes.BIGINT(), DataTypes.BIGINT(), DataTypes.TINYINT(), + DataTypes.BOOLEAN()]) self.t_env.register_table_sink("Results", table_sink) t = self.t_env.from_elements( - [(1, None, 1)], + [(1, None, 1, True)], DataTypes.ROW( [DataTypes.FIELD("a", DataTypes.BIGINT()), DataTypes.FIELD("b", DataTypes.BIGINT()), - DataTypes.FIELD("c", DataTypes.TINYINT())])) + DataTypes.FIELD("c", DataTypes.TINYINT()), + DataTypes.FIELD("d", DataTypes.BOOLEAN())])) t.select("bigint_func(a), bigint_func_none(b)," - "tinyint_func(c)") \ + "tinyint_func(c), boolean_func(d)") \ .insert_into("Results") self.t_env.execute("test") actual = source_sink_utils.results() - self.assert_equals(actual, ["1,null,1"]) + self.assert_equals(actual, ["1,null,1,true"]) class PyFlinkStreamUserDefinedFunctionTests(UserDefinedFunctionTests, diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java index 36cc57db7791..8590f7f808c6 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java @@ -20,12 +20,14 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.BooleanSerializer; import org.apache.flink.api.common.typeutils.base.ByteSerializer; import org.apache.flink.api.common.typeutils.base.LongSerializer; import org.apache.flink.api.java.typeutils.runtime.RowSerializer; import org.apache.flink.fnexecution.v1.FlinkFnApi; import org.apache.flink.table.runtime.typeutils.serializers.python.BaseRowSerializer; import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.TinyIntType; @@ -53,6 +55,11 @@ public static FlinkFnApi.Schema.FieldType toProtoType(LogicalType logicalType) { } private static class LogicalTypeToTypeSerializerConverter extends LogicalTypeDefaultVisitor { + @Override + public TypeSerializer visit(BooleanType booleanType) { + return BooleanSerializer.INSTANCE; + } + @Override public TypeSerializer visit(TinyIntType tinyIntType) { return ByteSerializer.INSTANCE; @@ -92,6 +99,14 @@ public TypeSerializer visit(RowType rowType) { } private static class LogicalTypeToProtoTypeConverter extends LogicalTypeDefaultVisitor { + @Override + public FlinkFnApi.Schema.FieldType visit(BooleanType booleanType) { + return FlinkFnApi.Schema.FieldType.newBuilder() + .setTypeName(FlinkFnApi.Schema.TypeName.BOOLEAN) + .setNullable(booleanType.isNullable()) + .build(); + } + @Override public FlinkFnApi.Schema.FieldType visit(TinyIntType tinyIntType) { return FlinkFnApi.Schema.FieldType.newBuilder() From 90d33d0d59245712013d69570dbed09ff2c08eb6 Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Fri, 25 Oct 2019 20:24:32 +0800 Subject: [PATCH 235/746] [FLINK-14497][python] Support SmallInt for Python UDF --- .../pyflink/fn_execution/coder_impl.py | 11 +++++++++ flink-python/pyflink/fn_execution/coders.py | 16 ++++++++++++- .../fn_execution/tests/coders_test_common.py | 6 ++++- flink-python/pyflink/table/tests/test_udf.py | 23 ++++++++++++++----- .../runtime/typeutils/PythonTypeUtils.java | 15 ++++++++++++ 5 files changed, 63 insertions(+), 8 deletions(-) diff --git a/flink-python/pyflink/fn_execution/coder_impl.py b/flink-python/pyflink/fn_execution/coder_impl.py index 8e7232bbb293..3686979c082b 100644 --- a/flink-python/pyflink/fn_execution/coder_impl.py +++ b/flink-python/pyflink/fn_execution/coder_impl.py @@ -94,6 +94,17 @@ def decode_from_stream(self, in_stream, nested): return int(in_stream.read_byte()) +class SmallIntImpl(StreamCoderImpl): + + def encode_to_stream(self, value, out_stream, nested): + import struct + out_stream.write(struct.pack('>h', value)) + + def decode_from_stream(self, in_stream, nested): + import struct + return struct.unpack('>h', in_stream.read(2))[0] + + class BooleanCoderImpl(StreamCoderImpl): def encode_to_stream(self, value, out_stream, nested): diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index 448db5a4b6a3..cc00ab115e40 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -27,7 +27,8 @@ FLINK_SCHEMA_CODER_URN = "flink:coder:schema:v1" -__all__ = ['RowCoder', 'BigIntCoder', 'TinyIntCoder', 'BooleanCoder'] +__all__ = ['RowCoder', 'BigIntCoder', 'TinyIntCoder', 'BooleanCoder', + 'SmallIntCoder'] class RowCoder(FastCoder): @@ -109,6 +110,18 @@ def to_type_hint(self): return bool +class SmallIntCoder(DeterministicCoder): + """ + Coder for Short. + """ + + def _create_impl(self): + return coder_impl.SmallIntImpl() + + def to_type_hint(self): + return int + + @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]) @@ -117,6 +130,7 @@ def _pickle_from_runner_api_parameter(schema_proto, unused_components, unused_co type_name = flink_fn_execution_pb2.Schema.TypeName _type_name_mappings = { type_name.TINYINT: TinyIntCoder(), + type_name.SMALLINT: SmallIntCoder(), type_name.BIGINT: BigIntCoder(), type_name.BOOLEAN: BooleanCoder(), } diff --git a/flink-python/pyflink/fn_execution/tests/coders_test_common.py b/flink-python/pyflink/fn_execution/tests/coders_test_common.py index e3f1eb231ac6..9d26941f6b32 100644 --- a/flink-python/pyflink/fn_execution/tests/coders_test_common.py +++ b/flink-python/pyflink/fn_execution/tests/coders_test_common.py @@ -20,7 +20,7 @@ import logging import unittest -from pyflink.fn_execution.coders import BigIntCoder, TinyIntCoder, BooleanCoder +from pyflink.fn_execution.coders import BigIntCoder, TinyIntCoder, BooleanCoder, SmallIntCoder class CodersTest(unittest.TestCase): @@ -41,6 +41,10 @@ def test_boolean_coder(self): coder = BooleanCoder() self.check_coder(coder, True, False) + def test_smallint_coder(self): + coder = SmallIntCoder() + self.check_coder(coder, 32767, -32768, 0) + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) diff --git a/flink-python/pyflink/table/tests/test_udf.py b/flink-python/pyflink/table/tests/test_udf.py index 562b2ed03486..df2d50c04030 100644 --- a/flink-python/pyflink/table/tests/test_udf.py +++ b/flink-python/pyflink/table/tests/test_udf.py @@ -340,6 +340,12 @@ def tinyint_func(tinyint_param): % type(tinyint_param) return tinyint_param + def smallint_func(smallint_param): + assert isinstance(smallint_param, int), 'smallint_param of wrong type %s !' \ + % type(smallint_param) + assert smallint_param == 32767, 'smallint_param of wrong value %s' % smallint_param + return smallint_param + def bigint_func(bigint_param): assert isinstance(bigint_param, int), 'bigint_param of wrong type %s !' \ % type(bigint_param) @@ -355,6 +361,9 @@ def bigint_func_none(bigint_param): self.t_env.register_function( "tinyint_func", udf(tinyint_func, [DataTypes.TINYINT()], DataTypes.TINYINT())) + self.t_env.register_function( + "smallint_func", udf(smallint_func, [DataTypes.SMALLINT()], DataTypes.SMALLINT())) + self.t_env.register_function( "bigint_func", udf(bigint_func, [DataTypes.BIGINT()], DataTypes.BIGINT())) @@ -362,25 +371,27 @@ def bigint_func_none(bigint_param): "bigint_func_none", udf(bigint_func_none, [DataTypes.BIGINT()], DataTypes.BIGINT())) table_sink = source_sink_utils.TestAppendSink( - ['a', 'b', 'c', 'd'], + ['a', 'b', 'c', 'd', 'e'], [DataTypes.BIGINT(), DataTypes.BIGINT(), DataTypes.TINYINT(), - DataTypes.BOOLEAN()]) + DataTypes.BOOLEAN(), DataTypes.SMALLINT()]) self.t_env.register_table_sink("Results", table_sink) t = self.t_env.from_elements( - [(1, None, 1, True)], + [(1, None, 1, True, 32767)], DataTypes.ROW( [DataTypes.FIELD("a", DataTypes.BIGINT()), DataTypes.FIELD("b", DataTypes.BIGINT()), DataTypes.FIELD("c", DataTypes.TINYINT()), - DataTypes.FIELD("d", DataTypes.BOOLEAN())])) + DataTypes.FIELD("d", DataTypes.BOOLEAN()), + DataTypes.FIELD("e", DataTypes.SMALLINT())])) t.select("bigint_func(a), bigint_func_none(b)," - "tinyint_func(c), boolean_func(d)") \ + "tinyint_func(c), boolean_func(d)," + "smallint_func(e)") \ .insert_into("Results") self.t_env.execute("test") actual = source_sink_utils.results() - self.assert_equals(actual, ["1,null,1,true"]) + self.assert_equals(actual, ["1,null,1,true,32767"]) class PyFlinkStreamUserDefinedFunctionTests(UserDefinedFunctionTests, diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java index 8590f7f808c6..78c9ab584151 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.typeutils.base.BooleanSerializer; import org.apache.flink.api.common.typeutils.base.ByteSerializer; import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.common.typeutils.base.ShortSerializer; import org.apache.flink.api.java.typeutils.runtime.RowSerializer; import org.apache.flink.fnexecution.v1.FlinkFnApi; import org.apache.flink.table.runtime.typeutils.serializers.python.BaseRowSerializer; @@ -30,6 +31,7 @@ import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.SmallIntType; import org.apache.flink.table.types.logical.TinyIntType; import org.apache.flink.table.types.logical.utils.LogicalTypeDefaultVisitor; @@ -65,6 +67,11 @@ public TypeSerializer visit(TinyIntType tinyIntType) { return ByteSerializer.INSTANCE; } + @Override + public TypeSerializer visit(SmallIntType smallIntType) { + return ShortSerializer.INSTANCE; + } + @Override public TypeSerializer visit(BigIntType bigIntType) { return LongSerializer.INSTANCE; @@ -115,6 +122,14 @@ public FlinkFnApi.Schema.FieldType visit(TinyIntType tinyIntType) { .build(); } + @Override + public FlinkFnApi.Schema.FieldType visit(SmallIntType smallIntType) { + return FlinkFnApi.Schema.FieldType.newBuilder() + .setTypeName(FlinkFnApi.Schema.TypeName.SMALLINT) + .setNullable(smallIntType.isNullable()) + .build(); + } + @Override public FlinkFnApi.Schema.FieldType visit(BigIntType bigIntType) { return FlinkFnApi.Schema.FieldType.newBuilder() From a13ee19d18f255e026f1f616d4b2cc8d2470f9fc Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Fri, 25 Oct 2019 20:24:33 +0800 Subject: [PATCH 236/746] [FLINK-14497][python] Support IntType for Python UDF --- .../pyflink/fn_execution/coder_impl.py | 8 +++++++ flink-python/pyflink/fn_execution/coders.py | 15 ++++++++++++- .../fn_execution/tests/coders_test_common.py | 7 +++++- flink-python/pyflink/table/tests/test_udf.py | 22 ++++++++++++++----- .../runtime/typeutils/PythonTypeUtils.java | 15 +++++++++++++ 5 files changed, 59 insertions(+), 8 deletions(-) diff --git a/flink-python/pyflink/fn_execution/coder_impl.py b/flink-python/pyflink/fn_execution/coder_impl.py index 3686979c082b..348038ce32cf 100644 --- a/flink-python/pyflink/fn_execution/coder_impl.py +++ b/flink-python/pyflink/fn_execution/coder_impl.py @@ -105,6 +105,14 @@ def decode_from_stream(self, in_stream, nested): return struct.unpack('>h', in_stream.read(2))[0] +class IntCoderImpl(StreamCoderImpl): + def encode_to_stream(self, value, out_stream, nested): + out_stream.write_bigendian_int32(value) + + def decode_from_stream(self, in_stream, nested): + return in_stream.read_bigendian_int32() + + class BooleanCoderImpl(StreamCoderImpl): def encode_to_stream(self, value, out_stream, nested): diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index cc00ab115e40..d6d2247349be 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -28,7 +28,7 @@ __all__ = ['RowCoder', 'BigIntCoder', 'TinyIntCoder', 'BooleanCoder', - 'SmallIntCoder'] + 'SmallIntCoder', 'IntCoder'] class RowCoder(FastCoder): @@ -122,6 +122,18 @@ def to_type_hint(self): return int +class IntCoder(DeterministicCoder): + """ + Coder for 4 bytes int. + """ + + def _create_impl(self): + return coder_impl.IntCoderImpl() + + def to_type_hint(self): + return int + + @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]) @@ -131,6 +143,7 @@ def _pickle_from_runner_api_parameter(schema_proto, unused_components, unused_co _type_name_mappings = { type_name.TINYINT: TinyIntCoder(), type_name.SMALLINT: SmallIntCoder(), + type_name.INT: IntCoder(), type_name.BIGINT: BigIntCoder(), type_name.BOOLEAN: BooleanCoder(), } diff --git a/flink-python/pyflink/fn_execution/tests/coders_test_common.py b/flink-python/pyflink/fn_execution/tests/coders_test_common.py index 9d26941f6b32..50a5045b5575 100644 --- a/flink-python/pyflink/fn_execution/tests/coders_test_common.py +++ b/flink-python/pyflink/fn_execution/tests/coders_test_common.py @@ -20,7 +20,8 @@ import logging import unittest -from pyflink.fn_execution.coders import BigIntCoder, TinyIntCoder, BooleanCoder, SmallIntCoder +from pyflink.fn_execution.coders import BigIntCoder, TinyIntCoder, BooleanCoder, \ + SmallIntCoder, IntCoder class CodersTest(unittest.TestCase): @@ -45,6 +46,10 @@ def test_smallint_coder(self): coder = SmallIntCoder() self.check_coder(coder, 32767, -32768, 0) + def test_int_coder(self): + coder = IntCoder() + self.check_coder(coder, -2147483648, 2147483647) + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) diff --git a/flink-python/pyflink/table/tests/test_udf.py b/flink-python/pyflink/table/tests/test_udf.py index df2d50c04030..5d580aec873e 100644 --- a/flink-python/pyflink/table/tests/test_udf.py +++ b/flink-python/pyflink/table/tests/test_udf.py @@ -346,6 +346,12 @@ def smallint_func(smallint_param): assert smallint_param == 32767, 'smallint_param of wrong value %s' % smallint_param return smallint_param + def int_func(int_param): + assert isinstance(int_param, int), 'int_param of wrong type %s !' \ + % type(int_param) + assert int_param == -2147483648, 'int_param of wrong value %s' % int_param + return int_param + def bigint_func(bigint_param): assert isinstance(bigint_param, int), 'bigint_param of wrong type %s !' \ % type(bigint_param) @@ -364,6 +370,9 @@ def bigint_func_none(bigint_param): self.t_env.register_function( "smallint_func", udf(smallint_func, [DataTypes.SMALLINT()], DataTypes.SMALLINT())) + self.t_env.register_function( + "int_func", udf(int_func, [DataTypes.INT()], DataTypes.INT())) + self.t_env.register_function( "bigint_func", udf(bigint_func, [DataTypes.BIGINT()], DataTypes.BIGINT())) @@ -371,27 +380,28 @@ def bigint_func_none(bigint_param): "bigint_func_none", udf(bigint_func_none, [DataTypes.BIGINT()], DataTypes.BIGINT())) table_sink = source_sink_utils.TestAppendSink( - ['a', 'b', 'c', 'd', 'e'], + ['a', 'b', 'c', 'd', 'e', 'f'], [DataTypes.BIGINT(), DataTypes.BIGINT(), DataTypes.TINYINT(), - DataTypes.BOOLEAN(), DataTypes.SMALLINT()]) + DataTypes.BOOLEAN(), DataTypes.SMALLINT(), DataTypes.INT()]) self.t_env.register_table_sink("Results", table_sink) t = self.t_env.from_elements( - [(1, None, 1, True, 32767)], + [(1, None, 1, True, 32767, -2147483648)], DataTypes.ROW( [DataTypes.FIELD("a", DataTypes.BIGINT()), DataTypes.FIELD("b", DataTypes.BIGINT()), DataTypes.FIELD("c", DataTypes.TINYINT()), DataTypes.FIELD("d", DataTypes.BOOLEAN()), - DataTypes.FIELD("e", DataTypes.SMALLINT())])) + DataTypes.FIELD("e", DataTypes.SMALLINT()), + DataTypes.FIELD("f", DataTypes.INT())])) t.select("bigint_func(a), bigint_func_none(b)," "tinyint_func(c), boolean_func(d)," - "smallint_func(e)") \ + "smallint_func(e),int_func(f)") \ .insert_into("Results") self.t_env.execute("test") actual = source_sink_utils.results() - self.assert_equals(actual, ["1,null,1,true,32767"]) + self.assert_equals(actual, ["1,null,1,true,32767,-2147483648"]) class PyFlinkStreamUserDefinedFunctionTests(UserDefinedFunctionTests, diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java index 78c9ab584151..52fe359379d2 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.BooleanSerializer; import org.apache.flink.api.common.typeutils.base.ByteSerializer; +import org.apache.flink.api.common.typeutils.base.IntSerializer; import org.apache.flink.api.common.typeutils.base.LongSerializer; import org.apache.flink.api.common.typeutils.base.ShortSerializer; import org.apache.flink.api.java.typeutils.runtime.RowSerializer; @@ -29,6 +30,7 @@ import org.apache.flink.table.runtime.typeutils.serializers.python.BaseRowSerializer; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.SmallIntType; @@ -72,6 +74,11 @@ public TypeSerializer visit(SmallIntType smallIntType) { return ShortSerializer.INSTANCE; } + @Override + public TypeSerializer visit(IntType intType) { + return IntSerializer.INSTANCE; + } + @Override public TypeSerializer visit(BigIntType bigIntType) { return LongSerializer.INSTANCE; @@ -130,6 +137,14 @@ public FlinkFnApi.Schema.FieldType visit(SmallIntType smallIntType) { .build(); } + @Override + public FlinkFnApi.Schema.FieldType visit(IntType intType) { + return FlinkFnApi.Schema.FieldType.newBuilder() + .setTypeName(FlinkFnApi.Schema.TypeName.INT) + .setNullable(intType.isNullable()) + .build(); + } + @Override public FlinkFnApi.Schema.FieldType visit(BigIntType bigIntType) { return FlinkFnApi.Schema.FieldType.newBuilder() From 9569c0e245c417517c69bdd07140896bff31044b Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Fri, 25 Oct 2019 20:24:35 +0800 Subject: [PATCH 237/746] [FLINK-14497][python] Support FloatType for Python UDF --- .../pyflink/fn_execution/coder_impl.py | 10 ++++++ flink-python/pyflink/fn_execution/coders.py | 15 ++++++++- .../fn_execution/tests/coders_test_common.py | 17 ++++++++-- flink-python/pyflink/table/tests/test_udf.py | 31 +++++++++++++------ .../runtime/typeutils/PythonTypeUtils.java | 15 +++++++++ 5 files changed, 76 insertions(+), 12 deletions(-) diff --git a/flink-python/pyflink/fn_execution/coder_impl.py b/flink-python/pyflink/fn_execution/coder_impl.py index 348038ce32cf..b66c9044363e 100644 --- a/flink-python/pyflink/fn_execution/coder_impl.py +++ b/flink-python/pyflink/fn_execution/coder_impl.py @@ -16,6 +16,7 @@ # limitations under the License. ################################################################################ +import struct from apache_beam.coders.coder_impl import StreamCoderImpl @@ -120,3 +121,12 @@ def encode_to_stream(self, value, out_stream, nested): def decode_from_stream(self, in_stream, nested): return not not in_stream.read_byte() + + +class FloatCoderImpl(StreamCoderImpl): + + def encode_to_stream(self, value, out_stream, nested): + out_stream.write(struct.pack('>f', value)) + + def decode_from_stream(self, in_stream, nested): + return struct.unpack('>f', in_stream.read(4))[0] diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index d6d2247349be..d7e16b129270 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -28,7 +28,7 @@ __all__ = ['RowCoder', 'BigIntCoder', 'TinyIntCoder', 'BooleanCoder', - 'SmallIntCoder', 'IntCoder'] + 'SmallIntCoder', 'IntCoder', 'FloatCoder'] class RowCoder(FastCoder): @@ -134,6 +134,18 @@ def to_type_hint(self): return int +class FloatCoder(DeterministicCoder): + """ + Coder for Float. + """ + + def _create_impl(self): + return coder_impl.FloatCoderImpl() + + def to_type_hint(self): + return float + + @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]) @@ -146,6 +158,7 @@ def _pickle_from_runner_api_parameter(schema_proto, unused_components, unused_co type_name.INT: IntCoder(), type_name.BIGINT: BigIntCoder(), type_name.BOOLEAN: BooleanCoder(), + type_name.FLOAT: FloatCoder(), } diff --git a/flink-python/pyflink/fn_execution/tests/coders_test_common.py b/flink-python/pyflink/fn_execution/tests/coders_test_common.py index 50a5045b5575..d8b99c8bb45d 100644 --- a/flink-python/pyflink/fn_execution/tests/coders_test_common.py +++ b/flink-python/pyflink/fn_execution/tests/coders_test_common.py @@ -21,14 +21,23 @@ import unittest from pyflink.fn_execution.coders import BigIntCoder, TinyIntCoder, BooleanCoder, \ - SmallIntCoder, IntCoder + SmallIntCoder, IntCoder, FloatCoder class CodersTest(unittest.TestCase): def check_coder(self, coder, *values): for v in values: - self.assertEqual(v, coder.decode(coder.encode(v))) + if isinstance(v, float): + from pyflink.table.tests.test_udf import float_equal + assert float_equal(v, coder.decode(coder.encode(v)), 1e-6) + else: + self.assertEqual(v, coder.decode(coder.encode(v))) + + # decide whether two floats are equal + @staticmethod + 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) def test_bigint_coder(self): coder = BigIntCoder() @@ -50,6 +59,10 @@ def test_int_coder(self): coder = IntCoder() self.check_coder(coder, -2147483648, 2147483647) + def test_float_coder(self): + coder = FloatCoder() + self.check_coder(coder, 1.02, 1.32) + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) diff --git a/flink-python/pyflink/table/tests/test_udf.py b/flink-python/pyflink/table/tests/test_udf.py index 5d580aec873e..29a64877e037 100644 --- a/flink-python/pyflink/table/tests/test_udf.py +++ b/flink-python/pyflink/table/tests/test_udf.py @@ -120,9 +120,6 @@ def udf_with_constant_params(p, null_param, tinyint_param, smallint_param, int_p 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 @@ -361,6 +358,11 @@ def bigint_func_none(bigint_param): assert bigint_param is None, 'bigint_param %s should be None!' % bigint_param return bigint_param + def float_func(float_param): + assert isinstance(float_param, float) and float_equal(float_param, 1.23, 1e-6), \ + 'float_param is wrong value %s !' % float_param + return float_param + self.t_env.register_function( "boolean_func", udf(boolean_func, [DataTypes.BOOLEAN()], DataTypes.BOOLEAN())) @@ -379,29 +381,40 @@ def bigint_func_none(bigint_param): self.t_env.register_function( "bigint_func_none", udf(bigint_func_none, [DataTypes.BIGINT()], DataTypes.BIGINT())) + self.t_env.register_function( + "float_func", udf(float_func, [DataTypes.FLOAT()], DataTypes.FLOAT())) + table_sink = source_sink_utils.TestAppendSink( - ['a', 'b', 'c', 'd', 'e', 'f'], + ['a', 'b', 'c', 'd', 'e', 'f', 'g'], [DataTypes.BIGINT(), DataTypes.BIGINT(), DataTypes.TINYINT(), - DataTypes.BOOLEAN(), DataTypes.SMALLINT(), DataTypes.INT()]) + DataTypes.BOOLEAN(), DataTypes.SMALLINT(), DataTypes.INT(), + DataTypes.FLOAT()]) self.t_env.register_table_sink("Results", table_sink) t = self.t_env.from_elements( - [(1, None, 1, True, 32767, -2147483648)], + [(1, None, 1, True, 32767, -2147483648, 1.23)], DataTypes.ROW( [DataTypes.FIELD("a", DataTypes.BIGINT()), DataTypes.FIELD("b", DataTypes.BIGINT()), DataTypes.FIELD("c", DataTypes.TINYINT()), DataTypes.FIELD("d", DataTypes.BOOLEAN()), DataTypes.FIELD("e", DataTypes.SMALLINT()), - DataTypes.FIELD("f", DataTypes.INT())])) + DataTypes.FIELD("f", DataTypes.INT()), + DataTypes.FIELD("g", DataTypes.FLOAT())])) t.select("bigint_func(a), bigint_func_none(b)," "tinyint_func(c), boolean_func(d)," - "smallint_func(e),int_func(f)") \ + "smallint_func(e),int_func(f)," + "float_func(g)") \ .insert_into("Results") self.t_env.execute("test") actual = source_sink_utils.results() - self.assert_equals(actual, ["1,null,1,true,32767,-2147483648"]) + self.assert_equals(actual, ["1,null,1,true,32767,-2147483648,1.23"]) + + +# 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) class PyFlinkStreamUserDefinedFunctionTests(UserDefinedFunctionTests, diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java index 52fe359379d2..9f81ff935d6f 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.BooleanSerializer; import org.apache.flink.api.common.typeutils.base.ByteSerializer; +import org.apache.flink.api.common.typeutils.base.FloatSerializer; import org.apache.flink.api.common.typeutils.base.IntSerializer; import org.apache.flink.api.common.typeutils.base.LongSerializer; import org.apache.flink.api.common.typeutils.base.ShortSerializer; @@ -30,6 +31,7 @@ import org.apache.flink.table.runtime.typeutils.serializers.python.BaseRowSerializer; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.FloatType; import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; @@ -84,6 +86,11 @@ public TypeSerializer visit(BigIntType bigIntType) { return LongSerializer.INSTANCE; } + @Override + public TypeSerializer visit(FloatType floatType) { + return FloatSerializer.INSTANCE; + } + @Override public TypeSerializer visit(RowType rowType) { final TypeSerializer[] fieldTypeSerializers = rowType.getFields() @@ -153,6 +160,14 @@ public FlinkFnApi.Schema.FieldType visit(BigIntType bigIntType) { .build(); } + @Override + public FlinkFnApi.Schema.FieldType visit(FloatType floatType) { + return FlinkFnApi.Schema.FieldType.newBuilder() + .setTypeName(FlinkFnApi.Schema.TypeName.FLOAT) + .setNullable(floatType.isNullable()) + .build(); + } + @Override public FlinkFnApi.Schema.FieldType visit(RowType rowType) { FlinkFnApi.Schema.FieldType.Builder builder = From c2674401159fc45e6e05e45a5e3c087cabacb1f2 Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Fri, 25 Oct 2019 20:24:36 +0800 Subject: [PATCH 238/746] [FLINK-14497][python] Support DoubleType for Python UDF --- .../pyflink/fn_execution/coder_impl.py | 9 ++++++++ flink-python/pyflink/fn_execution/coders.py | 15 ++++++++++++- .../fn_execution/tests/coders_test_common.py | 6 +++++- flink-python/pyflink/table/tests/test_udf.py | 21 +++++++++++++------ .../runtime/typeutils/PythonTypeUtils.java | 15 +++++++++++++ 5 files changed, 58 insertions(+), 8 deletions(-) diff --git a/flink-python/pyflink/fn_execution/coder_impl.py b/flink-python/pyflink/fn_execution/coder_impl.py index b66c9044363e..92711e97278f 100644 --- a/flink-python/pyflink/fn_execution/coder_impl.py +++ b/flink-python/pyflink/fn_execution/coder_impl.py @@ -130,3 +130,12 @@ def encode_to_stream(self, value, out_stream, nested): def decode_from_stream(self, in_stream, nested): return struct.unpack('>f', in_stream.read(4))[0] + + +class DoubleCoderImpl(StreamCoderImpl): + + def encode_to_stream(self, value, out_stream, nested): + out_stream.write_bigendian_double(value) + + def decode_from_stream(self, in_stream, nested): + return in_stream.read_bigendian_double() diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index d7e16b129270..eb6c9eee3d61 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -28,7 +28,7 @@ __all__ = ['RowCoder', 'BigIntCoder', 'TinyIntCoder', 'BooleanCoder', - 'SmallIntCoder', 'IntCoder', 'FloatCoder'] + 'SmallIntCoder', 'IntCoder', 'FloatCoder', 'DoubleCoder'] class RowCoder(FastCoder): @@ -146,6 +146,18 @@ def to_type_hint(self): return float +class DoubleCoder(DeterministicCoder): + """ + Coder for Double. + """ + + def _create_impl(self): + return coder_impl.DoubleCoderImpl() + + def to_type_hint(self): + return float + + @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]) @@ -159,6 +171,7 @@ def _pickle_from_runner_api_parameter(schema_proto, unused_components, unused_co type_name.BIGINT: BigIntCoder(), type_name.BOOLEAN: BooleanCoder(), type_name.FLOAT: FloatCoder(), + type_name.DOUBLE: DoubleCoder(), } diff --git a/flink-python/pyflink/fn_execution/tests/coders_test_common.py b/flink-python/pyflink/fn_execution/tests/coders_test_common.py index d8b99c8bb45d..1addec42bda9 100644 --- a/flink-python/pyflink/fn_execution/tests/coders_test_common.py +++ b/flink-python/pyflink/fn_execution/tests/coders_test_common.py @@ -21,7 +21,7 @@ import unittest from pyflink.fn_execution.coders import BigIntCoder, TinyIntCoder, BooleanCoder, \ - SmallIntCoder, IntCoder, FloatCoder + SmallIntCoder, IntCoder, FloatCoder, DoubleCoder class CodersTest(unittest.TestCase): @@ -63,6 +63,10 @@ def test_float_coder(self): coder = FloatCoder() self.check_coder(coder, 1.02, 1.32) + def test_double_coder(self): + coder = DoubleCoder() + self.check_coder(coder, -12.02, 1.98932) + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) diff --git a/flink-python/pyflink/table/tests/test_udf.py b/flink-python/pyflink/table/tests/test_udf.py index 29a64877e037..0ad9ef7caa96 100644 --- a/flink-python/pyflink/table/tests/test_udf.py +++ b/flink-python/pyflink/table/tests/test_udf.py @@ -363,6 +363,11 @@ def float_func(float_param): 'float_param is wrong value %s !' % float_param return float_param + def double_func(double_param): + assert isinstance(double_param, float) and float_equal(double_param, 1.98932, 1e-7), \ + 'double_param is wrong value %s !' % double_param + return double_param + self.t_env.register_function( "boolean_func", udf(boolean_func, [DataTypes.BOOLEAN()], DataTypes.BOOLEAN())) @@ -384,15 +389,18 @@ def float_func(float_param): self.t_env.register_function( "float_func", udf(float_func, [DataTypes.FLOAT()], DataTypes.FLOAT())) + self.t_env.register_function( + "double_func", udf(double_func, [DataTypes.DOUBLE()], DataTypes.DOUBLE())) + table_sink = source_sink_utils.TestAppendSink( - ['a', 'b', 'c', 'd', 'e', 'f', 'g'], + ['a', 'b', 'c', 'd', 'e', 'f', 'g', 'h'], [DataTypes.BIGINT(), DataTypes.BIGINT(), DataTypes.TINYINT(), DataTypes.BOOLEAN(), DataTypes.SMALLINT(), DataTypes.INT(), - DataTypes.FLOAT()]) + DataTypes.FLOAT(), DataTypes.DOUBLE()]) self.t_env.register_table_sink("Results", table_sink) t = self.t_env.from_elements( - [(1, None, 1, True, 32767, -2147483648, 1.23)], + [(1, None, 1, True, 32767, -2147483648, 1.23, 1.98932)], DataTypes.ROW( [DataTypes.FIELD("a", DataTypes.BIGINT()), DataTypes.FIELD("b", DataTypes.BIGINT()), @@ -400,16 +408,17 @@ def float_func(float_param): DataTypes.FIELD("d", DataTypes.BOOLEAN()), DataTypes.FIELD("e", DataTypes.SMALLINT()), DataTypes.FIELD("f", DataTypes.INT()), - DataTypes.FIELD("g", DataTypes.FLOAT())])) + DataTypes.FIELD("g", DataTypes.FLOAT()), + DataTypes.FIELD("h", DataTypes.DOUBLE())])) t.select("bigint_func(a), bigint_func_none(b)," "tinyint_func(c), boolean_func(d)," "smallint_func(e),int_func(f)," - "float_func(g)") \ + "float_func(g),double_func(h)") \ .insert_into("Results") self.t_env.execute("test") actual = source_sink_utils.results() - self.assert_equals(actual, ["1,null,1,true,32767,-2147483648,1.23"]) + self.assert_equals(actual, ["1,null,1,true,32767,-2147483648,1.23,1.98932"]) # decide whether two floats are equal diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java index 9f81ff935d6f..e9f683bd6f3d 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.BooleanSerializer; import org.apache.flink.api.common.typeutils.base.ByteSerializer; +import org.apache.flink.api.common.typeutils.base.DoubleSerializer; import org.apache.flink.api.common.typeutils.base.FloatSerializer; import org.apache.flink.api.common.typeutils.base.IntSerializer; import org.apache.flink.api.common.typeutils.base.LongSerializer; @@ -31,6 +32,7 @@ import org.apache.flink.table.runtime.typeutils.serializers.python.BaseRowSerializer; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.DoubleType; import org.apache.flink.table.types.logical.FloatType; import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.LogicalType; @@ -91,6 +93,11 @@ public TypeSerializer visit(FloatType floatType) { return FloatSerializer.INSTANCE; } + @Override + public TypeSerializer visit(DoubleType doubleType) { + return DoubleSerializer.INSTANCE; + } + @Override public TypeSerializer visit(RowType rowType) { final TypeSerializer[] fieldTypeSerializers = rowType.getFields() @@ -168,6 +175,14 @@ public FlinkFnApi.Schema.FieldType visit(FloatType floatType) { .build(); } + @Override + public FlinkFnApi.Schema.FieldType visit(DoubleType doubleType) { + return FlinkFnApi.Schema.FieldType.newBuilder() + .setTypeName(FlinkFnApi.Schema.TypeName.DOUBLE) + .setNullable(doubleType.isNullable()) + .build(); + } + @Override public FlinkFnApi.Schema.FieldType visit(RowType rowType) { FlinkFnApi.Schema.FieldType.Builder builder = From 67413f9bbbda825c31b4456cadbe11f79313d602 Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Fri, 25 Oct 2019 20:24:38 +0800 Subject: [PATCH 239/746] [FLINK-14497][python] Support BinaryType/VarBinaryType for Python UDF --- .../pyflink/fn_execution/coder_impl.py | 11 +++++++ flink-python/pyflink/fn_execution/coders.py | 17 ++++++++++- .../fn_execution/tests/coders_test_common.py | 6 +++- flink-python/pyflink/table/tests/test_udf.py | 25 ++++++++++++---- .../runtime/typeutils/PythonTypeUtils.java | 29 +++++++++++++++++++ 5 files changed, 80 insertions(+), 8 deletions(-) diff --git a/flink-python/pyflink/fn_execution/coder_impl.py b/flink-python/pyflink/fn_execution/coder_impl.py index 92711e97278f..146cf7182fc4 100644 --- a/flink-python/pyflink/fn_execution/coder_impl.py +++ b/flink-python/pyflink/fn_execution/coder_impl.py @@ -139,3 +139,14 @@ def encode_to_stream(self, value, out_stream, nested): def decode_from_stream(self, in_stream, nested): return in_stream.read_bigendian_double() + + +class BinaryCoderImpl(StreamCoderImpl): + + def encode_to_stream(self, value, out_stream, nested): + out_stream.write_bigendian_int32(len(value)) + out_stream.write(value, False) + + def decode_from_stream(self, in_stream, nested): + size = in_stream.read_bigendian_int32() + return in_stream.read(size) diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index eb6c9eee3d61..0b9318249966 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -28,7 +28,8 @@ __all__ = ['RowCoder', 'BigIntCoder', 'TinyIntCoder', 'BooleanCoder', - 'SmallIntCoder', 'IntCoder', 'FloatCoder', 'DoubleCoder'] + 'SmallIntCoder', 'IntCoder', 'FloatCoder', 'DoubleCoder', + 'BinaryCoder'] class RowCoder(FastCoder): @@ -158,6 +159,18 @@ def to_type_hint(self): return float +class BinaryCoder(DeterministicCoder): + """ + Coder for Byte Array. + """ + + def _create_impl(self): + return coder_impl.BinaryCoderImpl() + + def to_type_hint(self): + return bytes + + @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]) @@ -172,6 +185,8 @@ def _pickle_from_runner_api_parameter(schema_proto, unused_components, unused_co type_name.BOOLEAN: BooleanCoder(), type_name.FLOAT: FloatCoder(), type_name.DOUBLE: DoubleCoder(), + type_name.BINARY: BinaryCoder(), + type_name.VARBINARY: BinaryCoder(), } diff --git a/flink-python/pyflink/fn_execution/tests/coders_test_common.py b/flink-python/pyflink/fn_execution/tests/coders_test_common.py index 1addec42bda9..486e50fc2492 100644 --- a/flink-python/pyflink/fn_execution/tests/coders_test_common.py +++ b/flink-python/pyflink/fn_execution/tests/coders_test_common.py @@ -21,7 +21,7 @@ import unittest from pyflink.fn_execution.coders import BigIntCoder, TinyIntCoder, BooleanCoder, \ - SmallIntCoder, IntCoder, FloatCoder, DoubleCoder + SmallIntCoder, IntCoder, FloatCoder, DoubleCoder, BinaryCoder class CodersTest(unittest.TestCase): @@ -67,6 +67,10 @@ def test_double_coder(self): coder = DoubleCoder() self.check_coder(coder, -12.02, 1.98932) + def test_binary_coder(self): + coder = BinaryCoder() + self.check_coder(coder, b'pyflink') + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) diff --git a/flink-python/pyflink/table/tests/test_udf.py b/flink-python/pyflink/table/tests/test_udf.py index 0ad9ef7caa96..caae132727e2 100644 --- a/flink-python/pyflink/table/tests/test_udf.py +++ b/flink-python/pyflink/table/tests/test_udf.py @@ -368,6 +368,11 @@ def double_func(double_param): 'double_param is wrong value %s !' % double_param return double_param + def bytes_func(bytes_param): + assert bytes_param == b'flink', \ + 'bytes_param is wrong value %s !' % bytes_param + return bytes_param + self.t_env.register_function( "boolean_func", udf(boolean_func, [DataTypes.BOOLEAN()], DataTypes.BOOLEAN())) @@ -392,15 +397,19 @@ def double_func(double_param): self.t_env.register_function( "double_func", udf(double_func, [DataTypes.DOUBLE()], DataTypes.DOUBLE())) + self.t_env.register_function( + "bytes_func", udf(bytes_func, [DataTypes.BYTES()], DataTypes.BYTES())) + table_sink = source_sink_utils.TestAppendSink( - ['a', 'b', 'c', 'd', 'e', 'f', 'g', 'h'], + ['a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i'], [DataTypes.BIGINT(), DataTypes.BIGINT(), DataTypes.TINYINT(), DataTypes.BOOLEAN(), DataTypes.SMALLINT(), DataTypes.INT(), - DataTypes.FLOAT(), DataTypes.DOUBLE()]) + DataTypes.FLOAT(), DataTypes.DOUBLE(), DataTypes.BYTES()]) self.t_env.register_table_sink("Results", table_sink) t = self.t_env.from_elements( - [(1, None, 1, True, 32767, -2147483648, 1.23, 1.98932)], + [(1, None, 1, True, 32767, -2147483648, 1.23, 1.98932, + bytearray(b'flink'))], DataTypes.ROW( [DataTypes.FIELD("a", DataTypes.BIGINT()), DataTypes.FIELD("b", DataTypes.BIGINT()), @@ -409,16 +418,20 @@ def double_func(double_param): DataTypes.FIELD("e", DataTypes.SMALLINT()), DataTypes.FIELD("f", DataTypes.INT()), DataTypes.FIELD("g", DataTypes.FLOAT()), - DataTypes.FIELD("h", DataTypes.DOUBLE())])) + DataTypes.FIELD("h", DataTypes.DOUBLE()), + DataTypes.FIELD("i", DataTypes.BYTES())])) t.select("bigint_func(a), bigint_func_none(b)," "tinyint_func(c), boolean_func(d)," "smallint_func(e),int_func(f)," - "float_func(g),double_func(h)") \ + "float_func(g),double_func(h)," + "bytes_func(i)") \ .insert_into("Results") self.t_env.execute("test") actual = source_sink_utils.results() - self.assert_equals(actual, ["1,null,1,true,32767,-2147483648,1.23,1.98932"]) + self.assert_equals(actual, + ["1,null,1,true,32767,-2147483648,1.23,1.98932," + "[102, 108, 105, 110, 107]"]) # decide whether two floats are equal diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java index e9f683bd6f3d..de987194ca78 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java @@ -27,10 +27,12 @@ import org.apache.flink.api.common.typeutils.base.IntSerializer; import org.apache.flink.api.common.typeutils.base.LongSerializer; import org.apache.flink.api.common.typeutils.base.ShortSerializer; +import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer; import org.apache.flink.api.java.typeutils.runtime.RowSerializer; import org.apache.flink.fnexecution.v1.FlinkFnApi; import org.apache.flink.table.runtime.typeutils.serializers.python.BaseRowSerializer; import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.BinaryType; import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.DoubleType; import org.apache.flink.table.types.logical.FloatType; @@ -39,6 +41,7 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.SmallIntType; import org.apache.flink.table.types.logical.TinyIntType; +import org.apache.flink.table.types.logical.VarBinaryType; import org.apache.flink.table.types.logical.utils.LogicalTypeDefaultVisitor; /** @@ -98,6 +101,16 @@ public TypeSerializer visit(DoubleType doubleType) { return DoubleSerializer.INSTANCE; } + @Override + public TypeSerializer visit(BinaryType binaryType) { + return BytePrimitiveArraySerializer.INSTANCE; + } + + @Override + public TypeSerializer visit(VarBinaryType varBinaryType) { + return BytePrimitiveArraySerializer.INSTANCE; + } + @Override public TypeSerializer visit(RowType rowType) { final TypeSerializer[] fieldTypeSerializers = rowType.getFields() @@ -183,6 +196,22 @@ public FlinkFnApi.Schema.FieldType visit(DoubleType doubleType) { .build(); } + @Override + public FlinkFnApi.Schema.FieldType visit(BinaryType binaryType) { + return FlinkFnApi.Schema.FieldType.newBuilder() + .setTypeName(FlinkFnApi.Schema.TypeName.BINARY) + .setNullable(binaryType.isNullable()) + .build(); + } + + @Override + public FlinkFnApi.Schema.FieldType visit(VarBinaryType varBinaryType) { + return FlinkFnApi.Schema.FieldType.newBuilder() + .setTypeName(FlinkFnApi.Schema.TypeName.VARBINARY) + .setNullable(varBinaryType.isNullable()) + .build(); + } + @Override public FlinkFnApi.Schema.FieldType visit(RowType rowType) { FlinkFnApi.Schema.FieldType.Builder builder = From 23a9694c0d1fb9204c066e7603f9ad10fd21744c Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Fri, 25 Oct 2019 20:24:39 +0800 Subject: [PATCH 240/746] [FLINK-14497][python] Support CharType/VarCharType for Python UDF --- .../pyflink/fn_execution/coder_impl.py | 11 ++ flink-python/pyflink/fn_execution/coders.py | 15 ++- .../fn_execution/tests/coders_test_common.py | 6 +- flink-python/pyflink/table/tests/test_udf.py | 22 +++- .../runtime/typeutils/PythonTypeUtils.java | 39 ++++++ .../serializers/python/StringSerializer.java | 122 ++++++++++++++++++ .../python/StringSerializerTest.java | 48 +++++++ 7 files changed, 255 insertions(+), 8 deletions(-) create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/StringSerializer.java create mode 100644 flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/serializers/python/StringSerializerTest.java diff --git a/flink-python/pyflink/fn_execution/coder_impl.py b/flink-python/pyflink/fn_execution/coder_impl.py index 146cf7182fc4..da9fc188648d 100644 --- a/flink-python/pyflink/fn_execution/coder_impl.py +++ b/flink-python/pyflink/fn_execution/coder_impl.py @@ -150,3 +150,14 @@ def encode_to_stream(self, value, out_stream, nested): def decode_from_stream(self, in_stream, nested): size = in_stream.read_bigendian_int32() return in_stream.read(size) + + +class CharCoderImpl(StreamCoderImpl): + + def encode_to_stream(self, value, out_stream, nested): + out_stream.write_bigendian_int32(len(value)) + out_stream.write(value.encode("utf-8"), False) + + def decode_from_stream(self, in_stream, nested): + size = in_stream.read_bigendian_int32() + return in_stream.read(size).decode("utf-8") diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index 0b9318249966..79fe4c1ef880 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -29,7 +29,7 @@ __all__ = ['RowCoder', 'BigIntCoder', 'TinyIntCoder', 'BooleanCoder', 'SmallIntCoder', 'IntCoder', 'FloatCoder', 'DoubleCoder', - 'BinaryCoder'] + 'BinaryCoder', 'CharCoder'] class RowCoder(FastCoder): @@ -171,6 +171,17 @@ def to_type_hint(self): return bytes +class CharCoder(DeterministicCoder): + """ + Coder for Character String. + """ + def _create_impl(self): + return coder_impl.CharCoderImpl() + + def to_type_hint(self): + return str + + @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]) @@ -187,6 +198,8 @@ def _pickle_from_runner_api_parameter(schema_proto, unused_components, unused_co type_name.DOUBLE: DoubleCoder(), type_name.BINARY: BinaryCoder(), type_name.VARBINARY: BinaryCoder(), + type_name.CHAR: CharCoder(), + type_name.VARCHAR: CharCoder(), } diff --git a/flink-python/pyflink/fn_execution/tests/coders_test_common.py b/flink-python/pyflink/fn_execution/tests/coders_test_common.py index 486e50fc2492..105add561e01 100644 --- a/flink-python/pyflink/fn_execution/tests/coders_test_common.py +++ b/flink-python/pyflink/fn_execution/tests/coders_test_common.py @@ -21,7 +21,7 @@ import unittest from pyflink.fn_execution.coders import BigIntCoder, TinyIntCoder, BooleanCoder, \ - SmallIntCoder, IntCoder, FloatCoder, DoubleCoder, BinaryCoder + SmallIntCoder, IntCoder, FloatCoder, DoubleCoder, BinaryCoder, CharCoder class CodersTest(unittest.TestCase): @@ -71,6 +71,10 @@ def test_binary_coder(self): coder = BinaryCoder() self.check_coder(coder, b'pyflink') + def test_char_coder(self): + coder = CharCoder() + self.check_coder(coder, 'flink') + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) diff --git a/flink-python/pyflink/table/tests/test_udf.py b/flink-python/pyflink/table/tests/test_udf.py index caae132727e2..525d8afac012 100644 --- a/flink-python/pyflink/table/tests/test_udf.py +++ b/flink-python/pyflink/table/tests/test_udf.py @@ -373,6 +373,11 @@ def bytes_func(bytes_param): 'bytes_param is wrong value %s !' % bytes_param return bytes_param + def str_func(str_param): + assert str_param == 'pyflink', \ + 'str_param is wrong value %s !' % str_param + return str_param + self.t_env.register_function( "boolean_func", udf(boolean_func, [DataTypes.BOOLEAN()], DataTypes.BOOLEAN())) @@ -400,16 +405,20 @@ def bytes_func(bytes_param): self.t_env.register_function( "bytes_func", udf(bytes_func, [DataTypes.BYTES()], DataTypes.BYTES())) + self.t_env.register_function( + "str_func", udf(str_func, [DataTypes.STRING()], DataTypes.STRING())) + table_sink = source_sink_utils.TestAppendSink( - ['a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i'], + ['a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j'], [DataTypes.BIGINT(), DataTypes.BIGINT(), DataTypes.TINYINT(), DataTypes.BOOLEAN(), DataTypes.SMALLINT(), DataTypes.INT(), - DataTypes.FLOAT(), DataTypes.DOUBLE(), DataTypes.BYTES()]) + DataTypes.FLOAT(), DataTypes.DOUBLE(), DataTypes.BYTES(), + DataTypes.STRING()]) self.t_env.register_table_sink("Results", table_sink) t = self.t_env.from_elements( [(1, None, 1, True, 32767, -2147483648, 1.23, 1.98932, - bytearray(b'flink'))], + bytearray(b'flink'), 'pyflink')], DataTypes.ROW( [DataTypes.FIELD("a", DataTypes.BIGINT()), DataTypes.FIELD("b", DataTypes.BIGINT()), @@ -419,19 +428,20 @@ def bytes_func(bytes_param): DataTypes.FIELD("f", DataTypes.INT()), DataTypes.FIELD("g", DataTypes.FLOAT()), DataTypes.FIELD("h", DataTypes.DOUBLE()), - DataTypes.FIELD("i", DataTypes.BYTES())])) + DataTypes.FIELD("i", DataTypes.BYTES()), + DataTypes.FIELD("j", DataTypes.STRING())])) t.select("bigint_func(a), bigint_func_none(b)," "tinyint_func(c), boolean_func(d)," "smallint_func(e),int_func(f)," "float_func(g),double_func(h)," - "bytes_func(i)") \ + "bytes_func(i),str_func(j)") \ .insert_into("Results") self.t_env.execute("test") actual = source_sink_utils.results() self.assert_equals(actual, ["1,null,1,true,32767,-2147483648,1.23,1.98932," - "[102, 108, 105, 110, 107]"]) + "[102, 108, 105, 110, 107],pyflink"]) # decide whether two floats are equal diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java index de987194ca78..fb3067d902e6 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java @@ -31,9 +31,11 @@ import org.apache.flink.api.java.typeutils.runtime.RowSerializer; import org.apache.flink.fnexecution.v1.FlinkFnApi; import org.apache.flink.table.runtime.typeutils.serializers.python.BaseRowSerializer; +import org.apache.flink.table.runtime.typeutils.serializers.python.StringSerializer; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BinaryType; import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.CharType; import org.apache.flink.table.types.logical.DoubleType; import org.apache.flink.table.types.logical.FloatType; import org.apache.flink.table.types.logical.IntType; @@ -42,6 +44,7 @@ import org.apache.flink.table.types.logical.SmallIntType; import org.apache.flink.table.types.logical.TinyIntType; import org.apache.flink.table.types.logical.VarBinaryType; +import org.apache.flink.table.types.logical.VarCharType; import org.apache.flink.table.types.logical.utils.LogicalTypeDefaultVisitor; /** @@ -111,6 +114,16 @@ public TypeSerializer visit(VarBinaryType varBinaryType) { return BytePrimitiveArraySerializer.INSTANCE; } + @Override + public TypeSerializer visit(VarCharType varCharType) { + return StringSerializer.INSTANCE; + } + + @Override + public TypeSerializer visit(CharType charType) { + return StringSerializer.INSTANCE; + } + @Override public TypeSerializer visit(RowType rowType) { final TypeSerializer[] fieldTypeSerializers = rowType.getFields() @@ -137,6 +150,16 @@ public TypeSerializer visit(RowType rowType) { .toArray(TypeSerializer[]::new); return new BaseRowSerializer(rowType.getChildren().toArray(new LogicalType[0]), fieldTypeSerializers); } + + @Override + public TypeSerializer visit(VarCharType varCharType) { + return BinaryStringSerializer.INSTANCE; + } + + @Override + public TypeSerializer visit(CharType charType) { + return BinaryStringSerializer.INSTANCE; + } } private static class LogicalTypeToProtoTypeConverter extends LogicalTypeDefaultVisitor { @@ -212,6 +235,22 @@ public FlinkFnApi.Schema.FieldType visit(VarBinaryType varBinaryType) { .build(); } + @Override + public FlinkFnApi.Schema.FieldType visit(CharType charType) { + return FlinkFnApi.Schema.FieldType.newBuilder() + .setTypeName(FlinkFnApi.Schema.TypeName.CHAR) + .setNullable(charType.isNullable()) + .build(); + } + + @Override + public FlinkFnApi.Schema.FieldType visit(VarCharType varCharType) { + return FlinkFnApi.Schema.FieldType.newBuilder() + .setTypeName(FlinkFnApi.Schema.TypeName.VARCHAR) + .setNullable(varCharType.isNullable()) + .build(); + } + @Override public FlinkFnApi.Schema.FieldType visit(RowType rowType) { FlinkFnApi.Schema.FieldType.Builder builder = diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/StringSerializer.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/StringSerializer.java new file mode 100644 index 000000000000..0901bdb4bdae --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/StringSerializer.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.table.runtime.typeutils.serializers.python; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.table.runtime.util.StringUtf8Utils; + +import java.io.IOException; + +/** + *

We create the StringSerializer instead of using the StringSerializer of flink-core module + * because the StringSerializer of flink-core module serialize every Char of String in serialize + * method and deserialize the Char to build the String. We want to convert String to UTF-8 bytes + * to serialize which is compatible with BinaryStringSerializer in blink.

+ * + *

So we create this StringSerializer (only used in Java and Python data communication in udf). + * + *

StringSerializer for String. + */ +@Internal +public class StringSerializer extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + public static final StringSerializer INSTANCE = new StringSerializer(); + + private static final String EMPTY = ""; + + @Override + public boolean isImmutableType() { + return true; + } + + @Override + public String createInstance() { + return EMPTY; + } + + @Override + public String copy(String from) { + return from; + } + + @Override + public String copy(String from, String reuse) { + return from; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(String record, DataOutputView target) throws IOException { + if (record == null) { + throw new IllegalArgumentException("The String record must not be null."); + } + byte[] bytes = StringUtf8Utils.encodeUTF8(record); + target.writeInt(bytes.length); + target.write(bytes); + } + + @Override + public String deserialize(DataInputView source) throws IOException { + int len = source.readInt(); + byte[] bytes = new byte[len]; + source.read(bytes); + return StringUtf8Utils.decodeUTF8(bytes, 0, len); + } + + @Override + public String deserialize(String reuse, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + int len = source.readInt(); + target.writeInt(len); + byte[] bytes = new byte[len]; + source.read(bytes); + target.write(bytes); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new StringSerializerSnapshot(); + } + + /** + * Serializer configuration snapshot for compatibility and format evolution. + */ + @SuppressWarnings("WeakerAccess") + public static final class StringSerializerSnapshot extends SimpleTypeSerializerSnapshot { + + public StringSerializerSnapshot() { + super(() -> INSTANCE); + } + } +} diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/serializers/python/StringSerializerTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/serializers/python/StringSerializerTest.java new file mode 100644 index 000000000000..e5b6f8bd1604 --- /dev/null +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/serializers/python/StringSerializerTest.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.table.runtime.typeutils.serializers.python; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; + +/** + * Test for {@link StringSerializer}. + */ +public class StringSerializerTest extends SerializerTestBase { + + @Override + protected TypeSerializer createSerializer() { + return StringSerializer.INSTANCE; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return String.class; + } + + @Override + protected String[] getTestData() { + return new String[]{"pyflink", "flink"}; + } +} From dd3d9dc2bd3edbe7f61bfb7265fb2f0b2768b91b Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Fri, 25 Oct 2019 20:24:41 +0800 Subject: [PATCH 241/746] [FLINK-14497][python] Support DateType for Python UDF This closes #9977. --- .../pyflink/fn_execution/coder_impl.py | 20 ++- flink-python/pyflink/fn_execution/coders.py | 17 ++- .../fn_execution/tests/coders_test_common.py | 7 +- flink-python/pyflink/table/tests/test_udf.py | 24 +++- .../runtime/typeutils/PythonTypeUtils.java | 20 +++ .../serializers/python/DateSerializer.java | 131 ++++++++++++++++++ .../python/DateSerializerTest.java | 50 +++++++ 7 files changed, 259 insertions(+), 10 deletions(-) create mode 100644 flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/DateSerializer.java create mode 100644 flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/serializers/python/DateSerializerTest.java diff --git a/flink-python/pyflink/fn_execution/coder_impl.py b/flink-python/pyflink/fn_execution/coder_impl.py index da9fc188648d..712308505f29 100644 --- a/flink-python/pyflink/fn_execution/coder_impl.py +++ b/flink-python/pyflink/fn_execution/coder_impl.py @@ -16,6 +16,7 @@ # limitations under the License. ################################################################################ +import datetime import struct from apache_beam.coders.coder_impl import StreamCoderImpl @@ -98,11 +99,9 @@ def decode_from_stream(self, in_stream, nested): class SmallIntImpl(StreamCoderImpl): def encode_to_stream(self, value, out_stream, nested): - import struct out_stream.write(struct.pack('>h', value)) def decode_from_stream(self, in_stream, nested): - import struct return struct.unpack('>h', in_stream.read(2))[0] @@ -161,3 +160,20 @@ def encode_to_stream(self, value, out_stream, nested): def decode_from_stream(self, in_stream, nested): size = in_stream.read_bigendian_int32() return in_stream.read(size).decode("utf-8") + + +class DateCoderImpl(StreamCoderImpl): + EPOCH_ORDINAL = datetime.datetime(1970, 1, 1).toordinal() + + def encode_to_stream(self, value, out_stream, nested): + out_stream.write_bigendian_int32(self.date_to_internal(value)) + + def decode_from_stream(self, in_stream, nested): + value = in_stream.read_bigendian_int32() + return self.internal_to_date(value) + + def date_to_internal(self, d): + return d.toordinal() - self.EPOCH_ORDINAL + + def internal_to_date(self, v): + return datetime.date.fromordinal(v + self.EPOCH_ORDINAL) diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index 79fe4c1ef880..f98561a3bb63 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -18,6 +18,8 @@ from abc import ABC + +import datetime from apache_beam.coders import Coder from apache_beam.coders.coders import FastCoder @@ -29,7 +31,7 @@ __all__ = ['RowCoder', 'BigIntCoder', 'TinyIntCoder', 'BooleanCoder', 'SmallIntCoder', 'IntCoder', 'FloatCoder', 'DoubleCoder', - 'BinaryCoder', 'CharCoder'] + 'BinaryCoder', 'CharCoder', 'DateCoder'] class RowCoder(FastCoder): @@ -182,6 +184,18 @@ def to_type_hint(self): return str +class DateCoder(DeterministicCoder): + """ + Coder for Date + """ + + def _create_impl(self): + return coder_impl.DateCoderImpl() + + def to_type_hint(self): + return datetime.date + + @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]) @@ -200,6 +214,7 @@ def _pickle_from_runner_api_parameter(schema_proto, unused_components, unused_co type_name.VARBINARY: BinaryCoder(), type_name.CHAR: CharCoder(), type_name.VARCHAR: CharCoder(), + type_name.DATE: DateCoder(), } diff --git a/flink-python/pyflink/fn_execution/tests/coders_test_common.py b/flink-python/pyflink/fn_execution/tests/coders_test_common.py index 105add561e01..6af26bf30921 100644 --- a/flink-python/pyflink/fn_execution/tests/coders_test_common.py +++ b/flink-python/pyflink/fn_execution/tests/coders_test_common.py @@ -21,7 +21,7 @@ import unittest from pyflink.fn_execution.coders import BigIntCoder, TinyIntCoder, BooleanCoder, \ - SmallIntCoder, IntCoder, FloatCoder, DoubleCoder, BinaryCoder, CharCoder + SmallIntCoder, IntCoder, FloatCoder, DoubleCoder, BinaryCoder, CharCoder, DateCoder class CodersTest(unittest.TestCase): @@ -75,6 +75,11 @@ def test_char_coder(self): coder = CharCoder() self.check_coder(coder, 'flink') + def test_date_coder(self): + import datetime + coder = DateCoder() + self.check_coder(coder, datetime.date(2019, 9, 10)) + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) diff --git a/flink-python/pyflink/table/tests/test_udf.py b/flink-python/pyflink/table/tests/test_udf.py index 525d8afac012..a433c904f4fa 100644 --- a/flink-python/pyflink/table/tests/test_udf.py +++ b/flink-python/pyflink/table/tests/test_udf.py @@ -378,6 +378,12 @@ def str_func(str_param): 'str_param is wrong value %s !' % str_param return str_param + def date_func(date_param): + from datetime import date + assert date_param == date(year=2014, month=9, day=13), \ + 'date_param is wrong value %s !' % date_param + return date_param + self.t_env.register_function( "boolean_func", udf(boolean_func, [DataTypes.BOOLEAN()], DataTypes.BOOLEAN())) @@ -408,17 +414,21 @@ def str_func(str_param): self.t_env.register_function( "str_func", udf(str_func, [DataTypes.STRING()], DataTypes.STRING())) + self.t_env.register_function( + "date_func", udf(date_func, [DataTypes.DATE()], DataTypes.DATE())) + table_sink = source_sink_utils.TestAppendSink( - ['a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j'], + ['a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k'], [DataTypes.BIGINT(), DataTypes.BIGINT(), DataTypes.TINYINT(), DataTypes.BOOLEAN(), DataTypes.SMALLINT(), DataTypes.INT(), DataTypes.FLOAT(), DataTypes.DOUBLE(), DataTypes.BYTES(), - DataTypes.STRING()]) + DataTypes.STRING(), DataTypes.DATE()]) self.t_env.register_table_sink("Results", table_sink) + import datetime t = self.t_env.from_elements( [(1, None, 1, True, 32767, -2147483648, 1.23, 1.98932, - bytearray(b'flink'), 'pyflink')], + bytearray(b'flink'), 'pyflink', datetime.date(2014, 9, 13))], DataTypes.ROW( [DataTypes.FIELD("a", DataTypes.BIGINT()), DataTypes.FIELD("b", DataTypes.BIGINT()), @@ -429,19 +439,21 @@ def str_func(str_param): DataTypes.FIELD("g", DataTypes.FLOAT()), DataTypes.FIELD("h", DataTypes.DOUBLE()), DataTypes.FIELD("i", DataTypes.BYTES()), - DataTypes.FIELD("j", DataTypes.STRING())])) + DataTypes.FIELD("j", DataTypes.STRING()), + DataTypes.FIELD("k", DataTypes.DATE())])) t.select("bigint_func(a), bigint_func_none(b)," "tinyint_func(c), boolean_func(d)," "smallint_func(e),int_func(f)," "float_func(g),double_func(h)," - "bytes_func(i),str_func(j)") \ + "bytes_func(i),str_func(j)," + "date_func(k)") \ .insert_into("Results") self.t_env.execute("test") actual = source_sink_utils.results() self.assert_equals(actual, ["1,null,1,true,32767,-2147483648,1.23,1.98932," - "[102, 108, 105, 110, 107],pyflink"]) + "[102, 108, 105, 110, 107],pyflink,2014-09-13"]) # decide whether two floats are equal diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java index fb3067d902e6..fbe4db04d165 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java @@ -31,11 +31,13 @@ import org.apache.flink.api.java.typeutils.runtime.RowSerializer; import org.apache.flink.fnexecution.v1.FlinkFnApi; import org.apache.flink.table.runtime.typeutils.serializers.python.BaseRowSerializer; +import org.apache.flink.table.runtime.typeutils.serializers.python.DateSerializer; import org.apache.flink.table.runtime.typeutils.serializers.python.StringSerializer; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BinaryType; import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.CharType; +import org.apache.flink.table.types.logical.DateType; import org.apache.flink.table.types.logical.DoubleType; import org.apache.flink.table.types.logical.FloatType; import org.apache.flink.table.types.logical.IntType; @@ -124,6 +126,11 @@ public TypeSerializer visit(CharType charType) { return StringSerializer.INSTANCE; } + @Override + public TypeSerializer visit(DateType dateType) { + return DateSerializer.INSTANCE; + } + @Override public TypeSerializer visit(RowType rowType) { final TypeSerializer[] fieldTypeSerializers = rowType.getFields() @@ -160,6 +167,11 @@ public TypeSerializer visit(VarCharType varCharType) { public TypeSerializer visit(CharType charType) { return BinaryStringSerializer.INSTANCE; } + + @Override + public TypeSerializer visit(DateType dateType) { + return IntSerializer.INSTANCE; + } } private static class LogicalTypeToProtoTypeConverter extends LogicalTypeDefaultVisitor { @@ -251,6 +263,14 @@ public FlinkFnApi.Schema.FieldType visit(VarCharType varCharType) { .build(); } + @Override + public FlinkFnApi.Schema.FieldType visit(DateType dateType) { + return FlinkFnApi.Schema.FieldType.newBuilder() + .setTypeName(FlinkFnApi.Schema.TypeName.DATE) + .setNullable(dateType.isNullable()) + .build(); + } + @Override public FlinkFnApi.Schema.FieldType visit(RowType rowType) { FlinkFnApi.Schema.FieldType.Builder builder = diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/DateSerializer.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/DateSerializer.java new file mode 100644 index 000000000000..5c279d95f822 --- /dev/null +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/serializers/python/DateSerializer.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.typeutils.serializers.python; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; +import java.sql.Date; +import java.util.TimeZone; + +/** + * Takes int instead of long as the serialized value. It not only reduces the length of + * the serialized value, but also makes the serialized value consistent between + * the legacy planner and the blink planner. + */ +@Internal +public class DateSerializer extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + /** + * The local time zone. + */ + private static final TimeZone LOCAL_TZ = TimeZone.getDefault(); + + private static final long MILLIS_PER_DAY = 86400000L; // = 24 * 60 * 60 * 1000 + + public static final DateSerializer INSTANCE = new DateSerializer(); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public Date createInstance() { + return new Date(0L); + } + + @Override + public Date copy(Date from) { + if (from == null) { + return null; + } + return new Date(from.getTime()); + } + + @Override + public Date copy(Date from, Date reuse) { + if (from == null) { + return null; + } + reuse.setTime(from.getTime()); + return reuse; + } + + @Override + public int getLength() { + return 4; + } + + @Override + public void serialize(Date record, DataOutputView target) throws IOException { + if (record == null) { + throw new IllegalArgumentException("The Date record must not be null."); + } + target.writeInt(dateToInternal(record)); + } + + @Override + public Date deserialize(DataInputView source) throws IOException { + return internalToDate(source.readInt()); + } + + private int dateToInternal(Date date) { + long ts = date.getTime() + LOCAL_TZ.getOffset(date.getTime()); + return (int) (ts / MILLIS_PER_DAY); + } + + private Date internalToDate(int v) { + final long t = v * MILLIS_PER_DAY; + return new Date(t - LOCAL_TZ.getOffset(t)); + } + + @Override + public Date deserialize(Date reuse, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new DateSerializerSnapshot(); + } + + /** + * Serializer configuration snapshot for compatibility and format evolution. + */ + @SuppressWarnings("WeakerAccess") + public static final class DateSerializerSnapshot extends SimpleTypeSerializerSnapshot { + + public DateSerializerSnapshot() { + super(() -> INSTANCE); + } + } +} diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/serializers/python/DateSerializerTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/serializers/python/DateSerializerTest.java new file mode 100644 index 000000000000..1a22c72c0230 --- /dev/null +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/typeutils/serializers/python/DateSerializerTest.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.table.runtime.typeutils.serializers.python; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; + +import java.sql.Date; + +/** + * Test for {@link DateSerializer}. + */ +public class DateSerializerTest extends SerializerTestBase { + + @Override + protected TypeSerializer createSerializer() { + return DateSerializer.INSTANCE; + } + + @Override + protected int getLength() { + return 4; + } + + @Override + protected Class getTypeClass() { + return Date.class; + } + + @Override + protected Date[] getTestData() { + return new Date[]{Date.valueOf("2014-09-13")}; + } +} From 098d5e2099563afe70deb28b4c9893a65988def3 Mon Sep 17 00:00:00 2001 From: Wei Zhong Date: Thu, 24 Oct 2019 14:51:50 +0800 Subject: [PATCH 242/746] [FLINK-14509][python] Improve the README.md in flink-python to prepare for PyPI release. PyFlink will be released on PyPi from FLINK 1.10.0. This commit adds corresponding document for it. This closes #9982. --- docs/tutorials/python_table_api.md | 3 +- docs/tutorials/python_table_api.zh.md | 4 +-- flink-python/README.md | 42 +++++++++++++-------------- 3 files changed, 25 insertions(+), 24 deletions(-) diff --git a/docs/tutorials/python_table_api.md b/docs/tutorials/python_table_api.md index 89d339716d5d..f882830df579 100644 --- a/docs/tutorials/python_table_api.md +++ b/docs/tutorials/python_table_api.md @@ -32,7 +32,8 @@ to running a Python Table API program. ## Setting up a Python Project Firstly, you can fire up your favorite IDE and create a Python project and then -you need to install the PyFlink package. Please +you need to install the PyFlink package. You can install the latest PyFlink from PyPI via +this command: `pip install apache-flink`. If you want to build and install PyFlink from source code, please see [Build PyFlink]({{ site.baseurl }}/flinkDev/building.html#build-pyflink) for more details about this. diff --git a/docs/tutorials/python_table_api.zh.md b/docs/tutorials/python_table_api.zh.md index bab36e169603..6ba3b37e2106 100644 --- a/docs/tutorials/python_table_api.zh.md +++ b/docs/tutorials/python_table_api.zh.md @@ -30,8 +30,8 @@ under the License. ## 创建一个Python Table API项目 -首先,你可以使用你最熟悉的IDE,创建一个Python项目。然后,你需要安装PyFlink包, -请参考[构建PyFlink]({{ site.baseurl }}/zh/flinkDev/building.html#build-pyflink)了解详细信息。 +首先,使用您最熟悉的IDE创建一个Python项目。之后执行命令`pip install apache-flink`从PyPI下载安装PyFlink包。 +如果您想从源码安装,请参考[构建PyFlink]({{ site.baseurl }}/zh/flinkDev/building.html#build-pyflink)了解详细信息。 ## 编写一个Flink Python Table API程序 diff --git a/flink-python/README.md b/flink-python/README.md index 29dc3c419fbd..cb97357b59c2 100644 --- a/flink-python/README.md +++ b/flink-python/README.md @@ -1,37 +1,28 @@ -# Apache Flink Python API +# Apache Flink -Apache Flink is an open source stream processing framework with the powerful stream- and batch-processing capabilities. +Apache Flink is a framework and distributed processing engine for stateful computations over unbounded and bounded data streams. Flink has been designed to run in all common cluster environments, perform computations at in-memory speed and at any scale. Learn more about Flink at [https://flink.apache.org/](https://flink.apache.org/) -This packaging allows you to write Flink programs in Python, but it is currently a very initial version and will change in future versions. - -In this initial version only Table API is supported, you can find the documentation at [https://ci.apache.org/projects/flink/flink-docs-master/dev/table/tableApi.html](https://ci.apache.org/projects/flink/flink-docs-master/dev/table/tableApi.html) +## Python Packaging -## Installation +This packaging allows you to write Flink programs in Python, but it is currently a very initial version and will change in future versions. -Currently, we can install PyFlink from Flink source code. Enter the directory where this README.md file is located and install PyFlink on your device by executing +In this initial version only Table API is supported, you can find the documentation at [https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/tableApi.html](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/tableApi.html) -``` -python setup.py install -``` +The tutorial can be found at [https://ci.apache.org/projects/flink/flink-docs-stable/tutorials/python_table_api.html](https://ci.apache.org/projects/flink/flink-docs-stable/tutorials/python_table_api.html) -## Running test cases - -Currently, we use conda and tox to verify the compatibility of the Flink Python API for multiple versions of Python and will integrate some useful plugins with tox, such as flake8. -We can enter the directory where this README.md file is located and run test cases by executing - -``` -./dev/lint-python.sh -``` +The auto-generated Python docs can be found at [https://ci.apache.org/projects/flink/flink-docs-stable/api/python/](https://ci.apache.org/projects/flink/flink-docs-stable/api/python/) ## Python Requirements -PyFlink depends on Py4J (currently version 0.10.8.1) and CloudPickle (currently version 1.2.2). +Apache Flink Python API depends on Py4J (currently version 0.10.8.1), CloudPickle (currently version 1.2.2), python-dateutil(currently version 2.8.0) and Apache Beam (currently version 2.15.0). + +## Development Notices -## Development notices +### Protobuf Code Generation -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 +Protocol buffer is used in file `flink_fn_execution_pb2.py` and the file 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 @@ -41,3 +32,12 @@ 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 (>=7.1.0) + +### Running Test Cases + +Currently, we use conda and tox to verify the compatibility of the Flink Python API for multiple versions of Python and will integrate some useful plugins with tox, such as flake8. +We can enter the directory where this README.md file is located and run test cases by executing + +``` +./dev/lint-python.sh +``` From 4ef6ba13acfa03bc5eebeaacb07ff02cbb9ff0ce Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 1 Oct 2019 14:45:58 +0200 Subject: [PATCH 243/746] [hotfix] Enrich DispatcherRunnerFactory to pass in additional services --- .../runner/DispatcherRunnerFactory.java | 8 +++- .../runner/DispatcherRunnerFactoryImpl.java | 8 +++- ...atcherResourceManagerComponentFactory.java | 6 ++- .../HaServicesJobGraphStoreFactory.java | 48 +++++++++++++++++++ 4 files changed, 67 insertions(+), 3 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HaServicesJobGraphStoreFactory.java 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 9cf8232563a4..ba42ccfb7cf1 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 @@ -19,6 +19,9 @@ package org.apache.flink.runtime.dispatcher.runner; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; +import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; +import org.apache.flink.runtime.leaderelection.LeaderElectionService; +import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; import java.util.concurrent.Executor; @@ -29,7 +32,10 @@ public interface DispatcherRunnerFactory { DispatcherRunner createDispatcherRunner( - RpcService rpcService, + LeaderElectionService leaderElectionService, + FatalErrorHandler fatalErrorHandler, + JobGraphStoreFactory jobGraphStoreFactory, Executor ioExecutor, + RpcService rpcService, 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 dc2654714ad9..67020679d4fd 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 @@ -21,6 +21,9 @@ import org.apache.flink.runtime.dispatcher.DispatcherFactory; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; import org.apache.flink.runtime.dispatcher.StandaloneDispatcher; +import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; +import org.apache.flink.runtime.leaderelection.LeaderElectionService; +import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; import java.util.concurrent.Executor; @@ -38,8 +41,11 @@ public DispatcherRunnerFactoryImpl(DispatcherFactory dispatcherFactory) { @Override public DispatcherRunnerImpl createDispatcherRunner( - RpcService rpcService, + LeaderElectionService leaderElectionService, + FatalErrorHandler fatalErrorHandler, + JobGraphStoreFactory jobGraphStoreFactory, Executor ioExecutor, + RpcService rpcService, PartialDispatcherServices partialDispatcherServices) throws Exception { return new DispatcherRunnerImpl(dispatcherFactory, rpcService, partialDispatcherServices); } 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 9b72997cdc4c..76ea174d0a34 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 @@ -38,6 +38,7 @@ import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.jobmanager.HaServicesJobGraphStoreFactory; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; @@ -200,8 +201,11 @@ public DispatcherResourceManagerComponent create( log.debug("Starting Dispatcher."); dispatcherRunner = dispatcherRunnerFactory.createDispatcherRunner( - rpcService, + highAvailabilityServices.getDispatcherLeaderElectionService(), + fatalErrorHandler, + new HaServicesJobGraphStoreFactory(highAvailabilityServices), ioExecutor, + rpcService, partialDispatcherServices); log.debug("Starting ResourceManager."); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HaServicesJobGraphStoreFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HaServicesJobGraphStoreFactory.java new file mode 100644 index 000000000000..fcde69aaddc4 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HaServicesJobGraphStoreFactory.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.jobmanager; + +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.util.FlinkRuntimeException; + +/** + * {@link JobGraphStoreFactory} implementation which creates a {@link JobGraphStore} + * using the provided {@link HighAvailabilityServices}. + */ +public class HaServicesJobGraphStoreFactory implements JobGraphStoreFactory { + private final HighAvailabilityServices highAvailabilityServices; + + public HaServicesJobGraphStoreFactory(HighAvailabilityServices highAvailabilityServices) { + this.highAvailabilityServices = highAvailabilityServices; + } + + @Override + public JobGraphStore create() { + try { + return highAvailabilityServices.getJobGraphStore(); + } catch (Exception e) { + throw new FlinkRuntimeException( + String.format( + "Could not create %s from %s.", + JobGraphStore.class.getSimpleName(), + highAvailabilityServices.getClass().getSimpleName()), + e); + } + } +} From 5201a057d31cd7673626abc28adcdf3757e77663 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 1 Oct 2019 15:02:24 +0200 Subject: [PATCH 244/746] [hotfix] Introduce JobManagerMetricGroupFactory and use for DispatcherServices creation The JobManagerMetricGroupFactory allows to create a fresh JobManagerMetricGroup whenever we create a new Dispatcher instance. --- .../dispatcher/DispatcherServices.java | 10 +++--- .../JobManagerMetricGroupFactory.java | 32 +++++++++++++++++++ .../dispatcher/PartialDispatcherServices.java | 11 +++---- ...atcherResourceManagerComponentFactory.java | 12 +------ .../dispatcher/MiniDispatcherTest.java | 2 +- .../runtime/dispatcher/TestingDispatcher.java | 2 +- 6 files changed, 46 insertions(+), 23 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobManagerMetricGroupFactory.java 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 bdb64638a596..af43c155d9a7 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 @@ -74,22 +74,22 @@ public DispatcherServices( @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 JobManagerMetricGroup jobManagerMetricGroup, @Nonnull JobManagerRunnerFactory jobManagerRunnerFactory) { 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.jobManagerMetricGroup = jobManagerMetricGroup; this.jobManagerRunnerFactory = jobManagerRunnerFactory; } @@ -148,18 +148,20 @@ JobManagerRunnerFactory getJobManagerRunnerFactory() { return jobManagerRunnerFactory; } - public static DispatcherServices from(@Nonnull PartialDispatcherServices partialDispatcherServices, @Nonnull JobManagerRunnerFactory 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(), + partialDispatcherServices.getJobManagerMetricGroupFactory().create(), jobManagerRunnerFactory); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobManagerMetricGroupFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobManagerMetricGroupFactory.java new file mode 100644 index 000000000000..792b300020e4 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobManagerMetricGroupFactory.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.dispatcher; + +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; + +/** + * Factory interface for {@link JobManagerMetricGroup}. + */ +public interface JobManagerMetricGroupFactory { + + /** + * @return newly created {@link JobManagerMetricGroup} + */ + JobManagerMetricGroup create(); +} 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 index 3425ec71513b..89049a09b4cc 100644 --- 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 @@ -22,7 +22,6 @@ 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; @@ -52,7 +51,7 @@ public class PartialDispatcherServices { private final HeartbeatServices heartbeatServices; @Nonnull - private final JobManagerMetricGroup jobManagerMetricGroup; + private final JobManagerMetricGroupFactory jobManagerMetricGroupFactory; @Nonnull private final ArchivedExecutionGraphStore archivedExecutionGraphStore; @@ -72,7 +71,7 @@ public PartialDispatcherServices( @Nonnull GatewayRetriever resourceManagerGatewayRetriever, @Nonnull BlobServer blobServer, @Nonnull HeartbeatServices heartbeatServices, - @Nonnull JobManagerMetricGroup jobManagerMetricGroup, + @Nonnull JobManagerMetricGroupFactory jobManagerMetricGroupFactory, @Nonnull ArchivedExecutionGraphStore archivedExecutionGraphStore, @Nonnull FatalErrorHandler fatalErrorHandler, @Nonnull HistoryServerArchivist historyServerArchivist, @@ -82,7 +81,7 @@ public PartialDispatcherServices( this.resourceManagerGatewayRetriever = resourceManagerGatewayRetriever; this.blobServer = blobServer; this.heartbeatServices = heartbeatServices; - this.jobManagerMetricGroup = jobManagerMetricGroup; + this.jobManagerMetricGroupFactory = jobManagerMetricGroupFactory; this.archivedExecutionGraphStore = archivedExecutionGraphStore; this.fatalErrorHandler = fatalErrorHandler; this.historyServerArchivist = historyServerArchivist; @@ -115,8 +114,8 @@ public HeartbeatServices getHeartbeatServices() { } @Nonnull - public JobManagerMetricGroup getJobManagerMetricGroup() { - return jobManagerMetricGroup; + public JobManagerMetricGroupFactory getJobManagerMetricGroupFactory() { + return jobManagerMetricGroupFactory; } @Nonnull 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 76ea174d0a34..2b2f531fb690 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 @@ -41,7 +41,6 @@ import org.apache.flink.runtime.jobmanager.HaServicesJobGraphStoreFactory; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.metrics.MetricRegistry; -import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; import org.apache.flink.runtime.metrics.groups.ResourceManagerMetricGroup; import org.apache.flink.runtime.metrics.util.MetricUtils; import org.apache.flink.runtime.resourcemanager.ResourceManager; @@ -117,7 +116,6 @@ public DispatcherResourceManagerComponent create( LeaderRetrievalService resourceManagerRetrievalService = null; WebMonitorEndpoint webMonitorEndpoint = null; ResourceManager resourceManager = null; - JobManagerMetricGroup jobManagerMetricGroup = null; ResourceManagerMetricGroup resourceManagerMetricGroup = null; DispatcherRunner dispatcherRunner = null; @@ -183,17 +181,13 @@ public DispatcherResourceManagerComponent create( final HistoryServerArchivist historyServerArchivist = HistoryServerArchivist.createHistoryServerArchivist(configuration, webMonitorEndpoint); - jobManagerMetricGroup = MetricUtils.instantiateJobManagerMetricGroup( - metricRegistry, - hostname); - final PartialDispatcherServices partialDispatcherServices = new PartialDispatcherServices( configuration, highAvailabilityServices, resourceManagerGatewayRetriever, blobServer, heartbeatServices, - jobManagerMetricGroup, + () -> MetricUtils.instantiateJobManagerMetricGroup(metricRegistry, hostname), archivedExecutionGraphStore, fatalErrorHandler, historyServerArchivist, @@ -261,10 +255,6 @@ public DispatcherResourceManagerComponent create( exception = ExceptionUtils.firstOrSuppressed(e, exception); } - if (jobManagerMetricGroup != null) { - jobManagerMetricGroup.close(); - } - if (resourceManagerMetricGroup != null) { resourceManagerMetricGroup.close(); } 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 8f855298c3a2..7e9aab7da357 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 @@ -235,11 +235,11 @@ private MiniDispatcher createMiniDispatcher(ClusterEntrypoint.ExecutionMode exec () -> CompletableFuture.completedFuture(resourceManagerGateway), blobServer, heartbeatServices, - UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), archivedExecutionGraphStore, testingFatalErrorHandler, VoidHistoryServerArchivist.INSTANCE, null, + UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), 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 90b70532d819..b06699f2aa38 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 @@ -66,11 +66,11 @@ class TestingDispatcher extends Dispatcher { resourceManagerGatewayRetriever, blobServer, heartbeatServices, - jobManagerMetricGroup, archivedExecutionGraphStore, fatalErrorHandler, VoidHistoryServerArchivist.INSTANCE, metricQueryServiceAddress, + jobManagerMetricGroup, jobManagerRunnerFactory), highAvailabilityServices.getJobGraphStore()); From d477ae3c0d29b23cf20eea071c4df5e45b7ae890 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 21 Oct 2019 18:42:33 +0200 Subject: [PATCH 245/746] [hotfix] Stop DispatcherResourceManagerComponent if Dispatcher completes shutDownFuture in MiniCluster --- .../runtime/minicluster/MiniCluster.java | 36 +++++++++++++------ 1 file changed, 26 insertions(+), 10 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 7da4c6ddf243..0216f816f30a 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 @@ -34,6 +34,7 @@ import org.apache.flink.runtime.client.ClientUtils; import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.client.JobStatusMessage; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.dispatcher.DispatcherGateway; @@ -325,16 +326,7 @@ public void start() throws Exception { MetricQueryServiceRetriever metricQueryServiceRetriever = new RpcMetricQueryServiceRetriever(metricRegistry.getMetricQueryServiceRpcService()); - dispatcherResourceManagerComponents.addAll(createDispatcherResourceManagerComponents( - configuration, - dispatcherResourceManagreComponentRpcServiceFactory, - haServices, - blobServer, - heartbeatServices, - metricRegistry, - metricQueryServiceRetriever, - new ShutDownFatalErrorHandler() - )); + setupDispatcherResourceManagerComponents(configuration, dispatcherResourceManagreComponentRpcServiceFactory, metricQueryServiceRetriever); resourceManagerLeaderRetriever = haServices.getResourceManagerLeaderRetriever(); dispatcherLeaderRetriever = haServices.getDispatcherLeaderRetriever(); @@ -378,6 +370,30 @@ public void start() throws Exception { } } + @GuardedBy("lock") + private void setupDispatcherResourceManagerComponents(Configuration configuration, RpcServiceFactory dispatcherResourceManagreComponentRpcServiceFactory, MetricQueryServiceRetriever metricQueryServiceRetriever) throws Exception { + dispatcherResourceManagerComponents.addAll(createDispatcherResourceManagerComponents( + configuration, + dispatcherResourceManagreComponentRpcServiceFactory, + haServices, + blobServer, + heartbeatServices, + metricRegistry, + metricQueryServiceRetriever, + new ShutDownFatalErrorHandler() + )); + + final Collection> shutDownFutures = new ArrayList<>(dispatcherResourceManagerComponents.size()); + + for (DispatcherResourceManagerComponent dispatcherResourceManagerComponent : dispatcherResourceManagerComponents) { + final CompletableFuture shutDownFuture = dispatcherResourceManagerComponent.getShutDownFuture(); + FutureUtils.assertNoException(shutDownFuture.thenRun(dispatcherResourceManagerComponent::closeAsync)); + shutDownFutures.add(shutDownFuture); + } + + FutureUtils.assertNoException(FutureUtils.completeAll(shutDownFutures).thenRun(this::closeAsync)); + } + @VisibleForTesting protected Collection createDispatcherResourceManagerComponents( Configuration configuration, From 7035fbdeb0fb112f51e88f9ac6d0c9d426cb36a9 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 21 Oct 2019 18:43:14 +0200 Subject: [PATCH 246/746] [hotfix] Rework ZooKeeperLeaderElectionITCase to not rely on DispatcherResourceManagerComponent internals --- .../ZooKeeperLeaderElectionITCase.java | 122 ++---------------- 1 file changed, 12 insertions(+), 110 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java index 4d90b6e148fb..a13addcc7297 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java @@ -20,25 +20,22 @@ import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.time.Time; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.ClusterOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.dispatcher.Dispatcher; import org.apache.flink.runtime.dispatcher.runner.DispatcherRunner; import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponent; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.execution.Environment; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; 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.tasks.AbstractInvokable; import org.apache.flink.runtime.jobmaster.JobResult; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; 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.runtime.testutils.ZooKeeperTestUtils; -import org.apache.flink.runtime.webmonitor.retriever.LeaderRetriever; import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.TestLogger; @@ -50,13 +47,9 @@ import org.junit.experimental.categories.Category; import org.junit.rules.TemporaryFolder; -import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.time.Duration; -import java.util.Collection; -import java.util.Optional; -import java.util.UUID; import java.util.concurrent.CompletableFuture; import static org.hamcrest.Matchers.is; @@ -117,10 +110,9 @@ public void testJobExecutionOnClusterWithLeaderChange() throws Exception { .setNumSlotsPerTaskManager(numSlotsPerTM) .build(); - LeaderRetrievalService dispatcherLeaderRetriever = null; + Deadline timeout = Deadline.fromNow(TEST_TIMEOUT); try (TestingMiniCluster miniCluster = new TestingMiniCluster(miniClusterConfiguration)) { - Deadline timeout = Deadline.fromNow(TEST_TIMEOUT); miniCluster.start(); final int parallelism = numTMs * numSlotsPerTM; @@ -128,119 +120,29 @@ public void testJobExecutionOnClusterWithLeaderChange() throws Exception { miniCluster.submitJob(jobGraph).get(); - Collection dispatcherResourceManagerComponents = miniCluster.getDispatcherResourceManagerComponents(); - - final NewLeaderRetriever newLeaderRetriever = new NewLeaderRetriever(); - final HighAvailabilityServices highAvailabilityServices = miniCluster.getHighAvailabilityServices(); - dispatcherLeaderRetriever = highAvailabilityServices.getDispatcherLeaderRetriever(); - dispatcherLeaderRetriever.start(newLeaderRetriever); + String previousLeaderAddress = null; for (int i = 0; i < numDispatchers - 1; i++) { - final DispatcherResourceManagerComponent leadingDispatcherResourceManagerComponent = getLeadingDispatcherResourceManagerComponent( - dispatcherResourceManagerComponents, - newLeaderRetriever); - - final DispatcherRunner dispatcherRunner = leadingDispatcherResourceManagerComponent.getDispatcherRunner(); - final Dispatcher dispatcher = dispatcherRunner.getDispatcher(); + final DispatcherGateway leaderDispatcherGateway = getNextLeadingDispatcherGateway(miniCluster, previousLeaderAddress, timeout); + previousLeaderAddress = leaderDispatcherGateway.getAddress(); - assertThat(dispatcher, is(notNullValue())); + CommonTestUtils.waitUntilCondition(() -> leaderDispatcherGateway.requestJobStatus(jobGraph.getJobID(), RPC_TIMEOUT).get() == JobStatus.RUNNING, timeout, 50L); - CommonTestUtils.waitUntilCondition(() -> dispatcher.requestJobStatus(jobGraph.getJobID(), RPC_TIMEOUT).get() == JobStatus.RUNNING, timeout, 50L); - - leadingDispatcherResourceManagerComponent.closeAsync(); + leaderDispatcherGateway.shutDownCluster(); } - final DispatcherResourceManagerComponent leadingDispatcherResourceManagerComponent = getLeadingDispatcherResourceManagerComponent( - dispatcherResourceManagerComponents, - newLeaderRetriever); - - final DispatcherRunner dispatcherRunner = leadingDispatcherResourceManagerComponent.getDispatcherRunner(); - final Dispatcher dispatcher = dispatcherRunner.getDispatcher(); + final DispatcherGateway leaderDispatcherGateway = getNextLeadingDispatcherGateway(miniCluster, previousLeaderAddress, timeout); - assertThat(dispatcher, is(notNullValue())); - - CompletableFuture jobResultFuture = dispatcher.requestJobResult(jobGraph.getJobID(), RPC_TIMEOUT); + CompletableFuture jobResultFuture = leaderDispatcherGateway.requestJobResult(jobGraph.getJobID(), RPC_TIMEOUT); BlockingOperator.unblock(); assertThat(jobResultFuture.get().isSuccess(), is(true)); - } finally { - if (dispatcherLeaderRetriever != null) { - dispatcherLeaderRetriever.stop(); - } - } - } - - @Nonnull - protected DispatcherResourceManagerComponent getLeadingDispatcherResourceManagerComponent( - Collection dispatcherResourceManagerComponents, - NewLeaderRetriever newLeaderRetriever) throws Exception { - final Tuple2 leaderInformation = newLeaderRetriever.waitUntilNewLeader().get(); - - final String leaderAddress = leaderInformation.f0; - - return findLeadingDispatcherResourceManagerComponent( - dispatcherResourceManagerComponents, - leaderAddress).orElseThrow(() -> new Exception(String.format("Could not find the leading Dispatcher with address %s", leaderAddress))); - } - - @Nonnull - private static Optional findLeadingDispatcherResourceManagerComponent(Collection dispatcherResourceManagerComponents, String address) { - for (DispatcherResourceManagerComponent dispatcherResourceManagerComponent : dispatcherResourceManagerComponents) { - final DispatcherRunner dispatcherRunner = dispatcherResourceManagerComponent.getDispatcherRunner(); - final Dispatcher dispatcher = dispatcherRunner.getDispatcher(); - - if (dispatcher != null && dispatcher.getAddress().equals(address)) { - return Optional.of(dispatcherResourceManagerComponent); - } } - - return Optional.empty(); } - private static class NewLeaderRetriever extends LeaderRetriever { - - private final Object lock = new Object(); - - @Nullable - private Tuple2 lastAddress = null; - - private CompletableFuture> newLeaderFuture = new CompletableFuture<>(); - - CompletableFuture> waitUntilNewLeader() { - synchronized (lock) { - if (newLeaderFuture.isDone()) { - CompletableFuture> newLeader = newLeaderFuture; - newLeaderFuture = new CompletableFuture<>(); - - return newLeader; - } else { - return newLeaderFuture.thenApply(stringUUIDTuple2 -> { - synchronized (lock) { - newLeaderFuture = new CompletableFuture<>(); - } - return stringUUIDTuple2; - }); - } - } - } - - @Override - protected void notifyNewLeaderAddress(CompletableFuture> newLeaderAddressFuture) { - newLeaderAddressFuture.whenComplete((newLeaderAddress, throwable) -> { - synchronized (lock) { - if (throwable != null) { - newLeaderFuture.completeExceptionally(throwable); - } else if (!newLeaderAddress.equals(lastAddress)) { - lastAddress = newLeaderAddress; - if (newLeaderFuture.isDone()) { - newLeaderFuture = CompletableFuture.completedFuture(newLeaderAddress); - } else { - newLeaderFuture.complete(newLeaderAddress); - } - } - } - }); - } + private DispatcherGateway getNextLeadingDispatcherGateway(TestingMiniCluster miniCluster, @Nullable String previousLeaderAddress, Deadline timeout) throws Exception { + CommonTestUtils.waitUntilCondition(() -> !miniCluster.getDispatcherGatewayFuture().get().getAddress().equals(previousLeaderAddress), timeout, 20L); + return miniCluster.getDispatcherGatewayFuture().get(); } private JobGraph createJobGraph(int parallelism) { From 1f5540d3c9f86bf79eaf4b004577695cd4111be6 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 21 Oct 2019 18:44:08 +0200 Subject: [PATCH 247/746] [hotfix] Remove unused methods from MiniCluster --- .../flink/runtime/minicluster/MiniCluster.java | 15 --------------- .../runtime/minicluster/TestingMiniCluster.java | 6 ------ 2 files changed, 21 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 0216f816f30a..7899b540a80f 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 @@ -221,25 +221,10 @@ public ClusterInformation getClusterInformation() { } } - public HighAvailabilityServices getHighAvailabilityServices() { - synchronized (lock) { - checkState(running, "MiniCluster is not yet running."); - return haServices; - } - } - protected Executor getIOExecutor() { return ioExecutor; } - @VisibleForTesting - @Nonnull - protected Collection getDispatcherResourceManagerComponents() { - synchronized (lock) { - return Collections.unmodifiableCollection(dispatcherResourceManagerComponents); - } - } - // ------------------------------------------------------------------------ // life cycle // ------------------------------------------------------------------------ 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 44c2d947794a..2b8e32b7d485 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 @@ -67,12 +67,6 @@ public TestingMiniCluster(TestingMiniClusterConfiguration miniClusterConfigurati this(miniClusterConfiguration, null); } - @Nonnull - @Override - public Collection getDispatcherResourceManagerComponents() { - return super.getDispatcherResourceManagerComponents(); - } - @Nonnull @Override public CompletableFuture terminateTaskExecutor(int index) { From 8772b07c27598b1ac1ad259ab33bb2d939a1f264 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 26 Sep 2019 13:07:14 +0200 Subject: [PATCH 248/746] [hotfix] Improve logging in DispatcherResourceManagerComponent --- .../component/DispatcherResourceManagerComponent.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponent.java index 770e92aea3bc..f8c43f68333c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponent.java @@ -29,6 +29,9 @@ import org.apache.flink.util.AutoCloseableAsync; import org.apache.flink.util.ExceptionUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -43,6 +46,8 @@ */ public class DispatcherResourceManagerComponent implements AutoCloseableAsync { + private static final Logger LOG = LoggerFactory.getLogger(DispatcherResourceManagerComponent.class); + @Nonnull private final DispatcherRunner dispatcherRunner; @@ -130,6 +135,8 @@ private CompletableFuture deregisterApplication( } private CompletableFuture closeAsyncInternal() { + LOG.info("Closing components."); + Exception exception = null; final Collection> terminationFutures = new ArrayList<>(3); From 89e9b6f000d07fe40420680b8b03fa6ce4076491 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 6 May 2019 09:43:45 +0200 Subject: [PATCH 249/746] [FLINK-11665] Add DispatcherRunnerImplTest for FLINK-11665 --- .../dispatcher/runner/DispatcherRunner.java | 8 +- .../runner/DispatcherRunnerImpl.java | 35 ++- .../ZooKeeperDispatcherRunnerImplTest.java | 218 ++++++++++++++++++ .../ZooKeeperLeaderElectionITCase.java | 4 - 4 files changed, 255 insertions(+), 10 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDispatcherRunnerImplTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunner.java index 2157564cbcc0..c049ecbc8e52 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunner.java @@ -20,6 +20,7 @@ import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.dispatcher.Dispatcher; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.util.AutoCloseableAsync; import java.util.concurrent.CompletableFuture; @@ -30,11 +31,12 @@ public interface DispatcherRunner extends AutoCloseableAsync { /** - * Get the currently running {@link Dispatcher}. + * Return a future which is completed once the {@link Dispatcher} gains + * leadership. * - * @return the currently running dispatcher + * @return Future which is completed with the leader's gateway */ - Dispatcher getDispatcher(); + CompletableFuture getDispatcherGateway(); /** * Return shut down future of this runner. The shut down future is being diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java index a5f7b6a58335..8c6a149a1df4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java @@ -19,11 +19,17 @@ package org.apache.flink.runtime.dispatcher.runner; import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.dispatcher.Dispatcher; import org.apache.flink.runtime.dispatcher.DispatcherFactory; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.webmonitor.retriever.LeaderRetriever; +import java.util.ArrayList; +import java.util.Collection; import java.util.concurrent.CompletableFuture; /** @@ -33,6 +39,10 @@ class DispatcherRunnerImpl implements DispatcherRunner { private final Dispatcher dispatcher; + private final LeaderRetrievalService leaderRetrievalService; + + private final LeaderRetriever leaderRetriever; + DispatcherRunnerImpl( DispatcherFactory dispatcherFactory, RpcService rpcService, @@ -40,18 +50,37 @@ class DispatcherRunnerImpl implements DispatcherRunner { this.dispatcher = dispatcherFactory.createDispatcher( rpcService, partialDispatcherServices); + this.leaderRetrievalService = partialDispatcherServices.getHighAvailabilityServices().getDispatcherLeaderRetriever(); + this.leaderRetriever = new LeaderRetriever(); + leaderRetrievalService.start(leaderRetriever); dispatcher.start(); } @Override - public Dispatcher getDispatcher() { - return dispatcher; + public CompletableFuture getDispatcherGateway() { + return leaderRetriever.getLeaderFuture().thenApply(ignored -> dispatcher.getSelfGateway(DispatcherGateway.class)); } @Override public CompletableFuture closeAsync() { - return dispatcher.closeAsync(); + Exception exception = null; + + try { + leaderRetrievalService.stop(); + } catch (Exception e) { + exception = e; + } + + Collection> terminationFutures = new ArrayList<>(); + + terminationFutures.add(dispatcher.closeAsync()); + + if (exception != null) { + terminationFutures.add(FutureUtils.completedExceptionally(exception)); + } + + return FutureUtils.completeAll(terminationFutures); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDispatcherRunnerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDispatcherRunnerImplTest.java new file mode 100644 index 000000000000..7199a38089b4 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDispatcherRunnerImplTest.java @@ -0,0 +1,218 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.runner; + +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.HighAvailabilityOptions; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.blob.BlobUtils; +import org.apache.flink.runtime.blob.PermanentBlobKey; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; +import org.apache.flink.runtime.dispatcher.MemoryArchivedExecutionGraphStore; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; +import org.apache.flink.runtime.dispatcher.SessionDispatcherFactory; +import org.apache.flink.runtime.dispatcher.VoidHistoryServerArchivist; +import org.apache.flink.runtime.heartbeat.TestingHeartbeatServices; +import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; +import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; +import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServicesBuilder; +import org.apache.flink.runtime.highavailability.zookeeper.ZooKeeperRunningJobsRegistry; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobmanager.ZooKeeperJobGraphStore; +import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; +import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; +import org.apache.flink.runtime.rpc.TestingRpcService; +import org.apache.flink.runtime.rpc.TestingRpcServiceResource; +import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.runtime.util.TestingFatalErrorHandler; +import org.apache.flink.runtime.util.ZooKeeperUtils; +import org.apache.flink.runtime.zookeeper.ZooKeeperResource; +import org.apache.flink.util.TestLogger; + +import org.apache.curator.framework.CuratorFramework; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Ignore; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.time.Duration; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; + +import static org.hamcrest.Matchers.emptyArray; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + +/** + * Tests for the interaction between the {@link DispatcherRunnerImpl} and ZooKeeper. + */ +public class ZooKeeperDispatcherRunnerImplTest extends TestLogger { + + private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperDispatcherRunnerImplTest.class); + + private static final Time TESTING_TIMEOUT = Time.seconds(10L); + + private static final Duration VERIFICATION_TIMEOUT = Duration.ofSeconds(10L); + + @ClassRule + public static ZooKeeperResource zooKeeperResource = new ZooKeeperResource(); + + @ClassRule + public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @ClassRule + public static TestingRpcServiceResource testingRpcServiceResource = new TestingRpcServiceResource(); + + private BlobServer blobServer; + + private TestingFatalErrorHandler fatalErrorHandler; + + private File clusterHaStorageDir; + + private Configuration configuration; + + @Before + public void setup() throws IOException { + fatalErrorHandler = new TestingFatalErrorHandler(); + configuration = new Configuration(); + configuration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper"); + configuration.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zooKeeperResource.getConnectString()); + configuration.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.newFolder().getAbsolutePath()); + + clusterHaStorageDir = new File(HighAvailabilityServicesUtils.getClusterHighAvailableStoragePath(configuration).toString()); + blobServer = new BlobServer(configuration, BlobUtils.createBlobStoreFromConfig(configuration)); + } + + @After + public void teardown() throws Exception { + if (blobServer != null) { + blobServer.close(); + } + + if (fatalErrorHandler != null) { + fatalErrorHandler.rethrowError(); + } + } + + /** + * See FLINK-11665. + */ + @Test + @Ignore + public void testResourceCleanupUnderLeadershipChange() throws Exception { + final TestingRpcService rpcService = testingRpcServiceResource.getTestingRpcService(); + final TestingLeaderElectionService dispatcherLeaderElectionService = new TestingLeaderElectionService(); + final SettableLeaderRetrievalService dispatcherLeaderRetriever = new SettableLeaderRetrievalService(); + + final CuratorFramework client = ZooKeeperUtils.startCuratorFramework(configuration); + try (final TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServicesBuilder() + .setJobGraphStore(ZooKeeperUtils.createJobGraphs(client, configuration)) + .setRunningJobsRegistry(new ZooKeeperRunningJobsRegistry(client, configuration)) + .setDispatcherLeaderElectionService(dispatcherLeaderElectionService) + .setDispatcherLeaderRetriever(dispatcherLeaderRetriever) + .setJobMasterLeaderRetrieverFunction(jobId -> ZooKeeperUtils.createLeaderRetrievalService(client, configuration)) + .build()) { + + final PartialDispatcherServices partialDispatcherServices = new PartialDispatcherServices( + configuration, + highAvailabilityServices, + () -> new CompletableFuture<>(), + blobServer, + new TestingHeartbeatServices(), + UnregisteredMetricGroups::createUnregisteredJobManagerMetricGroup, + new MemoryArchivedExecutionGraphStore(), + fatalErrorHandler, + VoidHistoryServerArchivist.INSTANCE, + null); + + final JobGraph jobGraph = createJobGraphWithBlobs(); + + try (final DispatcherRunnerImpl dispatcherRunner = new DispatcherRunnerImpl(SessionDispatcherFactory.INSTANCE, rpcService, partialDispatcherServices)) { + // initial run + DispatcherGateway dispatcherGateway = grantLeadership(dispatcherLeaderElectionService, dispatcherLeaderRetriever, dispatcherRunner); + + LOG.info("Initial job submission {}.", jobGraph.getJobID()); + dispatcherGateway.submitJob(jobGraph, TESTING_TIMEOUT).get(); + + dispatcherLeaderElectionService.notLeader(); + + // recovering submitted jobs + LOG.info("Re-grant leadership first time."); + dispatcherGateway = grantLeadership(dispatcherLeaderElectionService, dispatcherLeaderRetriever, dispatcherRunner); + + LOG.info("Cancel recovered job {}.", jobGraph.getJobID()); + // cancellation of the job should remove everything + final CompletableFuture jobResultFuture = dispatcherGateway.requestJobResult(jobGraph.getJobID(), TESTING_TIMEOUT); + dispatcherGateway.cancelJob(jobGraph.getJobID(), TESTING_TIMEOUT).get(); + + // a successful cancellation should eventually remove all job information + final JobResult jobResult = jobResultFuture.get(); + + assertThat(jobResult.getApplicationStatus(), is(ApplicationStatus.CANCELED)); + + dispatcherLeaderElectionService.notLeader(); + + // check that the job has been removed from ZooKeeper + final ZooKeeperJobGraphStore submittedJobGraphStore = ZooKeeperUtils.createJobGraphs(client, configuration); + + CommonTestUtils.waitUntilCondition(() -> submittedJobGraphStore.getJobIds().isEmpty(), Deadline.fromNow(VERIFICATION_TIMEOUT), 20L); + } + } + + // check resource clean up + assertThat(clusterHaStorageDir.listFiles(), is(emptyArray())); + } + + private DispatcherGateway grantLeadership(TestingLeaderElectionService dispatcherLeaderElectionService, SettableLeaderRetrievalService dispatcherLeaderRetriever, DispatcherRunnerImpl dispatcherRunner) throws InterruptedException, java.util.concurrent.ExecutionException { + final UUID leaderSessionId = UUID.randomUUID(); + dispatcherLeaderElectionService.isLeader(leaderSessionId).get(); + // TODO: Remove once runner properly works + dispatcherLeaderRetriever.notifyListener("foobar", leaderSessionId); + + return dispatcherRunner.getDispatcherGateway().get(); + } + + private JobGraph createJobGraphWithBlobs() throws IOException { + final JobVertex vertex = new JobVertex("test vertex"); + vertex.setInvokableClass(NoOpInvokable.class); + vertex.setParallelism(1); + + final JobGraph jobGraph = new JobGraph("Test job graph", vertex); + jobGraph.setAllowQueuedScheduling(true); + + final PermanentBlobKey permanentBlobKey = blobServer.putPermanent(jobGraph.getJobID(), new byte[256]); + jobGraph.addUserJarBlobKey(permanentBlobKey); + + return jobGraph; + } + +} diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java index a13addcc7297..a35a3175116e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java @@ -22,9 +22,6 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.ClusterOptions; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.dispatcher.Dispatcher; -import org.apache.flink.runtime.dispatcher.runner.DispatcherRunner; -import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponent; import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -53,7 +50,6 @@ import java.util.concurrent.CompletableFuture; import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.notNullValue; import static org.junit.Assert.assertThat; /** From 431e2a0423103e747a95f9e2196e3aa8000cb198 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 6 May 2019 17:48:35 +0200 Subject: [PATCH 250/746] [FLINK-11843] Add test case for FLINK-11843 DispatcherRunnerImplTest#testJobRecoveryUnderLeaderChange fails only occasionally. --- .../TestingJobManagerRunnerFactory.java | 2 +- .../runner/DispatcherRunnerImplTest.java | 176 ++++++++++++++++++ 2 files changed, 177 insertions(+), 1 deletion(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java 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 c6f34d726414..1f3216bb6bfe 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 @@ -36,7 +36,7 @@ /** * Testing implementation of {@link JobManagerRunnerFactory} which returns a {@link TestingJobManagerRunner}. */ -class TestingJobManagerRunnerFactory implements JobManagerRunnerFactory { +public class TestingJobManagerRunnerFactory implements JobManagerRunnerFactory { private final BlockingQueue createdJobManagerRunner = new ArrayBlockingQueue<>(16); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java new file mode 100644 index 000000000000..9703781b7276 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.runner; + +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.blob.VoidBlobStore; +import org.apache.flink.runtime.dispatcher.Dispatcher; +import org.apache.flink.runtime.dispatcher.DispatcherFactory; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; +import org.apache.flink.runtime.dispatcher.DispatcherServices; +import org.apache.flink.runtime.dispatcher.JobManagerRunnerFactory; +import org.apache.flink.runtime.dispatcher.MemoryArchivedExecutionGraphStore; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; +import org.apache.flink.runtime.dispatcher.SingleJobJobGraphStore; +import org.apache.flink.runtime.dispatcher.StandaloneDispatcher; +import org.apache.flink.runtime.dispatcher.TestingJobManagerRunnerFactory; +import org.apache.flink.runtime.dispatcher.VoidHistoryServerArchivist; +import org.apache.flink.runtime.heartbeat.TestingHeartbeatServices; +import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; +import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServicesBuilder; +import org.apache.flink.runtime.jobgraph.JobGraph; +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.rpc.RpcService; +import org.apache.flink.runtime.rpc.TestingRpcService; +import org.apache.flink.runtime.rpc.TestingRpcServiceResource; +import org.apache.flink.runtime.util.TestingFatalErrorHandler; +import org.apache.flink.util.TestLogger; + +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Ignore; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; + +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + +/** + * Tests for the {@link DispatcherRunnerImpl}. + */ +public class DispatcherRunnerImplTest extends TestLogger { + + private static final Logger LOG = LoggerFactory.getLogger(DispatcherRunnerImplTest.class); + + private static final Time TESTING_TIMEOUT = Time.seconds(10L); + + @ClassRule + public static TestingRpcServiceResource testingRpcServiceResource = new TestingRpcServiceResource(); + + private TestingFatalErrorHandler fatalErrorHandler; + + @Before + public void setup() { + fatalErrorHandler = new TestingFatalErrorHandler(); + } + + @After + public void teardown() throws Exception { + if (fatalErrorHandler != null) { + fatalErrorHandler.rethrowError(); + } + } + + /** + * See FLINK-11843. This is a probabilistic test which needs to be executed several times to fail. + */ + @Test + @Ignore + public void testJobRecoveryUnderLeaderChange() throws Exception { + final TestingRpcService rpcService = testingRpcServiceResource.getTestingRpcService(); + final TestingLeaderElectionService dispatcherLeaderElectionService = new TestingLeaderElectionService(); + final Configuration configuration = new Configuration(); + + final JobGraph jobGraph = new JobGraph(); + final SingleJobJobGraphStore submittedJobGraphStore = new SingleJobJobGraphStore(jobGraph); + + try (final BlobServer blobServer = new BlobServer(configuration, new VoidBlobStore()); + final TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServicesBuilder() + .setJobGraphStore(submittedJobGraphStore) + .setDispatcherLeaderElectionService(dispatcherLeaderElectionService) + .build()) { + + final PartialDispatcherServices partialDispatcherServices = new PartialDispatcherServices( + configuration, + highAvailabilityServices, + CompletableFuture::new, + blobServer, + new TestingHeartbeatServices(), + UnregisteredMetricGroups::createUnregisteredJobManagerMetricGroup, + new MemoryArchivedExecutionGraphStore(), + fatalErrorHandler, + VoidHistoryServerArchivist.INSTANCE, + null); + + final TestingJobManagerRunnerFactory jobManagerRunnerFactory = new TestingJobManagerRunnerFactory(1); + try (final DispatcherRunnerImpl dispatcherRunner = new DispatcherRunnerImpl( + new TestingDispatcherFactory(jobManagerRunnerFactory), + rpcService, + partialDispatcherServices)) { + // initial run + grantLeadership(dispatcherLeaderElectionService, dispatcherRunner); + final TestingJobManagerRunner testingJobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); + + dispatcherLeaderElectionService.notLeader(); + + LOG.info("Re-grant leadership first time."); + dispatcherLeaderElectionService.isLeader(UUID.randomUUID()); + + // give the Dispatcher some time to recover jobs + Thread.sleep(1L); + + dispatcherLeaderElectionService.notLeader(); + + LOG.info("Re-grant leadership second time."); + final UUID leaderSessionId = UUID.randomUUID(); + final CompletableFuture leaderFuture = dispatcherLeaderElectionService.isLeader(leaderSessionId); + + LOG.info("Complete the termination of the first job manager runner."); + testingJobManagerRunner.completeTerminationFuture(); + + assertThat(leaderFuture.get(TESTING_TIMEOUT.toMilliseconds(), TimeUnit.MILLISECONDS), is(equalTo(leaderSessionId))); + } + } + } + + private DispatcherGateway grantLeadership( + TestingLeaderElectionService dispatcherLeaderElectionService, + DispatcherRunner dispatcherRunner) throws InterruptedException, java.util.concurrent.ExecutionException { + final UUID leaderSessionId = UUID.randomUUID(); + dispatcherLeaderElectionService.isLeader(leaderSessionId).get(); + + return dispatcherRunner.getDispatcherGateway().get(); + } + + private static class TestingDispatcherFactory implements DispatcherFactory { + private final JobManagerRunnerFactory jobManagerRunnerFactory; + + private TestingDispatcherFactory(JobManagerRunnerFactory jobManagerRunnerFactory) { + this.jobManagerRunnerFactory = jobManagerRunnerFactory; + } + + @Override + public Dispatcher createDispatcher(@Nonnull RpcService rpcService, @Nonnull PartialDispatcherServices partialDispatcherServices) throws Exception { + return new StandaloneDispatcher(rpcService, getEndpointId(), DispatcherServices.from(partialDispatcherServices, jobManagerRunnerFactory)); + } + } +} From 637a6e9eaeed504131d7df7ff96ae346b1be4849 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 12 Sep 2019 16:53:43 +0200 Subject: [PATCH 251/746] [FLINK-11843] Add DispatcherRunnerImplNG with DispatcherLeaderProcess abstraction --- .../runner/DispatcherLeaderProcess.java | 40 +++ .../DispatcherLeaderProcessFactory.java | 29 ++ .../runner/DispatcherRunnerImplNG.java | 208 ++++++++++++ .../StoppedDispatcherLeaderProcess.java | 61 ++++ .../runner/DispatcherRunnerImplNGTest.java | 302 ++++++++++++++++++ .../TestingDispatcherLeaderProcess.java | 128 ++++++++ ...TestingDispatcherLeaderProcessFactory.java | 54 ++++ 7 files changed, 822 insertions(+) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcess.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessFactory.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNG.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StoppedDispatcherLeaderProcess.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherLeaderProcess.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherLeaderProcessFactory.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcess.java new file mode 100644 index 000000000000..6d5c29bddb43 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcess.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.runtime.dispatcher.runner; + +import org.apache.flink.runtime.dispatcher.Dispatcher; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; +import org.apache.flink.util.AutoCloseableAsync; + +import java.util.UUID; +import java.util.concurrent.CompletableFuture; + +/** + * Leader process which encapsulates the lifecycle of the {@link Dispatcher} component. + */ +interface DispatcherLeaderProcess extends AutoCloseableAsync { + + void start(); + + UUID getLeaderSessionId(); + + CompletableFuture getDispatcherGateway(); + + CompletableFuture getConfirmLeaderSessionFuture(); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessFactory.java new file mode 100644 index 000000000000..d3bd700024ef --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessFactory.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.dispatcher.runner; + +import java.util.UUID; + +/** + * Factory for the {@link DispatcherLeaderProcess}. + */ +public interface DispatcherLeaderProcessFactory { + + DispatcherLeaderProcess create(UUID leaderSessionID); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNG.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNG.java new file mode 100644 index 000000000000..16f79657d49d --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNG.java @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.runner; + +import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; +import org.apache.flink.runtime.leaderelection.LeaderContender; +import org.apache.flink.runtime.leaderelection.LeaderElectionService; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.util.FlinkException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; + +/** + * Runner for the {@link org.apache.flink.runtime.dispatcher.Dispatcher} which is responsible for the + * leader election. + */ +public class DispatcherRunnerImplNG implements DispatcherRunner, LeaderContender { + + private static final Logger LOG = LoggerFactory.getLogger(DispatcherRunnerImplNG.class); + + private final Object lock = new Object(); + + private final LeaderElectionService leaderElectionService; + + private final FatalErrorHandler fatalErrorHandler; + + private final DispatcherLeaderProcessFactory dispatcherLeaderProcessFactory; + + private final CompletableFuture terminationFuture; + + private final CompletableFuture shutDownFuture; + + private boolean isRunning; + + private DispatcherLeaderProcess dispatcherLeaderProcess; + + private CompletableFuture previousDispatcherLeaderProcessTerminationFuture; + + private CompletableFuture dispatcherGatewayFuture; + + DispatcherRunnerImplNG( + LeaderElectionService leaderElectionService, + FatalErrorHandler fatalErrorHandler, + DispatcherLeaderProcessFactory dispatcherLeaderProcessFactory) throws Exception { + this.leaderElectionService = leaderElectionService; + this.fatalErrorHandler = fatalErrorHandler; + this.dispatcherLeaderProcessFactory = dispatcherLeaderProcessFactory; + this.terminationFuture = new CompletableFuture<>(); + this.shutDownFuture = new CompletableFuture<>(); + + this.isRunning = true; + this.dispatcherLeaderProcess = StoppedDispatcherLeaderProcess.INSTANCE; + this.previousDispatcherLeaderProcessTerminationFuture = CompletableFuture.completedFuture(null); + this.dispatcherGatewayFuture = new CompletableFuture<>(); + + startDispatcherRunner(leaderElectionService); + } + + private void startDispatcherRunner(LeaderElectionService leaderElectionService) throws Exception { + LOG.info("Starting {}.", getClass().getName()); + + leaderElectionService.start(this); + } + + @Override + public CompletableFuture getDispatcherGateway() { + synchronized (lock) { + return dispatcherGatewayFuture; + } + } + + @Override + public CompletableFuture getShutDownFuture() { + return shutDownFuture; + } + + @Override + public CompletableFuture closeAsync() { + synchronized (lock) { + if (!isRunning) { + return terminationFuture; + } else { + isRunning = false; + } + } + + stopDispatcherLeaderProcess(); + final CompletableFuture servicesTerminationFuture = stopServices(); + + FutureUtils.forward( + FutureUtils.completeAll( + Arrays.asList( + previousDispatcherLeaderProcessTerminationFuture, + servicesTerminationFuture)), + terminationFuture); + + return terminationFuture; + } + + private CompletableFuture stopServices() { + Exception exception = null; + + try { + leaderElectionService.stop(); + } catch (Exception e) { + exception = e; + } + if (exception == null) { + return CompletableFuture.completedFuture(null); + } else { + return FutureUtils.completedExceptionally(exception); + } + } + + // --------------------------------------------------------------- + // Leader election + // --------------------------------------------------------------- + + @Override + public void grantLeadership(UUID leaderSessionID) { + runActionIfRunning(() -> startNewDispatcherLeaderProcess(leaderSessionID)); + } + + private void startNewDispatcherLeaderProcess(UUID leaderSessionID) { + stopDispatcherLeaderProcess(); + + createAndAssignNewDispatcherLeaderProcess(leaderSessionID); + + final DispatcherLeaderProcess newDispatcherLeaderProcess = dispatcherLeaderProcess; + FutureUtils.assertNoException( + previousDispatcherLeaderProcessTerminationFuture.thenRun(newDispatcherLeaderProcess::start)); + } + + private void createAndAssignNewDispatcherLeaderProcess(UUID leaderSessionID) { + dispatcherLeaderProcess = createNewDispatcherLeaderProcess(leaderSessionID); + + final CompletableFuture newDispatcherGatewayFuture = dispatcherLeaderProcess.getDispatcherGateway(); + FutureUtils.forward(newDispatcherGatewayFuture, dispatcherGatewayFuture); + dispatcherGatewayFuture = newDispatcherGatewayFuture; + + FutureUtils.assertNoException( + dispatcherLeaderProcess.getConfirmLeaderSessionFuture().thenAccept( + leaderAddress -> { + if (leaderElectionService.hasLeadership(leaderSessionID)) { + leaderElectionService.confirmLeadership(leaderSessionID, leaderAddress); + } + })); + } + + private void stopDispatcherLeaderProcess() { + final CompletableFuture terminationFuture = dispatcherLeaderProcess.closeAsync(); + previousDispatcherLeaderProcessTerminationFuture = FutureUtils.completeAll( + Arrays.asList( + previousDispatcherLeaderProcessTerminationFuture, + terminationFuture)); + } + + private DispatcherLeaderProcess createNewDispatcherLeaderProcess(UUID leaderSessionID) { + LOG.debug("Create new {} with leader session id {}.", DispatcherLeaderProcess.class.getSimpleName(), leaderSessionID); + return dispatcherLeaderProcessFactory.create(leaderSessionID); + } + + @Override + public void revokeLeadership() { + runActionIfRunning(this::stopDispatcherLeaderProcess); + } + + private void runActionIfRunning(Runnable runnable) { + synchronized (lock) { + if (isRunning) { + runnable.run(); + } else { + LOG.debug("Ignoring action because {} has already been stopped.", getClass().getSimpleName()); + } + } + } + + @Override + public void handleError(Exception exception) { + fatalErrorHandler.onFatalError( + new FlinkException( + String.format("Exception during leader election of %s occurred.", getClass().getSimpleName()), + exception)); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StoppedDispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StoppedDispatcherLeaderProcess.java new file mode 100644 index 000000000000..d63102b965fe --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StoppedDispatcherLeaderProcess.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.runner; + +import org.apache.flink.runtime.dispatcher.DispatcherGateway; + +import java.util.UUID; +import java.util.concurrent.CompletableFuture; + +/** + * {@link DispatcherLeaderProcess} implementation which is stopped. This class + * is useful as the initial state of the {@link DispatcherRunnerImplNG}. + */ +public enum StoppedDispatcherLeaderProcess implements DispatcherLeaderProcess { + INSTANCE; + + private static final CompletableFuture TERMINATION_FUTURE = CompletableFuture.completedFuture(null); + private static final UUID LEADER_SESSION_ID = new UUID(0L, 0L); + private static final CompletableFuture NEVER_COMPLETED_FUTURE = new CompletableFuture<>(); + + @Override + public void start() { + + } + + @Override + public UUID getLeaderSessionId() { + return LEADER_SESSION_ID; + } + + @Override + public CompletableFuture getDispatcherGateway() { + return null; + } + + @Override + public CompletableFuture getConfirmLeaderSessionFuture() { + return NEVER_COMPLETED_FUTURE; + } + + @Override + public CompletableFuture closeAsync() { + return TERMINATION_FUTURE; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGTest.java new file mode 100644 index 000000000000..e6eb8002b9db --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGTest.java @@ -0,0 +1,302 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.runner; + +import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; +import org.apache.flink.runtime.dispatcher.DispatcherId; +import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; +import org.apache.flink.runtime.util.LeaderConnectionInfo; +import org.apache.flink.runtime.util.TestingFatalErrorHandler; +import org.apache.flink.runtime.webmonitor.TestingDispatcherGateway; +import org.apache.flink.util.TestLogger; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.fail; + +/** + * Tests for the {@link DispatcherRunnerImplNG}. + */ +public class DispatcherRunnerImplNGTest extends TestLogger { + + private TestingLeaderElectionService testingLeaderElectionService; + private TestingFatalErrorHandler testingFatalErrorHandler; + private TestingDispatcherLeaderProcessFactory testingDispatcherLeaderProcessFactory; + + @Before + public void setup() { + testingLeaderElectionService = new TestingLeaderElectionService(); + testingFatalErrorHandler = new TestingFatalErrorHandler(); + testingDispatcherLeaderProcessFactory = TestingDispatcherLeaderProcessFactory.defaultValue(); + } + + @After + public void teardown() throws Exception { + if (testingLeaderElectionService != null) { + testingLeaderElectionService.stop(); + testingLeaderElectionService = null; + } + + if (testingFatalErrorHandler != null) { + testingFatalErrorHandler.rethrowError(); + testingFatalErrorHandler = null; + } + } + + @Test + public void closeAsync_withUncompletedShutDownFuture_completesShutDownFuture() throws Exception { + final DispatcherRunnerImplNG dispatcherRunner = createDispatcherRunner(); + + final CompletableFuture terminationFuture = dispatcherRunner.closeAsync(); + terminationFuture.get(); + + final CompletableFuture shutDownFuture = dispatcherRunner.getShutDownFuture(); + assertThat(shutDownFuture.isDone(), is(true)); + assertThat(shutDownFuture.get(), is(ApplicationStatus.UNKNOWN)); + } + + @Test + public void getDispatcherGateway_beforeDispatcherLeaderProcessCompletes_returnsDispatcherGateway() throws Exception { + final UUID leaderSessionId = UUID.randomUUID(); + final TestingDispatcherGateway expectedDispatcherGateway = createDispatcherGateway(leaderSessionId); + final TestingDispatcherLeaderProcess testingDispatcherLeaderProcess = TestingDispatcherLeaderProcess.newBuilder(leaderSessionId) + .setDispatcherGatewayFuture(CompletableFuture.completedFuture(expectedDispatcherGateway)) + .build(); + + testingDispatcherLeaderProcessFactory = TestingDispatcherLeaderProcessFactory.from(testingDispatcherLeaderProcess); + try (final DispatcherRunnerImplNG dispatcherRunner = createDispatcherRunner()) { + + final CompletableFuture dispatcherGatewayFuture = dispatcherRunner.getDispatcherGateway(); + + assertThat(dispatcherGatewayFuture.isDone(), is(false)); + + testingLeaderElectionService.isLeader(leaderSessionId); + + assertThat(dispatcherGatewayFuture.get(), is(expectedDispatcherGateway)); + } + } + + @Test + public void getDispatcherGateway_withChangingLeaders_returnsLeadingDispatcherGateway() throws Exception { + final UUID firstLeaderSessionId = UUID.randomUUID(); + final UUID secondLeaderSessionId = UUID.randomUUID(); + final TestingDispatcherGateway firstDispatcherGateway = createDispatcherGateway(firstLeaderSessionId); + final TestingDispatcherGateway secondDispatcherGateway = createDispatcherGateway(secondLeaderSessionId); + + final TestingDispatcherLeaderProcess firstDispatcherLeaderProcess = TestingDispatcherLeaderProcess.newBuilder(firstLeaderSessionId) + .setDispatcherGatewayFuture(CompletableFuture.completedFuture(firstDispatcherGateway)) + .build(); + final TestingDispatcherLeaderProcess secondDispatcherLeaderProcess = TestingDispatcherLeaderProcess.newBuilder(secondLeaderSessionId) + .setDispatcherGatewayFuture(CompletableFuture.completedFuture(secondDispatcherGateway)) + .build(); + + testingDispatcherLeaderProcessFactory = TestingDispatcherLeaderProcessFactory.from( + firstDispatcherLeaderProcess, + secondDispatcherLeaderProcess); + + try (final DispatcherRunnerImplNG dispatcherRunner = createDispatcherRunner()) { + testingLeaderElectionService.isLeader(firstLeaderSessionId); + + final CompletableFuture firstDispatcherGatewayFuture = dispatcherRunner.getDispatcherGateway(); + + testingLeaderElectionService.notLeader(); + testingLeaderElectionService.isLeader(secondLeaderSessionId); + + final CompletableFuture secondDispatcherGatewayFuture = dispatcherRunner.getDispatcherGateway(); + + assertThat(firstDispatcherGatewayFuture.get(), is(firstDispatcherGateway)); + assertThat(secondDispatcherGatewayFuture.get(), is(secondDispatcherGateway)); + } + } + + @Test + public void grantLeadership_withExistingLeader_waitsForTerminationOfFirstLeader() throws Exception { + final UUID firstLeaderSessionId = UUID.randomUUID(); + final UUID secondLeaderSessionId = UUID.randomUUID(); + + final StartStopTestingDispatcherLeaderProcess firstTestingDispatcherLeaderProcess = StartStopTestingDispatcherLeaderProcess.create(firstLeaderSessionId); + final StartStopTestingDispatcherLeaderProcess secondTestingDispatcherLeaderProcess = StartStopTestingDispatcherLeaderProcess.create(secondLeaderSessionId); + + testingDispatcherLeaderProcessFactory = TestingDispatcherLeaderProcessFactory.from( + firstTestingDispatcherLeaderProcess.asTestingDispatcherLeaderProcess(), + secondTestingDispatcherLeaderProcess.asTestingDispatcherLeaderProcess()); + + try (final DispatcherRunnerImplNG dispatcherRunner = createDispatcherRunner()) { + testingLeaderElectionService.isLeader(firstLeaderSessionId); + + assertThat(firstTestingDispatcherLeaderProcess.isStarted(), is(true)); + + testingLeaderElectionService.isLeader(secondLeaderSessionId); + + assertThat(secondTestingDispatcherLeaderProcess.isStarted(), is(false)); + firstTestingDispatcherLeaderProcess.terminateProcess(); + assertThat(secondTestingDispatcherLeaderProcess.isStarted(), is(true)); + secondTestingDispatcherLeaderProcess.terminateProcess(); // make the dispatcherRunner terminate + } + } + + @Test + public void grantLeadership_validLeader_confirmsLeaderSession() throws Exception { + final UUID leaderSessionId = UUID.randomUUID(); + + try (final DispatcherRunnerImplNG dispatcherRunner = createDispatcherRunner()) { + testingLeaderElectionService.isLeader(leaderSessionId); + + final CompletableFuture confirmationFuture = testingLeaderElectionService.getConfirmationFuture(); + + final LeaderConnectionInfo leaderConnectionInfo = confirmationFuture.get(); + assertThat(leaderConnectionInfo.getLeaderSessionId(), is(leaderSessionId)); + } + } + + @Test + public void grantLeadership_oldLeader_doesNotConfirmLeaderSession() throws Exception { + final UUID leaderSessionId = UUID.randomUUID(); + final CompletableFuture contenderConfirmationFuture = new CompletableFuture<>(); + final TestingDispatcherLeaderProcess testingDispatcherLeaderProcess = TestingDispatcherLeaderProcess.newBuilder(leaderSessionId) + .setConfirmLeaderSessionFuture(contenderConfirmationFuture) + .build(); + + testingDispatcherLeaderProcessFactory = TestingDispatcherLeaderProcessFactory.from(testingDispatcherLeaderProcess); + + try (final DispatcherRunnerImplNG dispatcherRunner = createDispatcherRunner()) { + testingLeaderElectionService.isLeader(leaderSessionId); + + testingLeaderElectionService.notLeader(); + + // complete the confirmation future after losing the leadership + contenderConfirmationFuture.complete("leader address"); + + final CompletableFuture leaderElectionConfirmationFuture = testingLeaderElectionService.getConfirmationFuture(); + + try { + leaderElectionConfirmationFuture.get(5L, TimeUnit.MILLISECONDS); + fail("No valid leader should exist."); + } catch (TimeoutException expected) {} + } + } + + @Test + public void grantLeadership_multipleLeaderChanges_lastDispatcherLeaderProcessWaitsForOthersToTerminateBeforeItStarts() throws Exception { + final UUID firstLeaderSession = UUID.randomUUID(); + final UUID secondLeaderSession = UUID.randomUUID(); + final UUID thirdLeaderSession = UUID.randomUUID(); + + final CompletableFuture firstDispatcherLeaderProcessTerminationFuture = new CompletableFuture<>(); + final TestingDispatcherLeaderProcess firstDispatcherLeaderProcess = TestingDispatcherLeaderProcess.newBuilder(firstLeaderSession) + .setCloseAsyncSupplier(() -> firstDispatcherLeaderProcessTerminationFuture) + .build(); + final CompletableFuture secondDispatcherLeaderProcessTerminationFuture = new CompletableFuture<>(); + final TestingDispatcherLeaderProcess secondDispatcherLeaderProcess = TestingDispatcherLeaderProcess.newBuilder(secondLeaderSession) + .setCloseAsyncSupplier(() -> secondDispatcherLeaderProcessTerminationFuture) + .build(); + final CompletableFuture thirdDispatcherLeaderProcessHasBeenStartedFuture = new CompletableFuture<>(); + final TestingDispatcherLeaderProcess thirdDispatcherLeaderProcess = TestingDispatcherLeaderProcess.newBuilder(thirdLeaderSession) + .setStartConsumer(thirdDispatcherLeaderProcessHasBeenStartedFuture::complete) + .build(); + + testingDispatcherLeaderProcessFactory = TestingDispatcherLeaderProcessFactory.from( + firstDispatcherLeaderProcess, + secondDispatcherLeaderProcess, + thirdDispatcherLeaderProcess); + + final DispatcherRunnerImplNG dispatcherRunner = createDispatcherRunner(); + + try { + testingLeaderElectionService.isLeader(firstLeaderSession); + testingLeaderElectionService.isLeader(secondLeaderSession); + testingLeaderElectionService.isLeader(thirdLeaderSession); + + firstDispatcherLeaderProcessTerminationFuture.complete(null); + + assertThat(thirdDispatcherLeaderProcessHasBeenStartedFuture.isDone(), is(false)); + + secondDispatcherLeaderProcessTerminationFuture.complete(null); + + assertThat(thirdDispatcherLeaderProcessHasBeenStartedFuture.isDone(), is(true)); + } finally { + firstDispatcherLeaderProcessTerminationFuture.complete(null); + secondDispatcherLeaderProcessTerminationFuture.complete(null); + + dispatcherRunner.close(); + } + } + + private static class StartStopTestingDispatcherLeaderProcess { + + private final TestingDispatcherLeaderProcess testingDispatcherLeaderProcess; + private final CompletableFuture startFuture; + private final CompletableFuture terminationFuture; + + private StartStopTestingDispatcherLeaderProcess( + TestingDispatcherLeaderProcess testingDispatcherLeaderProcess, + CompletableFuture startFuture, + CompletableFuture terminationFuture) { + this.testingDispatcherLeaderProcess = testingDispatcherLeaderProcess; + this.startFuture = startFuture; + this.terminationFuture = terminationFuture; + } + + private TestingDispatcherLeaderProcess asTestingDispatcherLeaderProcess() { + return testingDispatcherLeaderProcess; + } + + private boolean isStarted() { + return startFuture.isDone(); + } + + private void terminateProcess() { + terminationFuture.complete(null); + } + + private static StartStopTestingDispatcherLeaderProcess create(UUID leaderSessionId) { + final CompletableFuture processStartFuture = new CompletableFuture<>(); + final CompletableFuture processTerminationFuture = new CompletableFuture<>(); + final TestingDispatcherLeaderProcess dispatcherLeaderProcess = TestingDispatcherLeaderProcess.newBuilder(leaderSessionId) + .setStartConsumer(processStartFuture::complete) + .setCloseAsyncSupplier(() -> processTerminationFuture) + .build(); + + return new StartStopTestingDispatcherLeaderProcess(dispatcherLeaderProcess, processStartFuture, processTerminationFuture); + } + } + + private TestingDispatcherGateway createDispatcherGateway(UUID leaderSessionId) { + return new TestingDispatcherGateway.Builder() + .setFencingToken(DispatcherId.fromUuid(leaderSessionId)) + .build(); + } + + private DispatcherRunnerImplNG createDispatcherRunner() throws Exception { + return new DispatcherRunnerImplNG( + testingLeaderElectionService, + testingFatalErrorHandler, + testingDispatcherLeaderProcessFactory); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherLeaderProcess.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherLeaderProcess.java new file mode 100644 index 000000000000..7f4d51ae3adc --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherLeaderProcess.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.runner; + +import org.apache.flink.runtime.dispatcher.DispatcherGateway; + +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.function.Consumer; +import java.util.function.Supplier; + +class TestingDispatcherLeaderProcess implements DispatcherLeaderProcess { + private final UUID leaderSessionId; + + private final CompletableFuture dispatcherGatewayFuture; + private final Consumer startConsumer; + private final Supplier> closeAsyncSupplier; + private final CompletableFuture confirmLeaderSessionFuture; + + private CompletableFuture terminationFuture = null; + + private TestingDispatcherLeaderProcess( + UUID leaderSessionId, + CompletableFuture dispatcherGatewayFuture, + Consumer startConsumer, + Supplier> closeAsyncSupplier, + CompletableFuture confirmLeaderSessionFuture) { + this.leaderSessionId = leaderSessionId; + this.dispatcherGatewayFuture = dispatcherGatewayFuture; + this.startConsumer = startConsumer; + this.closeAsyncSupplier = closeAsyncSupplier; + this.confirmLeaderSessionFuture = confirmLeaderSessionFuture; + } + + @Override + public void start() { + startConsumer.accept(null); + } + + @Override + public UUID getLeaderSessionId() { + return leaderSessionId; + } + + @Override + public CompletableFuture getDispatcherGateway() { + return dispatcherGatewayFuture; + } + + @Override + public CompletableFuture getConfirmLeaderSessionFuture() { + return confirmLeaderSessionFuture; + } + + @Override + public CompletableFuture closeAsync() { + if (terminationFuture == null) { + terminationFuture = closeAsyncSupplier.get(); + } + + return terminationFuture; + } + + public static Builder newBuilder(UUID leaderSessionId) { + return new Builder(leaderSessionId); + } + + public static class Builder { + private final UUID leaderSessionId; + + private CompletableFuture dispatcherGatewayFuture = new CompletableFuture<>(); + + private Consumer startConsumer = (ignored) -> {}; + + private Supplier> closeAsyncSupplier = () -> CompletableFuture.completedFuture(null); + + private CompletableFuture confirmLeaderSessionFuture = CompletableFuture.completedFuture("Unknown address"); + + private Builder(UUID leaderSessionId) { + this.leaderSessionId = leaderSessionId; + } + + public Builder setDispatcherGatewayFuture(CompletableFuture dispatcherGatewayFuture) { + this.dispatcherGatewayFuture = dispatcherGatewayFuture; + return this; + } + + public Builder setStartConsumer(Consumer startConsumer) { + this.startConsumer = startConsumer; + return this; + } + + public Builder setCloseAsyncSupplier(Supplier> closeAsyncSupplier) { + this.closeAsyncSupplier = closeAsyncSupplier; + return this; + } + + public Builder setConfirmLeaderSessionFuture(CompletableFuture confirmLeaderSessionFuture) { + this.confirmLeaderSessionFuture = confirmLeaderSessionFuture; + return this; + } + + public TestingDispatcherLeaderProcess build() { + return new TestingDispatcherLeaderProcess( + leaderSessionId, + dispatcherGatewayFuture, + startConsumer, + closeAsyncSupplier, + confirmLeaderSessionFuture); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherLeaderProcessFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherLeaderProcessFactory.java new file mode 100644 index 000000000000..cb4340721979 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherLeaderProcessFactory.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.dispatcher.runner; + +import org.apache.flink.util.Preconditions; + +import java.util.ArrayDeque; +import java.util.Arrays; +import java.util.Queue; +import java.util.UUID; + +class TestingDispatcherLeaderProcessFactory implements DispatcherLeaderProcessFactory { + private final Queue processes; + + private TestingDispatcherLeaderProcessFactory(Queue processes) { + this.processes = processes; + } + + @Override + public TestingDispatcherLeaderProcess create(UUID leaderSessionID) { + if (processes.isEmpty()) { + return TestingDispatcherLeaderProcess.newBuilder(leaderSessionID).build(); + } else { + final TestingDispatcherLeaderProcess nextProcess = processes.poll(); + Preconditions.checkState(leaderSessionID.equals(nextProcess.getLeaderSessionId())); + + return nextProcess; + } + } + + public static TestingDispatcherLeaderProcessFactory from(TestingDispatcherLeaderProcess... processes) { + return new TestingDispatcherLeaderProcessFactory(new ArrayDeque<>(Arrays.asList(processes))); + } + + public static TestingDispatcherLeaderProcessFactory defaultValue() { + return new TestingDispatcherLeaderProcessFactory(new ArrayDeque<>(0)); + } +} From d734ac0bc633386e509d49d8a2ea1ef7bb66a29c Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 13 Sep 2019 23:48:19 +0200 Subject: [PATCH 252/746] [FLINK-11843] Add DispatcherLeaderProcessImpl --- .../runner/DispatcherLeaderProcessImpl.java | 415 ++++++++++++++++++ .../DispatcherLeaderProcessImplTest.java | 250 +++++++++++ .../runner/TestingDispatcherService.java | 85 ++++ .../TestingDispatcherServiceFactory.java | 57 +++ .../testutils/TestingJobGraphStore.java | 17 +- 5 files changed, 822 insertions(+), 2 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherService.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java new file mode 100644 index 000000000000..b079ab141c01 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java @@ -0,0 +1,415 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.runner; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.dispatcher.Dispatcher; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmanager.JobGraphStore; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcUtils; +import org.apache.flink.runtime.webmonitor.RestfulGateway; +import org.apache.flink.util.AutoCloseableAsync; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.function.FunctionUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.function.Predicate; +import java.util.function.Supplier; + +/** + * Process which encapsulates the job recovery logic and life cycle management of a + * {@link Dispatcher}. + */ +public class DispatcherLeaderProcessImpl implements DispatcherLeaderProcess, JobGraphStore.JobGraphListener { + + private static final Logger LOG = LoggerFactory.getLogger(DispatcherLeaderProcessImpl.class); + + private final Object lock = new Object(); + + private final UUID leaderSessionId; + + private final DispatcherServiceFactory dispatcherFactory; + + private final JobGraphStore jobGraphStore; + + private final Executor ioExecutor; + + private final FatalErrorHandler fatalErrorHandler; + + private final CompletableFuture dispatcherGatewayFuture; + + private final CompletableFuture confirmLeaderSessionFuture; + + private final CompletableFuture terminationFuture; + + private State state; + + @Nullable + private DispatcherService dispatcher; + + private CompletableFuture onGoingRecoveryOperation = FutureUtils.completedVoidFuture(); + + private DispatcherLeaderProcessImpl( + UUID leaderSessionId, + DispatcherServiceFactory dispatcherFactory, + JobGraphStore jobGraphStore, + Executor ioExecutor, + FatalErrorHandler fatalErrorHandler) { + this.leaderSessionId = leaderSessionId; + this.dispatcherFactory = dispatcherFactory; + this.jobGraphStore = jobGraphStore; + this.ioExecutor = ioExecutor; + this.fatalErrorHandler = fatalErrorHandler; + + this.dispatcherGatewayFuture = new CompletableFuture<>(); + this.confirmLeaderSessionFuture = dispatcherGatewayFuture.thenApply(RestfulGateway::getAddress); + this.terminationFuture = new CompletableFuture<>(); + + this.state = State.CREATED; + this.dispatcher = null; + } + + State getState() { + synchronized (lock) { + return state; + } + } + + @Override + public void start() { + runIfStateIs( + State.CREATED, + this::startInternal); + } + + private void startInternal() { + LOG.info("Start {}.", getClass().getSimpleName()); + state = State.RUNNING; + startServices(); + + onGoingRecoveryOperation = recoverJobsAsync() + .thenAccept(this::createDispatcherIfRunning) + .handle(this::onErrorIfRunning); + } + + private void startServices() { + try { + jobGraphStore.start(this); + } catch (Exception e) { + throw new FlinkRuntimeException( + String.format( + "Could not start %s when trying to start the %s.", + jobGraphStore.getClass().getSimpleName(), + getClass().getSimpleName()), + e); + } + } + + private Void onErrorIfRunning(T ignored, Throwable throwable) { + synchronized (lock) { + if (state != State.RUNNING) { + return null; + } + } + + if (throwable != null) { + closeAsync(); + fatalErrorHandler.onFatalError(throwable); + } + + return null; + } + + private void createDispatcherIfRunning(Collection jobGraphs) { + runIfStateIs(State.RUNNING, () -> createDispatcher(jobGraphs)); + } + + private void createDispatcher(Collection jobGraphs) { + dispatcher = dispatcherFactory.create(jobGraphs, jobGraphStore); + dispatcherGatewayFuture.complete(dispatcher.getGateway()); + } + + private CompletableFuture> recoverJobsAsync() { + return CompletableFuture.supplyAsync( + this::recoverJobsIfRunning, + ioExecutor); + } + + private Collection recoverJobsIfRunning() { + return supplyUnsynchronizedIfRunning(this::recoverJobs).orElse(Collections.emptyList()); + + } + + private Collection recoverJobs() { + LOG.info("Recover all persisted job graphs."); + final Collection jobIds = getJobIds(); + final Collection recoveredJobGraphs = new ArrayList<>(); + + for (JobID jobId : jobIds) { + recoveredJobGraphs.add(recoverJob(jobId)); + } + + LOG.info("Successfully recovered {} persisted job graphs.", recoveredJobGraphs.size()); + + return recoveredJobGraphs; + } + + private Collection getJobIds() { + try { + return jobGraphStore.getJobIds(); + } catch (Exception e) { + throw new FlinkRuntimeException( + "Could not retrieve job ids of persisted jobs.", + e); + } + } + + private JobGraph recoverJob(JobID jobId) { + LOG.info("Trying to recover job with job id {}.", jobId); + try { + return jobGraphStore.recoverJobGraph(jobId); + } catch (Exception e) { + throw new FlinkRuntimeException( + String.format("Could not recover job with job id %s.", jobId), + e); + } + } + + @Override + public UUID getLeaderSessionId() { + return leaderSessionId; + } + + @Override + public CompletableFuture getDispatcherGateway() { + return dispatcherGatewayFuture; + } + + @Override + public CompletableFuture getConfirmLeaderSessionFuture() { + return confirmLeaderSessionFuture; + } + + @Override + public CompletableFuture closeAsync() { + runIfStateIsNot( + State.STOPPED, + this::closeInternal); + + return terminationFuture; + } + + private void closeInternal() { + LOG.info("Stopping {}.", getClass().getSimpleName()); + final CompletableFuture dispatcherTerminationFuture; + + if (dispatcher != null) { + dispatcherTerminationFuture = dispatcher.closeAsync(); + } else { + dispatcherTerminationFuture = FutureUtils.completedVoidFuture(); + } + + final CompletableFuture stopServicesFuture = FutureUtils.runAfterwardsAsync( + dispatcherTerminationFuture, + this::stopServices, + ioExecutor); + + FutureUtils.forward( + stopServicesFuture, + terminationFuture); + + state = State.STOPPED; + } + + private void stopServices() throws Exception { + Exception exception = null; + + try { + jobGraphStore.stop(); + } catch (Exception e) { + exception = e; + } + + ExceptionUtils.tryRethrowException(exception); + } + + // ------------------------------------------------------------ + // JobGraphListener + // ------------------------------------------------------------ + + @Override + public void onAddedJobGraph(JobID jobId) { + runIfStateIs( + State.RUNNING, + () -> handleAddedJobGraph(jobId)); + } + + private void handleAddedJobGraph(JobID jobId) { + // serialize all ongoing recovery operations + onGoingRecoveryOperation = onGoingRecoveryOperation + .thenApplyAsync( + ignored -> recoverJobIfRunning(jobId), + ioExecutor) + .thenCompose(optionalJobGraph -> optionalJobGraph + .flatMap(this::submitAddedJobIfRunning) + .orElse(FutureUtils.completedVoidFuture())) + .handle(this::onErrorIfRunning); + } + + private Optional> submitAddedJobIfRunning(JobGraph jobGraph) { + return supplyIfRunning(() -> submitAddedJob(jobGraph)); + } + + private CompletableFuture submitAddedJob(JobGraph jobGraph) { + final DispatcherGateway dispatcherGateway = getDispatcherGatewayInternal(); + + // TODO: Filter out duplicate job submissions which can happen with the JobGraphListener + return dispatcherGateway + .submitJob(jobGraph, RpcUtils.INF_TIMEOUT) + .thenApply(FunctionUtils.nullFn()); + } + + private DispatcherGateway getDispatcherGatewayInternal() { + return Preconditions.checkNotNull(dispatcherGatewayFuture.getNow(null)); + } + + private Optional recoverJobIfRunning(JobID jobId) { + return supplyUnsynchronizedIfRunning(() -> recoverJob(jobId)); + } + + @Override + public void onRemovedJobGraph(JobID jobId) { + runIfStateIs( + State.RUNNING, + () -> handleRemovedJobGraph(jobId)); + } + + private void handleRemovedJobGraph(JobID jobId) { + onGoingRecoveryOperation = onGoingRecoveryOperation + .thenCompose(ignored -> removeJobGraphIfRunning(jobId).orElse(FutureUtils.completedVoidFuture())) + .handle(this::onErrorIfRunning); + } + + private Optional> removeJobGraphIfRunning(JobID jobId) { + return supplyIfRunning(() -> removeJobGraph(jobId)); + } + + private CompletableFuture removeJobGraph(JobID jobId) { + final DispatcherGateway dispatcherGateway = getDispatcherGatewayInternal(); + + // TODO: replace cancel with other fail method + return dispatcherGateway + .cancelJob(jobId, RpcUtils.INF_TIMEOUT) + .thenApply(FunctionUtils.nullFn()); + } + + // --------------------------------------------------------------- + // Factory methods + // --------------------------------------------------------------- + + public static DispatcherLeaderProcessImpl create( + UUID leaderSessionId, + DispatcherServiceFactory dispatcherFactory, + JobGraphStore jobGraphStore, + Executor ioExecutor, + FatalErrorHandler fatalErrorHandler) { + return new DispatcherLeaderProcessImpl( + leaderSessionId, + dispatcherFactory, + jobGraphStore, + ioExecutor, + fatalErrorHandler); + } + + // --------------------------------------------------------------- + // Internal helper methods + // --------------------------------------------------------------- + + private Optional supplyUnsynchronizedIfRunning(Supplier supplier) { + synchronized (lock) { + if (state != State.RUNNING) { + return Optional.empty(); + } + } + + return Optional.of(supplier.get()); + } + + private Optional supplyIfRunning(Supplier supplier) { + synchronized (lock) { + if (state != State.RUNNING) { + return Optional.empty(); + } + + return Optional.of(supplier.get()); + } + } + + private void runIfStateIs(State expectedState, Runnable action) { + runIfState(expectedState::equals, action); + } + + private void runIfStateIsNot(State notExpectedState, Runnable action) { + runIfState( + state -> !notExpectedState.equals(state), + action); + } + + private void runIfState(Predicate actionPredicate, Runnable action) { + synchronized (lock) { + if (actionPredicate.test(state)) { + action.run(); + } + } + } + + // ------------------------------------------------------------ + // Internal classes + // ------------------------------------------------------------ + + interface DispatcherServiceFactory { + DispatcherService create(Collection recoveredJobs, JobGraphStore jobGraphStore); + } + + interface DispatcherService extends AutoCloseableAsync { + DispatcherGateway getGateway(); + } + + enum State { + CREATED, + RUNNING, + STOPPED + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java new file mode 100644 index 000000000000..02ad01041b19 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java @@ -0,0 +1,250 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.runner; + +import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmanager.JobGraphStore; +import org.apache.flink.runtime.testutils.TestingJobGraphStore; +import org.apache.flink.runtime.util.TestingFatalErrorHandler; +import org.apache.flink.runtime.webmonitor.TestingDispatcherGateway; +import org.apache.flink.util.ExecutorUtils; +import org.apache.flink.util.TestLogger; +import org.apache.flink.util.function.BiFunctionWithException; + +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.Collection; +import java.util.Collections; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.hasSize; +import static org.junit.Assert.fail; + +/** + * Tests for the {@link DispatcherLeaderProcessImpl}. + */ +public class DispatcherLeaderProcessImplTest extends TestLogger { + + private static final JobGraph PERSISTED_JOB_GRAPH = new JobGraph("Persisted JobGraph"); + + private static ExecutorService ioExecutor; + + private final UUID leaderSessionId = UUID.randomUUID(); + + private TestingFatalErrorHandler fatalErrorHandler; + + private JobGraphStore jobGraphStore; + + private TestingDispatcherServiceFactory dispatcherServiceFactory; + + @BeforeClass + public static void setupClass() { + ioExecutor = Executors.newSingleThreadExecutor(); + } + + @Before + public void setup() { + fatalErrorHandler = new TestingFatalErrorHandler(); + jobGraphStore = TestingJobGraphStore.newBuilder().build(); + dispatcherServiceFactory = TestingDispatcherServiceFactory.newBuilder().build(); + } + + @After + public void teardown() throws Exception { + if (fatalErrorHandler != null) { + fatalErrorHandler.rethrowError(); + fatalErrorHandler = null; + } + } + + @AfterClass + public static void teardownClass() { + if (ioExecutor != null) { + ExecutorUtils.gracefulShutdown(5L, TimeUnit.SECONDS, ioExecutor); + } + } + + @Test + public void start_afterClose_doesNotHaveAnEffect() throws Exception { + final DispatcherLeaderProcessImpl dispatcherLeaderProcess = createDispatcherLeaderProcess(); + + dispatcherLeaderProcess.close(); + dispatcherLeaderProcess.start(); + + assertThat(dispatcherLeaderProcess.getState(), is(DispatcherLeaderProcessImpl.State.STOPPED)); + } + + @Test + public void start_triggersJobGraphRecoveryAndDispatcherServiceCreation() throws Exception { + jobGraphStore = TestingJobGraphStore.newBuilder() + .setInitialJobGraphs(Collections.singleton(PERSISTED_JOB_GRAPH)) + .build(); + + final CompletableFuture> recoveredJobGraphsFuture = new CompletableFuture<>(); + dispatcherServiceFactory = TestingDispatcherServiceFactory.newBuilder() + .setCreateFunction( + (recoveredJobGraphs, jobGraphStore) -> { + recoveredJobGraphsFuture.complete(recoveredJobGraphs); + return TestingDispatcherService.newBuilder().build(); + } + ) + .build(); + + try (final DispatcherLeaderProcessImpl dispatcherLeaderProcess = createDispatcherLeaderProcess()) { + dispatcherLeaderProcess.start(); + assertThat(dispatcherLeaderProcess.getState(), is(DispatcherLeaderProcessImpl.State.RUNNING)); + + final Collection recoveredJobGraphs = recoveredJobGraphsFuture.get(); + + assertThat(recoveredJobGraphs, hasSize(1)); + assertThat(recoveredJobGraphs, containsInAnyOrder(PERSISTED_JOB_GRAPH)); + } + } + + @Test + public void closeAsync_stopsJobGraphStoreAndDispatcher() throws Exception { + final CompletableFuture jobGraphStopFuture = new CompletableFuture<>(); + jobGraphStore = TestingJobGraphStore.newBuilder() + .setStopRunnable(() -> jobGraphStopFuture.complete(null)) + .build(); + + final CompletableFuture dispatcherServiceTerminationFuture = new CompletableFuture<>(); + final OneShotLatch dispatcherServiceShutdownLatch = new OneShotLatch(); + dispatcherServiceFactory = TestingDispatcherServiceFactory.newBuilder() + .setCreateFunction((ignoredA, ignoredB) -> TestingDispatcherService.newBuilder() + .setTerminationFutureSupplier(() -> { + dispatcherServiceShutdownLatch.trigger(); + return dispatcherServiceTerminationFuture; + }) + .build()) + .build(); + + try (final DispatcherLeaderProcessImpl dispatcherLeaderProcess = createDispatcherLeaderProcess()) { + dispatcherLeaderProcess.start(); + + // wait for the creation of the DispatcherService + dispatcherLeaderProcess.getDispatcherGateway().get(); + + final CompletableFuture terminationFuture = dispatcherLeaderProcess.closeAsync(); + + assertThat(jobGraphStopFuture.isDone(), is(false)); + assertThat(terminationFuture.isDone(), is(false)); + + dispatcherServiceShutdownLatch.await(); + dispatcherServiceTerminationFuture.complete(null); + + // verify that we shut down the JobGraphStore + jobGraphStopFuture.get(); + + // verify that we completed the dispatcher leader process shut down + terminationFuture.get(); + } + } + + @Test + public void confirmLeaderSessionFuture_completesAfterDispatcherServiceHasBeenStarted() throws Exception { + final OneShotLatch createDispatcherServiceLatch = new OneShotLatch(); + final String dispatcherAddress = "myAddress"; + final TestingDispatcherGateway dispatcherGateway = new TestingDispatcherGateway.Builder() + .setAddress(dispatcherAddress) + .build(); + + dispatcherServiceFactory = TestingDispatcherServiceFactory.newBuilder() + .setCreateFunction( + BiFunctionWithException.unchecked((ignoredA, ignoredB) -> { + createDispatcherServiceLatch.await(); + return TestingDispatcherService.newBuilder() + .setDispatcherGateway(dispatcherGateway) + .build(); + })) + .build(); + + try (final DispatcherLeaderProcessImpl dispatcherLeaderProcess = createDispatcherLeaderProcess()) { + final CompletableFuture confirmLeaderSessionFuture = dispatcherLeaderProcess.getConfirmLeaderSessionFuture(); + + dispatcherLeaderProcess.start(); + + assertThat(confirmLeaderSessionFuture.isDone(), is(false)); + + createDispatcherServiceLatch.trigger(); + + assertThat(confirmLeaderSessionFuture.get(), is(dispatcherAddress)); + } + } + + @Test + public void closeAsync_duringJobRecovery_preventsDispatcherServiceCreation() throws Exception { + final OneShotLatch jobRecoveryStarted = new OneShotLatch(); + final OneShotLatch completeJobRecovery = new OneShotLatch(); + final OneShotLatch createDispatcherService = new OneShotLatch(); + + this.jobGraphStore = TestingJobGraphStore.newBuilder() + .setJobIdsFunction(storedJobs -> { + jobRecoveryStarted.trigger(); + completeJobRecovery.await(); + return storedJobs; + }) + .build(); + + this.dispatcherServiceFactory = TestingDispatcherServiceFactory.newBuilder() + .setCreateFunction( + (ignoredA, ignoredB) -> { + createDispatcherService.trigger(); + return TestingDispatcherService.newBuilder().build(); + }) + .build(); + + try (final DispatcherLeaderProcessImpl dispatcherLeaderProcess = createDispatcherLeaderProcess()) { + dispatcherLeaderProcess.start(); + + jobRecoveryStarted.await(); + + dispatcherLeaderProcess.closeAsync(); + + completeJobRecovery.trigger(); + + try { + createDispatcherService.await(10L, TimeUnit.MILLISECONDS); + fail("No dispatcher service should be created after the process has been stopped."); + } catch (TimeoutException expected) {} + } + } + + private DispatcherLeaderProcessImpl createDispatcherLeaderProcess() { + return DispatcherLeaderProcessImpl.create( + leaderSessionId, + dispatcherServiceFactory, + jobGraphStore, + ioExecutor, + fatalErrorHandler); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherService.java new file mode 100644 index 000000000000..9e9ba7d24737 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherService.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.runtime.dispatcher.runner; + +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; +import org.apache.flink.runtime.webmonitor.TestingDispatcherGateway; + +import java.util.concurrent.CompletableFuture; +import java.util.function.Supplier; + +class TestingDispatcherService implements DispatcherLeaderProcessImpl.DispatcherService { + + private final Object lock = new Object(); + + private final Supplier> terminationFutureSupplier; + + private final DispatcherGateway dispatcherGateway; + + private CompletableFuture terminationFuture; + + private TestingDispatcherService(Supplier> terminationFutureSupplier, DispatcherGateway dispatcherGateway) { + this.terminationFutureSupplier = terminationFutureSupplier; + this.dispatcherGateway = dispatcherGateway; + } + + @Override + public DispatcherGateway getGateway() { + return dispatcherGateway; + } + + @Override + public CompletableFuture closeAsync() { + synchronized (lock) { + if (terminationFuture == null) { + terminationFuture = terminationFutureSupplier.get(); + } + + return terminationFuture; + } + } + + public static Builder newBuilder() { + return new Builder(); + } + + public static class Builder { + + private Supplier> terminationFutureSupplier = FutureUtils::completedVoidFuture; + + private DispatcherGateway dispatcherGateway = new TestingDispatcherGateway.Builder().build(); + + private Builder() {} + + public Builder setTerminationFutureSupplier(Supplier> terminationFutureSupplier) { + this.terminationFutureSupplier = terminationFutureSupplier; + return this; + } + + public Builder setDispatcherGateway(DispatcherGateway dispatcherGateway) { + this.dispatcherGateway = dispatcherGateway; + return this; + } + + public TestingDispatcherService build() { + return new TestingDispatcherService(terminationFutureSupplier, dispatcherGateway); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java new file mode 100644 index 000000000000..7b0faeed98cd --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.runner; + +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmanager.JobGraphStore; + +import java.util.Collection; +import java.util.function.BiFunction; + +class TestingDispatcherServiceFactory implements DispatcherLeaderProcessImpl.DispatcherServiceFactory { + private final BiFunction, JobGraphStore, DispatcherLeaderProcessImpl.DispatcherService> createFunction; + + private TestingDispatcherServiceFactory(BiFunction, JobGraphStore, DispatcherLeaderProcessImpl.DispatcherService> createFunction) { + this.createFunction = createFunction; + } + + @Override + public DispatcherLeaderProcessImpl.DispatcherService create(Collection recoveredJobs, JobGraphStore jobGraphStore) { + return createFunction.apply(recoveredJobs, jobGraphStore); + } + + public static Builder newBuilder() { + return new Builder(); + } + + public static class Builder { + private BiFunction, JobGraphStore, DispatcherLeaderProcessImpl.DispatcherService> createFunction = (ignoredA, ignoredB) -> TestingDispatcherService.newBuilder().build(); + + private Builder() {} + + Builder setCreateFunction(BiFunction, JobGraphStore, DispatcherLeaderProcessImpl.DispatcherService> createFunction) { + this.createFunction = createFunction; + return this; + } + + public TestingDispatcherServiceFactory build() { + return new TestingDispatcherServiceFactory(createFunction); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobGraphStore.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobGraphStore.java index bdf6f0b56c67..0376d448546a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobGraphStore.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobGraphStore.java @@ -65,7 +65,8 @@ private TestingJobGraphStore( BiFunctionWithException, JobGraph, ? extends Exception> recoverJobGraphFunction, ThrowingConsumer putJobGraphConsumer, ThrowingConsumer removeJobGraphConsumer, - ThrowingConsumer releaseJobGraphConsumer) { + ThrowingConsumer releaseJobGraphConsumer, + Collection initialJobGraphs) { this.startConsumer = startConsumer; this.stopRunnable = stopRunnable; this.jobIdsFunction = jobIdsFunction; @@ -73,6 +74,10 @@ private TestingJobGraphStore( this.putJobGraphConsumer = putJobGraphConsumer; this.removeJobGraphConsumer = removeJobGraphConsumer; this.releaseJobGraphConsumer = releaseJobGraphConsumer; + + for (JobGraph initialJobGraph : initialJobGraphs) { + storedJobs.put(initialJobGraph.getJobID(), initialJobGraph); + } } @Override @@ -146,6 +151,8 @@ public static class Builder { private ThrowingConsumer releaseJobGraphConsumer = ignored -> {}; + private Collection initialJobGraphs = Collections.emptyList(); + private Builder() {} public Builder setStartConsumer(ThrowingConsumer startConsumer) { @@ -183,6 +190,11 @@ public Builder setReleaseJobGraphConsumer(ThrowingConsumer initialJobGraphs) { + this.initialJobGraphs = initialJobGraphs; + return this; + } + public TestingJobGraphStore build() { return new TestingJobGraphStore( startConsumer, @@ -191,7 +203,8 @@ public TestingJobGraphStore build() { recoverJobGraphFunction, putJobGraphConsumer, removeJobGraphConsumer, - releaseJobGraphConsumer); + releaseJobGraphConsumer, + initialJobGraphs); } } From 7956a14c4e7a46f8033b4cb98fb25951ce47b697 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 14 Sep 2019 18:14:58 +0200 Subject: [PATCH 253/746] [FLINK-11843] Add DispatcherRunnerImplNGFactory implementation --- .../DispatcherLeaderProcessImplFactory.java | 57 ++++++++++++++++ .../runner/DispatcherRunnerImplNGFactory.java | 66 +++++++++++++++++++ .../runner/DispatcherServiceImpl.java | 49 ++++++++++++++ .../runner/DispatcherServiceImplFactory.java | 66 +++++++++++++++++++ 4 files changed, 238 insertions(+) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplFactory.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGFactory.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImpl.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplFactory.java new file mode 100644 index 000000000000..085f290d37f3 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplFactory.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.runner; + +import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; +import org.apache.flink.runtime.rpc.FatalErrorHandler; + +import java.util.UUID; +import java.util.concurrent.Executor; + +/** + * Factory for the {@link DispatcherLeaderProcessImpl}. + */ +class DispatcherLeaderProcessImplFactory implements DispatcherLeaderProcessFactory { + + private final DispatcherLeaderProcessImpl.DispatcherServiceFactory dispatcherServiceFactory; + private final JobGraphStoreFactory jobGraphStoreFactory; + private final Executor ioExecutor; + private final FatalErrorHandler fatalErrorHandler; + + DispatcherLeaderProcessImplFactory( + DispatcherLeaderProcessImpl.DispatcherServiceFactory dispatcherServiceFactory, + JobGraphStoreFactory jobGraphStoreFactory, + Executor ioExecutor, + FatalErrorHandler fatalErrorHandler) { + this.dispatcherServiceFactory = dispatcherServiceFactory; + this.jobGraphStoreFactory = jobGraphStoreFactory; + this.ioExecutor = ioExecutor; + this.fatalErrorHandler = fatalErrorHandler; + } + + @Override + public DispatcherLeaderProcess create(UUID leaderSessionID) { + return DispatcherLeaderProcessImpl.create( + leaderSessionID, + dispatcherServiceFactory, + jobGraphStoreFactory.create(), + ioExecutor, + fatalErrorHandler); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGFactory.java new file mode 100644 index 000000000000..e6831a09d2eb --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGFactory.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.runtime.dispatcher.runner; + +import org.apache.flink.runtime.dispatcher.DispatcherFactory; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; +import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; +import org.apache.flink.runtime.leaderelection.LeaderElectionService; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcService; + +import java.util.concurrent.Executor; + +/** + * {@link DispatcherRunnerFactory} implementation which creates {@link DispatcherRunnerImplNG} + * instances. + */ +public class DispatcherRunnerImplNGFactory implements DispatcherRunnerFactory { + private final DispatcherFactory dispatcherFactory; + + public DispatcherRunnerImplNGFactory(DispatcherFactory dispatcherFactory) { + this.dispatcherFactory = dispatcherFactory; + } + + @Override + public DispatcherRunner createDispatcherRunner( + LeaderElectionService leaderElectionService, + FatalErrorHandler fatalErrorHandler, + JobGraphStoreFactory jobGraphStoreFactory, + Executor ioExecutor, + RpcService rpcService, + PartialDispatcherServices partialDispatcherServices) throws Exception { + + final DispatcherLeaderProcessImpl.DispatcherServiceFactory dispatcherServiceFactory = new DispatcherServiceImplFactory( + dispatcherFactory, + rpcService, + partialDispatcherServices); + + final DispatcherLeaderProcessFactory dispatcherLeaderProcessFactory = new DispatcherLeaderProcessImplFactory( + dispatcherServiceFactory, + jobGraphStoreFactory, + ioExecutor, + fatalErrorHandler); + + return new DispatcherRunnerImplNG( + leaderElectionService, + fatalErrorHandler, + dispatcherLeaderProcessFactory); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImpl.java new file mode 100644 index 000000000000..facda4971a0d --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImpl.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.runner; + +import org.apache.flink.runtime.dispatcher.Dispatcher; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; + +import java.util.concurrent.CompletableFuture; + +class DispatcherServiceImpl implements DispatcherLeaderProcessImpl.DispatcherService { + + private final Dispatcher dispatcher; + private final DispatcherGateway dispatcherGateway; + + private DispatcherServiceImpl(Dispatcher dispatcher) { + this.dispatcher = dispatcher; + this.dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); + } + + @Override + public DispatcherGateway getGateway() { + return dispatcherGateway; + } + + @Override + public CompletableFuture closeAsync() { + return dispatcher.closeAsync(); + } + + public static DispatcherServiceImpl from(Dispatcher dispatcher) { + return new DispatcherServiceImpl(dispatcher); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.java new file mode 100644 index 000000000000..6bfb234f54d6 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.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.runtime.dispatcher.runner; + +import org.apache.flink.runtime.dispatcher.Dispatcher; +import org.apache.flink.runtime.dispatcher.DispatcherFactory; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmanager.JobGraphStore; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.util.FlinkRuntimeException; + +import java.util.Collection; + +/** + * Factory for the {@link DispatcherServiceImpl}. + */ +class DispatcherServiceImplFactory implements DispatcherLeaderProcessImpl.DispatcherServiceFactory { + + private final DispatcherFactory dispatcherFactory; + + private final RpcService rpcService; + + private final PartialDispatcherServices partialDispatcherServices; + + DispatcherServiceImplFactory( + DispatcherFactory dispatcherFactory, + RpcService rpcService, + PartialDispatcherServices partialDispatcherServices) { + this.dispatcherFactory = dispatcherFactory; + this.rpcService = rpcService; + this.partialDispatcherServices = partialDispatcherServices; + } + + @Override + public DispatcherLeaderProcessImpl.DispatcherService create(Collection recoveredJobs, JobGraphStore jobGraphStore) { + final Dispatcher dispatcher; + try { + dispatcher = dispatcherFactory.createDispatcher( + rpcService, + partialDispatcherServices); + } catch (Exception e) { + throw new FlinkRuntimeException("Could not create the Dispatcher rpc endpoint.", e); + } + + dispatcher.start(); + + return DispatcherServiceImpl.from(dispatcher); + } +} From 9d447ada31c85ab899d01c05ca60f4887f0f1218 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 14 Sep 2019 19:01:12 +0200 Subject: [PATCH 254/746] [FLINK-11843] Allow to pass JobGraphStore into Dispatcher via DispatcherServices This commit introduces DispatcherFactoryServices which is an extension of PartialDispatcherFactoryServices. The extension allows to set a specific JobGraphStore which is being forwarded via the DispatcherServices to the Dispatcher. --- .../runtime/dispatcher/DispatcherFactory.java | 2 +- .../dispatcher/DispatcherServices.java | 36 +++++--- .../dispatcher/JobDispatcherFactory.java | 6 +- ...alDispatcherServicesWithJobGraphStore.java | 86 +++++++++++++++++++ .../dispatcher/SessionDispatcherFactory.java | 4 +- .../dispatcher/StandaloneDispatcher.java | 2 +- .../runner/DispatcherRunnerFactoryImpl.java | 6 +- .../runner/DispatcherRunnerImpl.java | 8 +- .../runner/DispatcherServiceImplFactory.java | 3 +- .../dispatcher/MiniDispatcherTest.java | 4 +- .../runtime/dispatcher/TestingDispatcher.java | 1 + .../runner/DispatcherRunnerImplTest.java | 13 +-- .../ZooKeeperDispatcherRunnerImplTest.java | 9 +- .../SessionDispatcherWithUUIDFactory.java | 6 +- 14 files changed, 147 insertions(+), 39 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobGraphStore.java 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 4e7ea67ebc1c..933a413d7047 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 @@ -34,7 +34,7 @@ public interface DispatcherFactory { */ Dispatcher createDispatcher( @Nonnull RpcService rpcService, - @Nonnull PartialDispatcherServices partialDispatcherServices) throws Exception; + @Nonnull PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) 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 index af43c155d9a7..5bca3fec84cd 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 @@ -22,6 +22,7 @@ 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.jobmanager.JobGraphStore; import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rpc.FatalErrorHandler; @@ -65,6 +66,9 @@ public class DispatcherServices { @Nullable private final String metricQueryServiceAddress; + @Nonnull + private final JobGraphStore jobGraphStore; + @Nonnull private final JobManagerRunnerFactory jobManagerRunnerFactory; @@ -79,6 +83,7 @@ public DispatcherServices( @Nonnull HistoryServerArchivist historyServerArchivist, @Nullable String metricQueryServiceAddress, @Nonnull JobManagerMetricGroup jobManagerMetricGroup, + @Nonnull JobGraphStore jobGraphStore, @Nonnull JobManagerRunnerFactory jobManagerRunnerFactory) { this.configuration = configuration; this.highAvailabilityServices = highAvailabilityServices; @@ -90,6 +95,7 @@ public DispatcherServices( this.historyServerArchivist = historyServerArchivist; this.metricQueryServiceAddress = metricQueryServiceAddress; this.jobManagerMetricGroup = jobManagerMetricGroup; + this.jobGraphStore = jobGraphStore; this.jobManagerRunnerFactory = jobManagerRunnerFactory; } @@ -143,25 +149,31 @@ public String getMetricQueryServiceAddress() { return metricQueryServiceAddress; } + @Nonnull + public JobGraphStore getJobGraphStore() { + return jobGraphStore; + } + @Nonnull JobManagerRunnerFactory getJobManagerRunnerFactory() { return jobManagerRunnerFactory; } public static DispatcherServices from( - @Nonnull PartialDispatcherServices partialDispatcherServices, - @Nonnull JobManagerRunnerFactory jobManagerRunnerFactory) { + @Nonnull PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore, + @Nonnull JobManagerRunnerFactory jobManagerRunnerFactory) { return new DispatcherServices( - partialDispatcherServices.getConfiguration(), - partialDispatcherServices.getHighAvailabilityServices(), - partialDispatcherServices.getResourceManagerGatewayRetriever(), - partialDispatcherServices.getBlobServer(), - partialDispatcherServices.getHeartbeatServices(), - partialDispatcherServices.getArchivedExecutionGraphStore(), - partialDispatcherServices.getFatalErrorHandler(), - partialDispatcherServices.getHistoryServerArchivist(), - partialDispatcherServices.getMetricQueryServiceAddress(), - partialDispatcherServices.getJobManagerMetricGroupFactory().create(), + partialDispatcherServicesWithJobGraphStore.getConfiguration(), + partialDispatcherServicesWithJobGraphStore.getHighAvailabilityServices(), + partialDispatcherServicesWithJobGraphStore.getResourceManagerGatewayRetriever(), + partialDispatcherServicesWithJobGraphStore.getBlobServer(), + partialDispatcherServicesWithJobGraphStore.getHeartbeatServices(), + partialDispatcherServicesWithJobGraphStore.getArchivedExecutionGraphStore(), + partialDispatcherServicesWithJobGraphStore.getFatalErrorHandler(), + partialDispatcherServicesWithJobGraphStore.getHistoryServerArchivist(), + partialDispatcherServicesWithJobGraphStore.getMetricQueryServiceAddress(), + partialDispatcherServicesWithJobGraphStore.getJobManagerMetricGroupFactory().create(), + partialDispatcherServicesWithJobGraphStore.getJobGraphStore(), 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 6ca6157351da..a4349fee2ce0 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 @@ -42,8 +42,8 @@ public JobDispatcherFactory(JobGraphRetriever jobGraphRetriever) { @Override public MiniDispatcher createDispatcher( @Nonnull RpcService rpcService, - @Nonnull PartialDispatcherServices partialDispatcherServices) throws Exception { - final Configuration configuration = partialDispatcherServices.getConfiguration(); + @Nonnull PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { + final Configuration configuration = partialDispatcherServicesWithJobGraphStore.getConfiguration(); final JobGraph jobGraph = jobGraphRetriever.retrieveJobGraph(configuration); final String executionModeValue = configuration.getString(EXECUTION_MODE); @@ -53,7 +53,7 @@ public MiniDispatcher createDispatcher( return new MiniDispatcher( rpcService, getEndpointId(), - DispatcherServices.from(partialDispatcherServices, DefaultJobManagerRunnerFactory.INSTANCE), + DispatcherServices.from(partialDispatcherServicesWithJobGraphStore, DefaultJobManagerRunnerFactory.INSTANCE), jobGraph, executionMode); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobGraphStore.java new file mode 100644 index 000000000000..624c8c937973 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobGraphStore.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.jobmanager.JobGraphStore; +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 DispatcherFactory} services container. + */ +public class PartialDispatcherServicesWithJobGraphStore extends PartialDispatcherServices { + + @Nonnull + private final JobGraphStore jobGraphStore; + + public PartialDispatcherServicesWithJobGraphStore( + @Nonnull Configuration configuration, + @Nonnull HighAvailabilityServices highAvailabilityServices, + @Nonnull GatewayRetriever resourceManagerGatewayRetriever, + @Nonnull BlobServer blobServer, + @Nonnull HeartbeatServices heartbeatServices, + @Nonnull JobManagerMetricGroupFactory jobManagerMetricGroupFactory, + @Nonnull ArchivedExecutionGraphStore archivedExecutionGraphStore, + @Nonnull FatalErrorHandler fatalErrorHandler, + @Nonnull HistoryServerArchivist historyServerArchivist, + @Nullable String metricQueryServiceAddress, + @Nonnull JobGraphStore jobGraphStore) { + super( + configuration, + highAvailabilityServices, + resourceManagerGatewayRetriever, + blobServer, + heartbeatServices, + jobManagerMetricGroupFactory, + archivedExecutionGraphStore, + fatalErrorHandler, + historyServerArchivist, + metricQueryServiceAddress); + this.jobGraphStore = jobGraphStore; + } + + @Nonnull + public JobGraphStore getJobGraphStore() { + return jobGraphStore; + } + + public static PartialDispatcherServicesWithJobGraphStore from(PartialDispatcherServices partialDispatcherServices, JobGraphStore jobGraphStore) { + return new PartialDispatcherServicesWithJobGraphStore( + partialDispatcherServices.getConfiguration(), + partialDispatcherServices.getHighAvailabilityServices(), + partialDispatcherServices.getResourceManagerGatewayRetriever(), + partialDispatcherServices.getBlobServer(), + partialDispatcherServices.getHeartbeatServices(), + partialDispatcherServices.getJobManagerMetricGroupFactory(), + partialDispatcherServices.getArchivedExecutionGraphStore(), + partialDispatcherServices.getFatalErrorHandler(), + partialDispatcherServices.getHistoryServerArchivist(), + partialDispatcherServices.getMetricQueryServiceAddress(), + jobGraphStore); + } +} 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 a87992bb20a0..4be749e76456 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 @@ -29,11 +29,11 @@ public enum SessionDispatcherFactory implements DispatcherFactory { @Override public StandaloneDispatcher createDispatcher( RpcService rpcService, - PartialDispatcherServices partialDispatcherServices) throws Exception { + PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { // create the default dispatcher return new StandaloneDispatcher( rpcService, getEndpointId(), - DispatcherServices.from(partialDispatcherServices, DefaultJobManagerRunnerFactory.INSTANCE)); + DispatcherServices.from(partialDispatcherServicesWithJobGraphStore, 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 90b7faa647a4..31be67858bd0 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 @@ -36,6 +36,6 @@ public StandaloneDispatcher( rpcService, endpointId, dispatcherServices, - dispatcherServices.getHighAvailabilityServices().getJobGraphStore()); + dispatcherServices.getJobGraphStore()); } } 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 67020679d4fd..8acfbf1a49bf 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 @@ -20,6 +20,7 @@ import org.apache.flink.runtime.dispatcher.DispatcherFactory; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobGraphStore; import org.apache.flink.runtime.dispatcher.StandaloneDispatcher; import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; import org.apache.flink.runtime.leaderelection.LeaderElectionService; @@ -47,6 +48,9 @@ public DispatcherRunnerImpl createDispatcherRunner( Executor ioExecutor, RpcService rpcService, PartialDispatcherServices partialDispatcherServices) throws Exception { - return new DispatcherRunnerImpl(dispatcherFactory, rpcService, partialDispatcherServices); + return new DispatcherRunnerImpl( + dispatcherFactory, + rpcService, + PartialDispatcherServicesWithJobGraphStore.from(partialDispatcherServices, jobGraphStoreFactory.create())); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java index 8c6a149a1df4..7eb6c5967e5b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java @@ -23,7 +23,7 @@ import org.apache.flink.runtime.dispatcher.Dispatcher; import org.apache.flink.runtime.dispatcher.DispatcherFactory; import org.apache.flink.runtime.dispatcher.DispatcherGateway; -import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobGraphStore; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.webmonitor.retriever.LeaderRetriever; @@ -46,11 +46,11 @@ class DispatcherRunnerImpl implements DispatcherRunner { DispatcherRunnerImpl( DispatcherFactory dispatcherFactory, RpcService rpcService, - PartialDispatcherServices partialDispatcherServices) throws Exception { + PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { this.dispatcher = dispatcherFactory.createDispatcher( rpcService, - partialDispatcherServices); - this.leaderRetrievalService = partialDispatcherServices.getHighAvailabilityServices().getDispatcherLeaderRetriever(); + partialDispatcherServicesWithJobGraphStore); + this.leaderRetrievalService = partialDispatcherServicesWithJobGraphStore.getHighAvailabilityServices().getDispatcherLeaderRetriever(); this.leaderRetriever = new LeaderRetriever(); leaderRetrievalService.start(leaderRetriever); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.java index 6bfb234f54d6..ea8c10b7b723 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.java @@ -21,6 +21,7 @@ import org.apache.flink.runtime.dispatcher.Dispatcher; import org.apache.flink.runtime.dispatcher.DispatcherFactory; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobGraphStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.runtime.rpc.RpcService; @@ -54,7 +55,7 @@ public DispatcherLeaderProcessImpl.DispatcherService create(Collection try { dispatcher = dispatcherFactory.createDispatcher( rpcService, - partialDispatcherServices); + PartialDispatcherServicesWithJobGraphStore.from(partialDispatcherServices, jobGraphStore)); } catch (Exception e) { throw new FlinkRuntimeException("Could not create the Dispatcher rpc endpoint.", e); } 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 7e9aab7da357..bca7ecddd851 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 @@ -27,6 +27,7 @@ import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; +import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServicesBuilder; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobmaster.JobResult; @@ -114,7 +115,7 @@ public static void setupClass() throws IOException { @Before public void setup() throws Exception { dispatcherLeaderElectionService = new TestingLeaderElectionService(); - highAvailabilityServices = new TestingHighAvailabilityServices(); + highAvailabilityServices = new TestingHighAvailabilityServicesBuilder().build(); testingFatalErrorHandler = new TestingFatalErrorHandler(); highAvailabilityServices.setDispatcherLeaderElectionService(dispatcherLeaderElectionService); @@ -240,6 +241,7 @@ private MiniDispatcher createMiniDispatcher(ClusterEntrypoint.ExecutionMode exec VoidHistoryServerArchivist.INSTANCE, null, UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), + highAvailabilityServices.getJobGraphStore(), 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 b06699f2aa38..4f0419488fac 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 @@ -71,6 +71,7 @@ class TestingDispatcher extends Dispatcher { VoidHistoryServerArchivist.INSTANCE, metricQueryServiceAddress, jobManagerMetricGroup, + highAvailabilityServices.getJobGraphStore(), jobManagerRunnerFactory), highAvailabilityServices.getJobGraphStore()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java index 9703781b7276..f670ea21f77b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java @@ -28,7 +28,7 @@ import org.apache.flink.runtime.dispatcher.DispatcherServices; import org.apache.flink.runtime.dispatcher.JobManagerRunnerFactory; import org.apache.flink.runtime.dispatcher.MemoryArchivedExecutionGraphStore; -import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobGraphStore; import org.apache.flink.runtime.dispatcher.SingleJobJobGraphStore; import org.apache.flink.runtime.dispatcher.StandaloneDispatcher; import org.apache.flink.runtime.dispatcher.TestingJobManagerRunnerFactory; @@ -109,7 +109,7 @@ public void testJobRecoveryUnderLeaderChange() throws Exception { .setDispatcherLeaderElectionService(dispatcherLeaderElectionService) .build()) { - final PartialDispatcherServices partialDispatcherServices = new PartialDispatcherServices( + final PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore = new PartialDispatcherServicesWithJobGraphStore( configuration, highAvailabilityServices, CompletableFuture::new, @@ -119,13 +119,14 @@ public void testJobRecoveryUnderLeaderChange() throws Exception { new MemoryArchivedExecutionGraphStore(), fatalErrorHandler, VoidHistoryServerArchivist.INSTANCE, - null); + null, + highAvailabilityServices.getJobGraphStore()); final TestingJobManagerRunnerFactory jobManagerRunnerFactory = new TestingJobManagerRunnerFactory(1); try (final DispatcherRunnerImpl dispatcherRunner = new DispatcherRunnerImpl( new TestingDispatcherFactory(jobManagerRunnerFactory), rpcService, - partialDispatcherServices)) { + partialDispatcherServicesWithJobGraphStore)) { // initial run grantLeadership(dispatcherLeaderElectionService, dispatcherRunner); final TestingJobManagerRunner testingJobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); @@ -169,8 +170,8 @@ private TestingDispatcherFactory(JobManagerRunnerFactory jobManagerRunnerFactory } @Override - public Dispatcher createDispatcher(@Nonnull RpcService rpcService, @Nonnull PartialDispatcherServices partialDispatcherServices) throws Exception { - return new StandaloneDispatcher(rpcService, getEndpointId(), DispatcherServices.from(partialDispatcherServices, jobManagerRunnerFactory)); + public Dispatcher createDispatcher(@Nonnull RpcService rpcService, @Nonnull PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { + return new StandaloneDispatcher(rpcService, getEndpointId(), DispatcherServices.from(partialDispatcherServicesWithJobGraphStore, jobManagerRunnerFactory)); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDispatcherRunnerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDispatcherRunnerImplTest.java index 7199a38089b4..ba0059af9b06 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDispatcherRunnerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDispatcherRunnerImplTest.java @@ -28,7 +28,7 @@ import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.dispatcher.MemoryArchivedExecutionGraphStore; -import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobGraphStore; import org.apache.flink.runtime.dispatcher.SessionDispatcherFactory; import org.apache.flink.runtime.dispatcher.VoidHistoryServerArchivist; import org.apache.flink.runtime.heartbeat.TestingHeartbeatServices; @@ -142,7 +142,7 @@ public void testResourceCleanupUnderLeadershipChange() throws Exception { .setJobMasterLeaderRetrieverFunction(jobId -> ZooKeeperUtils.createLeaderRetrievalService(client, configuration)) .build()) { - final PartialDispatcherServices partialDispatcherServices = new PartialDispatcherServices( + final PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore = new PartialDispatcherServicesWithJobGraphStore( configuration, highAvailabilityServices, () -> new CompletableFuture<>(), @@ -152,11 +152,12 @@ public void testResourceCleanupUnderLeadershipChange() throws Exception { new MemoryArchivedExecutionGraphStore(), fatalErrorHandler, VoidHistoryServerArchivist.INSTANCE, - null); + null, + highAvailabilityServices.getJobGraphStore()); final JobGraph jobGraph = createJobGraphWithBlobs(); - try (final DispatcherRunnerImpl dispatcherRunner = new DispatcherRunnerImpl(SessionDispatcherFactory.INSTANCE, rpcService, partialDispatcherServices)) { + try (final DispatcherRunnerImpl dispatcherRunner = new DispatcherRunnerImpl(SessionDispatcherFactory.INSTANCE, rpcService, partialDispatcherServicesWithJobGraphStore)) { // initial run DispatcherGateway dispatcherGateway = grantLeadership(dispatcherLeaderElectionService, dispatcherLeaderRetriever, dispatcherRunner); 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 307c3cfeb38d..b681f4b312d1 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 @@ -21,7 +21,7 @@ import org.apache.flink.runtime.dispatcher.DefaultJobManagerRunnerFactory; import org.apache.flink.runtime.dispatcher.DispatcherFactory; import org.apache.flink.runtime.dispatcher.DispatcherServices; -import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobGraphStore; import org.apache.flink.runtime.dispatcher.StandaloneDispatcher; import org.apache.flink.runtime.rpc.RpcService; @@ -37,11 +37,11 @@ public enum SessionDispatcherWithUUIDFactory implements DispatcherFactory { @Override public StandaloneDispatcher createDispatcher( @Nonnull RpcService rpcService, - @Nonnull PartialDispatcherServices partialDispatcherServices) throws Exception { + @Nonnull PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { // create the default dispatcher return new StandaloneDispatcher( rpcService, generateEndpointIdWithUUID(), - DispatcherServices.from(partialDispatcherServices, DefaultJobManagerRunnerFactory.INSTANCE)); + DispatcherServices.from(partialDispatcherServicesWithJobGraphStore, DefaultJobManagerRunnerFactory.INSTANCE)); } } From 90db87ddb967d40551c8b540683bb2268fa46eb8 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 14 Sep 2019 21:56:46 +0200 Subject: [PATCH 255/746] [FLINK-11843] Remove JobGraphListener from Dispatcher --- .../flink/runtime/dispatcher/Dispatcher.java | 83 +------------ .../runner/DispatcherLeaderProcessImpl.java | 10 ++ .../runtime/dispatcher/DispatcherTest.java | 76 ------------ .../DispatcherLeaderProcessImplTest.java | 116 +++++++++++++++++- 4 files changed, 124 insertions(+), 161 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 c38ef28e6f5e..4b5b4f8806d4 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 @@ -99,7 +99,7 @@ * about the state of the Flink session cluster. */ public abstract class Dispatcher extends FencedRpcEndpoint implements - DispatcherGateway, LeaderContender, JobGraphStore.JobGraphListener { + DispatcherGateway, LeaderContender { public static final String DISPATCHER_NAME = "dispatcher"; @@ -201,7 +201,6 @@ public void onStart() throws Exception { private void startDispatcherServices() throws Exception { try { - jobGraphStore.start(this); leaderElectionService.start(this); registerDispatcherMetrics(jobManagerMetricGroup); @@ -243,12 +242,6 @@ private void stopDispatcherServices() throws Exception { exception = ExceptionUtils.firstOrSuppressed(e, exception); } - try { - jobGraphStore.stop(); - } catch (Exception e) { - exception = ExceptionUtils.firstOrSuppressed(e, exception); - } - try { leaderElectionService.stop(); } catch (Exception e) { @@ -1009,78 +1002,4 @@ public void revokeLeadership() { public void handleError(final Exception exception) { onFatalError(new DispatcherException("Received an error from the LeaderElectionService.", exception)); } - - //------------------------------------------------------ - // JobGraphListener - //------------------------------------------------------ - - @Override - public void onAddedJobGraph(final JobID jobId) { - runAsync( - () -> { - if (!jobManagerRunnerFutures.containsKey(jobId)) { - // IMPORTANT: onAddedJobGraph can generate false positives and, thus, we must expect that - // the specified job is already removed from the JobGraphStore. In this case, - // JobGraphStore.recoverJob returns null. - final CompletableFuture> recoveredJob = recoveryOperation.thenApplyAsync( - FunctionUtils.uncheckedFunction(ignored -> Optional.ofNullable(recoverJob(jobId))), - getRpcService().getExecutor()); - - final DispatcherId dispatcherId = getFencingToken(); - final CompletableFuture submissionFuture = recoveredJob.thenComposeAsync( - (Optional jobGraphOptional) -> jobGraphOptional.map( - FunctionUtils.uncheckedFunction(jobGraph -> tryRunRecoveredJobGraph(jobGraph, dispatcherId).thenAcceptAsync( - FunctionUtils.uncheckedConsumer((Boolean isRecoveredJobRunning) -> { - if (!isRecoveredJobRunning) { - jobGraphStore.releaseJobGraph(jobId); - } - }), - getRpcService().getExecutor()))) - .orElse(CompletableFuture.completedFuture(null)), - getUnfencedMainThreadExecutor()); - - submissionFuture.whenComplete( - (Void ignored, Throwable throwable) -> { - if (throwable != null) { - onFatalError( - new DispatcherException( - String.format("Could not start the added job %s", jobId), - ExceptionUtils.stripCompletionException(throwable))); - } - }); - - recoveryOperation = submissionFuture; - } - }); - } - - private CompletableFuture tryRunRecoveredJobGraph(JobGraph jobGraph, DispatcherId dispatcherId) throws Exception { - if (leaderElectionService.hasLeadership(dispatcherId.toUUID())) { - final JobID jobId = jobGraph.getJobID(); - if (jobManagerRunnerFutures.containsKey(jobId)) { - // we must not release the job graph lock since it can only be locked once and - // is currently being executed. Once we support multiple locks, we must release - // the JobGraph here - log.debug("Ignore added JobGraph because the job {} is already running.", jobId); - return CompletableFuture.completedFuture(true); - } else if (runningJobsRegistry.getJobSchedulingStatus(jobId) != RunningJobsRegistry.JobSchedulingStatus.DONE) { - return waitForTerminatingJobManager(jobId, jobGraph, this::runJob).thenApply(ignored -> true); - } else { - log.debug("Ignore added JobGraph because the job {} has already been completed.", jobId); - } - } - - return CompletableFuture.completedFuture(false); - } - - @Override - public void onRemovedJobGraph(final JobID jobId) { - runAsync(() -> { - try { - removeJobAndRegisterTerminationFuture(jobId, false); - } catch (final Exception e) { - onFatalError(new DispatcherException(String.format("Could not remove job %s.", jobId), e)); - } - }); - } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java index b079ab141c01..2791f1851efd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java @@ -277,6 +277,11 @@ public void onAddedJobGraph(JobID jobId) { } private void handleAddedJobGraph(JobID jobId) { + LOG.debug( + "Job {} has been added to the {} by another process.", + jobId, + jobGraphStore.getClass().getSimpleName()); + // serialize all ongoing recovery operations onGoingRecoveryOperation = onGoingRecoveryOperation .thenApplyAsync( @@ -317,6 +322,11 @@ public void onRemovedJobGraph(JobID jobId) { } private void handleRemovedJobGraph(JobID jobId) { + LOG.debug( + "Job {} has been removed from the {} by another process.", + jobId, + jobGraphStore.getClass().getSimpleName()); + onGoingRecoveryOperation = onGoingRecoveryOperation .thenCompose(ignored -> removeJobGraphIfRunning(jobId).orElse(FutureUtils.completedVoidFuture())) .handle(this::onErrorIfRunning); 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 46620142f154..bdb3b012913a 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 @@ -332,82 +332,6 @@ public void testLeaderElection() throws Exception { jobIdsFuture.get(TIMEOUT.toMilliseconds(), TimeUnit.MILLISECONDS); } - /** - * Test callbacks from - * {@link JobGraphStore.JobGraphListener}. - */ - @Test - public void testSubmittedJobGraphListener() throws Exception { - dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch)); - - dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); - - final DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); - - dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); - jobMasterLeaderElectionService.isLeader(UUID.randomUUID()).get(); - - final JobGraph jobGraph = jobGraphStore.recoverJobGraph(TEST_JOB_ID); - - // pretend that other Dispatcher has removed job from submittedJobGraphStore - 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); - jobGraphStore.putJobGraph(jobGraph); - dispatcher.onAddedJobGraph(TEST_JOB_ID); - createdJobManagerRunnerLatch.await(); - assertThat(dispatcherGateway.listJobs(TIMEOUT).get(), hasSize(1)); - } - - @Test - public void testOnAddedJobGraphRecoveryFailure() throws Exception { - final FlinkException expectedFailure = new FlinkException("Expected failure"); - 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(); - - jobGraphStore.putJobGraph(jobGraph); - dispatcher.onAddedJobGraph(TEST_JOB_ID); - - final CompletableFuture errorFuture = fatalErrorHandler.getErrorFuture(); - - final Throwable throwable = errorFuture.get(); - - assertThat(ExceptionUtils.findThrowable(throwable, expectedFailure::equals).isPresent(), is(true)); - - fatalErrorHandler.clearError(); - } - - @Test - public void testOnAddedJobGraphWithFinishedJob() throws Throwable { - dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch)); - - dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); - - jobGraphStore.putJobGraph(jobGraph); - runningJobsRegistry.setJobFinished(TEST_JOB_ID); - dispatcher.onAddedJobGraph(TEST_JOB_ID); - - // wait until the recovery is over - dispatcher.getRecoverOperationFuture(TIMEOUT).get(); - - final DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); - - // check that we did not start executing the added JobGraph - assertThat(dispatcherGateway.listJobs(TIMEOUT).get(), is(empty())); - } - /** * Test that {@link JobResult} is cached when the job finishes. */ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java index 02ad01041b19..61d2099d3635 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java @@ -18,20 +18,27 @@ package org.apache.flink.runtime.dispatcher.runner; +import org.apache.flink.api.common.JobID; import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.JobGraphStore; +import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.testutils.TestingJobGraphStore; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.runtime.webmonitor.TestingDispatcherGateway; +import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.ExecutorUtils; +import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; import org.apache.flink.util.function.BiFunctionWithException; +import org.hamcrest.core.Is; import org.junit.After; import org.junit.AfterClass; +import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.Ignore; import org.junit.Test; import java.util.Collection; @@ -54,7 +61,7 @@ */ public class DispatcherLeaderProcessImplTest extends TestLogger { - private static final JobGraph PERSISTED_JOB_GRAPH = new JobGraph("Persisted JobGraph"); + private static final JobGraph JOB_GRAPH = new JobGraph("JobGraph"); private static ExecutorService ioExecutor; @@ -106,7 +113,7 @@ public void start_afterClose_doesNotHaveAnEffect() throws Exception { @Test public void start_triggersJobGraphRecoveryAndDispatcherServiceCreation() throws Exception { jobGraphStore = TestingJobGraphStore.newBuilder() - .setInitialJobGraphs(Collections.singleton(PERSISTED_JOB_GRAPH)) + .setInitialJobGraphs(Collections.singleton(JOB_GRAPH)) .build(); final CompletableFuture> recoveredJobGraphsFuture = new CompletableFuture<>(); @@ -126,7 +133,7 @@ public void start_triggersJobGraphRecoveryAndDispatcherServiceCreation() throws final Collection recoveredJobGraphs = recoveredJobGraphsFuture.get(); assertThat(recoveredJobGraphs, hasSize(1)); - assertThat(recoveredJobGraphs, containsInAnyOrder(PERSISTED_JOB_GRAPH)); + assertThat(recoveredJobGraphs, containsInAnyOrder(JOB_GRAPH)); } } @@ -239,6 +246,109 @@ public void closeAsync_duringJobRecovery_preventsDispatcherServiceCreation() thr } } + @Test + public void onRemovedJobGraph_cancelsRunningJob() throws Exception { + jobGraphStore = TestingJobGraphStore.newBuilder() + .setInitialJobGraphs(Collections.singleton(JOB_GRAPH)) + .build(); + + final CompletableFuture cancelJobFuture = new CompletableFuture<>(); + final TestingDispatcherGateway testingDispatcherGateway = new TestingDispatcherGateway.Builder() + .setCancelJobFunction( + jobToCancel -> { + cancelJobFuture.complete(jobToCancel); + return CompletableFuture.completedFuture(Acknowledge.get()); + }) + .build(); + + dispatcherServiceFactory = createDispatcherServiceFactoryFor(testingDispatcherGateway); + + try (final DispatcherLeaderProcessImpl dispatcherLeaderProcess = createDispatcherLeaderProcess()) { + dispatcherLeaderProcess.start(); + + // wait for the dispatcher process to be created + dispatcherLeaderProcess.getDispatcherGateway().get(); + + // now remove the Job from the JobGraphStore and notify the dispatcher service + jobGraphStore.removeJobGraph(JOB_GRAPH.getJobID()); + dispatcherLeaderProcess.onRemovedJobGraph(JOB_GRAPH.getJobID()); + + assertThat(cancelJobFuture.get(), is(JOB_GRAPH.getJobID())); + } + } + + @Test + public void onAddedJobGraph_submitsRecoveredJob() throws Exception { + final CompletableFuture submittedJobFuture = new CompletableFuture<>(); + final TestingDispatcherGateway testingDispatcherGateway = new TestingDispatcherGateway.Builder() + .setSubmitFunction( + submittedJob -> { + submittedJobFuture.complete(submittedJob); + return CompletableFuture.completedFuture(Acknowledge.get()); + }) + .build(); + + dispatcherServiceFactory = createDispatcherServiceFactoryFor(testingDispatcherGateway); + + try (final DispatcherLeaderProcessImpl dispatcherLeaderProcess = createDispatcherLeaderProcess()) { + dispatcherLeaderProcess.start(); + + // wait first for the dispatcher service to be created + dispatcherLeaderProcess.getDispatcherGateway().get(); + + jobGraphStore.putJobGraph(JOB_GRAPH); + dispatcherLeaderProcess.onAddedJobGraph(JOB_GRAPH.getJobID()); + + final JobGraph submittedJobGraph = submittedJobFuture.get(); + + assertThat(submittedJobGraph.getJobID(), is(JOB_GRAPH.getJobID())); + } + } + + @Test + public void onAddedJobGraph_failingRecovery_propagatesTheFailure() throws Exception { + final FlinkException expectedFailure = new FlinkException("Expected failure"); + jobGraphStore = TestingJobGraphStore.newBuilder() + .setRecoverJobGraphFunction( + (ignoredA, ignoredB) -> { + throw expectedFailure; + }) + .build(); + + try (final DispatcherLeaderProcessImpl dispatcherLeaderProcess = createDispatcherLeaderProcess()) { + dispatcherLeaderProcess.start(); + + // wait first for the dispatcher service to be created + dispatcherLeaderProcess.getDispatcherGateway().get(); + + jobGraphStore.putJobGraph(JOB_GRAPH); + dispatcherLeaderProcess.onAddedJobGraph(JOB_GRAPH.getJobID()); + + final CompletableFuture errorFuture = fatalErrorHandler.getErrorFuture(); + final Throwable throwable = errorFuture.get(); + Assert.assertThat(ExceptionUtils.findThrowable(throwable, expectedFailure::equals).isPresent(), Is.is(true)); + + assertThat(dispatcherLeaderProcess.getState(), is(DispatcherLeaderProcessImpl.State.STOPPED)); + + fatalErrorHandler.clearError(); + } + } + + @Test + @Ignore + public void onAddedJobGraph_falsePositive_willBeIgnored() { + fail("Needs to be implemented once the proper deduplication mechanism is in place."); + } + + private TestingDispatcherServiceFactory createDispatcherServiceFactoryFor(TestingDispatcherGateway testingDispatcherGateway) { + return TestingDispatcherServiceFactory.newBuilder() + .setCreateFunction( + (ignoredA, ignoredB) -> TestingDispatcherService.newBuilder() + .setDispatcherGateway(testingDispatcherGateway) + .build()) + .build(); + } + private DispatcherLeaderProcessImpl createDispatcherLeaderProcess() { return DispatcherLeaderProcessImpl.create( leaderSessionId, From 0b9382bb3b39e86b894c4d845bf4d86857602507 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 14 Sep 2019 22:15:51 +0200 Subject: [PATCH 256/746] [FLINK-11843] Allow passing collection of recovered jobs to Dispatcher --- .../flink/runtime/dispatcher/Dispatcher.java | 18 +++++++++++++++++- .../runtime/dispatcher/DispatcherFactory.java | 3 +++ .../dispatcher/JobDispatcherFactory.java | 3 +++ .../runtime/dispatcher/MiniDispatcher.java | 2 ++ .../dispatcher/SessionDispatcherFactory.java | 5 +++++ .../dispatcher/StandaloneDispatcher.java | 4 ++++ .../runner/DispatcherRunnerImpl.java | 2 ++ .../runner/DispatcherServiceImplFactory.java | 1 + .../runtime/dispatcher/DispatcherHATest.java | 3 +++ .../DispatcherResourceCleanupTest.java | 2 ++ .../runtime/dispatcher/DispatcherTest.java | 1 + .../runtime/dispatcher/TestingDispatcher.java | 4 ++++ .../dispatcher/ZooKeeperHADispatcherTest.java | 2 ++ .../runner/DispatcherRunnerImplTest.java | 12 ++++++++++-- .../SessionDispatcherWithUUIDFactory.java | 5 +++++ 15 files changed, 64 insertions(+), 3 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 4b5b4f8806d4..8699fdfc8fda 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 @@ -117,6 +117,7 @@ public abstract class Dispatcher extends FencedRpcEndpoint impleme private final FatalErrorHandler fatalErrorHandler; private final Map> jobManagerRunnerFutures; + private final Collection recoveredJobs; private final LeaderElectionService leaderElectionService; @@ -140,6 +141,7 @@ public abstract class Dispatcher extends FencedRpcEndpoint impleme public Dispatcher( RpcService rpcService, String endpointId, + Collection recoveredJobs, DispatcherServices dispatcherServices, JobGraphStore jobGraphStore) throws Exception { super(rpcService, endpointId, null); @@ -174,6 +176,8 @@ public Dispatcher( this.jobManagerTerminationFutures = new HashMap<>(2); this.shutDownFuture = new CompletableFuture<>(); + + this.recoveredJobs = new HashSet<>(recoveredJobs); } //------------------------------------------------------ @@ -715,8 +719,10 @@ Collection recoverJobs() throws Exception { log.info("Recovering all persisted jobs."); final Collection jobIds = jobGraphStore.getJobIds(); + final Collection jobGraphs; + try { - return recoverJobGraphs(jobIds); + jobGraphs = recoverJobGraphs(jobIds); } catch (Exception e) { // release all recovered job graphs for (JobID jobId : jobIds) { @@ -728,6 +734,16 @@ Collection recoverJobs() throws Exception { } throw e; } + + return deduplicateJobs(jobGraphs); + } + + private Collection deduplicateJobs(Collection jobGraphs) { + final HashSet result = new HashSet<>(recoveredJobs); + + result.addAll(jobGraphs); + + return result; } @Nonnull 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 933a413d7047..44c8eafee263 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,10 +18,12 @@ package org.apache.flink.runtime.dispatcher; +import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.rpc.RpcService; import javax.annotation.Nonnull; +import java.util.Collection; import java.util.UUID; /** @@ -34,6 +36,7 @@ public interface DispatcherFactory { */ Dispatcher createDispatcher( @Nonnull RpcService rpcService, + @Nonnull Collection recoveredJobs, @Nonnull PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception; default String generateEndpointIdWithUUID() { 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 a4349fee2ce0..2b5c73474991 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 @@ -26,6 +26,8 @@ import javax.annotation.Nonnull; +import java.util.Collection; + import static org.apache.flink.runtime.entrypoint.ClusterEntrypoint.EXECUTION_MODE; /** @@ -42,6 +44,7 @@ public JobDispatcherFactory(JobGraphRetriever jobGraphRetriever) { @Override public MiniDispatcher createDispatcher( @Nonnull RpcService rpcService, + @Nonnull Collection recoveredJobs, @Nonnull PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { final Configuration configuration = partialDispatcherServicesWithJobGraphStore.getConfiguration(); final JobGraph jobGraph = jobGraphRetriever.retrieveJobGraph(configuration); 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 2fc805c34fbc..3405edb09035 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 @@ -30,6 +30,7 @@ import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.util.FlinkException; +import java.util.Collections; import java.util.concurrent.CompletableFuture; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -55,6 +56,7 @@ public MiniDispatcher( super( rpcService, endpointId, + Collections.singleton(jobGraph), dispatcherServices, new SingleJobJobGraphStore(jobGraph)); 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 4be749e76456..746d6f5d7fe4 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,8 +18,11 @@ package org.apache.flink.runtime.dispatcher; +import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.rpc.RpcService; +import java.util.Collection; + /** * {@link DispatcherFactory} which creates a {@link StandaloneDispatcher}. */ @@ -29,11 +32,13 @@ public enum SessionDispatcherFactory implements DispatcherFactory { @Override public StandaloneDispatcher createDispatcher( RpcService rpcService, + Collection recoveredJobs, PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { // create the default dispatcher return new StandaloneDispatcher( rpcService, getEndpointId(), + recoveredJobs, DispatcherServices.from(partialDispatcherServicesWithJobGraphStore, 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 31be67858bd0..c43711a469cc 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 @@ -22,6 +22,8 @@ import org.apache.flink.runtime.jobmaster.JobMaster; import org.apache.flink.runtime.rpc.RpcService; +import java.util.Collection; + /** * Dispatcher implementation which spawns a {@link JobMaster} for each * submitted {@link JobGraph} within in the same process. This dispatcher @@ -31,10 +33,12 @@ public class StandaloneDispatcher extends Dispatcher { public StandaloneDispatcher( RpcService rpcService, String endpointId, + Collection recoveredJobs, DispatcherServices dispatcherServices) throws Exception { super( rpcService, endpointId, + recoveredJobs, dispatcherServices, dispatcherServices.getJobGraphStore()); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java index 7eb6c5967e5b..55432dff0a45 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java @@ -30,6 +30,7 @@ import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.concurrent.CompletableFuture; /** @@ -49,6 +50,7 @@ class DispatcherRunnerImpl implements DispatcherRunner { PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { this.dispatcher = dispatcherFactory.createDispatcher( rpcService, + Collections.emptyList(), partialDispatcherServicesWithJobGraphStore); this.leaderRetrievalService = partialDispatcherServicesWithJobGraphStore.getHighAvailabilityServices().getDispatcherLeaderRetriever(); this.leaderRetriever = new LeaderRetriever(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.java index ea8c10b7b723..5f099f576fb6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.java @@ -55,6 +55,7 @@ public DispatcherLeaderProcessImpl.DispatcherService create(Collection try { dispatcher = dispatcherFactory.createDispatcher( rpcService, + recoveredJobs, PartialDispatcherServicesWithJobGraphStore.from(partialDispatcherServices, jobGraphStore)); } catch (Exception e) { throw new FlinkRuntimeException("Could not create the Dispatcher rpc endpoint.", e); 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 b25c60e7e638..474b39ac89e4 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 @@ -314,6 +314,7 @@ private HATestingDispatcher createDispatcher( return new HATestingDispatcher( rpcService, UUID.randomUUID().toString(), + Collections.emptyList(), configuration, highAvailabilityServices, () -> CompletableFuture.completedFuture(resourceManagerGateway), @@ -345,6 +346,7 @@ private static class HATestingDispatcher extends TestingDispatcher { HATestingDispatcher( RpcService rpcService, String endpointId, + Collection recoveredJobs, Configuration configuration, HighAvailabilityServices highAvailabilityServices, GatewayRetriever resourceManagerGatewayRetriever, @@ -359,6 +361,7 @@ private static class HATestingDispatcher extends TestingDispatcher { super( rpcService, endpointId, + recoveredJobs, configuration, highAvailabilityServices, resourceManagerGatewayRetriever, 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 b0979a64fc66..271b46455a9e 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 @@ -75,6 +75,7 @@ import java.io.File; import java.io.IOException; +import java.util.Collections; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -200,6 +201,7 @@ private void startDispatcher(JobManagerRunnerFactory jobManagerRunnerFactory) th dispatcher = new TestingDispatcher( rpcService, Dispatcher.DISPATCHER_NAME + UUID.randomUUID(), + Collections.emptyList(), configuration, highAvailabilityServices, () -> CompletableFuture.completedFuture(resourceManagerGateway), 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 bdb3b012913a..da956997e318 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 @@ -216,6 +216,7 @@ private TestingDispatcher createDispatcher(HeartbeatServices heartbeatServices, return new TestingDispatcher( rpcService, Dispatcher.DISPATCHER_NAME + '_' + name.getMethodName(), + Collections.emptyList(), configuration, haServices, () -> CompletableFuture.completedFuture(resourceManagerGateway), 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 4f0419488fac..803caa5127cc 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 @@ -25,6 +25,7 @@ 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.metrics.groups.JobManagerMetricGroup; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rpc.FatalErrorHandler; @@ -34,6 +35,7 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; +import java.util.Collection; import java.util.concurrent.CompletableFuture; import java.util.function.Function; @@ -47,6 +49,7 @@ class TestingDispatcher extends Dispatcher { TestingDispatcher( RpcService rpcService, String endpointId, + Collection recoveredJobs, Configuration configuration, HighAvailabilityServices highAvailabilityServices, GatewayRetriever resourceManagerGatewayRetriever, @@ -60,6 +63,7 @@ class TestingDispatcher extends Dispatcher { super( rpcService, endpointId, + recoveredJobs, new DispatcherServices( configuration, highAvailabilityServices, 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 b40946c7431f..d732075151d0 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 @@ -64,6 +64,7 @@ import java.io.IOException; import java.time.Duration; import java.util.Collection; +import java.util.Collections; import java.util.UUID; import java.util.concurrent.CompletableFuture; @@ -334,6 +335,7 @@ private TestingDispatcher createDispatcher(HighAvailabilityServices highAvailabi return new TestingDispatcher( rpcService, Dispatcher.DISPATCHER_NAME + '_' + name.getMethodName() + UUID.randomUUID(), + Collections.emptyList(), configuration, highAvailabilityServices, () -> CompletableFuture.completedFuture(resourceManagerGateway), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java index f670ea21f77b..57d1b3b38f45 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java @@ -56,6 +56,7 @@ import javax.annotation.Nonnull; +import java.util.Collection; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @@ -170,8 +171,15 @@ private TestingDispatcherFactory(JobManagerRunnerFactory jobManagerRunnerFactory } @Override - public Dispatcher createDispatcher(@Nonnull RpcService rpcService, @Nonnull PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { - return new StandaloneDispatcher(rpcService, getEndpointId(), DispatcherServices.from(partialDispatcherServicesWithJobGraphStore, jobManagerRunnerFactory)); + public Dispatcher createDispatcher( + @Nonnull RpcService rpcService, + @Nonnull Collection recoveredJobs, + @Nonnull PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { + return new StandaloneDispatcher( + rpcService, + getEndpointId(), + recoveredJobs, + DispatcherServices.from(partialDispatcherServicesWithJobGraphStore, jobManagerRunnerFactory)); } } } 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 b681f4b312d1..97a2871dea84 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 @@ -23,10 +23,13 @@ import org.apache.flink.runtime.dispatcher.DispatcherServices; import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobGraphStore; import org.apache.flink.runtime.dispatcher.StandaloneDispatcher; +import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.rpc.RpcService; import javax.annotation.Nonnull; +import java.util.Collection; + /** * {@link DispatcherFactory} which creates a {@link StandaloneDispatcher} which has an * endpoint id with a random UUID suffix. @@ -37,11 +40,13 @@ public enum SessionDispatcherWithUUIDFactory implements DispatcherFactory { @Override public StandaloneDispatcher createDispatcher( @Nonnull RpcService rpcService, + @Nonnull Collection recoveredJobs, @Nonnull PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { // create the default dispatcher return new StandaloneDispatcher( rpcService, generateEndpointIdWithUUID(), + recoveredJobs, DispatcherServices.from(partialDispatcherServicesWithJobGraphStore, DefaultJobManagerRunnerFactory.INSTANCE)); } } From 07c5b720fcf4cd74ee196da16465c1384e105eae Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 14 Sep 2019 22:37:49 +0200 Subject: [PATCH 257/746] [FLINK-11843] Remove job recovery from Dispatcher --- .../flink/runtime/dispatcher/Dispatcher.java | 79 +------------------ 1 file changed, 4 insertions(+), 75 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 8699fdfc8fda..e98ad42b23de 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 @@ -69,7 +69,6 @@ import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; import org.apache.flink.util.function.BiConsumerWithException; -import org.apache.flink.util.function.BiFunctionWithException; import org.apache.flink.util.function.CheckedSupplier; import org.apache.flink.util.function.FunctionUtils; import org.apache.flink.util.function.FunctionWithException; @@ -711,64 +710,6 @@ private CompletableFuture terminateJobManagerRunnersAndGetTerminationFutur return FutureUtils.completeAll(values); } - /** - * Recovers all jobs persisted via the submitted job graph store. - */ - @VisibleForTesting - Collection recoverJobs() throws Exception { - log.info("Recovering all persisted jobs."); - final Collection jobIds = jobGraphStore.getJobIds(); - - final Collection jobGraphs; - - try { - jobGraphs = recoverJobGraphs(jobIds); - } catch (Exception e) { - // release all recovered job graphs - for (JobID jobId : jobIds) { - try { - jobGraphStore.releaseJobGraph(jobId); - } catch (Exception ie) { - e.addSuppressed(ie); - } - } - throw e; - } - - return deduplicateJobs(jobGraphs); - } - - private Collection deduplicateJobs(Collection jobGraphs) { - final HashSet result = new HashSet<>(recoveredJobs); - - result.addAll(jobGraphs); - - return result; - } - - @Nonnull - private Collection recoverJobGraphs(Collection jobIds) throws Exception { - final List jobGraphs = new ArrayList<>(jobIds.size()); - - for (JobID jobId : jobIds) { - final JobGraph jobGraph = recoverJob(jobId); - - if (jobGraph == null) { - throw new FlinkJobNotFoundException(jobId); - } - - jobGraphs.add(jobGraph); - } - - return jobGraphs; - } - - @Nullable - private JobGraph recoverJob(JobID jobId) throws Exception { - log.debug("Recover job {}.", jobId); - return jobGraphStore.recoverJobGraph(jobId); - } - protected void onFatalError(Throwable throwable) { fatalErrorHandler.onFatalError(throwable); } @@ -892,26 +833,14 @@ public void grantLeadership(final UUID newLeaderSessionID) { () -> { log.info("Dispatcher {} was granted leadership with fencing token {}", getAddress(), newLeaderSessionID); - final CompletableFuture> recoveredJobsFuture = recoveryOperation.thenApplyAsync( - FunctionUtils.uncheckedFunction(ignored -> recoverJobs()), - getRpcService().getExecutor()); - - final CompletableFuture fencingTokenFuture = recoveredJobsFuture.thenComposeAsync( - (Collection recoveredJobs) -> tryAcceptLeadershipAndRunJobs(newLeaderSessionID, recoveredJobs), - getUnfencedMainThreadExecutor()); + final CompletableFuture fencingTokenFuture = tryAcceptLeadershipAndRunJobs(newLeaderSessionID, recoveredJobs); - final CompletableFuture confirmationFuture = fencingTokenFuture.thenCombineAsync( - recoveredJobsFuture, - BiFunctionWithException.unchecked((Boolean confirmLeadership, Collection recoveredJobs) -> { + final CompletableFuture confirmationFuture = fencingTokenFuture.thenAcceptAsync( + (Boolean confirmLeadership) -> { if (confirmLeadership) { leaderElectionService.confirmLeadership(newLeaderSessionID, getAddress()); - } else { - for (JobGraph recoveredJob : recoveredJobs) { - jobGraphStore.releaseJobGraph(recoveredJob.getJobID()); - } } - return null; - }), + }, getRpcService().getExecutor()); confirmationFuture.whenComplete( From 358d46c78447f37758a665cde25a05d6c0db5648 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 14 Sep 2019 22:43:54 +0200 Subject: [PATCH 258/746] [FLINK-11843] Restrict Dispatcher to only use JobGraphWriter instead of JobGraphStore --- .../flink/runtime/dispatcher/Dispatcher.java | 16 ++++++++-------- .../runtime/dispatcher/DispatcherServices.java | 14 +++++++------- ...rtialDispatcherServicesWithJobGraphStore.java | 16 ++++++++-------- .../runtime/dispatcher/StandaloneDispatcher.java | 2 +- .../runner/DispatcherLeaderProcessImpl.java | 3 ++- .../runner/DispatcherServiceImplFactory.java | 6 +++--- .../runner/TestingDispatcherServiceFactory.java | 14 +++++++------- 7 files changed, 36 insertions(+), 35 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 e98ad42b23de..d86814935c94 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 @@ -40,7 +40,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.jobmanager.JobGraphStore; +import org.apache.flink.runtime.jobmanager.JobGraphWriter; import org.apache.flink.runtime.jobmaster.JobManagerRunner; import org.apache.flink.runtime.jobmaster.JobManagerRunnerImpl; import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; @@ -104,7 +104,7 @@ public abstract class Dispatcher extends FencedRpcEndpoint impleme private final Configuration configuration; - private final JobGraphStore jobGraphStore; + private final JobGraphWriter jobGraphWriter; private final RunningJobsRegistry runningJobsRegistry; private final HighAvailabilityServices highAvailabilityServices; @@ -142,7 +142,7 @@ public Dispatcher( String endpointId, Collection recoveredJobs, DispatcherServices dispatcherServices, - JobGraphStore jobGraphStore) throws Exception { + JobGraphWriter jobGraphWriter) throws Exception { super(rpcService, endpointId, null); Preconditions.checkNotNull(dispatcherServices); @@ -152,7 +152,7 @@ public Dispatcher( this.heartbeatServices = dispatcherServices.getHeartbeatServices(); this.blobServer = dispatcherServices.getBlobServer(); this.fatalErrorHandler = dispatcherServices.getFatalErrorHandler(); - this.jobGraphStore = Preconditions.checkNotNull(jobGraphStore); + this.jobGraphWriter = Preconditions.checkNotNull(jobGraphWriter); this.jobManagerMetricGroup = dispatcherServices.getJobManagerMetricGroup(); this.metricServiceQueryAddress = dispatcherServices.getMetricQueryServiceAddress(); @@ -339,13 +339,13 @@ private CompletableFuture internalSubmitJob(JobGraph jobGraph) { } private CompletableFuture persistAndRunJob(JobGraph jobGraph) throws Exception { - jobGraphStore.putJobGraph(jobGraph); + jobGraphWriter.putJobGraph(jobGraph); final CompletableFuture runJobFuture = runJob(jobGraph); return runJobFuture.whenComplete(BiConsumerWithException.unchecked((Object ignored, Throwable throwable) -> { if (throwable != null) { - jobGraphStore.removeJobGraph(jobGraph.getJobID()); + jobGraphWriter.removeJobGraph(jobGraph.getJobID()); } })); } @@ -667,7 +667,7 @@ private void cleanUpJobData(JobID jobId, boolean cleanupHA) { boolean cleanupHABlobs = false; if (cleanupHA) { try { - jobGraphStore.removeJobGraph(jobId); + jobGraphWriter.removeJobGraph(jobId); // only clean up the HA blobs if we could remove the job from HA storage cleanupHABlobs = true; @@ -682,7 +682,7 @@ private void cleanUpJobData(JobID jobId, boolean cleanupHA) { } } else { try { - jobGraphStore.releaseJobGraph(jobId); + jobGraphWriter.releaseJobGraph(jobId); } catch (Exception e) { log.warn("Could not properly release job {} from submitted job graph store.", jobId, e); } 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 5bca3fec84cd..89cef28d869f 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 @@ -22,7 +22,7 @@ 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.jobmanager.JobGraphStore; +import org.apache.flink.runtime.jobmanager.JobGraphWriter; import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rpc.FatalErrorHandler; @@ -67,7 +67,7 @@ public class DispatcherServices { private final String metricQueryServiceAddress; @Nonnull - private final JobGraphStore jobGraphStore; + private final JobGraphWriter jobGraphWriter; @Nonnull private final JobManagerRunnerFactory jobManagerRunnerFactory; @@ -83,7 +83,7 @@ public DispatcherServices( @Nonnull HistoryServerArchivist historyServerArchivist, @Nullable String metricQueryServiceAddress, @Nonnull JobManagerMetricGroup jobManagerMetricGroup, - @Nonnull JobGraphStore jobGraphStore, + @Nonnull JobGraphWriter jobGraphWriter, @Nonnull JobManagerRunnerFactory jobManagerRunnerFactory) { this.configuration = configuration; this.highAvailabilityServices = highAvailabilityServices; @@ -95,7 +95,7 @@ public DispatcherServices( this.historyServerArchivist = historyServerArchivist; this.metricQueryServiceAddress = metricQueryServiceAddress; this.jobManagerMetricGroup = jobManagerMetricGroup; - this.jobGraphStore = jobGraphStore; + this.jobGraphWriter = jobGraphWriter; this.jobManagerRunnerFactory = jobManagerRunnerFactory; } @@ -150,8 +150,8 @@ public String getMetricQueryServiceAddress() { } @Nonnull - public JobGraphStore getJobGraphStore() { - return jobGraphStore; + public JobGraphWriter getJobGraphWriter() { + return jobGraphWriter; } @Nonnull @@ -173,7 +173,7 @@ public static DispatcherServices from( partialDispatcherServicesWithJobGraphStore.getHistoryServerArchivist(), partialDispatcherServicesWithJobGraphStore.getMetricQueryServiceAddress(), partialDispatcherServicesWithJobGraphStore.getJobManagerMetricGroupFactory().create(), - partialDispatcherServicesWithJobGraphStore.getJobGraphStore(), + partialDispatcherServicesWithJobGraphStore.getJobGraphWriter(), jobManagerRunnerFactory); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobGraphStore.java index 624c8c937973..0eec923c710c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobGraphStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobGraphStore.java @@ -22,7 +22,7 @@ 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.jobmanager.JobGraphStore; +import org.apache.flink.runtime.jobmanager.JobGraphWriter; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; @@ -36,7 +36,7 @@ public class PartialDispatcherServicesWithJobGraphStore extends PartialDispatcherServices { @Nonnull - private final JobGraphStore jobGraphStore; + private final JobGraphWriter jobGraphWriter; public PartialDispatcherServicesWithJobGraphStore( @Nonnull Configuration configuration, @@ -49,7 +49,7 @@ public PartialDispatcherServicesWithJobGraphStore( @Nonnull FatalErrorHandler fatalErrorHandler, @Nonnull HistoryServerArchivist historyServerArchivist, @Nullable String metricQueryServiceAddress, - @Nonnull JobGraphStore jobGraphStore) { + @Nonnull JobGraphWriter jobGraphWriter) { super( configuration, highAvailabilityServices, @@ -61,15 +61,15 @@ public PartialDispatcherServicesWithJobGraphStore( fatalErrorHandler, historyServerArchivist, metricQueryServiceAddress); - this.jobGraphStore = jobGraphStore; + this.jobGraphWriter = jobGraphWriter; } @Nonnull - public JobGraphStore getJobGraphStore() { - return jobGraphStore; + public JobGraphWriter getJobGraphWriter() { + return jobGraphWriter; } - public static PartialDispatcherServicesWithJobGraphStore from(PartialDispatcherServices partialDispatcherServices, JobGraphStore jobGraphStore) { + public static PartialDispatcherServicesWithJobGraphStore from(PartialDispatcherServices partialDispatcherServices, JobGraphWriter jobGraphWriter) { return new PartialDispatcherServicesWithJobGraphStore( partialDispatcherServices.getConfiguration(), partialDispatcherServices.getHighAvailabilityServices(), @@ -81,6 +81,6 @@ public static PartialDispatcherServicesWithJobGraphStore from(PartialDispatcherS partialDispatcherServices.getFatalErrorHandler(), partialDispatcherServices.getHistoryServerArchivist(), partialDispatcherServices.getMetricQueryServiceAddress(), - jobGraphStore); + jobGraphWriter); } } 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 c43711a469cc..96c1c705cf94 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 @@ -40,6 +40,6 @@ public StandaloneDispatcher( endpointId, recoveredJobs, dispatcherServices, - dispatcherServices.getJobGraphStore()); + dispatcherServices.getJobGraphWriter()); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java index 2791f1851efd..9b2f73a3441a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java @@ -24,6 +24,7 @@ import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.JobGraphStore; +import org.apache.flink.runtime.jobmanager.JobGraphWriter; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.webmonitor.RestfulGateway; @@ -410,7 +411,7 @@ private void runIfState(Predicate actionPredicate, Runnable action) { // ------------------------------------------------------------ interface DispatcherServiceFactory { - DispatcherService create(Collection recoveredJobs, JobGraphStore jobGraphStore); + DispatcherService create(Collection recoveredJobs, JobGraphWriter jobGraphWriter); } interface DispatcherService extends AutoCloseableAsync { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.java index 5f099f576fb6..860e0a088c74 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.java @@ -23,7 +23,7 @@ import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobGraphStore; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobmanager.JobGraphStore; +import org.apache.flink.runtime.jobmanager.JobGraphWriter; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.util.FlinkRuntimeException; @@ -50,13 +50,13 @@ class DispatcherServiceImplFactory implements DispatcherLeaderProcessImpl.Dispat } @Override - public DispatcherLeaderProcessImpl.DispatcherService create(Collection recoveredJobs, JobGraphStore jobGraphStore) { + public DispatcherLeaderProcessImpl.DispatcherService create(Collection recoveredJobs, JobGraphWriter jobGraphWriter) { final Dispatcher dispatcher; try { dispatcher = dispatcherFactory.createDispatcher( rpcService, recoveredJobs, - PartialDispatcherServicesWithJobGraphStore.from(partialDispatcherServices, jobGraphStore)); + PartialDispatcherServicesWithJobGraphStore.from(partialDispatcherServices, jobGraphWriter)); } catch (Exception e) { throw new FlinkRuntimeException("Could not create the Dispatcher rpc endpoint.", e); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java index 7b0faeed98cd..469709145cef 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java @@ -19,21 +19,21 @@ package org.apache.flink.runtime.dispatcher.runner; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobmanager.JobGraphStore; +import org.apache.flink.runtime.jobmanager.JobGraphWriter; import java.util.Collection; import java.util.function.BiFunction; class TestingDispatcherServiceFactory implements DispatcherLeaderProcessImpl.DispatcherServiceFactory { - private final BiFunction, JobGraphStore, DispatcherLeaderProcessImpl.DispatcherService> createFunction; + private final BiFunction, JobGraphWriter, DispatcherLeaderProcessImpl.DispatcherService> createFunction; - private TestingDispatcherServiceFactory(BiFunction, JobGraphStore, DispatcherLeaderProcessImpl.DispatcherService> createFunction) { + private TestingDispatcherServiceFactory(BiFunction, JobGraphWriter, DispatcherLeaderProcessImpl.DispatcherService> createFunction) { this.createFunction = createFunction; } @Override - public DispatcherLeaderProcessImpl.DispatcherService create(Collection recoveredJobs, JobGraphStore jobGraphStore) { - return createFunction.apply(recoveredJobs, jobGraphStore); + public DispatcherLeaderProcessImpl.DispatcherService create(Collection recoveredJobs, JobGraphWriter jobGraphWriter) { + return createFunction.apply(recoveredJobs, jobGraphWriter); } public static Builder newBuilder() { @@ -41,11 +41,11 @@ public static Builder newBuilder() { } public static class Builder { - private BiFunction, JobGraphStore, DispatcherLeaderProcessImpl.DispatcherService> createFunction = (ignoredA, ignoredB) -> TestingDispatcherService.newBuilder().build(); + private BiFunction, JobGraphWriter, DispatcherLeaderProcessImpl.DispatcherService> createFunction = (ignoredA, ignoredB) -> TestingDispatcherService.newBuilder().build(); private Builder() {} - Builder setCreateFunction(BiFunction, JobGraphStore, DispatcherLeaderProcessImpl.DispatcherService> createFunction) { + Builder setCreateFunction(BiFunction, JobGraphWriter, DispatcherLeaderProcessImpl.DispatcherService> createFunction) { this.createFunction = createFunction; return this; } From f67349db10b6dfc40ec0e76f8fe50f30f3f243e6 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 15 Sep 2019 16:37:16 +0200 Subject: [PATCH 259/746] [FLINK-11843] Make Dispatcher a PermanentlyFencedRpcEndpoint --- .../flink/runtime/dispatcher/Dispatcher.java | 152 ++++-------------- .../runtime/dispatcher/DispatcherFactory.java | 1 + .../dispatcher/JobDispatcherFactory.java | 2 + .../runtime/dispatcher/MiniDispatcher.java | 2 + .../dispatcher/SessionDispatcherFactory.java | 2 + .../dispatcher/StandaloneDispatcher.java | 2 + .../runner/DispatcherLeaderProcessImpl.java | 11 +- .../runner/DispatcherRunnerImpl.java | 2 + .../runner/DispatcherServiceImplFactory.java | 7 +- .../runtime/dispatcher/DispatcherHATest.java | 3 + .../DispatcherResourceCleanupTest.java | 1 + .../runtime/dispatcher/DispatcherTest.java | 1 + .../dispatcher/MiniDispatcherTest.java | 1 + .../runtime/dispatcher/TestingDispatcher.java | 8 +- .../dispatcher/ZooKeeperHADispatcherTest.java | 1 + .../DispatcherLeaderProcessImplTest.java | 12 +- .../runner/DispatcherRunnerImplTest.java | 3 + .../TestingDispatcherServiceFactory.java | 18 ++- .../SessionDispatcherWithUUIDFactory.java | 3 + 19 files changed, 86 insertions(+), 146 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 d86814935c94..87570438ee9f 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 @@ -18,7 +18,6 @@ package org.apache.flink.runtime.dispatcher; -import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.operators.ResourceSpec; import org.apache.flink.api.common.time.Time; @@ -48,8 +47,6 @@ import org.apache.flink.runtime.jobmaster.JobNotFinishedException; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.jobmaster.factories.DefaultJobManagerJobMetricGroupFactory; -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.messages.FlinkJobNotFoundException; import org.apache.flink.runtime.messages.webmonitor.ClusterOverview; @@ -62,7 +59,7 @@ import org.apache.flink.runtime.resourcemanager.ResourceOverview; import org.apache.flink.runtime.rest.handler.legacy.backpressure.OperatorBackPressureStatsResponse; import org.apache.flink.runtime.rpc.FatalErrorHandler; -import org.apache.flink.runtime.rpc.FencedRpcEndpoint; +import org.apache.flink.runtime.rpc.PermanentlyFencedRpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import org.apache.flink.util.ExceptionUtils; @@ -85,9 +82,9 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; +import java.util.function.BiFunction; import java.util.function.Function; import java.util.stream.Collectors; @@ -97,8 +94,7 @@ * the jobs and to recover them in case of a master failure. Furthermore, it knows * about the state of the Flink session cluster. */ -public abstract class Dispatcher extends FencedRpcEndpoint implements - DispatcherGateway, LeaderContender { +public abstract class Dispatcher extends PermanentlyFencedRpcEndpoint implements DispatcherGateway { public static final String DISPATCHER_NAME = "dispatcher"; @@ -116,9 +112,8 @@ public abstract class Dispatcher extends FencedRpcEndpoint impleme private final FatalErrorHandler fatalErrorHandler; private final Map> jobManagerRunnerFutures; - private final Collection recoveredJobs; - private final LeaderElectionService leaderElectionService; + private final Collection recoveredJobs; private final ArchivedExecutionGraphStore archivedExecutionGraphStore; @@ -133,17 +128,16 @@ public abstract class Dispatcher extends FencedRpcEndpoint impleme private final Map> jobManagerTerminationFutures; - private CompletableFuture recoveryOperation = CompletableFuture.completedFuture(null); - protected final CompletableFuture shutDownFuture; public Dispatcher( RpcService rpcService, String endpointId, + DispatcherId fencingToken, Collection recoveredJobs, DispatcherServices dispatcherServices, JobGraphWriter jobGraphWriter) throws Exception { - super(rpcService, endpointId, null); + super(rpcService, endpointId, fencingToken); Preconditions.checkNotNull(dispatcherServices); this.configuration = dispatcherServices.getConfiguration(); @@ -164,8 +158,6 @@ public Dispatcher( jobManagerRunnerFutures = new HashMap<>(16); - leaderElectionService = highAvailabilityServices.getDispatcherLeaderElectionService(); - this.historyServerArchivist = dispatcherServices.getHistoryServerArchivist(); this.archivedExecutionGraphStore = dispatcherServices.getArchivedExecutionGraphStore(); @@ -200,18 +192,37 @@ public void onStart() throws Exception { onFatalError(exception); throw exception; } + + startRecoveredJobs(); } private void startDispatcherServices() throws Exception { try { - leaderElectionService.start(this); - registerDispatcherMetrics(jobManagerMetricGroup); } catch (Exception e) { handleStartDispatcherServicesException(e); } } + private void startRecoveredJobs() { + for (JobGraph recoveredJob : recoveredJobs) { + FutureUtils.assertNoException(runJob(recoveredJob) + .handle(handleRecoveredJobStartError(recoveredJob.getJobID()))); + } + + recoveredJobs.clear(); + } + + private BiFunction handleRecoveredJobStartError(JobID jobId) { + return (ignored, throwable) -> { + if (throwable != null) { + onFatalError(new DispatcherException(String.format("Could not start recovered job %s.", jobId), throwable)); + } + + return null; + }; + } + private void handleStartDispatcherServicesException(Exception e) throws Exception { try { stopDispatcherServices(); @@ -245,12 +256,6 @@ private void stopDispatcherServices() throws Exception { exception = ExceptionUtils.firstOrSuppressed(e, exception); } - try { - leaderElectionService.stop(); - } catch (Exception e) { - exception = ExceptionUtils.firstOrSuppressed(e, exception); - } - jobManagerMetricGroup.close(); ExceptionUtils.tryRethrowException(exception); @@ -643,7 +648,7 @@ private void registerJobManagerRunnerTerminationFuture(JobID jobId, CompletableF jobManagerTerminationFutures.put(jobId, terminationFuture); } }, - getUnfencedMainThreadExecutor()); + getMainThreadExecutor()); } private CompletableFuture removeJob(JobID jobId, boolean cleanupHA) { @@ -818,66 +823,6 @@ private List>> queryJobMastersForInformation(F return optionalJobInformation; } - //------------------------------------------------------ - // Leader contender - //------------------------------------------------------ - - /** - * Callback method when current resourceManager is granted leadership. - * - * @param newLeaderSessionID unique leadershipID - */ - @Override - public void grantLeadership(final UUID newLeaderSessionID) { - runAsyncWithoutFencing( - () -> { - log.info("Dispatcher {} was granted leadership with fencing token {}", getAddress(), newLeaderSessionID); - - final CompletableFuture fencingTokenFuture = tryAcceptLeadershipAndRunJobs(newLeaderSessionID, recoveredJobs); - - final CompletableFuture confirmationFuture = fencingTokenFuture.thenAcceptAsync( - (Boolean confirmLeadership) -> { - if (confirmLeadership) { - leaderElectionService.confirmLeadership(newLeaderSessionID, getAddress()); - } - }, - getRpcService().getExecutor()); - - confirmationFuture.whenComplete( - (Void ignored, Throwable throwable) -> { - if (throwable != null) { - onFatalError( - new DispatcherException( - String.format("Failed to take leadership with session id %s.", newLeaderSessionID), - (ExceptionUtils.stripCompletionException(throwable)))); - } - }); - - recoveryOperation = confirmationFuture; - }); - } - - private CompletableFuture tryAcceptLeadershipAndRunJobs(UUID newLeaderSessionID, Collection recoveredJobs) { - final DispatcherId dispatcherId = DispatcherId.fromUuid(newLeaderSessionID); - - if (leaderElectionService.hasLeadership(newLeaderSessionID)) { - log.debug("Dispatcher {} accepted leadership with fencing token {}. Start recovered jobs.", getAddress(), dispatcherId); - setNewFencingToken(dispatcherId); - - Collection> runFutures = new ArrayList<>(recoveredJobs.size()); - - for (JobGraph recoveredJob : recoveredJobs) { - final CompletableFuture runFuture = waitForTerminatingJobManager(recoveredJob.getJobID(), recoveredJob, this::runJob); - runFutures.add(runFuture); - } - - return FutureUtils.waitForAll(runFutures).thenApply(ignored -> true); - } else { - log.debug("Dispatcher {} lost leadership before accepting it. Stop recovering jobs for fencing token {}.", getAddress(), dispatcherId); - return CompletableFuture.completedFuture(false); - } - } - private CompletableFuture waitForTerminatingJobManager(JobID jobId, JobGraph jobGraph, FunctionWithException, ?> action) { final CompletableFuture jobManagerTerminationFuture = getJobTerminationFuture(jobId) .exceptionally((Throwable throwable) -> { @@ -902,49 +847,8 @@ CompletableFuture getJobTerminationFuture(JobID jobId) { } } - @VisibleForTesting - CompletableFuture getRecoveryOperation() { - return recoveryOperation; - } - - private void setNewFencingToken(@Nullable DispatcherId dispatcherId) { - // clear the state if we've been the leader before - if (getFencingToken() != null) { - clearDispatcherState(); - } - - setFencingToken(dispatcherId); - } - - private void clearDispatcherState() { - terminateJobManagerRunners(); - } - private void registerDispatcherMetrics(MetricGroup jobManagerMetricGroup) { jobManagerMetricGroup.gauge(MetricNames.NUM_RUNNING_JOBS, () -> (long) jobManagerRunnerFutures.size()); } - - /** - * Callback method when current resourceManager loses leadership. - */ - @Override - public void revokeLeadership() { - runAsyncWithoutFencing( - () -> { - log.info("Dispatcher {} was revoked leadership.", getAddress()); - - setNewFencingToken(null); - }); - } - - /** - * Handles error occurring in the leader election service. - * - * @param exception Exception being thrown in the leader election service - */ - @Override - public void handleError(final Exception exception) { - onFatalError(new DispatcherException("Received an error from the LeaderElectionService.", exception)); - } } 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 44c8eafee263..508acf853f66 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 @@ -36,6 +36,7 @@ public interface DispatcherFactory { */ Dispatcher createDispatcher( @Nonnull RpcService rpcService, + @Nonnull DispatcherId fencingToken, @Nonnull Collection recoveredJobs, @Nonnull PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception; 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 2b5c73474991..ab37e7ff8031 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 @@ -44,6 +44,7 @@ public JobDispatcherFactory(JobGraphRetriever jobGraphRetriever) { @Override public MiniDispatcher createDispatcher( @Nonnull RpcService rpcService, + @Nonnull DispatcherId fencingToken, @Nonnull Collection recoveredJobs, @Nonnull PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { final Configuration configuration = partialDispatcherServicesWithJobGraphStore.getConfiguration(); @@ -56,6 +57,7 @@ public MiniDispatcher createDispatcher( return new MiniDispatcher( rpcService, getEndpointId(), + fencingToken, DispatcherServices.from(partialDispatcherServicesWithJobGraphStore, 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 3405edb09035..3f7acc629e5e 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 @@ -50,12 +50,14 @@ public class MiniDispatcher extends Dispatcher { public MiniDispatcher( RpcService rpcService, String endpointId, + DispatcherId fencingToken, DispatcherServices dispatcherServices, JobGraph jobGraph, JobClusterEntrypoint.ExecutionMode executionMode) throws Exception { super( rpcService, endpointId, + fencingToken, Collections.singleton(jobGraph), dispatcherServices, new SingleJobJobGraphStore(jobGraph)); 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 746d6f5d7fe4..fbd88a5f7fc7 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 @@ -32,12 +32,14 @@ public enum SessionDispatcherFactory implements DispatcherFactory { @Override public StandaloneDispatcher createDispatcher( RpcService rpcService, + DispatcherId fencingToken, Collection recoveredJobs, PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { // create the default dispatcher return new StandaloneDispatcher( rpcService, getEndpointId(), + fencingToken, recoveredJobs, DispatcherServices.from(partialDispatcherServicesWithJobGraphStore, 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 96c1c705cf94..8a4c429b9345 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 @@ -33,11 +33,13 @@ public class StandaloneDispatcher extends Dispatcher { public StandaloneDispatcher( RpcService rpcService, String endpointId, + DispatcherId fencingToken, Collection recoveredJobs, DispatcherServices dispatcherServices) throws Exception { super( rpcService, endpointId, + fencingToken, recoveredJobs, dispatcherServices, dispatcherServices.getJobGraphWriter()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java index 9b2f73a3441a..dc20e980a36d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java @@ -22,6 +22,7 @@ import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.dispatcher.Dispatcher; import org.apache.flink.runtime.dispatcher.DispatcherGateway; +import org.apache.flink.runtime.dispatcher.DispatcherId; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.runtime.jobmanager.JobGraphWriter; @@ -158,7 +159,10 @@ private void createDispatcherIfRunning(Collection jobGraphs) { } private void createDispatcher(Collection jobGraphs) { - dispatcher = dispatcherFactory.create(jobGraphs, jobGraphStore); + dispatcher = dispatcherFactory.create( + DispatcherId.fromUuid(leaderSessionId), + jobGraphs, + jobGraphStore); dispatcherGatewayFuture.complete(dispatcher.getGateway()); } @@ -411,7 +415,10 @@ private void runIfState(Predicate actionPredicate, Runnable action) { // ------------------------------------------------------------ interface DispatcherServiceFactory { - DispatcherService create(Collection recoveredJobs, JobGraphWriter jobGraphWriter); + DispatcherService create( + DispatcherId fencingToken, + Collection recoveredJobs, + JobGraphWriter jobGraphWriter); } interface DispatcherService extends AutoCloseableAsync { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java index 55432dff0a45..dead9b5cc4a5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java @@ -23,6 +23,7 @@ import org.apache.flink.runtime.dispatcher.Dispatcher; 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.PartialDispatcherServicesWithJobGraphStore; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.rpc.RpcService; @@ -50,6 +51,7 @@ class DispatcherRunnerImpl implements DispatcherRunner { PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { this.dispatcher = dispatcherFactory.createDispatcher( rpcService, + DispatcherId.generate(), Collections.emptyList(), partialDispatcherServicesWithJobGraphStore); this.leaderRetrievalService = partialDispatcherServicesWithJobGraphStore.getHighAvailabilityServices().getDispatcherLeaderRetriever(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.java index 860e0a088c74..d538f32ab5d0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.java @@ -20,6 +20,7 @@ import org.apache.flink.runtime.dispatcher.Dispatcher; import org.apache.flink.runtime.dispatcher.DispatcherFactory; +import org.apache.flink.runtime.dispatcher.DispatcherId; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobGraphStore; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -50,11 +51,15 @@ class DispatcherServiceImplFactory implements DispatcherLeaderProcessImpl.Dispat } @Override - public DispatcherLeaderProcessImpl.DispatcherService create(Collection recoveredJobs, JobGraphWriter jobGraphWriter) { + public DispatcherLeaderProcessImpl.DispatcherService create( + DispatcherId fencingToken, + Collection recoveredJobs, + JobGraphWriter jobGraphWriter) { final Dispatcher dispatcher; try { dispatcher = dispatcherFactory.createDispatcher( rpcService, + fencingToken, recoveredJobs, PartialDispatcherServicesWithJobGraphStore.from(partialDispatcherServices, jobGraphWriter)); } catch (Exception e) { 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 474b39ac89e4..dca304eb57e0 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 @@ -314,6 +314,7 @@ private HATestingDispatcher createDispatcher( return new HATestingDispatcher( rpcService, UUID.randomUUID().toString(), + DispatcherId.generate(), Collections.emptyList(), configuration, highAvailabilityServices, @@ -346,6 +347,7 @@ private static class HATestingDispatcher extends TestingDispatcher { HATestingDispatcher( RpcService rpcService, String endpointId, + DispatcherId fencingToken, Collection recoveredJobs, Configuration configuration, HighAvailabilityServices highAvailabilityServices, @@ -361,6 +363,7 @@ private static class HATestingDispatcher extends TestingDispatcher { super( rpcService, endpointId, + fencingToken, recoveredJobs, configuration, highAvailabilityServices, 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 271b46455a9e..b07400cbbfbc 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 @@ -201,6 +201,7 @@ private void startDispatcher(JobManagerRunnerFactory jobManagerRunnerFactory) th dispatcher = new TestingDispatcher( rpcService, Dispatcher.DISPATCHER_NAME + UUID.randomUUID(), + DispatcherId.generate(), Collections.emptyList(), configuration, highAvailabilityServices, 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 da956997e318..e50cad774172 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 @@ -216,6 +216,7 @@ private TestingDispatcher createDispatcher(HeartbeatServices heartbeatServices, return new TestingDispatcher( rpcService, Dispatcher.DISPATCHER_NAME + '_' + name.getMethodName(), + DispatcherId.generate(), Collections.emptyList(), configuration, haServices, 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 bca7ecddd851..2a0781677ee1 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 @@ -230,6 +230,7 @@ private MiniDispatcher createMiniDispatcher(ClusterEntrypoint.ExecutionMode exec return new MiniDispatcher( rpcService, UUID.randomUUID().toString(), + DispatcherId.generate(), new DispatcherServices( configuration, highAvailabilityServices, 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 803caa5127cc..c29ce48169a6 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 @@ -49,6 +49,7 @@ class TestingDispatcher extends Dispatcher { TestingDispatcher( RpcService rpcService, String endpointId, + DispatcherId fencingToken, Collection recoveredJobs, Configuration configuration, HighAvailabilityServices highAvailabilityServices, @@ -63,6 +64,7 @@ class TestingDispatcher extends Dispatcher { super( rpcService, endpointId, + fencingToken, recoveredJobs, new DispatcherServices( configuration, @@ -105,12 +107,6 @@ CompletableFuture getJobTerminationFuture(@Nonnull JobID jobId, @Nonnull T timeout).thenCompose(Function.identity()); } - CompletableFuture getRecoverOperationFuture(@Nonnull Time timeout) { - return callAsyncWithoutFencing( - this::getRecoveryOperation, - timeout).thenCompose(Function.identity()); - } - CompletableFuture getNumberJobs(Time timeout) { return callAsyncWithoutFencing( () -> listJobs(timeout).get().size(), 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 d732075151d0..1f74b548764a 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 @@ -335,6 +335,7 @@ private TestingDispatcher createDispatcher(HighAvailabilityServices highAvailabi return new TestingDispatcher( rpcService, Dispatcher.DISPATCHER_NAME + '_' + name.getMethodName() + UUID.randomUUID(), + DispatcherId.generate(), Collections.emptyList(), configuration, highAvailabilityServices, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java index 61d2099d3635..c1403b642aba 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java @@ -30,7 +30,7 @@ import org.apache.flink.util.ExecutorUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; -import org.apache.flink.util.function.BiFunctionWithException; +import org.apache.flink.util.function.TriFunctionWithException; import org.hamcrest.core.Is; import org.junit.After; @@ -119,7 +119,7 @@ public void start_triggersJobGraphRecoveryAndDispatcherServiceCreation() throws final CompletableFuture> recoveredJobGraphsFuture = new CompletableFuture<>(); dispatcherServiceFactory = TestingDispatcherServiceFactory.newBuilder() .setCreateFunction( - (recoveredJobGraphs, jobGraphStore) -> { + (fencingToken, recoveredJobGraphs, jobGraphStore) -> { recoveredJobGraphsFuture.complete(recoveredJobGraphs); return TestingDispatcherService.newBuilder().build(); } @@ -147,7 +147,7 @@ public void closeAsync_stopsJobGraphStoreAndDispatcher() throws Exception { final CompletableFuture dispatcherServiceTerminationFuture = new CompletableFuture<>(); final OneShotLatch dispatcherServiceShutdownLatch = new OneShotLatch(); dispatcherServiceFactory = TestingDispatcherServiceFactory.newBuilder() - .setCreateFunction((ignoredA, ignoredB) -> TestingDispatcherService.newBuilder() + .setCreateFunction((ignoredA, ignoredB, ignoredC) -> TestingDispatcherService.newBuilder() .setTerminationFutureSupplier(() -> { dispatcherServiceShutdownLatch.trigger(); return dispatcherServiceTerminationFuture; @@ -187,7 +187,7 @@ public void confirmLeaderSessionFuture_completesAfterDispatcherServiceHasBeenSta dispatcherServiceFactory = TestingDispatcherServiceFactory.newBuilder() .setCreateFunction( - BiFunctionWithException.unchecked((ignoredA, ignoredB) -> { + TriFunctionWithException.unchecked((ignoredA, ignoredB, ignoredC) -> { createDispatcherServiceLatch.await(); return TestingDispatcherService.newBuilder() .setDispatcherGateway(dispatcherGateway) @@ -224,7 +224,7 @@ public void closeAsync_duringJobRecovery_preventsDispatcherServiceCreation() thr this.dispatcherServiceFactory = TestingDispatcherServiceFactory.newBuilder() .setCreateFunction( - (ignoredA, ignoredB) -> { + (ignoredA, ignoredB, ignoredC) -> { createDispatcherService.trigger(); return TestingDispatcherService.newBuilder().build(); }) @@ -343,7 +343,7 @@ public void onAddedJobGraph_falsePositive_willBeIgnored() { private TestingDispatcherServiceFactory createDispatcherServiceFactoryFor(TestingDispatcherGateway testingDispatcherGateway) { return TestingDispatcherServiceFactory.newBuilder() .setCreateFunction( - (ignoredA, ignoredB) -> TestingDispatcherService.newBuilder() + (ignoredA, ignoredB, ignoredC) -> TestingDispatcherService.newBuilder() .setDispatcherGateway(testingDispatcherGateway) .build()) .build(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java index 57d1b3b38f45..b28befd64152 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.dispatcher.Dispatcher; 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.DispatcherServices; import org.apache.flink.runtime.dispatcher.JobManagerRunnerFactory; import org.apache.flink.runtime.dispatcher.MemoryArchivedExecutionGraphStore; @@ -173,11 +174,13 @@ private TestingDispatcherFactory(JobManagerRunnerFactory jobManagerRunnerFactory @Override public Dispatcher createDispatcher( @Nonnull RpcService rpcService, + @Nonnull DispatcherId fencingToken, @Nonnull Collection recoveredJobs, @Nonnull PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { return new StandaloneDispatcher( rpcService, getEndpointId(), + fencingToken, recoveredJobs, DispatcherServices.from(partialDispatcherServicesWithJobGraphStore, jobManagerRunnerFactory)); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java index 469709145cef..4f9b766e3bab 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java @@ -18,22 +18,26 @@ package org.apache.flink.runtime.dispatcher.runner; +import org.apache.flink.runtime.dispatcher.DispatcherId; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.JobGraphWriter; +import org.apache.flink.util.function.TriFunction; import java.util.Collection; -import java.util.function.BiFunction; class TestingDispatcherServiceFactory implements DispatcherLeaderProcessImpl.DispatcherServiceFactory { - private final BiFunction, JobGraphWriter, DispatcherLeaderProcessImpl.DispatcherService> createFunction; + private final TriFunction, JobGraphWriter, DispatcherLeaderProcessImpl.DispatcherService> createFunction; - private TestingDispatcherServiceFactory(BiFunction, JobGraphWriter, DispatcherLeaderProcessImpl.DispatcherService> createFunction) { + private TestingDispatcherServiceFactory(TriFunction, JobGraphWriter, DispatcherLeaderProcessImpl.DispatcherService> createFunction) { this.createFunction = createFunction; } @Override - public DispatcherLeaderProcessImpl.DispatcherService create(Collection recoveredJobs, JobGraphWriter jobGraphWriter) { - return createFunction.apply(recoveredJobs, jobGraphWriter); + public DispatcherLeaderProcessImpl.DispatcherService create( + DispatcherId fencingToken, + Collection recoveredJobs, + JobGraphWriter jobGraphWriter) { + return createFunction.apply(fencingToken, recoveredJobs, jobGraphWriter); } public static Builder newBuilder() { @@ -41,11 +45,11 @@ public static Builder newBuilder() { } public static class Builder { - private BiFunction, JobGraphWriter, DispatcherLeaderProcessImpl.DispatcherService> createFunction = (ignoredA, ignoredB) -> TestingDispatcherService.newBuilder().build(); + private TriFunction, JobGraphWriter, DispatcherLeaderProcessImpl.DispatcherService> createFunction = (ignoredA, ignoredB, ignoredC) -> TestingDispatcherService.newBuilder().build(); private Builder() {} - Builder setCreateFunction(BiFunction, JobGraphWriter, DispatcherLeaderProcessImpl.DispatcherService> createFunction) { + Builder setCreateFunction(TriFunction, JobGraphWriter, DispatcherLeaderProcessImpl.DispatcherService> createFunction) { this.createFunction = createFunction; return this; } 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 97a2871dea84..c77147c03b06 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 @@ -20,6 +20,7 @@ import org.apache.flink.runtime.dispatcher.DefaultJobManagerRunnerFactory; import org.apache.flink.runtime.dispatcher.DispatcherFactory; +import org.apache.flink.runtime.dispatcher.DispatcherId; import org.apache.flink.runtime.dispatcher.DispatcherServices; import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobGraphStore; import org.apache.flink.runtime.dispatcher.StandaloneDispatcher; @@ -40,12 +41,14 @@ public enum SessionDispatcherWithUUIDFactory implements DispatcherFactory { @Override public StandaloneDispatcher createDispatcher( @Nonnull RpcService rpcService, + @Nonnull DispatcherId fencingToken, @Nonnull Collection recoveredJobs, @Nonnull PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { // create the default dispatcher return new StandaloneDispatcher( rpcService, generateEndpointIdWithUUID(), + fencingToken, recoveredJobs, DispatcherServices.from(partialDispatcherServicesWithJobGraphStore, DefaultJobManagerRunnerFactory.INSTANCE)); } From cb5fba5bba03a06e63495d226eaab8c39ad3998b Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 16 Sep 2019 18:15:03 +0200 Subject: [PATCH 260/746] [FLINK-11843] Enable DispatcherRunnerImplNG --- .../DefaultDispatcherResourceManagerComponentFactory.java | 5 +++-- .../apache/flink/runtime/minicluster/TestingMiniCluster.java | 4 ++-- 2 files changed, 5 insertions(+), 4 deletions(-) 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 2b2f531fb690..be9e7a1a8b1a 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 @@ -35,6 +35,7 @@ import org.apache.flink.runtime.dispatcher.runner.DispatcherRunner; import org.apache.flink.runtime.dispatcher.runner.DispatcherRunnerFactory; import org.apache.flink.runtime.dispatcher.runner.DispatcherRunnerFactoryImpl; +import org.apache.flink.runtime.dispatcher.runner.DispatcherRunnerImplNGFactory; import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; @@ -266,7 +267,7 @@ public DispatcherResourceManagerComponent create( public static DefaultDispatcherResourceManagerComponentFactory createSessionComponentFactory( ResourceManagerFactory resourceManagerFactory) { return new DefaultDispatcherResourceManagerComponentFactory( - new DispatcherRunnerFactoryImpl(SessionDispatcherFactory.INSTANCE), + new DispatcherRunnerImplNGFactory(SessionDispatcherFactory.INSTANCE), resourceManagerFactory, SessionRestEndpointFactory.INSTANCE); } @@ -275,7 +276,7 @@ public static DefaultDispatcherResourceManagerComponentFactory createJobComponen ResourceManagerFactory resourceManagerFactory, JobGraphRetriever jobGraphRetriever) { return new DefaultDispatcherResourceManagerComponentFactory( - new DispatcherRunnerFactoryImpl(new JobDispatcherFactory(jobGraphRetriever)), + new DispatcherRunnerImplNGFactory(new JobDispatcherFactory(jobGraphRetriever)), resourceManagerFactory, JobRestEndpointFactory.INSTANCE); } 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 2b8e32b7d485..44e818db6898 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 @@ -22,7 +22,7 @@ import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.dispatcher.MemoryArchivedExecutionGraphStore; -import org.apache.flink.runtime.dispatcher.runner.DispatcherRunnerFactoryImpl; +import org.apache.flink.runtime.dispatcher.runner.DispatcherRunnerImplNGFactory; import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponent; import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponentFactory; import org.apache.flink.runtime.entrypoint.component.TestingDefaultDispatcherResourceManagerComponentFactory; @@ -131,7 +131,7 @@ public CompletableFuture getDispatcherGatewayFuture() { private DispatcherResourceManagerComponentFactory createTestingDispatcherResourceManagerComponentFactory() { return TestingDefaultDispatcherResourceManagerComponentFactory.createSessionComponentFactory( - new DispatcherRunnerFactoryImpl(SessionDispatcherWithUUIDFactory.INSTANCE), + new DispatcherRunnerImplNGFactory(SessionDispatcherWithUUIDFactory.INSTANCE), StandaloneResourceManagerWithUUIDFactory.INSTANCE); } } From 8467acc100793ceeb8e79d184091fc2466f7ff49 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 16 Sep 2019 19:17:23 +0200 Subject: [PATCH 261/746] [FLINK-11665] Port ZooKeeperDispatcherRunnerImplTest to use DispatcherRunnerImplNG --- .../runner/DispatcherRunnerImplNGFactory.java | 2 +- .../ZooKeeperDispatcherRunnerImplTest.java | 53 +++++++++++++++---- 2 files changed, 43 insertions(+), 12 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGFactory.java index e6831a09d2eb..8e846d4b5bc9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGFactory.java @@ -39,7 +39,7 @@ public DispatcherRunnerImplNGFactory(DispatcherFactory dispatcherFactory) { } @Override - public DispatcherRunner createDispatcherRunner( + public DispatcherRunnerImplNG createDispatcherRunner( LeaderElectionService leaderElectionService, FatalErrorHandler fatalErrorHandler, JobGraphStoreFactory jobGraphStoreFactory, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDispatcherRunnerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDispatcherRunnerImplTest.java index ba0059af9b06..6550decc73e7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDispatcherRunnerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDispatcherRunnerImplTest.java @@ -28,7 +28,7 @@ import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.dispatcher.MemoryArchivedExecutionGraphStore; -import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobGraphStore; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; import org.apache.flink.runtime.dispatcher.SessionDispatcherFactory; import org.apache.flink.runtime.dispatcher.VoidHistoryServerArchivist; import org.apache.flink.runtime.heartbeat.TestingHeartbeatServices; @@ -38,6 +38,7 @@ import org.apache.flink.runtime.highavailability.zookeeper.ZooKeeperRunningJobsRegistry; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; import org.apache.flink.runtime.jobmanager.ZooKeeperJobGraphStore; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; @@ -45,18 +46,19 @@ import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.rpc.TestingRpcServiceResource; +import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testtasks.NoOpInvokable; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.runtime.util.ZooKeeperUtils; import org.apache.flink.runtime.zookeeper.ZooKeeperResource; +import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; import org.apache.curator.framework.CuratorFramework; import org.junit.After; import org.junit.Before; import org.junit.ClassRule; -import org.junit.Ignore; import org.junit.Test; import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; @@ -127,7 +129,6 @@ public void teardown() throws Exception { * See FLINK-11665. */ @Test - @Ignore public void testResourceCleanupUnderLeadershipChange() throws Exception { final TestingRpcService rpcService = testingRpcServiceResource.getTestingRpcService(); final TestingLeaderElectionService dispatcherLeaderElectionService = new TestingLeaderElectionService(); @@ -135,29 +136,35 @@ public void testResourceCleanupUnderLeadershipChange() throws Exception { final CuratorFramework client = ZooKeeperUtils.startCuratorFramework(configuration); try (final TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServicesBuilder() - .setJobGraphStore(ZooKeeperUtils.createJobGraphs(client, configuration)) .setRunningJobsRegistry(new ZooKeeperRunningJobsRegistry(client, configuration)) .setDispatcherLeaderElectionService(dispatcherLeaderElectionService) .setDispatcherLeaderRetriever(dispatcherLeaderRetriever) .setJobMasterLeaderRetrieverFunction(jobId -> ZooKeeperUtils.createLeaderRetrievalService(client, configuration)) .build()) { - final PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore = new PartialDispatcherServicesWithJobGraphStore( + final PartialDispatcherServices partialDispatcherServices = new PartialDispatcherServices( configuration, highAvailabilityServices, - () -> new CompletableFuture<>(), + CompletableFuture::new, blobServer, new TestingHeartbeatServices(), UnregisteredMetricGroups::createUnregisteredJobManagerMetricGroup, new MemoryArchivedExecutionGraphStore(), fatalErrorHandler, VoidHistoryServerArchivist.INSTANCE, - null, - highAvailabilityServices.getJobGraphStore()); + null); final JobGraph jobGraph = createJobGraphWithBlobs(); - try (final DispatcherRunnerImpl dispatcherRunner = new DispatcherRunnerImpl(SessionDispatcherFactory.INSTANCE, rpcService, partialDispatcherServicesWithJobGraphStore)) { + final DispatcherRunnerImplNGFactory dispatcherRunnerImplNGFactory = new DispatcherRunnerImplNGFactory(SessionDispatcherFactory.INSTANCE); + + try (final DispatcherRunner dispatcherRunner = createDispatcherRunner( + rpcService, + dispatcherLeaderElectionService, + () -> createZooKeeperJobGraphStore(client), + partialDispatcherServices, + dispatcherRunnerImplNGFactory)) { + // initial run DispatcherGateway dispatcherGateway = grantLeadership(dispatcherLeaderElectionService, dispatcherLeaderRetriever, dispatcherRunner); @@ -183,7 +190,7 @@ public void testResourceCleanupUnderLeadershipChange() throws Exception { dispatcherLeaderElectionService.notLeader(); // check that the job has been removed from ZooKeeper - final ZooKeeperJobGraphStore submittedJobGraphStore = ZooKeeperUtils.createJobGraphs(client, configuration); + final ZooKeeperJobGraphStore submittedJobGraphStore = createZooKeeperJobGraphStore(client); CommonTestUtils.waitUntilCondition(() -> submittedJobGraphStore.getJobIds().isEmpty(), Deadline.fromNow(VERIFICATION_TIMEOUT), 20L); } @@ -193,7 +200,31 @@ public void testResourceCleanupUnderLeadershipChange() throws Exception { assertThat(clusterHaStorageDir.listFiles(), is(emptyArray())); } - private DispatcherGateway grantLeadership(TestingLeaderElectionService dispatcherLeaderElectionService, SettableLeaderRetrievalService dispatcherLeaderRetriever, DispatcherRunnerImpl dispatcherRunner) throws InterruptedException, java.util.concurrent.ExecutionException { + private DispatcherRunner createDispatcherRunner( + TestingRpcService rpcService, + TestingLeaderElectionService dispatcherLeaderElectionService, + JobGraphStoreFactory jobGraphStoreFactory, + PartialDispatcherServices partialDispatcherServices, + DispatcherRunnerFactory dispatcherRunnerFactory) throws Exception { + return dispatcherRunnerFactory.createDispatcherRunner( + dispatcherLeaderElectionService, + fatalErrorHandler, + jobGraphStoreFactory, + TestingUtils.defaultExecutor(), + rpcService, + partialDispatcherServices); + } + + private ZooKeeperJobGraphStore createZooKeeperJobGraphStore(CuratorFramework client) { + try { + return ZooKeeperUtils.createJobGraphs(client, configuration); + } catch (Exception e) { + ExceptionUtils.rethrow(e); + return null; + } + } + + private DispatcherGateway grantLeadership(TestingLeaderElectionService dispatcherLeaderElectionService, SettableLeaderRetrievalService dispatcherLeaderRetriever, DispatcherRunner dispatcherRunner) throws InterruptedException, java.util.concurrent.ExecutionException { final UUID leaderSessionId = UUID.randomUUID(); dispatcherLeaderElectionService.isLeader(leaderSessionId).get(); // TODO: Remove once runner properly works From b6128ca6c87d61dc45572bb2727aa9d3930e0104 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 16 Sep 2019 22:45:38 +0200 Subject: [PATCH 262/746] [FLINK-11843] Enable DispatcherRunnerImplTest for DispatcherRunnerImplNG --- .../runner/DispatcherRunnerImplTest.java | 42 +++++++++++++------ 1 file changed, 30 insertions(+), 12 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java index b28befd64152..5afa743bcc9d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java @@ -29,6 +29,7 @@ import org.apache.flink.runtime.dispatcher.DispatcherServices; import org.apache.flink.runtime.dispatcher.JobManagerRunnerFactory; import org.apache.flink.runtime.dispatcher.MemoryArchivedExecutionGraphStore; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobGraphStore; import org.apache.flink.runtime.dispatcher.SingleJobJobGraphStore; import org.apache.flink.runtime.dispatcher.StandaloneDispatcher; @@ -38,19 +39,20 @@ import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServicesBuilder; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; 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.rpc.RpcService; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.rpc.TestingRpcServiceResource; +import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.util.TestLogger; import org.junit.After; import org.junit.Before; import org.junit.ClassRule; -import org.junit.Ignore; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -96,22 +98,19 @@ public void teardown() throws Exception { * See FLINK-11843. This is a probabilistic test which needs to be executed several times to fail. */ @Test - @Ignore public void testJobRecoveryUnderLeaderChange() throws Exception { final TestingRpcService rpcService = testingRpcServiceResource.getTestingRpcService(); final TestingLeaderElectionService dispatcherLeaderElectionService = new TestingLeaderElectionService(); final Configuration configuration = new Configuration(); final JobGraph jobGraph = new JobGraph(); - final SingleJobJobGraphStore submittedJobGraphStore = new SingleJobJobGraphStore(jobGraph); try (final BlobServer blobServer = new BlobServer(configuration, new VoidBlobStore()); final TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServicesBuilder() - .setJobGraphStore(submittedJobGraphStore) .setDispatcherLeaderElectionService(dispatcherLeaderElectionService) .build()) { - final PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore = new PartialDispatcherServicesWithJobGraphStore( + final PartialDispatcherServices partialDispatcherServices = new PartialDispatcherServices( configuration, highAvailabilityServices, CompletableFuture::new, @@ -121,14 +120,18 @@ public void testJobRecoveryUnderLeaderChange() throws Exception { new MemoryArchivedExecutionGraphStore(), fatalErrorHandler, VoidHistoryServerArchivist.INSTANCE, - null, - highAvailabilityServices.getJobGraphStore()); + null); final TestingJobManagerRunnerFactory jobManagerRunnerFactory = new TestingJobManagerRunnerFactory(1); - try (final DispatcherRunnerImpl dispatcherRunner = new DispatcherRunnerImpl( - new TestingDispatcherFactory(jobManagerRunnerFactory), - rpcService, - partialDispatcherServicesWithJobGraphStore)) { + + final DispatcherRunnerImplNGFactory dispatcherRunnerFactory = new DispatcherRunnerImplNGFactory(new TestingDispatcherFactory(jobManagerRunnerFactory)); + + try (final DispatcherRunner dispatcherRunner = createDispatcherRunner( + rpcService, + dispatcherLeaderElectionService, + partialDispatcherServices, + dispatcherRunnerFactory, + () -> new SingleJobJobGraphStore(jobGraph))) { // initial run grantLeadership(dispatcherLeaderElectionService, dispatcherRunner); final TestingJobManagerRunner testingJobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); @@ -155,6 +158,21 @@ public void testJobRecoveryUnderLeaderChange() throws Exception { } } + private DispatcherRunner createDispatcherRunner( + TestingRpcService rpcService, + TestingLeaderElectionService dispatcherLeaderElectionService, + PartialDispatcherServices partialDispatcherServices, + DispatcherRunnerFactory dispatcherRunnerFactory, + JobGraphStoreFactory jobGraphStoreFactory) throws Exception { + return dispatcherRunnerFactory.createDispatcherRunner( + dispatcherLeaderElectionService, + fatalErrorHandler, + jobGraphStoreFactory, + TestingUtils.defaultExecutor(), + rpcService, + partialDispatcherServices); + } + private DispatcherGateway grantLeadership( TestingLeaderElectionService dispatcherLeaderElectionService, DispatcherRunner dispatcherRunner) throws InterruptedException, java.util.concurrent.ExecutionException { @@ -179,7 +197,7 @@ public Dispatcher createDispatcher( @Nonnull PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { return new StandaloneDispatcher( rpcService, - getEndpointId(), + generateEndpointIdWithUUID(), fencingToken, recoveredJobs, DispatcherServices.from(partialDispatcherServicesWithJobGraphStore, jobManagerRunnerFactory)); From 7a23e028324bc97dbf07ebb875ed3a9ff76f77e5 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 17 Sep 2019 19:01:50 +0200 Subject: [PATCH 263/746] [FLINK-11843] Port MiniDispatcherTest to not do leader election for the MiniDispatcher --- .../runtime/dispatcher/MiniDispatcherTest.java | 15 --------------- 1 file changed, 15 deletions(-) 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 2a0781677ee1..39903fc59734 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 @@ -32,7 +32,6 @@ 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; import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedExecutionGraphBuilder; @@ -87,8 +86,6 @@ public class MiniDispatcherTest extends TestLogger { private final ArchivedExecutionGraphStore archivedExecutionGraphStore = new MemoryArchivedExecutionGraphStore(); - private TestingLeaderElectionService dispatcherLeaderElectionService; - private TestingHighAvailabilityServices highAvailabilityServices; private TestingFatalErrorHandler testingFatalErrorHandler; @@ -114,12 +111,9 @@ public static void setupClass() throws IOException { @Before public void setup() throws Exception { - dispatcherLeaderElectionService = new TestingLeaderElectionService(); highAvailabilityServices = new TestingHighAvailabilityServicesBuilder().build(); testingFatalErrorHandler = new TestingFatalErrorHandler(); - highAvailabilityServices.setDispatcherLeaderElectionService(dispatcherLeaderElectionService); - testingJobManagerRunnerFactory = new TestingJobManagerRunnerFactory(); } @@ -150,9 +144,6 @@ public void testSingleJobRecovery() throws Exception { miniDispatcher.start(); try { - // wait until the Dispatcher is the leader - dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); - final TestingJobManagerRunner testingJobManagerRunner = testingJobManagerRunnerFactory.takeCreatedJobManagerRunner(); assertThat(testingJobManagerRunner.getJobID(), is(jobGraph.getJobID())); @@ -172,9 +163,6 @@ public void testTerminationAfterJobCompletion() throws Exception { miniDispatcher.start(); try { - // wait until the Dispatcher is the leader - dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); - // wait until we have submitted the job final TestingJobManagerRunner testingJobManagerRunner = testingJobManagerRunnerFactory.takeCreatedJobManagerRunner(); @@ -198,9 +186,6 @@ public void testJobResultRetrieval() throws Exception { miniDispatcher.start(); try { - // wait until the Dispatcher is the leader - dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); - // wait until we have submitted the job final TestingJobManagerRunner testingJobManagerRunner = testingJobManagerRunnerFactory.takeCreatedJobManagerRunner(); From c5dee3141cfcd46c0adfec135482234449b6f294 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 19 Sep 2019 15:58:55 +0200 Subject: [PATCH 264/746] [FLINK-11843] Create AbstractDispatcherLeaderProcess --- .../AbstractDispatcherLeaderProcess.java | 185 ++++++++++++++++++ .../runner/DispatcherLeaderProcessImpl.java | 163 ++------------- 2 files changed, 200 insertions(+), 148 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java new file mode 100644 index 000000000000..8bfa964f8e3f --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.runner; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.webmonitor.RestfulGateway; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.function.Predicate; +import java.util.function.Supplier; + +abstract class AbstractDispatcherLeaderProcess implements DispatcherLeaderProcess { + + protected final Logger log = LoggerFactory.getLogger(getClass()); + + private final Object lock = new Object(); + + private final UUID leaderSessionId; + + private final FatalErrorHandler fatalErrorHandler; + + private final CompletableFuture dispatcherGatewayFuture; + + private final CompletableFuture confirmLeaderSessionFuture; + + private final CompletableFuture terminationFuture; + + private State state; + + AbstractDispatcherLeaderProcess(UUID leaderSessionId, FatalErrorHandler fatalErrorHandler) { + this.leaderSessionId = leaderSessionId; + this.fatalErrorHandler = fatalErrorHandler; + + this.dispatcherGatewayFuture = new CompletableFuture<>(); + this.confirmLeaderSessionFuture = dispatcherGatewayFuture.thenApply(RestfulGateway::getAddress); + this.terminationFuture = new CompletableFuture<>(); + + this.state = State.CREATED; + } + + @VisibleForTesting + State getState() { + synchronized (lock) { + return state; + } + } + + @Override + public final void start() { + runIfStateIs( + State.CREATED, + this::startInternal); + } + + private void startInternal() { + log.info("Start {}.", getClass().getSimpleName()); + state = State.RUNNING; + onStart(); + } + + @Override + public final UUID getLeaderSessionId() { + return leaderSessionId; + } + + @Override + public final CompletableFuture getDispatcherGateway() { + return dispatcherGatewayFuture; + } + + @Override + public final CompletableFuture getConfirmLeaderSessionFuture() { + return confirmLeaderSessionFuture; + } + + @Override + public final CompletableFuture closeAsync() { + runIfStateIsNot( + State.STOPPED, + this::closeInternal); + + return terminationFuture; + } + + private void closeInternal() { + FutureUtils.forward( + onClose(), + terminationFuture); + + state = State.STOPPED; + } + + protected abstract void onStart(); + + protected CompletableFuture onClose() { + return FutureUtils.completedVoidFuture(); + } + + protected void completeDispatcherGatewayFuture(DispatcherGateway dispatcherGateway) { + dispatcherGatewayFuture.complete(dispatcherGateway); + } + + final Optional supplyUnsynchronizedIfRunning(Supplier supplier) { + synchronized (lock) { + if (state != State.RUNNING) { + return Optional.empty(); + } + } + + return Optional.of(supplier.get()); + } + + final Optional supplyIfRunning(Supplier supplier) { + synchronized (lock) { + if (state != State.RUNNING) { + return Optional.empty(); + } + + return Optional.of(supplier.get()); + } + } + + final void runIfStateIs(State expectedState, Runnable action) { + runIfState(expectedState::equals, action); + } + + private void runIfStateIsNot(State notExpectedState, Runnable action) { + runIfState( + state -> !notExpectedState.equals(state), + action); + } + + private void runIfState(Predicate actionPredicate, Runnable action) { + synchronized (lock) { + if (actionPredicate.test(state)) { + action.run(); + } + } + } + + final Void onErrorIfRunning(T ignored, Throwable throwable) { + synchronized (lock) { + if (state != State.RUNNING) { + return null; + } + } + + if (throwable != null) { + closeAsync(); + fatalErrorHandler.onFatalError(throwable); + } + + return null; + } + + protected enum State { + CREATED, + RUNNING, + STOPPED + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java index dc20e980a36d..02b708f0113e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java @@ -28,16 +28,12 @@ import org.apache.flink.runtime.jobmanager.JobGraphWriter; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcUtils; -import org.apache.flink.runtime.webmonitor.RestfulGateway; import org.apache.flink.util.AutoCloseableAsync; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.Preconditions; import org.apache.flink.util.function.FunctionUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import javax.annotation.Nullable; import java.util.ArrayList; @@ -47,20 +43,12 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; -import java.util.function.Predicate; -import java.util.function.Supplier; /** * Process which encapsulates the job recovery logic and life cycle management of a * {@link Dispatcher}. */ -public class DispatcherLeaderProcessImpl implements DispatcherLeaderProcess, JobGraphStore.JobGraphListener { - - private static final Logger LOG = LoggerFactory.getLogger(DispatcherLeaderProcessImpl.class); - - private final Object lock = new Object(); - - private final UUID leaderSessionId; +public class DispatcherLeaderProcessImpl extends AbstractDispatcherLeaderProcess implements JobGraphStore.JobGraphListener { private final DispatcherServiceFactory dispatcherFactory; @@ -68,16 +56,6 @@ public class DispatcherLeaderProcessImpl implements DispatcherLeaderProcess, Job private final Executor ioExecutor; - private final FatalErrorHandler fatalErrorHandler; - - private final CompletableFuture dispatcherGatewayFuture; - - private final CompletableFuture confirmLeaderSessionFuture; - - private final CompletableFuture terminationFuture; - - private State state; - @Nullable private DispatcherService dispatcher; @@ -89,36 +67,17 @@ private DispatcherLeaderProcessImpl( JobGraphStore jobGraphStore, Executor ioExecutor, FatalErrorHandler fatalErrorHandler) { - this.leaderSessionId = leaderSessionId; + super(leaderSessionId, fatalErrorHandler); + this.dispatcherFactory = dispatcherFactory; this.jobGraphStore = jobGraphStore; this.ioExecutor = ioExecutor; - this.fatalErrorHandler = fatalErrorHandler; - this.dispatcherGatewayFuture = new CompletableFuture<>(); - this.confirmLeaderSessionFuture = dispatcherGatewayFuture.thenApply(RestfulGateway::getAddress); - this.terminationFuture = new CompletableFuture<>(); - - this.state = State.CREATED; this.dispatcher = null; } - State getState() { - synchronized (lock) { - return state; - } - } - @Override - public void start() { - runIfStateIs( - State.CREATED, - this::startInternal); - } - - private void startInternal() { - LOG.info("Start {}.", getClass().getSimpleName()); - state = State.RUNNING; + protected void onStart() { startServices(); onGoingRecoveryOperation = recoverJobsAsync() @@ -139,31 +98,16 @@ private void startServices() { } } - private Void onErrorIfRunning(T ignored, Throwable throwable) { - synchronized (lock) { - if (state != State.RUNNING) { - return null; - } - } - - if (throwable != null) { - closeAsync(); - fatalErrorHandler.onFatalError(throwable); - } - - return null; - } - private void createDispatcherIfRunning(Collection jobGraphs) { runIfStateIs(State.RUNNING, () -> createDispatcher(jobGraphs)); } private void createDispatcher(Collection jobGraphs) { dispatcher = dispatcherFactory.create( - DispatcherId.fromUuid(leaderSessionId), + DispatcherId.fromUuid(getLeaderSessionId()), jobGraphs, jobGraphStore); - dispatcherGatewayFuture.complete(dispatcher.getGateway()); + completeDispatcherGatewayFuture(dispatcher.getGateway()); } private CompletableFuture> recoverJobsAsync() { @@ -178,7 +122,7 @@ private Collection recoverJobsIfRunning() { } private Collection recoverJobs() { - LOG.info("Recover all persisted job graphs."); + log.info("Recover all persisted job graphs."); final Collection jobIds = getJobIds(); final Collection recoveredJobGraphs = new ArrayList<>(); @@ -186,7 +130,7 @@ private Collection recoverJobs() { recoveredJobGraphs.add(recoverJob(jobId)); } - LOG.info("Successfully recovered {} persisted job graphs.", recoveredJobGraphs.size()); + log.info("Successfully recovered {} persisted job graphs.", recoveredJobGraphs.size()); return recoveredJobGraphs; } @@ -202,7 +146,7 @@ private Collection getJobIds() { } private JobGraph recoverJob(JobID jobId) { - LOG.info("Trying to recover job with job id {}.", jobId); + log.info("Trying to recover job with job id {}.", jobId); try { return jobGraphStore.recoverJobGraph(jobId); } catch (Exception e) { @@ -213,31 +157,8 @@ private JobGraph recoverJob(JobID jobId) { } @Override - public UUID getLeaderSessionId() { - return leaderSessionId; - } - - @Override - public CompletableFuture getDispatcherGateway() { - return dispatcherGatewayFuture; - } - - @Override - public CompletableFuture getConfirmLeaderSessionFuture() { - return confirmLeaderSessionFuture; - } - - @Override - public CompletableFuture closeAsync() { - runIfStateIsNot( - State.STOPPED, - this::closeInternal); - - return terminationFuture; - } - - private void closeInternal() { - LOG.info("Stopping {}.", getClass().getSimpleName()); + protected CompletableFuture onClose() { + log.info("Stopping {}.", getClass().getSimpleName()); final CompletableFuture dispatcherTerminationFuture; if (dispatcher != null) { @@ -246,16 +167,10 @@ private void closeInternal() { dispatcherTerminationFuture = FutureUtils.completedVoidFuture(); } - final CompletableFuture stopServicesFuture = FutureUtils.runAfterwardsAsync( + return FutureUtils.runAfterwardsAsync( dispatcherTerminationFuture, this::stopServices, ioExecutor); - - FutureUtils.forward( - stopServicesFuture, - terminationFuture); - - state = State.STOPPED; } private void stopServices() throws Exception { @@ -282,7 +197,7 @@ public void onAddedJobGraph(JobID jobId) { } private void handleAddedJobGraph(JobID jobId) { - LOG.debug( + log.debug( "Job {} has been added to the {} by another process.", jobId, jobGraphStore.getClass().getSimpleName()); @@ -312,7 +227,7 @@ private CompletableFuture submitAddedJob(JobGraph jobGraph) { } private DispatcherGateway getDispatcherGatewayInternal() { - return Preconditions.checkNotNull(dispatcherGatewayFuture.getNow(null)); + return Preconditions.checkNotNull(getDispatcherGateway().getNow(null)); } private Optional recoverJobIfRunning(JobID jobId) { @@ -327,7 +242,7 @@ public void onRemovedJobGraph(JobID jobId) { } private void handleRemovedJobGraph(JobID jobId) { - LOG.debug( + log.debug( "Job {} has been removed from the {} by another process.", jobId, jobGraphStore.getClass().getSimpleName()); @@ -368,48 +283,6 @@ public static DispatcherLeaderProcessImpl create( fatalErrorHandler); } - // --------------------------------------------------------------- - // Internal helper methods - // --------------------------------------------------------------- - - private Optional supplyUnsynchronizedIfRunning(Supplier supplier) { - synchronized (lock) { - if (state != State.RUNNING) { - return Optional.empty(); - } - } - - return Optional.of(supplier.get()); - } - - private Optional supplyIfRunning(Supplier supplier) { - synchronized (lock) { - if (state != State.RUNNING) { - return Optional.empty(); - } - - return Optional.of(supplier.get()); - } - } - - private void runIfStateIs(State expectedState, Runnable action) { - runIfState(expectedState::equals, action); - } - - private void runIfStateIsNot(State notExpectedState, Runnable action) { - runIfState( - state -> !notExpectedState.equals(state), - action); - } - - private void runIfState(Predicate actionPredicate, Runnable action) { - synchronized (lock) { - if (actionPredicate.test(state)) { - action.run(); - } - } - } - // ------------------------------------------------------------ // Internal classes // ------------------------------------------------------------ @@ -424,10 +297,4 @@ DispatcherService create( interface DispatcherService extends AutoCloseableAsync { DispatcherGateway getGateway(); } - - enum State { - CREATED, - RUNNING, - STOPPED - } } From c1f67ef1f27f51571c3a0004453357403a98f0df Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 19 Sep 2019 16:21:06 +0200 Subject: [PATCH 265/746] [FLINK-11843] Add proper per-job mode support for DispatcherRunnerImplNG --- .../dispatcher/JobDispatcherFactory.java | 17 ++--- ...DispatcherLeaderProcessFactoryFactory.java | 39 ++++++++++ .../runner/DispatcherRunnerImplNGFactory.java | 27 ++++--- .../runner/JobDispatcherLeaderProcess.java | 70 ++++++++++++++++++ .../JobDispatcherLeaderProcessFactory.java | 49 ++++++++++++ ...DispatcherLeaderProcessFactoryFactory.java | 74 +++++++++++++++++++ ...DispatcherLeaderProcessFactoryFactory.java | 62 ++++++++++++++++ ...atcherResourceManagerComponentFactory.java | 6 +- .../jobmanager/ThrowingJobGraphWriter.java | 41 ++++++++++ .../runner/DispatcherRunnerImplTest.java | 2 +- .../ZooKeeperDispatcherRunnerImplTest.java | 2 +- .../minicluster/TestingMiniCluster.java | 2 +- 12 files changed, 363 insertions(+), 28 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessFactoryFactory.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactory.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactoryFactory.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ThrowingJobGraphWriter.java 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 ab37e7ff8031..784bb42710d9 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 @@ -20,10 +20,11 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; -import org.apache.flink.runtime.entrypoint.component.JobGraphRetriever; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables; + import javax.annotation.Nonnull; import java.util.Collection; @@ -33,13 +34,8 @@ /** * {@link DispatcherFactory} which creates a {@link MiniDispatcher}. */ -public class JobDispatcherFactory implements DispatcherFactory { - - private final JobGraphRetriever jobGraphRetriever; - - public JobDispatcherFactory(JobGraphRetriever jobGraphRetriever) { - this.jobGraphRetriever = jobGraphRetriever; - } +public enum JobDispatcherFactory implements DispatcherFactory { + INSTANCE; @Override public MiniDispatcher createDispatcher( @@ -47,11 +43,10 @@ public MiniDispatcher createDispatcher( @Nonnull DispatcherId fencingToken, @Nonnull Collection recoveredJobs, @Nonnull PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { - final Configuration configuration = partialDispatcherServicesWithJobGraphStore.getConfiguration(); - final JobGraph jobGraph = jobGraphRetriever.retrieveJobGraph(configuration); + final JobGraph jobGraph = Iterables.getOnlyElement(recoveredJobs); + final Configuration configuration = partialDispatcherServicesWithJobGraphStore.getConfiguration(); final String executionModeValue = configuration.getString(EXECUTION_MODE); - final ClusterEntrypoint.ExecutionMode executionMode = ClusterEntrypoint.ExecutionMode.valueOf(executionModeValue); return new MiniDispatcher( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessFactoryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessFactoryFactory.java new file mode 100644 index 000000000000..8bde2df78580 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessFactoryFactory.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.runner; + +import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; +import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcService; + +import java.util.concurrent.Executor; + +/** + * Factory for {@link DispatcherLeaderProcessFactory}. + */ +public interface DispatcherLeaderProcessFactoryFactory { + + DispatcherLeaderProcessFactory createFactory( + JobGraphStoreFactory jobGraphStoreFactory, + Executor ioExecutor, + RpcService rpcService, + PartialDispatcherServices partialDispatcherServices, + FatalErrorHandler fatalErrorHandler); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGFactory.java index 8e846d4b5bc9..5ca79f0fcd0e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGFactory.java @@ -20,6 +20,7 @@ import org.apache.flink.runtime.dispatcher.DispatcherFactory; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; +import org.apache.flink.runtime.entrypoint.component.JobGraphRetriever; import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.rpc.FatalErrorHandler; @@ -32,10 +33,10 @@ * instances. */ public class DispatcherRunnerImplNGFactory implements DispatcherRunnerFactory { - private final DispatcherFactory dispatcherFactory; + private final DispatcherLeaderProcessFactoryFactory dispatcherLeaderProcessFactoryFactory; - public DispatcherRunnerImplNGFactory(DispatcherFactory dispatcherFactory) { - this.dispatcherFactory = dispatcherFactory; + private DispatcherRunnerImplNGFactory(DispatcherLeaderProcessFactoryFactory dispatcherLeaderProcessFactoryFactory) { + this.dispatcherLeaderProcessFactoryFactory = dispatcherLeaderProcessFactoryFactory; } @Override @@ -47,15 +48,11 @@ public DispatcherRunnerImplNG createDispatcherRunner( RpcService rpcService, PartialDispatcherServices partialDispatcherServices) throws Exception { - final DispatcherLeaderProcessImpl.DispatcherServiceFactory dispatcherServiceFactory = new DispatcherServiceImplFactory( - dispatcherFactory, - rpcService, - partialDispatcherServices); - - final DispatcherLeaderProcessFactory dispatcherLeaderProcessFactory = new DispatcherLeaderProcessImplFactory( - dispatcherServiceFactory, + final DispatcherLeaderProcessFactory dispatcherLeaderProcessFactory = dispatcherLeaderProcessFactoryFactory.createFactory( jobGraphStoreFactory, ioExecutor, + rpcService, + partialDispatcherServices, fatalErrorHandler); return new DispatcherRunnerImplNG( @@ -63,4 +60,14 @@ public DispatcherRunnerImplNG createDispatcherRunner( fatalErrorHandler, dispatcherLeaderProcessFactory); } + + public static DispatcherRunnerImplNGFactory createSessionRunner(DispatcherFactory dispatcherFactory) { + return new DispatcherRunnerImplNGFactory( + SessionDispatcherLeaderProcessFactoryFactory.create(dispatcherFactory)); + } + + public static DispatcherRunnerImplNGFactory createJobRunner(JobGraphRetriever jobGraphRetriever) { + return new DispatcherRunnerImplNGFactory( + JobDispatcherLeaderProcessFactoryFactory.create(jobGraphRetriever)); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java new file mode 100644 index 000000000000..fb3f8aa282f2 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.runner; + +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.dispatcher.DispatcherId; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmanager.ThrowingJobGraphWriter; +import org.apache.flink.runtime.rpc.FatalErrorHandler; + +import java.util.Collections; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; + +/** + * {@link DispatcherLeaderProcess} implementation for the per-job mode. + */ +public class JobDispatcherLeaderProcess extends AbstractDispatcherLeaderProcess { + + private final DispatcherLeaderProcessImpl.DispatcherServiceFactory dispatcherServiceFactory; + + private final JobGraph jobGraph; + + private DispatcherLeaderProcessImpl.DispatcherService dispatcherService; + + JobDispatcherLeaderProcess( + UUID leaderSessionId, + DispatcherLeaderProcessImpl.DispatcherServiceFactory dispatcherServiceFactory, + JobGraph jobGraph, + FatalErrorHandler fatalErrorHandler) { + super(leaderSessionId, fatalErrorHandler); + this.jobGraph = jobGraph; + this.dispatcherServiceFactory = dispatcherServiceFactory; + } + + @Override + protected void onStart() { + dispatcherService = dispatcherServiceFactory.create( + DispatcherId.fromUuid(getLeaderSessionId()), + Collections.singleton(jobGraph), + ThrowingJobGraphWriter.INSTANCE); + + completeDispatcherGatewayFuture(dispatcherService.getGateway()); + } + + @Override + protected CompletableFuture onClose() { + if (dispatcherService != null) { + return dispatcherService.closeAsync(); + } else { + return FutureUtils.completedVoidFuture(); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactory.java new file mode 100644 index 000000000000..db8f4e97c6f7 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactory.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.runner; + +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.rpc.FatalErrorHandler; + +import java.util.UUID; + +/** + * Factory for the {@link JobDispatcherLeaderProcess}. + */ +public class JobDispatcherLeaderProcessFactory implements DispatcherLeaderProcessFactory { + private final DispatcherLeaderProcessImpl.DispatcherServiceFactory dispatcherServiceFactory; + + private final JobGraph jobGraph; + + private final FatalErrorHandler fatalErrorHandler; + + JobDispatcherLeaderProcessFactory( + DispatcherLeaderProcessImpl.DispatcherServiceFactory dispatcherServiceFactory, + JobGraph jobGraph, + FatalErrorHandler fatalErrorHandler) { + this.dispatcherServiceFactory = dispatcherServiceFactory; + this.jobGraph = jobGraph; + this.fatalErrorHandler = fatalErrorHandler; + } + + @Override + public DispatcherLeaderProcess create(UUID leaderSessionID) { + return new JobDispatcherLeaderProcess(leaderSessionID, dispatcherServiceFactory, jobGraph, fatalErrorHandler); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java new file mode 100644 index 000000000000..137cf42846eb --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.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.runtime.dispatcher.runner; + +import org.apache.flink.runtime.dispatcher.JobDispatcherFactory; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; +import org.apache.flink.runtime.entrypoint.component.JobGraphRetriever; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.FlinkRuntimeException; + +import java.util.concurrent.Executor; + +/** + * Factory for the {@link JobDispatcherLeaderProcessFactory}. + */ +public class JobDispatcherLeaderProcessFactoryFactory implements DispatcherLeaderProcessFactoryFactory { + + private final JobGraphRetriever jobGraphRetriever; + + private JobDispatcherLeaderProcessFactoryFactory(JobGraphRetriever jobGraphRetriever) { + this.jobGraphRetriever = jobGraphRetriever; + } + + @Override + public DispatcherLeaderProcessFactory createFactory( + JobGraphStoreFactory jobGraphStoreFactory, + Executor ioExecutor, + RpcService rpcService, + PartialDispatcherServices partialDispatcherServices, + FatalErrorHandler fatalErrorHandler) { + + final JobGraph jobGraph; + + try { + jobGraph = jobGraphRetriever.retrieveJobGraph(partialDispatcherServices.getConfiguration()); + } catch (FlinkException e) { + throw new FlinkRuntimeException("Could not retrieve the JobGraph.", e); + } + + final DispatcherServiceImplFactory dispatcherServiceImplFactory = new DispatcherServiceImplFactory( + JobDispatcherFactory.INSTANCE, + rpcService, + partialDispatcherServices); + + return new JobDispatcherLeaderProcessFactory( + dispatcherServiceImplFactory, + jobGraph, + fatalErrorHandler); + } + + public static JobDispatcherLeaderProcessFactoryFactory create(JobGraphRetriever jobGraphRetriever) { + return new JobDispatcherLeaderProcessFactoryFactory(jobGraphRetriever); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactoryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactoryFactory.java new file mode 100644 index 000000000000..975b6ff91e1a --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactoryFactory.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.runner; + +import org.apache.flink.runtime.dispatcher.DispatcherFactory; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; +import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcService; + +import java.util.concurrent.Executor; + +/** + * Factory for the {@link DispatcherLeaderProcessImplFactory}. + */ +public class SessionDispatcherLeaderProcessFactoryFactory implements DispatcherLeaderProcessFactoryFactory { + + private final DispatcherFactory dispatcherFactory; + + private SessionDispatcherLeaderProcessFactoryFactory(DispatcherFactory dispatcherFactory) { + this.dispatcherFactory = dispatcherFactory; + } + + @Override + public DispatcherLeaderProcessFactory createFactory( + JobGraphStoreFactory jobGraphStoreFactory, + Executor ioExecutor, + RpcService rpcService, + PartialDispatcherServices partialDispatcherServices, + FatalErrorHandler fatalErrorHandler) { + final DispatcherLeaderProcessImpl.DispatcherServiceFactory dispatcherServiceFactory = new DispatcherServiceImplFactory( + dispatcherFactory, + rpcService, + partialDispatcherServices); + + return new DispatcherLeaderProcessImplFactory( + dispatcherServiceFactory, + jobGraphStoreFactory, + ioExecutor, + fatalErrorHandler); + } + + public static SessionDispatcherLeaderProcessFactoryFactory create(DispatcherFactory dispatcherFactory) { + return new SessionDispatcherLeaderProcessFactoryFactory(dispatcherFactory); + } +} 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 be9e7a1a8b1a..0650a13f2d00 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 @@ -29,12 +29,10 @@ 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.JobDispatcherFactory; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; import org.apache.flink.runtime.dispatcher.SessionDispatcherFactory; import org.apache.flink.runtime.dispatcher.runner.DispatcherRunner; import org.apache.flink.runtime.dispatcher.runner.DispatcherRunnerFactory; -import org.apache.flink.runtime.dispatcher.runner.DispatcherRunnerFactoryImpl; import org.apache.flink.runtime.dispatcher.runner.DispatcherRunnerImplNGFactory; import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.heartbeat.HeartbeatServices; @@ -267,7 +265,7 @@ public DispatcherResourceManagerComponent create( public static DefaultDispatcherResourceManagerComponentFactory createSessionComponentFactory( ResourceManagerFactory resourceManagerFactory) { return new DefaultDispatcherResourceManagerComponentFactory( - new DispatcherRunnerImplNGFactory(SessionDispatcherFactory.INSTANCE), + DispatcherRunnerImplNGFactory.createSessionRunner(SessionDispatcherFactory.INSTANCE), resourceManagerFactory, SessionRestEndpointFactory.INSTANCE); } @@ -276,7 +274,7 @@ public static DefaultDispatcherResourceManagerComponentFactory createJobComponen ResourceManagerFactory resourceManagerFactory, JobGraphRetriever jobGraphRetriever) { return new DefaultDispatcherResourceManagerComponentFactory( - new DispatcherRunnerImplNGFactory(new JobDispatcherFactory(jobGraphRetriever)), + DispatcherRunnerImplNGFactory.createJobRunner(jobGraphRetriever), resourceManagerFactory, JobRestEndpointFactory.INSTANCE); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ThrowingJobGraphWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ThrowingJobGraphWriter.java new file mode 100644 index 000000000000..5ea03ba1f725 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ThrowingJobGraphWriter.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.runtime.jobmanager; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.JobGraph; + +/** + * {@link JobGraphWriter} implementation which does not allow to store + * {@link JobGraph}. + */ +public enum ThrowingJobGraphWriter implements JobGraphWriter { + INSTANCE; + + @Override + public void putJobGraph(JobGraph jobGraph) { + throw new UnsupportedOperationException("Cannot store job graphs."); + } + + @Override + public void removeJobGraph(JobID jobId) {} + + @Override + public void releaseJobGraph(JobID jobId) {} +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java index 5afa743bcc9d..3aa13d2917d2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java @@ -124,7 +124,7 @@ public void testJobRecoveryUnderLeaderChange() throws Exception { final TestingJobManagerRunnerFactory jobManagerRunnerFactory = new TestingJobManagerRunnerFactory(1); - final DispatcherRunnerImplNGFactory dispatcherRunnerFactory = new DispatcherRunnerImplNGFactory(new TestingDispatcherFactory(jobManagerRunnerFactory)); + final DispatcherRunnerImplNGFactory dispatcherRunnerFactory = DispatcherRunnerImplNGFactory.createSessionRunner(new TestingDispatcherFactory(jobManagerRunnerFactory)); try (final DispatcherRunner dispatcherRunner = createDispatcherRunner( rpcService, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDispatcherRunnerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDispatcherRunnerImplTest.java index 6550decc73e7..0c30b19a2179 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDispatcherRunnerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDispatcherRunnerImplTest.java @@ -156,7 +156,7 @@ public void testResourceCleanupUnderLeadershipChange() throws Exception { final JobGraph jobGraph = createJobGraphWithBlobs(); - final DispatcherRunnerImplNGFactory dispatcherRunnerImplNGFactory = new DispatcherRunnerImplNGFactory(SessionDispatcherFactory.INSTANCE); + final DispatcherRunnerImplNGFactory dispatcherRunnerImplNGFactory = DispatcherRunnerImplNGFactory.createSessionRunner(SessionDispatcherFactory.INSTANCE); try (final DispatcherRunner dispatcherRunner = createDispatcherRunner( rpcService, 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 44e818db6898..b0b4ffa40fcb 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 @@ -131,7 +131,7 @@ public CompletableFuture getDispatcherGatewayFuture() { private DispatcherResourceManagerComponentFactory createTestingDispatcherResourceManagerComponentFactory() { return TestingDefaultDispatcherResourceManagerComponentFactory.createSessionComponentFactory( - new DispatcherRunnerImplNGFactory(SessionDispatcherWithUUIDFactory.INSTANCE), + DispatcherRunnerImplNGFactory.createSessionRunner(SessionDispatcherWithUUIDFactory.INSTANCE), StandaloneResourceManagerWithUUIDFactory.INSTANCE); } } From 17c42a7ff937bca748e04564f1d21a7150739e9a Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 19 Sep 2019 16:23:15 +0200 Subject: [PATCH 266/746] [hotfix] Remove explicit JobGraphWriter from Dispatcher constructor --- .../java/org/apache/flink/runtime/dispatcher/Dispatcher.java | 5 ++--- .../org/apache/flink/runtime/dispatcher/MiniDispatcher.java | 3 +-- .../flink/runtime/dispatcher/StandaloneDispatcher.java | 3 +-- .../apache/flink/runtime/dispatcher/TestingDispatcher.java | 3 +-- 4 files changed, 5 insertions(+), 9 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 87570438ee9f..9cf16a666888 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 @@ -135,8 +135,7 @@ public Dispatcher( String endpointId, DispatcherId fencingToken, Collection recoveredJobs, - DispatcherServices dispatcherServices, - JobGraphWriter jobGraphWriter) throws Exception { + DispatcherServices dispatcherServices) throws Exception { super(rpcService, endpointId, fencingToken); Preconditions.checkNotNull(dispatcherServices); @@ -146,7 +145,7 @@ public Dispatcher( this.heartbeatServices = dispatcherServices.getHeartbeatServices(); this.blobServer = dispatcherServices.getBlobServer(); this.fatalErrorHandler = dispatcherServices.getFatalErrorHandler(); - this.jobGraphWriter = Preconditions.checkNotNull(jobGraphWriter); + this.jobGraphWriter = dispatcherServices.getJobGraphWriter(); this.jobManagerMetricGroup = dispatcherServices.getJobManagerMetricGroup(); this.metricServiceQueryAddress = dispatcherServices.getMetricQueryServiceAddress(); 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 3f7acc629e5e..1703441a5701 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 @@ -59,8 +59,7 @@ public MiniDispatcher( endpointId, fencingToken, Collections.singleton(jobGraph), - dispatcherServices, - new SingleJobJobGraphStore(jobGraph)); + dispatcherServices); this.executionMode = checkNotNull(executionMode); } 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 8a4c429b9345..bde5d10c83cd 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 @@ -41,7 +41,6 @@ public StandaloneDispatcher( endpointId, fencingToken, recoveredJobs, - dispatcherServices, - dispatcherServices.getJobGraphWriter()); + dispatcherServices); } } 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 c29ce48169a6..86d5b9a0da33 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 @@ -78,8 +78,7 @@ class TestingDispatcher extends Dispatcher { metricQueryServiceAddress, jobManagerMetricGroup, highAvailabilityServices.getJobGraphStore(), - jobManagerRunnerFactory), - highAvailabilityServices.getJobGraphStore()); + jobManagerRunnerFactory)); this.startFuture = new CompletableFuture<>(); } From 7851cb696602ca7c900c3547820b177acfdc8e40 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 24 Sep 2019 11:41:21 +0200 Subject: [PATCH 267/746] [FLINK-11843] Port and remove DispatcherTest [FLINK-11843] Remove DispatcherTest#testLeaderElection DispatcherTest#testLeaderElection is superseded by DispatcherRunnerImplNGTest#grantLeadership_validLeader_confirmsLeaderSession. [FLINK-11843] Port DispatcherTest#testJobSuspensionWhenDispatcherLosesLeadership Changed DispatcherTest#testJobSuspensionWhenDispatcherLosesLeadership into DispatcherRunnerImplNGTest#revokeLeadership_withExistingLeader_stopsLeaderProcess to cover test logic of #testJobSuspensionWhenDispatcherLosesLeadership. [FLINK-11843] Port DispatcherTest#testFatalErrorAfterJobRecoveryFailure Port DipsatcherTest#testFatalErrorAfterJobRecoveryFailure to DispatcherLeaderProcessImplTest#recoverJobs_withRecoveryFailure_failsFatally [FLINK-11843] Port DispatcherTest#testFatalErrorAfterJobIdRecoveryFailure Port DispatcherTest#testFatalErrorAfterJobIdRecoveryFailure to DispatcherLeaderProcessImplTest#recoverJobs_withRecoveryFailure_failsFatally. [FLINK-11843] Port DispatcherTest#testJobSubmissionErrorAfterJobRecovery Port DispatcherTest#testJobSubmissionErrorAfterJobRecovery to [FLINK-11843] Port DispatcherTest#testJobRecovery Replace DispatcherTest#testJobRecovery with DispatcherRunnerImplNGITCase#leaderChange_afterJobSubmission_recoversSubmittedJob. [FLINK-11843] Clean up DispatcherTest --- .../runtime/dispatcher/DispatcherTest.java | 278 +++++------------- .../DispatcherLeaderProcessImplTest.java | 40 +++ .../runner/DispatcherRunnerImplNGITCase.java | 154 ++++++++++ .../runner/DispatcherRunnerImplNGTest.java | 29 ++ 4 files changed, 295 insertions(+), 206 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGITCase.java 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 e50cad774172..83714e3fa63e 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 @@ -34,12 +34,11 @@ import org.apache.flink.runtime.executiongraph.ErrorInfo; 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.jobmanager.JobGraphStore; +import org.apache.flink.runtime.jobmanager.StandaloneJobGraphStore; import org.apache.flink.runtime.jobmaster.JobManagerRunner; import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; import org.apache.flink.runtime.jobmaster.JobNotFinishedException; @@ -91,7 +90,6 @@ import java.nio.file.Paths; import java.util.Collection; import java.util.Collections; -import java.util.Map; import java.util.Optional; import java.util.UUID; import java.util.concurrent.ArrayBlockingQueue; @@ -101,15 +99,12 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.core.Is.is; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -135,14 +130,8 @@ public class DispatcherTest extends TestLogger { private TestingFatalErrorHandler fatalErrorHandler; - private TestingJobGraphStore jobGraphStore; - - private TestingLeaderElectionService dispatcherLeaderElectionService; - private TestingLeaderElectionService jobMasterLeaderElectionService; - private RunningJobsRegistry runningJobsRegistry; - private CountDownLatch createdJobManagerRunnerLatch; private Configuration configuration; @@ -179,18 +168,14 @@ public void setUp() throws Exception { fatalErrorHandler = new TestingFatalErrorHandler(); heartbeatServices = new HeartbeatServices(1000L, 10000L); - jobGraphStore = TestingJobGraphStore.newBuilder().build(); - dispatcherLeaderElectionService = new TestingLeaderElectionService(); jobMasterLeaderElectionService = new TestingLeaderElectionService(); haServices = new TestingHighAvailabilityServices(); - haServices.setDispatcherLeaderElectionService(dispatcherLeaderElectionService); - haServices.setJobGraphStore(jobGraphStore); + haServices.setJobGraphStore(new StandaloneJobGraphStore()); haServices.setJobMasterLeaderElectionService(TEST_JOB_ID, jobMasterLeaderElectionService); haServices.setCheckpointRecoveryFactory(new StandaloneCheckpointRecoveryFactory()); haServices.setResourceManagerLeaderRetriever(new SettableLeaderRetrievalService()); - runningJobsRegistry = haServices.getRunningJobsRegistry(); configuration = new Configuration(); @@ -203,31 +188,69 @@ public void setUp() throws Exception { } @Nonnull - private TestingDispatcher createAndStartDispatcher(HeartbeatServices heartbeatServices, TestingHighAvailabilityServices haServices, JobManagerRunnerFactory jobManagerRunnerFactory) throws Exception { - final TestingDispatcher dispatcher = createDispatcher(heartbeatServices, haServices, jobManagerRunnerFactory); + private TestingDispatcher createAndStartDispatcher( + HeartbeatServices heartbeatServices, + TestingHighAvailabilityServices haServices, + JobManagerRunnerFactory jobManagerRunnerFactory) throws Exception { + final TestingDispatcher dispatcher = new TestingDispatcherBuilder() + .setHaServices(haServices) + .setHeartbeatServices(heartbeatServices) + .setJobManagerRunnerFactory(jobManagerRunnerFactory) + .build(); dispatcher.start(); return dispatcher; } - @Nonnull - private TestingDispatcher createDispatcher(HeartbeatServices heartbeatServices, TestingHighAvailabilityServices haServices, JobManagerRunnerFactory jobManagerRunnerFactory) throws Exception { - TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway(); - return new TestingDispatcher( - rpcService, - Dispatcher.DISPATCHER_NAME + '_' + name.getMethodName(), - DispatcherId.generate(), - Collections.emptyList(), - configuration, - haServices, - () -> CompletableFuture.completedFuture(resourceManagerGateway), - blobServer, - heartbeatServices, - UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), - null, - new MemoryArchivedExecutionGraphStore(), - jobManagerRunnerFactory, - fatalErrorHandler); + private class TestingDispatcherBuilder { + + private Collection initialJobGraphs = Collections.emptyList(); + + private HeartbeatServices heartbeatServices = DispatcherTest.this.heartbeatServices; + + private HighAvailabilityServices haServices = DispatcherTest.this.haServices; + + private JobManagerRunnerFactory jobManagerRunnerFactory = DefaultJobManagerRunnerFactory.INSTANCE; + + TestingDispatcherBuilder setHeartbeatServices(HeartbeatServices heartbeatServices) { + this.heartbeatServices = heartbeatServices; + return this; + } + + TestingDispatcherBuilder setHaServices(HighAvailabilityServices haServices) { + this.haServices = haServices; + return this; + } + + TestingDispatcherBuilder setInitialJobGraphs(Collection initialJobGraphs) { + this.initialJobGraphs = initialJobGraphs; + return this; + } + + TestingDispatcherBuilder setJobManagerRunnerFactory(JobManagerRunnerFactory jobManagerRunnerFactory) { + this.jobManagerRunnerFactory = jobManagerRunnerFactory; + return this; + } + + TestingDispatcher build() throws Exception { + TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway(); + + return new TestingDispatcher( + rpcService, + Dispatcher.DISPATCHER_NAME + '_' + name.getMethodName(), + DispatcherId.generate(), + initialJobGraphs, + configuration, + haServices, + () -> CompletableFuture.completedFuture(resourceManagerGateway), + blobServer, + heartbeatServices, + UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), + null, + new MemoryArchivedExecutionGraphStore(), + jobManagerRunnerFactory, + fatalErrorHandler); + } } @After @@ -257,11 +280,6 @@ public void tearDown() throws Exception { public void testJobSubmission() throws Exception { dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch)); - CompletableFuture leaderFuture = dispatcherLeaderElectionService.isLeader(UUID.randomUUID()); - - // wait for the leader to be elected - leaderFuture.get(); - DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); CompletableFuture acknowledgeFuture = dispatcherGateway.submitJob(jobGraph, TIMEOUT); @@ -270,7 +288,7 @@ public void testJobSubmission() throws Exception { assertTrue( "jobManagerRunner was not started", - dispatcherLeaderElectionService.getStartFuture().isDone()); + jobMasterLeaderElectionService.getStartFuture().isDone()); } /** @@ -281,11 +299,6 @@ public void testJobSubmission() throws Exception { public void testJobSubmissionWithPartialResourceConfigured() throws Exception { dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch)); - CompletableFuture leaderFuture = dispatcherLeaderElectionService.isLeader(UUID.randomUUID()); - - // wait for the leader to be elected - leaderFuture.get(); - DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); ResourceSpec resourceSpec = ResourceSpec.newBuilder().setCpuCores(2).build(); @@ -311,29 +324,6 @@ public void testJobSubmissionWithPartialResourceConfigured() throws Exception { } } - /** - * Tests that the dispatcher takes part in the leader election. - */ - @Test - public void testLeaderElection() throws Exception { - CompletableFuture jobIdsFuture = new CompletableFuture<>(); - 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(); - - // wait that we asked the JobGraphStore for the stored jobs - jobIdsFuture.get(TIMEOUT.toMilliseconds(), TimeUnit.MILLISECONDS); - } - /** * Test that {@link JobResult} is cached when the job finishes. */ @@ -341,8 +331,6 @@ public void testLeaderElection() throws Exception { public void testCacheJobExecutionResult() throws Exception { dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch)); - dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); - final DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); final JobID failedJobId = new JobID(); @@ -368,8 +356,6 @@ public void testCacheJobExecutionResult() throws Exception { public void testThrowExceptionIfJobExecutionResultNotFound() throws Exception { dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch)); - dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); - final DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); try { dispatcherGateway.requestJob(new JobID(), TIMEOUT).get(); @@ -379,44 +365,6 @@ public void testThrowExceptionIfJobExecutionResultNotFound() throws Exception { } } - /** - * Tests that a reelected Dispatcher can recover jobs. - */ - @Test - public void testJobRecovery() throws Exception { - dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch)); - - final DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); - - // elect the initial dispatcher as the leader - dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); - - // submit the job to the current leader - dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); - - // check that the job has been persisted - assertThat(jobGraphStore.getJobIds(), contains(jobGraph.getJobID())); - - jobMasterLeaderElectionService.isLeader(UUID.randomUUID()).get(); - - assertThat(runningJobsRegistry.getJobSchedulingStatus(jobGraph.getJobID()), is(RunningJobsRegistry.JobSchedulingStatus.RUNNING)); - - // revoke the leadership which will stop all currently running jobs - dispatcherLeaderElectionService.notLeader(); - - // re-grant the leadership, this should trigger the job recovery - dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); - - // wait until we have recovered the job - createdJobManagerRunnerLatch.await(); - - // check whether the job has been recovered - final Collection jobIds = dispatcherGateway.listJobs(TIMEOUT).get(); - - assertThat(jobIds, hasSize(1)); - assertThat(jobIds, contains(jobGraph.getJobID())); - } - /** * Tests that we can dispose a savepoint. */ @@ -429,8 +377,6 @@ public void testSavepointDisposal() throws Exception { final DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); - dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); - assertThat(Files.exists(savepointPath), is(true)); dispatcherGateway.disposeSavepoint(externalPointer.toString(), TIMEOUT).get(); @@ -466,8 +412,6 @@ public void testWaitingForJobMasterLeadership() throws Exception { final DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); - dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); - dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); final CompletableFuture jobStatusFuture = dispatcherGateway.requestJobStatus(jobGraph.getJobID(), TIMEOUT); @@ -487,80 +431,20 @@ public void testWaitingForJobMasterLeadership() throws Exception { } /** - * Tests that the {@link Dispatcher} terminates if it cannot recover jobs ids from - * the {@link JobGraphStore}. See FLINK-8943. - */ - @Test - public void testFatalErrorAfterJobIdRecoveryFailure() throws Exception { - final FlinkException testException = new FlinkException("Test exception"); - 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(); - - // we expect that a fatal error occurred - final Throwable error = fatalErrorHandler.getErrorFuture().get(TIMEOUT.toMilliseconds(), TimeUnit.MILLISECONDS); - - assertThat(ExceptionUtils.findThrowableWithMessage(error, testException.getMessage()).isPresent(), is(true)); - - fatalErrorHandler.clearError(); - } - - /** - * Tests that the {@link Dispatcher} terminates if it cannot recover jobs from - * the {@link JobGraphStore}. See FLINK-8943. - */ - @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(); - - jobGraphStore.putJobGraph(jobGraph); - - electDispatcher(); - - // we expect that a fatal error occurred - final Throwable error = fatalErrorHandler.getErrorFuture().get(TIMEOUT.toMilliseconds(), TimeUnit.MILLISECONDS); - - assertThat(ExceptionUtils.findThrowableWithMessage(error, testException.getMessage()).isPresent(), is(true)); - - fatalErrorHandler.clearError(); - } - - /** - * Tests that the {@link Dispatcher} fails fatally if the job submission of a recovered job fails. + * Tests that the {@link Dispatcher} fails fatally if the recoverd jobs cannot be started. * See FLINK-9097. */ @Test - public void testJobSubmissionErrorAfterJobRecovery() throws Exception { + public void testFatalErrorIfRecoveredJobsCannotBeStarted() throws Exception { final FlinkException testException = new FlinkException("Test exception"); - dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch)); - - dispatcher.waitUntilStarted(); - final JobGraph failingJobGraph = createFailingJobGraph(testException); - jobGraphStore.putJobGraph(failingJobGraph); + dispatcher = new TestingDispatcherBuilder() + .setInitialJobGraphs(Collections.singleton(failingJobGraph)) + .build(); - electDispatcher(); + dispatcher.start(); final Throwable error = fatalErrorHandler.getErrorFuture().get(TIMEOUT.toMilliseconds(), TimeUnit.MILLISECONDS); @@ -583,8 +467,6 @@ public void testBlockingJobManagerRunner() throws Exception { haServices, new BlockingJobManagerRunnerFactory(jobManagerRunnerCreationLatch::await)); - dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); - final DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); final CompletableFuture submissionFuture = dispatcherGateway.submitJob(jobGraph, TIMEOUT); @@ -622,8 +504,6 @@ public void testFailingJobManagerRunnerCleanup() throws Exception { } })); - dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); - final DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); CompletableFuture submissionFuture = dispatcherGateway.submitJob(jobGraph, TIMEOUT); @@ -649,14 +529,11 @@ public void testFailingJobManagerRunnerCleanup() throws Exception { @Test public void testPersistedJobGraphWhenDispatcherIsShutDown() throws Exception { final TestingJobGraphStore submittedJobGraphStore = TestingJobGraphStore.newBuilder().build(); + submittedJobGraphStore.start(null); haServices.setJobGraphStore(submittedJobGraphStore); dispatcher = createAndStartDispatcher(heartbeatServices, haServices, DefaultJobManagerRunnerFactory.INSTANCE); - // grant leadership and submit a single job - final DispatcherId expectedDispatcherId = DispatcherId.generate(); - dispatcherLeaderElectionService.isLeader(expectedDispatcherId.toUUID()).get(); - final DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); final CompletableFuture submissionFuture = dispatcherGateway.submitJob(jobGraph, TIMEOUT); submissionFuture.get(); @@ -668,14 +545,12 @@ public void testPersistedJobGraphWhenDispatcherIsShutDown() throws Exception { } /** - * Tests that a submitted job is suspended if the Dispatcher loses leadership. + * Tests that a submitted job is suspended if the Dispatcher is terminated. */ @Test - public void testJobSuspensionWhenDispatcherLosesLeadership() throws Exception { + public void testJobSuspensionWhenDispatcherIsTerminated() throws Exception { dispatcher = createAndStartDispatcher(heartbeatServices, haServices, new ExpectedJobIdJobManagerRunnerFactory(TEST_JOB_ID, createdJobManagerRunnerLatch)); - dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); - DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); @@ -684,7 +559,7 @@ public void testJobSuspensionWhenDispatcherLosesLeadership() throws Exception { assertThat(jobResultFuture.isDone(), is(false)); - dispatcherLeaderElectionService.notLeader(); + dispatcher.closeAsync(); try { jobResultFuture.get(); @@ -697,7 +572,6 @@ public void testJobSuspensionWhenDispatcherLosesLeadership() throws Exception { @Test public void testShutDownClusterShouldCompleteShutDownFuture() throws Exception { dispatcher = createAndStartDispatcher(heartbeatServices, haServices, DefaultJobManagerRunnerFactory.INSTANCE); - dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); final DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); dispatcherGateway.shutDownCluster().get(); @@ -705,7 +579,7 @@ public void testShutDownClusterShouldCompleteShutDownFuture() throws Exception { dispatcher.getShutDownFuture().get(); } - private final class BlockingJobManagerRunnerFactory extends TestingJobManagerRunnerFactory { + private static final class BlockingJobManagerRunnerFactory extends TestingJobManagerRunnerFactory { @Nonnull private final ThrowingRunnable jobManagerRunnerCreationLatch; @@ -722,14 +596,6 @@ public TestingJobManagerRunner createJobManagerRunner(JobGraph jobGraph, Configu } } - private void electDispatcher() { - UUID expectedLeaderSessionId = UUID.randomUUID(); - - assertNull(dispatcherLeaderElectionService.getConfirmationFuture()); - - dispatcherLeaderElectionService.isLeader(expectedLeaderSessionId); - } - private JobGraph createFailingJobGraph(Exception failureCause) { final FailingJobVertex jobVertex = new FailingJobVertex("Failing JobVertex", failureCause); jobVertex.setInvokableClass(NoOpInvokable.class); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java index c1403b642aba..e84a1f7f5dd2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java @@ -43,6 +43,7 @@ import java.util.Collection; import java.util.Collections; +import java.util.Map; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; @@ -334,6 +335,45 @@ public void onAddedJobGraph_failingRecovery_propagatesTheFailure() throws Except } } + @Test + public void recoverJobs_withRecoveryFailure_failsFatally() throws Exception { + final FlinkException testException = new FlinkException("Test exception"); + jobGraphStore = TestingJobGraphStore.newBuilder() + .setRecoverJobGraphFunction( + (JobID jobId, Map jobGraphs) -> { + throw testException; + }) + .setInitialJobGraphs(Collections.singleton(JOB_GRAPH)) + .build(); + + runJobRecoveryFailureTest(testException); + } + + @Test + public void recoverJobs_withJobIdRecoveryFailure_failsFatally() throws Exception { + final FlinkException testException = new FlinkException("Test exception"); + jobGraphStore = TestingJobGraphStore.newBuilder() + .setJobIdsFunction( + ignored -> { + throw testException; + }) + .build(); + + runJobRecoveryFailureTest(testException); + } + + private void runJobRecoveryFailureTest(FlinkException testException) throws Exception { + try (final DispatcherLeaderProcessImpl dispatcherLeaderProcess = createDispatcherLeaderProcess()) { + dispatcherLeaderProcess.start(); + + // we expect that a fatal error occurred + final Throwable error = fatalErrorHandler.getErrorFuture().get(); + Assert.assertThat(ExceptionUtils.findThrowableWithMessage(error, testException.getMessage()).isPresent(), is(true)); + + fatalErrorHandler.clearError(); + } + } + @Test @Ignore public void onAddedJobGraph_falsePositive_willBeIgnored() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGITCase.java new file mode 100644 index 000000000000..5f73e6c6436a --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGITCase.java @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.runner; + +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.dispatcher.DispatcherGateway; +import org.apache.flink.runtime.dispatcher.MemoryArchivedExecutionGraphStore; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; +import org.apache.flink.runtime.dispatcher.VoidHistoryServerArchivist; +import org.apache.flink.runtime.heartbeat.TestingHeartbeatServices; +import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServicesBuilder; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobmanager.JobGraphStore; +import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; +import org.apache.flink.runtime.minicluster.SessionDispatcherWithUUIDFactory; +import org.apache.flink.runtime.rpc.TestingRpcServiceResource; +import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.runtime.testutils.TestingJobGraphStore; +import org.apache.flink.runtime.util.BlobServerResource; +import org.apache.flink.runtime.util.TestingFatalErrorHandler; +import org.apache.flink.util.TestLogger; + +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; + +import java.util.Collection; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; + +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.hasSize; +import static org.junit.Assert.assertThat; + +/** + * Integration tests for the {@link DispatcherRunnerImplNG}. + */ +public class DispatcherRunnerImplNGITCase extends TestLogger { + + private static final Time TIMEOUT = Time.seconds(10L); + + private static final JobID TEST_JOB_ID = new JobID(); + + @ClassRule + public static TestingRpcServiceResource rpcServiceResource = new TestingRpcServiceResource(); + + @ClassRule + public static BlobServerResource blobServerResource = new BlobServerResource(); + + private JobGraph jobGraph; + + private TestingLeaderElectionService dispatcherLeaderElectionService; + + private TestingFatalErrorHandler fatalErrorHandler; + + private JobGraphStore jobGraphStore; + + private PartialDispatcherServices partialDispatcherServices; + + @Before + public void setup() { + jobGraph = createJobGraph(); + dispatcherLeaderElectionService = new TestingLeaderElectionService(); + fatalErrorHandler = new TestingFatalErrorHandler(); + jobGraphStore = TestingJobGraphStore.newBuilder().build(); + + partialDispatcherServices = new PartialDispatcherServices( + new Configuration(), + new TestingHighAvailabilityServicesBuilder().build(), + CompletableFuture::new, + blobServerResource.getBlobServer(), + new TestingHeartbeatServices(), + UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), + new MemoryArchivedExecutionGraphStore(), + fatalErrorHandler, + VoidHistoryServerArchivist.INSTANCE, + null); + } + + @After + public void teardown() throws Exception { + if (fatalErrorHandler != null) { + fatalErrorHandler.rethrowError(); + } + } + + @Test + public void leaderChange_afterJobSubmission_recoversSubmittedJob() throws Exception { + try (final DispatcherRunnerImplNG dispatcherRunner = createDispatcherRunner()) { + final UUID firstLeaderSessionId = UUID.randomUUID(); + + dispatcherLeaderElectionService.isLeader(firstLeaderSessionId); + + final DispatcherGateway firstDispatcherGateway = dispatcherRunner.getDispatcherGateway().get(); + + firstDispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); + + dispatcherLeaderElectionService.notLeader(); + + final UUID secondLeaderSessionId = UUID.randomUUID(); + dispatcherLeaderElectionService.isLeader(secondLeaderSessionId).get(); + + final DispatcherGateway secondDispatcherGateway = dispatcherRunner.getDispatcherGateway().get(); + + final Collection jobIds = secondDispatcherGateway.listJobs(TIMEOUT).get(); + + assertThat(jobIds, hasSize(1)); + assertThat(jobIds, contains(jobGraph.getJobID())); + } + } + + private static JobGraph createJobGraph() { + final JobVertex testVertex = new JobVertex("testVertex"); + testVertex.setInvokableClass(NoOpInvokable.class); + final JobGraph testJob = new JobGraph(TEST_JOB_ID, "testJob", testVertex); + testJob.setAllowQueuedScheduling(true); + + return testJob; + } + + private DispatcherRunnerImplNG createDispatcherRunner() throws Exception { + final DispatcherRunnerImplNGFactory runnerFactory = DispatcherRunnerImplNGFactory.createSessionRunner(SessionDispatcherWithUUIDFactory.INSTANCE); + + return runnerFactory.createDispatcherRunner( + dispatcherLeaderElectionService, + fatalErrorHandler, + () -> jobGraphStore, + TestingUtils.defaultExecutor(), + rpcServiceResource.getTestingRpcService(), + partialDispatcherServices); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGTest.java index e6eb8002b9db..d8fd33ecc583 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGTest.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.dispatcher.runner; import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.dispatcher.DispatcherId; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; @@ -135,6 +136,34 @@ public void getDispatcherGateway_withChangingLeaders_returnsLeadingDispatcherGat } } + @Test + public void revokeLeadership_withExistingLeader_stopsLeaderProcess() throws Exception { + final UUID leaderSessionId = UUID.randomUUID(); + + final CompletableFuture startFuture = new CompletableFuture<>(); + final CompletableFuture stopFuture = new CompletableFuture<>(); + testingDispatcherLeaderProcessFactory = TestingDispatcherLeaderProcessFactory.from( + TestingDispatcherLeaderProcess.newBuilder(leaderSessionId) + .setStartConsumer(startFuture::complete) + .setCloseAsyncSupplier( + () -> { + stopFuture.complete(null); + return FutureUtils.completedVoidFuture(); + }) + .build()); + try (final DispatcherRunnerImplNG dispatcherRunner = createDispatcherRunner()) { + testingLeaderElectionService.isLeader(leaderSessionId); + + // wait until the leader process has been started + startFuture.get(); + + testingLeaderElectionService.notLeader(); + + // verify that the leader gets stopped + stopFuture.get(); + } + } + @Test public void grantLeadership_withExistingLeader_waitsForTerminationOfFirstLeader() throws Exception { final UUID firstLeaderSessionId = UUID.randomUUID(); From 9bf06697a59cce8f9fd6de37c421bb8a49cf540f Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 24 Sep 2019 17:56:28 +0200 Subject: [PATCH 268/746] [FLINK-11843] Port DispatcherResourceCleanupTest Ports the DispatcherResourceCleanupTest to work with the new Dispatcher contract --- .../DispatcherResourceCleanupTest.java | 26 ++++++------------- .../testutils/TestingJobGraphStore.java | 20 +++++++++++++- 2 files changed, 27 insertions(+), 19 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 b07400cbbfbc..8e7dd5d90531 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 @@ -45,7 +45,6 @@ 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; @@ -108,8 +107,6 @@ public class DispatcherResourceCleanupTest extends TestLogger { private Configuration configuration; - private TestingLeaderElectionService dispatcherLeaderElectionService; - private SingleRunningJobsRegistry runningJobsRegistry; private TestingHighAvailabilityServices highAvailabilityServices; @@ -150,12 +147,10 @@ public void setup() throws Exception { configuration.setString(BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); highAvailabilityServices = new TestingHighAvailabilityServices(); - dispatcherLeaderElectionService = new TestingLeaderElectionService(); - highAvailabilityServices.setDispatcherLeaderElectionService(dispatcherLeaderElectionService); clearedJobLatch = new OneShotLatch(); runningJobsRegistry = new SingleRunningJobsRegistry(jobId, clearedJobLatch); highAvailabilityServices.setRunningJobsRegistry(runningJobsRegistry); - jobGraphStore = TestingJobGraphStore.newBuilder().build(); + jobGraphStore = TestingJobGraphStore.newBuilder().withAutomaticStart().build(); highAvailabilityServices.setJobGraphStore(jobGraphStore); storedHABlobFuture = new CompletableFuture<>(); @@ -217,8 +212,6 @@ private void startDispatcher(JobManagerRunnerFactory jobManagerRunnerFactory) th dispatcher.start(); dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); - - dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); } @After @@ -409,28 +402,24 @@ private void assertThatHABlobsHaveNotBeenRemoved() { } /** - * Tests that recovered jobs will only be started after the complete termination of any - * other previously running JobMasters for the same job. + * Tests that terminating the Dispatcher will wait for all JobMasters to be terminated. */ @Test - public void testJobRecoveryWithPendingTermination() throws Exception { + public void testDispatcherTerminationWaitsForJobMasterTerminations() throws Exception { final TestingJobManagerRunnerFactory jobManagerRunnerFactory = startDispatcherAndSubmitJob(1); - runningJobsRegistry.setJobRunning(jobId); - dispatcherLeaderElectionService.notLeader(); - final UUID leaderSessionId = UUID.randomUUID(); - final CompletableFuture leaderFuture = dispatcherLeaderElectionService.isLeader(leaderSessionId); + final CompletableFuture dispatcherTerminationFuture = dispatcher.closeAsync(); try { - leaderFuture.get(10L, TimeUnit.MILLISECONDS); - fail("We should not become leader before all previously running JobMasters have terminated."); + dispatcherTerminationFuture.get(10L, TimeUnit.MILLISECONDS); + fail("We should not terminate before all running JobMasters have terminated."); } catch (TimeoutException ignored) { // expected } finally { jobManagerRunnerFactory.takeCreatedJobManagerRunner().completeTerminationFuture(); } - assertThat(leaderFuture.get(), equalTo(leaderSessionId)); + dispatcherTerminationFuture.get(); } private static final class SingleRunningJobsRegistry implements RunningJobsRegistry { @@ -494,6 +483,7 @@ public void testHABlobsAreNotRemovedIfHAJobGraphRemovalFails() throws Exception ignored -> { throw new Exception("Failed to Remove future"); }) + .withAutomaticStart() .build(); highAvailabilityServices.setJobGraphStore(jobGraphStore); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobGraphStore.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobGraphStore.java index 0376d448546a..1e3232c0f95d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobGraphStore.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobGraphStore.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.JobGraphStore; +import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; import org.apache.flink.util.function.BiFunctionWithException; import org.apache.flink.util.function.FunctionWithException; @@ -153,6 +154,8 @@ public static class Builder { private Collection initialJobGraphs = Collections.emptyList(); + private boolean startJobGraphStore = false; + private Builder() {} public Builder setStartConsumer(ThrowingConsumer startConsumer) { @@ -195,8 +198,13 @@ public Builder setInitialJobGraphs(Collection initialJobGraphs) { return this; } + public Builder withAutomaticStart() { + this.startJobGraphStore = true; + return this; + } + public TestingJobGraphStore build() { - return new TestingJobGraphStore( + final TestingJobGraphStore jobGraphStore = new TestingJobGraphStore( startConsumer, stopRunnable, jobIdsFunction, @@ -205,6 +213,16 @@ public TestingJobGraphStore build() { removeJobGraphConsumer, releaseJobGraphConsumer, initialJobGraphs); + + if (startJobGraphStore) { + try { + jobGraphStore.start(null); + } catch (Exception e) { + ExceptionUtils.rethrow(e); + } + } + + return jobGraphStore; } } From 2e1731d0e592765ee20a49a8eb91afeb8d164aa2 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 24 Sep 2019 17:59:56 +0200 Subject: [PATCH 269/746] [hotfix] Pass in DispatcherServices into TestingDispatcher --- .../runtime/dispatcher/DispatcherHATest.java | 23 +++++----- .../DispatcherResourceCleanupTest.java | 25 ++++++----- .../runtime/dispatcher/DispatcherTest.java | 24 ++++++----- .../runtime/dispatcher/TestingDispatcher.java | 42 +++---------------- .../dispatcher/ZooKeeperHADispatcherTest.java | 25 ++++++----- 5 files changed, 63 insertions(+), 76 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 dca304eb57e0..326f1ed5b89f 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 @@ -365,16 +365,19 @@ private static class HATestingDispatcher extends TestingDispatcher { endpointId, fencingToken, recoveredJobs, - configuration, - highAvailabilityServices, - resourceManagerGatewayRetriever, - blobServer, - heartbeatServices, - jobManagerMetricGroup, - metricQueryServiceAddress, - archivedExecutionGraphStore, - jobManagerRunnerFactory, - fatalErrorHandler); + new DispatcherServices( + configuration, + highAvailabilityServices, + resourceManagerGatewayRetriever, + blobServer, + heartbeatServices, + jobManagerMetricGroup, + archivedExecutionGraphStore, + fatalErrorHandler, + VoidHistoryServerArchivist.INSTANCE, + metricQueryServiceAddress, + highAvailabilityServices.getJobGraphStore(), + jobManagerRunnerFactory)); this.fencingTokens = fencingTokens; } 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 8e7dd5d90531..3ff046b8cff5 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 @@ -193,21 +193,26 @@ private TestingJobManagerRunnerFactory startDispatcherAndSubmitJob(int numBlocki private void startDispatcher(JobManagerRunnerFactory jobManagerRunnerFactory) throws Exception { TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway(); + final HeartbeatServices heartbeatServices = new HeartbeatServices(1000L, 1000L); + final MemoryArchivedExecutionGraphStore archivedExecutionGraphStore = new MemoryArchivedExecutionGraphStore(); dispatcher = new TestingDispatcher( rpcService, Dispatcher.DISPATCHER_NAME + UUID.randomUUID(), DispatcherId.generate(), Collections.emptyList(), - configuration, - highAvailabilityServices, - () -> CompletableFuture.completedFuture(resourceManagerGateway), - blobServer, - new HeartbeatServices(1000L, 1000L), - UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), - null, - new MemoryArchivedExecutionGraphStore(), - jobManagerRunnerFactory, - fatalErrorHandler); + new DispatcherServices( + configuration, + highAvailabilityServices, + () -> CompletableFuture.completedFuture(resourceManagerGateway), + blobServer, + heartbeatServices, + UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), + archivedExecutionGraphStore, + fatalErrorHandler, + VoidHistoryServerArchivist.INSTANCE, + null, + highAvailabilityServices.getJobGraphStore(), + jobManagerRunnerFactory)); dispatcher.start(); 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 83714e3fa63e..4c70012127cf 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 @@ -235,21 +235,25 @@ TestingDispatcherBuilder setJobManagerRunnerFactory(JobManagerRunnerFactory jobM TestingDispatcher build() throws Exception { TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway(); + final MemoryArchivedExecutionGraphStore archivedExecutionGraphStore = new MemoryArchivedExecutionGraphStore(); return new TestingDispatcher( rpcService, Dispatcher.DISPATCHER_NAME + '_' + name.getMethodName(), DispatcherId.generate(), initialJobGraphs, - configuration, - haServices, - () -> CompletableFuture.completedFuture(resourceManagerGateway), - blobServer, - heartbeatServices, - UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), - null, - new MemoryArchivedExecutionGraphStore(), - jobManagerRunnerFactory, - fatalErrorHandler); + new DispatcherServices( + configuration, + haServices, + () -> CompletableFuture.completedFuture(resourceManagerGateway), + blobServer, + heartbeatServices, + UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), + archivedExecutionGraphStore, + fatalErrorHandler, + VoidHistoryServerArchivist.INSTANCE, + null, + haServices.getJobGraphStore(), + jobManagerRunnerFactory)); } } 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 86d5b9a0da33..85eebb544e1d 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 @@ -20,20 +20,11 @@ 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.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.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.Nonnull; -import javax.annotation.Nullable; import java.util.Collection; import java.util.concurrent.CompletableFuture; @@ -47,38 +38,17 @@ class TestingDispatcher extends Dispatcher { private final CompletableFuture startFuture; TestingDispatcher( - RpcService rpcService, - String endpointId, - DispatcherId fencingToken, - Collection recoveredJobs, - Configuration configuration, - HighAvailabilityServices highAvailabilityServices, - GatewayRetriever resourceManagerGatewayRetriever, - BlobServer blobServer, - HeartbeatServices heartbeatServices, - JobManagerMetricGroup jobManagerMetricGroup, - @Nullable String metricQueryServiceAddress, - ArchivedExecutionGraphStore archivedExecutionGraphStore, - JobManagerRunnerFactory jobManagerRunnerFactory, - FatalErrorHandler fatalErrorHandler) throws Exception { + RpcService rpcService, + String endpointId, + DispatcherId fencingToken, + Collection recoveredJobs, + DispatcherServices dispatcherServices) throws Exception { super( rpcService, endpointId, fencingToken, recoveredJobs, - new DispatcherServices( - configuration, - highAvailabilityServices, - resourceManagerGatewayRetriever, - blobServer, - heartbeatServices, - archivedExecutionGraphStore, - fatalErrorHandler, - VoidHistoryServerArchivist.INSTANCE, - metricQueryServiceAddress, - jobManagerMetricGroup, - highAvailabilityServices.getJobGraphStore(), - jobManagerRunnerFactory)); + dispatcherServices); this.startFuture = new CompletableFuture<>(); } 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 1f74b548764a..6bc1fff82a6a 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 @@ -332,20 +332,25 @@ public void testStandbyDispatcherJobRecovery() throws Exception { @Nonnull private TestingDispatcher createDispatcher(HighAvailabilityServices highAvailabilityServices, JobManagerRunnerFactory jobManagerRunnerFactory) throws Exception { TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway(); + final HeartbeatServices heartbeatServices = new HeartbeatServices(1000L, 1000L); + final MemoryArchivedExecutionGraphStore archivedExecutionGraphStore = new MemoryArchivedExecutionGraphStore(); return new TestingDispatcher( rpcService, Dispatcher.DISPATCHER_NAME + '_' + name.getMethodName() + UUID.randomUUID(), DispatcherId.generate(), Collections.emptyList(), - configuration, - highAvailabilityServices, - () -> CompletableFuture.completedFuture(resourceManagerGateway), - blobServer, - new HeartbeatServices(1000L, 1000L), - UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), - null, - new MemoryArchivedExecutionGraphStore(), - jobManagerRunnerFactory, - testingFatalErrorHandler); + new DispatcherServices( + configuration, + highAvailabilityServices, + () -> CompletableFuture.completedFuture(resourceManagerGateway), + blobServer, + heartbeatServices, + UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), + archivedExecutionGraphStore, + testingFatalErrorHandler, + VoidHistoryServerArchivist.INSTANCE, + null, + highAvailabilityServices.getJobGraphStore(), + jobManagerRunnerFactory)); } } From a17b9c0a3e36ea7e95578f4402bc16bb9a971f5f Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 24 Sep 2019 18:10:35 +0200 Subject: [PATCH 270/746] [FLINK-11843] Pass in explicit JobGraphWriter in DispatcherTests --- .../DispatcherResourceCleanupTest.java | 11 ++--- .../runtime/dispatcher/DispatcherTest.java | 19 ++++++-- .../dispatcher/NoOpJobGraphWriter.java | 45 +++++++++++++++++++ 3 files changed, 64 insertions(+), 11 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/NoOpJobGraphWriter.java 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 3ff046b8cff5..2d514fa4c941 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,7 +39,7 @@ 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.jobmanager.JobGraphWriter; import org.apache.flink.runtime.jobmaster.JobManagerRunner; import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; import org.apache.flink.runtime.jobmaster.JobNotFinishedException; @@ -128,7 +128,7 @@ public class DispatcherResourceCleanupTest extends TestLogger { private CompletableFuture storedHABlobFuture; private CompletableFuture deleteAllHABlobsFuture; private CompletableFuture cleanupJobFuture; - private TestingJobGraphStore jobGraphStore; + private JobGraphWriter jobGraphWriter = NoOpJobGraphWriter.INSTANCE; @BeforeClass public static void setupClass() { @@ -150,8 +150,6 @@ public void setup() throws Exception { clearedJobLatch = new OneShotLatch(); runningJobsRegistry = new SingleRunningJobsRegistry(jobId, clearedJobLatch); highAvailabilityServices.setRunningJobsRegistry(runningJobsRegistry); - jobGraphStore = TestingJobGraphStore.newBuilder().withAutomaticStart().build(); - highAvailabilityServices.setJobGraphStore(jobGraphStore); storedHABlobFuture = new CompletableFuture<>(); deleteAllHABlobsFuture = new CompletableFuture<>(); @@ -211,7 +209,7 @@ private void startDispatcher(JobManagerRunnerFactory jobManagerRunnerFactory) th fatalErrorHandler, VoidHistoryServerArchivist.INSTANCE, null, - highAvailabilityServices.getJobGraphStore(), + jobGraphWriter, jobManagerRunnerFactory)); dispatcher.start(); @@ -483,7 +481,7 @@ public void clearJob(JobID jobID) { @Test public void testHABlobsAreNotRemovedIfHAJobGraphRemovalFails() throws Exception { - final JobGraphStore jobGraphStore = TestingJobGraphStore.newBuilder() + jobGraphWriter = TestingJobGraphStore.newBuilder() .setRemoveJobGraphConsumer( ignored -> { throw new Exception("Failed to Remove future"); @@ -491,7 +489,6 @@ public void testHABlobsAreNotRemovedIfHAJobGraphRemovalFails() throws Exception .withAutomaticStart() .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 4c70012127cf..f51a7310f000 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 @@ -38,7 +38,7 @@ 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.StandaloneJobGraphStore; +import org.apache.flink.runtime.jobmanager.JobGraphWriter; import org.apache.flink.runtime.jobmaster.JobManagerRunner; import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; import org.apache.flink.runtime.jobmaster.JobNotFinishedException; @@ -172,7 +172,6 @@ public void setUp() throws Exception { jobMasterLeaderElectionService = new TestingLeaderElectionService(); haServices = new TestingHighAvailabilityServices(); - haServices.setJobGraphStore(new StandaloneJobGraphStore()); haServices.setJobMasterLeaderElectionService(TEST_JOB_ID, jobMasterLeaderElectionService); haServices.setCheckpointRecoveryFactory(new StandaloneCheckpointRecoveryFactory()); haServices.setResourceManagerLeaderRetriever(new SettableLeaderRetrievalService()); @@ -212,6 +211,8 @@ private class TestingDispatcherBuilder { private JobManagerRunnerFactory jobManagerRunnerFactory = DefaultJobManagerRunnerFactory.INSTANCE; + private JobGraphWriter jobGraphWriter = NoOpJobGraphWriter.INSTANCE; + TestingDispatcherBuilder setHeartbeatServices(HeartbeatServices heartbeatServices) { this.heartbeatServices = heartbeatServices; return this; @@ -232,10 +233,16 @@ TestingDispatcherBuilder setJobManagerRunnerFactory(JobManagerRunnerFactory jobM return this; } + TestingDispatcherBuilder setJobGraphWriter(JobGraphWriter jobGraphWriter) { + this.jobGraphWriter = jobGraphWriter; + return this; + } + TestingDispatcher build() throws Exception { TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway(); final MemoryArchivedExecutionGraphStore archivedExecutionGraphStore = new MemoryArchivedExecutionGraphStore(); + return new TestingDispatcher( rpcService, Dispatcher.DISPATCHER_NAME + '_' + name.getMethodName(), @@ -252,7 +259,7 @@ TestingDispatcher build() throws Exception { fatalErrorHandler, VoidHistoryServerArchivist.INSTANCE, null, - haServices.getJobGraphStore(), + jobGraphWriter, jobManagerRunnerFactory)); } } @@ -536,7 +543,11 @@ public void testPersistedJobGraphWhenDispatcherIsShutDown() throws Exception { submittedJobGraphStore.start(null); haServices.setJobGraphStore(submittedJobGraphStore); - dispatcher = createAndStartDispatcher(heartbeatServices, haServices, DefaultJobManagerRunnerFactory.INSTANCE); + dispatcher = new TestingDispatcherBuilder() + .setJobGraphWriter(submittedJobGraphStore) + .build(); + + dispatcher.start(); final DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); final CompletableFuture submissionFuture = dispatcherGateway.submitJob(jobGraph, TIMEOUT); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/NoOpJobGraphWriter.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/NoOpJobGraphWriter.java new file mode 100644 index 000000000000..9664849263d2 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/NoOpJobGraphWriter.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.runtime.dispatcher; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmanager.JobGraphWriter; + +/** + * Testing implementation of {@link JobGraphWriter} which does nothing. + */ +public enum NoOpJobGraphWriter implements JobGraphWriter { + INSTANCE; + + @Override + public void putJobGraph(JobGraph jobGraph) throws Exception { + + } + + @Override + public void removeJobGraph(JobID jobId) throws Exception { + + } + + @Override + public void releaseJobGraph(JobID jobId) throws Exception { + + } +} From 2d3426f33284eab4e7c809be5da6746f4676bce9 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 24 Sep 2019 18:50:21 +0200 Subject: [PATCH 271/746] [FLINK-11843] Port and remove ZooKeeperHADispatcherTest#testStandbyDispatcherJobExecution ZooKeeperHADispatcherTest#testStandbyDispatcherJobExecution has been replaced with DispatcherLeaderProcessImplTest#onAddedJobGraph_ifNotRunning_isBeingIgnored [FLINK-11843] Remove ZooKeeperHADispatcherTest#testJobGraphRelease ZooKeeperHADispatcherTest#testJobGraphRelease is already covered by DispatcherLeaderProcessImplTest#closeAsync_stopsJobGraphStoreAndDispatcher. [FLINK-11843] Delete ZooKeeperHADispatcherTest#testStandbyDispatcherJobRecovery Delete ZooKeeperHADispatcherTest#testStandbyDispatcherJobRecovery because it should no longer add test coverage. The test is superseded by DispatcherLeaderProcessImplTest#start_triggersJobGraphRecoveryAndDispatcherServiceCreation, DispatcherRunnerImplNGTest#grantLeadership_validLeader_confirmsLeaderSession and DispatcherTest#testPersistedJobGraphWhenDispatcherIsShutDown --- .../dispatcher/ZooKeeperHADispatcherTest.java | 356 ------------------ .../DispatcherLeaderProcessImplTest.java | 31 ++ 2 files changed, 31 insertions(+), 356 deletions(-) delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/ZooKeeperHADispatcherTest.java 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 deleted file mode 100644 index 6bc1fff82a6a..000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/ZooKeeperHADispatcherTest.java +++ /dev/null @@ -1,356 +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.api.common.time.Time; -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.heartbeat.HeartbeatServices; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; -import org.apache.flink.runtime.highavailability.zookeeper.ZooKeeperHaServices; -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; -import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway; -import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedExecutionGraphBuilder; -import org.apache.flink.runtime.rpc.RpcUtils; -import org.apache.flink.runtime.rpc.TestingRpcService; -import org.apache.flink.runtime.util.LeaderConnectionInfo; -import org.apache.flink.runtime.util.LeaderRetrievalUtils; -import org.apache.flink.runtime.util.TestingFatalErrorHandler; -import org.apache.flink.runtime.util.ZooKeeperUtils; -import org.apache.flink.runtime.zookeeper.ZooKeeperResource; -import org.apache.flink.util.TestLogger; - -import org.apache.curator.framework.CuratorFramework; -import org.hamcrest.Matchers; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.rules.TestName; - -import javax.annotation.Nonnull; - -import java.io.IOException; -import java.time.Duration; -import java.util.Collection; -import java.util.Collections; -import java.util.UUID; -import java.util.concurrent.CompletableFuture; - -import static org.hamcrest.Matchers.empty; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.notNullValue; -import static org.junit.Assert.assertThat; - -/** - * Test cases for the interaction between ZooKeeper HA and the {@link Dispatcher}. - */ -public class ZooKeeperHADispatcherTest extends TestLogger { - - private static final Time TIMEOUT = Time.seconds(10L); - - @Rule - public final ZooKeeperResource zooKeeperResource = new ZooKeeperResource(); - - @ClassRule - public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - private static Configuration configuration; - - private static TestingRpcService rpcService; - - private static BlobServer blobServer; - - @Rule - public TestName name = new TestName(); - - private TestingFatalErrorHandler testingFatalErrorHandler; - - @BeforeClass - public static void setupClass() throws IOException { - configuration = new Configuration(); - configuration.setString(HighAvailabilityOptions.HA_STORAGE_PATH, TEMPORARY_FOLDER.newFolder().getAbsolutePath()); - rpcService = new TestingRpcService(); - blobServer = new BlobServer(configuration, new VoidBlobStore()); - } - - @Before - public void setup() throws Exception { - configuration.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zooKeeperResource.getConnectString()); - testingFatalErrorHandler = new TestingFatalErrorHandler(); - } - - @After - public void teardown() throws Exception { - if (testingFatalErrorHandler != null) { - testingFatalErrorHandler.rethrowError(); - } - } - - @AfterClass - public static void teardownClass() throws Exception { - if (rpcService != null) { - RpcUtils.terminateRpcService(rpcService, TIMEOUT); - rpcService = null; - } - - if (blobServer != null) { - blobServer.close(); - blobServer = null; - } - } - - /** - * Tests that the {@link Dispatcher} releases a locked {@link JobGraph} if it - * lost the leadership. - */ - @Test - public void testJobGraphRelease() throws Exception { - final CuratorFramework client = ZooKeeperUtils.startCuratorFramework(configuration); - final CuratorFramework otherClient = ZooKeeperUtils.startCuratorFramework(configuration); - - try (final TestingHighAvailabilityServices testingHighAvailabilityServices = new TestingHighAvailabilityServices()) { - testingHighAvailabilityServices.setJobGraphStore(ZooKeeperUtils.createJobGraphs(client, configuration)); - - final ZooKeeperJobGraphStore otherJobGraphStore = ZooKeeperUtils.createJobGraphs( - otherClient, - configuration); - - otherJobGraphStore.start(NoOpJobGraphListener.INSTANCE); - - final TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService(); - testingHighAvailabilityServices.setDispatcherLeaderElectionService(leaderElectionService); - - final TestingDispatcher dispatcher = createDispatcher( - testingHighAvailabilityServices, - new TestingJobManagerRunnerFactory()); - - dispatcher.start(); - - try { - final DispatcherId expectedLeaderId = DispatcherId.generate(); - leaderElectionService.isLeader(expectedLeaderId.toUUID()).get(); - - final DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); - - final JobGraph nonEmptyJobGraph = DispatcherHATest.createNonEmptyJobGraph(); - final CompletableFuture submissionFuture = dispatcherGateway.submitJob(nonEmptyJobGraph, TIMEOUT); - submissionFuture.get(); - - Collection jobIds = otherJobGraphStore.getJobIds(); - - final JobID jobId = nonEmptyJobGraph.getJobID(); - assertThat(jobIds, Matchers.contains(jobId)); - - leaderElectionService.notLeader(); - - // wait for the job to properly terminate - final CompletableFuture jobTerminationFuture = dispatcher.getJobTerminationFuture(jobId, TIMEOUT); - jobTerminationFuture.get(); - - // recover the job - final JobGraph jobGraph = otherJobGraphStore.recoverJobGraph(jobId); - - assertThat(jobGraph, is(notNullValue())); - - // check that the other job graph store can remove the job graph after the original leader - // has lost its leadership - otherJobGraphStore.removeJobGraph(jobId); - - jobIds = otherJobGraphStore.getJobIds(); - - assertThat(jobIds, Matchers.not(Matchers.contains(jobId))); - } finally { - RpcUtils.terminateRpcEndpoint(dispatcher, TIMEOUT); - client.close(); - otherClient.close(); - } - } - } - - /** - * Tests that a standby Dispatcher does not interfere with the clean up of a completed - * job. - */ - @Test - public void testStandbyDispatcherJobExecution() throws Exception { - try (final TestingHighAvailabilityServices haServices1 = new TestingHighAvailabilityServices(); - final TestingHighAvailabilityServices haServices2 = new TestingHighAvailabilityServices(); - final CuratorFramework curatorFramework = ZooKeeperUtils.startCuratorFramework(configuration)) { - - final ZooKeeperJobGraphStore jobGraphStore1 = ZooKeeperUtils.createJobGraphs(curatorFramework, configuration); - haServices1.setJobGraphStore(jobGraphStore1); - final TestingLeaderElectionService leaderElectionService1 = new TestingLeaderElectionService(); - haServices1.setDispatcherLeaderElectionService(leaderElectionService1); - - final ZooKeeperJobGraphStore jobGraphStore2 = ZooKeeperUtils.createJobGraphs(curatorFramework, configuration); - haServices2.setJobGraphStore(jobGraphStore2); - final TestingLeaderElectionService leaderElectionService2 = new TestingLeaderElectionService(); - haServices2.setDispatcherLeaderElectionService(leaderElectionService2); - - final TestingJobManagerRunnerFactory jobManagerRunnerFactory = new TestingJobManagerRunnerFactory(); - final TestingDispatcher dispatcher1 = createDispatcher( - haServices1, - jobManagerRunnerFactory); - - final TestingDispatcher dispatcher2 = createDispatcher( - haServices2, - new TestingJobManagerRunnerFactory()); - - try { - dispatcher1.start(); - dispatcher2.start(); - - leaderElectionService1.isLeader(UUID.randomUUID()).get(); - final DispatcherGateway dispatcherGateway1 = dispatcher1.getSelfGateway(DispatcherGateway.class); - - final JobGraph jobGraph = DispatcherHATest.createNonEmptyJobGraph(); - - dispatcherGateway1.submitJob(jobGraph, TIMEOUT).get(); - - final CompletableFuture jobResultFuture = dispatcherGateway1.requestJobResult(jobGraph.getJobID(), TIMEOUT); - - final TestingJobManagerRunner testingJobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); - - // complete the job - testingJobManagerRunner.completeResultFuture(new ArchivedExecutionGraphBuilder().setJobID(jobGraph.getJobID()).setState(JobStatus.FINISHED).build()); - - final JobResult jobResult = jobResultFuture.get(); - - assertThat(jobResult.isSuccess(), is(true)); - - // wait for the completion of the job - dispatcher1.getJobTerminationFuture(jobGraph.getJobID(), TIMEOUT).get(); - - // change leadership - leaderElectionService1.notLeader(); - leaderElectionService2.isLeader(UUID.randomUUID()).get(); - - // Dispatcher 2 should not recover any jobs - final DispatcherGateway dispatcherGateway2 = dispatcher2.getSelfGateway(DispatcherGateway.class); - assertThat(dispatcherGateway2.listJobs(TIMEOUT).get(), is(empty())); - } finally { - RpcUtils.terminateRpcEndpoint(dispatcher1, TIMEOUT); - RpcUtils.terminateRpcEndpoint(dispatcher2, TIMEOUT); - } - } - } - - /** - * Tests that a standby {@link Dispatcher} can recover all submitted jobs. - */ - @Test - public void testStandbyDispatcherJobRecovery() throws Exception { - try (CuratorFramework curatorFramework = ZooKeeperUtils.startCuratorFramework(configuration)) { - - HighAvailabilityServices haServices = null; - Dispatcher dispatcher1 = null; - Dispatcher dispatcher2 = null; - - try { - haServices = new ZooKeeperHaServices(curatorFramework, rpcService.getExecutor(), configuration, new VoidBlobStore()); - - final TestingJobManagerRunnerFactory jobManagerRunnerFactory = new TestingJobManagerRunnerFactory(); - dispatcher1 = createDispatcher( - haServices, - jobManagerRunnerFactory); - dispatcher2 = createDispatcher( - haServices, - jobManagerRunnerFactory); - - dispatcher1.start(); - dispatcher2.start(); - - 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(); - - 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(); - } else { - dispatcher2.closeAsync(); - } - - final TestingJobManagerRunner testingJobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); - assertThat(testingJobManagerRunner.getJobID(), is(equalTo(nonEmptyJobGraph.getJobID()))); - } finally { - if (dispatcher1 != null) { - RpcUtils.terminateRpcEndpoint(dispatcher1, TIMEOUT); - } - - if (dispatcher2 != null) { - RpcUtils.terminateRpcEndpoint(dispatcher2, TIMEOUT); - } - - if (haServices != null) { - haServices.close(); - } - } - } - } - - @Nonnull - private TestingDispatcher createDispatcher(HighAvailabilityServices highAvailabilityServices, JobManagerRunnerFactory jobManagerRunnerFactory) throws Exception { - TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway(); - final HeartbeatServices heartbeatServices = new HeartbeatServices(1000L, 1000L); - final MemoryArchivedExecutionGraphStore archivedExecutionGraphStore = new MemoryArchivedExecutionGraphStore(); - return new TestingDispatcher( - rpcService, - Dispatcher.DISPATCHER_NAME + '_' + name.getMethodName() + UUID.randomUUID(), - DispatcherId.generate(), - Collections.emptyList(), - new DispatcherServices( - configuration, - highAvailabilityServices, - () -> CompletableFuture.completedFuture(resourceManagerGateway), - blobServer, - heartbeatServices, - UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), - archivedExecutionGraphStore, - testingFatalErrorHandler, - VoidHistoryServerArchivist.INSTANCE, - null, - highAvailabilityServices.getJobGraphStore(), - jobManagerRunnerFactory)); - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java index e84a1f7f5dd2..b037fdbf78f4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java @@ -306,6 +306,37 @@ public void onAddedJobGraph_submitsRecoveredJob() throws Exception { } } + @Test + public void onAddedJobGraph_ifNotRunning_isBeingIgnored() throws Exception { + final CompletableFuture recoveredJobFuture = new CompletableFuture<>(); + jobGraphStore = TestingJobGraphStore.newBuilder() + .setRecoverJobGraphFunction( + (jobId, jobGraphs) -> { + recoveredJobFuture.complete(jobId); + return jobGraphs.get(jobId); + }) + .build(); + + try (final DispatcherLeaderProcessImpl dispatcherLeaderProcess = createDispatcherLeaderProcess()) { + dispatcherLeaderProcess.start(); + + // wait until the process has started the dispatcher + dispatcherLeaderProcess.getDispatcherGateway().get(); + + // now add the job graph + jobGraphStore.putJobGraph(JOB_GRAPH); + + dispatcherLeaderProcess.closeAsync(); + + dispatcherLeaderProcess.onAddedJobGraph(JOB_GRAPH.getJobID()); + + try { + recoveredJobFuture.get(10L, TimeUnit.MILLISECONDS); + fail("onAddedJobGraph should be ignored if the leader process is not running."); + } catch (TimeoutException expected) {} + } + } + @Test public void onAddedJobGraph_failingRecovery_propagatesTheFailure() throws Exception { final FlinkException expectedFailure = new FlinkException("Expected failure"); From a123c8f50a53ddacefb7fd343b04f34c275a242a Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 24 Sep 2019 18:12:44 +0200 Subject: [PATCH 272/746] [FLINK-11843] Port and remove DispatcherHATest Remove DispatcherTest#testFailingRecoveryIsFatalError DispatcherHATest#testFailingRecoveryIsFatalError has been superseded by DispatcherLeaderProcessImplTest#recoverJobs_withRecoveryFailure_failsFatally. [FLINK-11843] Port DispatcherHATest#testRevokeLeadershipTerminatesJobManagerRunners DispatcherHATest#testRevokeLeadershipTerminatesJobManagerRunners has been replaced by DispatcherResourceCleanupTest#testDispatcherTerminationTerminatesRunningJobMasters. [FLINK-11843] Remove DispatcherHATest#testJobRecoveryWhenChangingLeadership DispatcherHATest#testJobRecoveryWhenChangingLeadership has been replaced with DispatcherRunnerImplNGITCase#leaderChange_afterJobSubmission_recoversSubmittedJob [FLINK-11843] Remove DispatcherHATest#testGrantingRevokingLeadership DispatcherHATest#testGrantingRevokingLeadership has been replaced with DispatcherLeaderProcessImplTest#closeAsync_duringJobRecovery_preventsDispatcherServiceCreation. [FLINK-11843] Delete DispatcherHATest --- .../runtime/dispatcher/DispatcherHATest.java | 498 ------------------ .../DispatcherResourceCleanupTest.java | 10 + .../jobmaster/TestingJobManagerRunner.java | 4 + 3 files changed, 14 insertions(+), 498 deletions(-) delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherHATest.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 deleted file mode 100644 index 326f1ed5b89f..000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherHATest.java +++ /dev/null @@ -1,498 +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.api.common.time.Time; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.testutils.OneShotLatch; -import org.apache.flink.runtime.blob.BlobServer; -import org.apache.flink.runtime.blob.VoidBlobStore; -import org.apache.flink.runtime.heartbeat.HeartbeatServices; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; -import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServicesBuilder; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobgraph.JobVertex; -import org.apache.flink.runtime.jobmanager.JobGraphStore; -import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; -import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; -import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; -import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; -import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway; -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.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.runtime.webmonitor.retriever.GatewayRetriever; -import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.FlinkException; -import org.apache.flink.util.Preconditions; -import org.apache.flink.util.TestLogger; - -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; - -import javax.annotation.Nonnull; -import javax.annotation.Nullable; - -import java.util.ArrayDeque; -import java.util.Collection; -import java.util.Collections; -import java.util.Queue; -import java.util.UUID; -import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.function.Supplier; - -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.is; -import static org.junit.Assert.assertThat; - -/** - * Tests the HA behaviour of the {@link Dispatcher}. - */ -public class DispatcherHATest extends TestLogger { - private static final DispatcherId NULL_FENCING_TOKEN = DispatcherId.fromUuid(new UUID(0L, 0L)); - - private static final Time timeout = Time.seconds(10L); - - private static TestingRpcService rpcService; - - private TestingFatalErrorHandler testingFatalErrorHandler; - - @BeforeClass - public static void setupClass() { - rpcService = new TestingRpcService(); - } - - @Before - public void setup() { - testingFatalErrorHandler = new TestingFatalErrorHandler(); - } - - @After - public void teardown() throws Exception { - if (testingFatalErrorHandler != null) { - testingFatalErrorHandler.rethrowError(); - } - } - - @AfterClass - public static void teardownClass() throws ExecutionException, InterruptedException { - if (rpcService != null) { - rpcService.stopService().get(); - rpcService = null; - } - } - - /** - * Tests that interleaved granting and revoking of the leadership won't interfere - * with the job recovery and the resulting internal state of the Dispatcher. - */ - @Test - public void testGrantingRevokingLeadership() throws Exception { - final TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices(); - final JobGraph nonEmptyJobGraph = createNonEmptyJobGraph(); - - final OneShotLatch enterGetJobIdsLatch = new OneShotLatch(); - final OneShotLatch proceedGetJobIdsLatch = new OneShotLatch(); - highAvailabilityServices.setJobGraphStore(new BlockingJobGraphStore(nonEmptyJobGraph, enterGetJobIdsLatch, proceedGetJobIdsLatch)); - final TestingLeaderElectionService dispatcherLeaderElectionService = new TestingLeaderElectionService(); - highAvailabilityServices.setDispatcherLeaderElectionService(dispatcherLeaderElectionService); - - final BlockingQueue fencingTokens = new ArrayBlockingQueue<>(2); - - final HATestingDispatcher dispatcher = createDispatcherWithObservableFencingTokens(highAvailabilityServices, fencingTokens); - - dispatcher.start(); - - try { - // wait until the election service has been started - dispatcherLeaderElectionService.getStartFuture().get(); - - final UUID leaderId = UUID.randomUUID(); - dispatcherLeaderElectionService.isLeader(leaderId); - - dispatcherLeaderElectionService.notLeader(); - - final DispatcherId firstFencingToken = fencingTokens.take(); - - assertThat(firstFencingToken, equalTo(NULL_FENCING_TOKEN)); - - enterGetJobIdsLatch.await(); - proceedGetJobIdsLatch.trigger(); - - assertThat(dispatcher.getNumberJobs(timeout).get(), is(0)); - - } finally { - RpcUtils.terminateRpcEndpoint(dispatcher, timeout); - } - } - - /** - * Tests that all JobManagerRunner are terminated if the leadership of the - * Dispatcher is revoked. - */ - @Test - public void testRevokeLeadershipTerminatesJobManagerRunners() throws Exception { - - final TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService(); - final TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServicesBuilder() - .setDispatcherLeaderElectionService(leaderElectionService) - .build(); - - final ArrayBlockingQueue fencingTokens = new ArrayBlockingQueue<>(2); - final HATestingDispatcher dispatcher = createDispatcherWithObservableFencingTokens( - highAvailabilityServices, - fencingTokens); - - dispatcher.start(); - - try { - // grant leadership and submit a single job - final DispatcherId expectedDispatcherId = DispatcherId.generate(); - - leaderElectionService.isLeader(expectedDispatcherId.toUUID()).get(); - - assertThat(fencingTokens.take(), is(equalTo(expectedDispatcherId))); - - final DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); - - final CompletableFuture submissionFuture = dispatcherGateway.submitJob(createNonEmptyJobGraph(), timeout); - - submissionFuture.get(); - - assertThat(dispatcher.getNumberJobs(timeout).get(), is(1)); - - // revoke the leadership --> this should stop all running JobManagerRunners - leaderElectionService.notLeader(); - - assertThat(fencingTokens.take(), is(equalTo(NULL_FENCING_TOKEN))); - - assertThat(dispatcher.getNumberJobs(timeout).get(), is(0)); - } finally { - RpcUtils.terminateRpcEndpoint(dispatcher, timeout); - } - } - - /** - * Tests that a Dispatcher does not remove the JobGraph from the submitted job graph store - * when losing leadership and recovers it when regaining leadership. - */ - @Test - public void testJobRecoveryWhenChangingLeadership() throws Exception { - final CompletableFuture recoveredJobFuture = new CompletableFuture<>(); - final TestingJobGraphStore submittedJobGraphStore = TestingJobGraphStore.newBuilder() - .setRecoverJobGraphFunction( - (jobID, jobIDSubmittedJobGraphMap) -> { - recoveredJobFuture.complete(jobID); - return jobIDSubmittedJobGraphMap.get(jobID); - }) - .build(); - - final TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService(); - - final TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServicesBuilder() - .setJobGraphStore(submittedJobGraphStore) - .setDispatcherLeaderElectionService(leaderElectionService) - .build(); - - final ArrayBlockingQueue fencingTokens = new ArrayBlockingQueue<>(2); - final HATestingDispatcher dispatcher = createDispatcherWithObservableFencingTokens( - highAvailabilityServices, - fencingTokens); - - dispatcher.start(); - - try { - // grant leadership and submit a single job - final DispatcherId expectedDispatcherId = DispatcherId.generate(); - leaderElectionService.isLeader(expectedDispatcherId.toUUID()).get(); - - assertThat(fencingTokens.take(), is(equalTo(expectedDispatcherId))); - - final DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); - - final JobGraph jobGraph = createNonEmptyJobGraph(); - final CompletableFuture submissionFuture = dispatcherGateway.submitJob(jobGraph, timeout); - - submissionFuture.get(); - - final JobID jobId = jobGraph.getJobID(); - assertThat(submittedJobGraphStore.contains(jobId), is(true)); - - // revoke the leadership --> this should stop all running JobManagerRunners - leaderElectionService.notLeader(); - - assertThat(fencingTokens.take(), is(equalTo(NULL_FENCING_TOKEN))); - - assertThat(submittedJobGraphStore.contains(jobId), is(true)); - - assertThat(recoveredJobFuture.isDone(), is(false)); - - // re-grant leadership - leaderElectionService.isLeader(DispatcherId.generate().toUUID()); - - assertThat(recoveredJobFuture.get(), is(equalTo(jobId))); - } finally { - RpcUtils.terminateRpcEndpoint(dispatcher, timeout); - } - } - - /** - * Tests that a fatal error is reported if the job recovery fails. - */ - @Test - public void testFailingRecoveryIsAFatalError() throws Exception { - final String exceptionMessage = "Job recovery test failure."; - final Supplier exceptionSupplier = () -> new FlinkException(exceptionMessage); - final TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServicesBuilder() - .setJobGraphStore(new FailingJobGraphStore(exceptionSupplier)) - .build(); - - final HATestingDispatcher dispatcher = createDispatcher(haServices); - dispatcher.start(); - - final Throwable failure = testingFatalErrorHandler.getErrorFuture().get(); - - assertThat(ExceptionUtils.findThrowableWithMessage(failure, exceptionMessage).isPresent(), is(true)); - - testingFatalErrorHandler.clearError(); - } - - @Nonnull - private HATestingDispatcher createDispatcherWithObservableFencingTokens(HighAvailabilityServices highAvailabilityServices, Queue fencingTokens) throws Exception { - return createDispatcher(highAvailabilityServices, fencingTokens, createTestingJobManagerRunnerFactory()); - } - - @Nonnull - private TestingJobManagerRunnerFactory createTestingJobManagerRunnerFactory() { - return new TestingJobManagerRunnerFactory(); - } - - private HATestingDispatcher createDispatcher(HighAvailabilityServices haServices) throws Exception { - return createDispatcher( - haServices, - new ArrayDeque<>(1), - createTestingJobManagerRunnerFactory()); - } - - @Nonnull - private HATestingDispatcher createDispatcher( - HighAvailabilityServices highAvailabilityServices, - @Nonnull Queue fencingTokens, - JobManagerRunnerFactory jobManagerRunnerFactory) throws Exception { - final Configuration configuration = new Configuration(); - - TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway(); - return new HATestingDispatcher( - rpcService, - UUID.randomUUID().toString(), - DispatcherId.generate(), - Collections.emptyList(), - configuration, - highAvailabilityServices, - () -> CompletableFuture.completedFuture(resourceManagerGateway), - new BlobServer(configuration, new VoidBlobStore()), - new HeartbeatServices(1000L, 1000L), - UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), - null, - new MemoryArchivedExecutionGraphStore(), - jobManagerRunnerFactory, - testingFatalErrorHandler, - fencingTokens); - } - - @Nonnull - static JobGraph createNonEmptyJobGraph() { - final JobVertex noOpVertex = new JobVertex("NoOp vertex"); - noOpVertex.setInvokableClass(NoOpInvokable.class); - final JobGraph jobGraph = new JobGraph(noOpVertex); - jobGraph.setAllowQueuedScheduling(true); - - return jobGraph; - } - - private static class HATestingDispatcher extends TestingDispatcher { - - @Nonnull - private final Queue fencingTokens; - - HATestingDispatcher( - RpcService rpcService, - String endpointId, - DispatcherId fencingToken, - Collection recoveredJobs, - Configuration configuration, - HighAvailabilityServices highAvailabilityServices, - GatewayRetriever resourceManagerGatewayRetriever, - BlobServer blobServer, - HeartbeatServices heartbeatServices, - JobManagerMetricGroup jobManagerMetricGroup, - @Nullable String metricQueryServiceAddress, - ArchivedExecutionGraphStore archivedExecutionGraphStore, - JobManagerRunnerFactory jobManagerRunnerFactory, - FatalErrorHandler fatalErrorHandler, - @Nonnull Queue fencingTokens) throws Exception { - super( - rpcService, - endpointId, - fencingToken, - recoveredJobs, - new DispatcherServices( - configuration, - highAvailabilityServices, - resourceManagerGatewayRetriever, - blobServer, - heartbeatServices, - jobManagerMetricGroup, - archivedExecutionGraphStore, - fatalErrorHandler, - VoidHistoryServerArchivist.INSTANCE, - metricQueryServiceAddress, - highAvailabilityServices.getJobGraphStore(), - jobManagerRunnerFactory)); - this.fencingTokens = fencingTokens; - } - - @Override - protected void setFencingToken(@Nullable DispatcherId newFencingToken) { - super.setFencingToken(newFencingToken); - - final DispatcherId fencingToken; - if (newFencingToken == null) { - fencingToken = NULL_FENCING_TOKEN; - } else { - fencingToken = newFencingToken; - } - - fencingTokens.offer(fencingToken); - } - } - - private static class BlockingJobGraphStore implements JobGraphStore { - - @Nonnull - private final JobGraph jobGraph; - - @Nonnull - private final OneShotLatch enterGetJobIdsLatch; - - @Nonnull - private final OneShotLatch proceedGetJobIdsLatch; - - private BlockingJobGraphStore(@Nonnull JobGraph jobGraph, @Nonnull OneShotLatch enterGetJobIdsLatch, @Nonnull OneShotLatch proceedGetJobIdsLatch) { - this.jobGraph = jobGraph; - this.enterGetJobIdsLatch = enterGetJobIdsLatch; - this.proceedGetJobIdsLatch = proceedGetJobIdsLatch; - } - - @Override - public void start(JobGraphListener jobGraphListener) { - } - - @Override - public void stop() { - } - - @Nullable - @Override - public JobGraph recoverJobGraph(JobID jobId) { - Preconditions.checkArgument(jobId.equals(jobGraph.getJobID())); - - return jobGraph; - } - - @Override - public void putJobGraph(JobGraph jobGraph) { - throw new UnsupportedOperationException("Should not be called."); - } - - @Override - public void removeJobGraph(JobID jobId) { - throw new UnsupportedOperationException("Should not be called."); - } - - @Override - public void releaseJobGraph(JobID jobId) {} - - @Override - public Collection getJobIds() throws Exception { - enterGetJobIdsLatch.trigger(); - proceedGetJobIdsLatch.await(); - return Collections.singleton(jobGraph.getJobID()); - } - } - - private static class FailingJobGraphStore implements JobGraphStore { - private final JobID jobId = new JobID(); - - private final Supplier exceptionSupplier; - - private FailingJobGraphStore(Supplier exceptionSupplier) { - this.exceptionSupplier = exceptionSupplier; - } - - @Override - public void start(JobGraphListener jobGraphListener) { - - } - - @Override - public void stop() { - - } - - @Nullable - @Override - public JobGraph recoverJobGraph(JobID jobId) throws Exception { - throw exceptionSupplier.get(); - } - - @Override - public void putJobGraph(JobGraph jobGraph) { - - } - - @Override - public void removeJobGraph(JobID jobId) { - - } - - @Override - public void releaseJobGraph(JobID jobId) { - - } - - @Override - public Collection getJobIds() { - return Collections.singleton(jobId); - } - } -} 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 2d514fa4c941..36110be2fb7e 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 @@ -404,6 +404,16 @@ private void assertThatHABlobsHaveNotBeenRemoved() { assertThat(blobFile.exists(), is(true)); } + @Test + public void testDispatcherTerminationTerminatesRunningJobMasters() throws Exception { + final TestingJobManagerRunnerFactory jobManagerRunnerFactory = startDispatcherAndSubmitJob(); + + dispatcher.closeAsync().get(); + + final TestingJobManagerRunner jobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); + assertThat(jobManagerRunner.getTerminationFuture().isDone(), is(true)); + } + /** * Tests that terminating the Dispatcher will wait for all JobMasters to be terminated. */ 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 b0a0b9c540fd..55cfde4b0b84 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 @@ -90,4 +90,8 @@ public void completeResultFutureExceptionally(Exception e) { public void completeTerminationFuture() { terminationFuture.complete(null); } + + public CompletableFuture getTerminationFuture() { + return terminationFuture; + } } From 4e9dd7f78896079498daa06e7fe0133900feb74f Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 25 Sep 2019 11:49:16 +0200 Subject: [PATCH 273/746] [FLINK-11843] Ignore duplicate job submission due to false positive onAddedJobGraph callbacks --- .../runner/DispatcherLeaderProcessImpl.java | 19 +++++- .../DispatcherLeaderProcessImplTest.java | 64 +++++++++++++++++-- 2 files changed, 77 insertions(+), 6 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java index 02b708f0113e..9a3c93580e3f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.dispatcher.runner; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.client.DuplicateJobSubmissionException; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.dispatcher.Dispatcher; import org.apache.flink.runtime.dispatcher.DispatcherGateway; @@ -42,6 +43,7 @@ import java.util.Optional; import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.concurrent.Executor; /** @@ -220,10 +222,23 @@ private Optional> submitAddedJobIfRunning(JobGraph jobGr private CompletableFuture submitAddedJob(JobGraph jobGraph) { final DispatcherGateway dispatcherGateway = getDispatcherGatewayInternal(); - // TODO: Filter out duplicate job submissions which can happen with the JobGraphListener return dispatcherGateway .submitJob(jobGraph, RpcUtils.INF_TIMEOUT) - .thenApply(FunctionUtils.nullFn()); + .thenApply(FunctionUtils.nullFn()) + .exceptionally(this::filterOutDuplicateJobSubmissionException); + } + + private Void filterOutDuplicateJobSubmissionException(Throwable throwable) { + final Throwable strippedException = ExceptionUtils.stripCompletionException(throwable); + if (strippedException instanceof DuplicateJobSubmissionException) { + final DuplicateJobSubmissionException duplicateJobSubmissionException = (DuplicateJobSubmissionException) strippedException; + + log.debug("Ignore recovered job {} because the job is currently being executed.", duplicateJobSubmissionException.getJobID(), duplicateJobSubmissionException); + + return null; + } else { + throw new CompletionException(throwable); + } } private DispatcherGateway getDispatcherGatewayInternal() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java index b037fdbf78f4..86994ce50a69 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java @@ -20,6 +20,9 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.client.DuplicateJobSubmissionException; +import org.apache.flink.runtime.client.JobSubmissionException; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.runtime.messages.Acknowledge; @@ -30,6 +33,7 @@ import org.apache.flink.util.ExecutorUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; +import org.apache.flink.util.function.ThrowingConsumer; import org.apache.flink.util.function.TriFunctionWithException; import org.hamcrest.core.Is; @@ -38,7 +42,6 @@ import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; -import org.junit.Ignore; import org.junit.Test; import java.util.Collection; @@ -55,6 +58,7 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.hasSize; +import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; /** @@ -406,9 +410,61 @@ private void runJobRecoveryFailureTest(FlinkException testException) throws Exce } @Test - @Ignore - public void onAddedJobGraph_falsePositive_willBeIgnored() { - fail("Needs to be implemented once the proper deduplication mechanism is in place."); + public void onAddedJobGraph_failingRecoveredJobSubmission_failsFatally() throws Exception { + final TestingDispatcherGateway dispatcherGateway = new TestingDispatcherGateway.Builder() + .setSubmitFunction(jobGraph -> FutureUtils.completedExceptionally(new JobSubmissionException(jobGraph.getJobID(), "test exception"))) + .build(); + + runOnAddedJobGraphTest(dispatcherGateway, this::verifyOnAddedJobGraphResultFailsFatally); + } + + private void verifyOnAddedJobGraphResultFailsFatally(TestingFatalErrorHandler fatalErrorHandler) { + final Throwable actualCause = fatalErrorHandler.getErrorFuture().join(); + + assertTrue(ExceptionUtils.findThrowable(actualCause, JobSubmissionException.class).isPresent()); + + fatalErrorHandler.clearError(); + } + + @Test + public void onAddedJobGraph_duplicateJobSubmissionDueToFalsePositive_willBeIgnored() throws Exception { + final TestingDispatcherGateway dispatcherGateway = new TestingDispatcherGateway.Builder() + .setSubmitFunction(jobGraph -> FutureUtils.completedExceptionally(new DuplicateJobSubmissionException(jobGraph.getJobID()))) + .build(); + + runOnAddedJobGraphTest(dispatcherGateway, this::verifyOnAddedJobGraphResultDidNotFail); + } + + private void runOnAddedJobGraphTest(TestingDispatcherGateway dispatcherGateway, ThrowingConsumer verificationLogic) throws Exception { + jobGraphStore = TestingJobGraphStore.newBuilder() + .setInitialJobGraphs(Collections.singleton(JOB_GRAPH)) + .build(); + dispatcherServiceFactory = TestingDispatcherServiceFactory.newBuilder() + .setCreateFunction((dispatcherId, jobGraphs, jobGraphWriter) -> { + assertThat(jobGraphs, containsInAnyOrder(JOB_GRAPH)); + + return TestingDispatcherService.newBuilder() + .setDispatcherGateway(dispatcherGateway) + .build(); + }) + .build(); + + try (final DispatcherLeaderProcessImpl dispatcherLeaderProcess = createDispatcherLeaderProcess()) { + dispatcherLeaderProcess.start(); + + dispatcherLeaderProcess.getDispatcherGateway().get(); + + dispatcherLeaderProcess.onAddedJobGraph(JOB_GRAPH.getJobID()); + + verificationLogic.accept(fatalErrorHandler); + } + } + + private void verifyOnAddedJobGraphResultDidNotFail(TestingFatalErrorHandler fatalErrorHandler) throws Exception { + try { + fatalErrorHandler.getErrorFuture().get(10L, TimeUnit.MILLISECONDS); + fail("Expected that duplicate job submissions due to false job recoveries are ignored."); + } catch (TimeoutException expected) {} } private TestingDispatcherServiceFactory createDispatcherServiceFactoryFor(TestingDispatcherGateway testingDispatcherGateway) { From c8675858f30d296cc42cfbe37bdae6af452cbce9 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 25 Sep 2019 12:53:08 +0200 Subject: [PATCH 274/746] [FLINK-11843] Properly remove job graphs which have been removed from the JobGraphStore Instead of cancelling job graphs, which have been removed from the JobGraphStore, this commit simply stops and removes jobs. This has the advantage that the job does not go into a globally terminal state. --- .../flink/runtime/dispatcher/Dispatcher.java | 6 +++ .../runner/DispatcherLeaderProcessImpl.java | 13 ++--- .../runner/DispatcherServiceImpl.java | 6 +++ .../runtime/dispatcher/DispatcherTest.java | 23 +++++++++ .../DispatcherLeaderProcessImplTest.java | 50 +++++++++++++++---- .../runner/TestingDispatcherService.java | 28 +++++++++-- 6 files changed, 106 insertions(+), 20 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 9cf16a666888..ffd9ab09ab9a 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 @@ -850,4 +850,10 @@ private void registerDispatcherMetrics(MetricGroup jobManagerMetricGroup) { jobManagerMetricGroup.gauge(MetricNames.NUM_RUNNING_JOBS, () -> (long) jobManagerRunnerFutures.size()); } + + public CompletableFuture onRemovedJobGraph(JobID jobId) { + return CompletableFuture.runAsync( + () -> removeJobAndRegisterTerminationFuture(jobId, false), + getMainThreadExecutor()); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java index 9a3c93580e3f..a3965d289873 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java @@ -272,12 +272,11 @@ private Optional> removeJobGraphIfRunning(JobID jobId) { } private CompletableFuture removeJobGraph(JobID jobId) { - final DispatcherGateway dispatcherGateway = getDispatcherGatewayInternal(); - - // TODO: replace cancel with other fail method - return dispatcherGateway - .cancelJob(jobId, RpcUtils.INF_TIMEOUT) - .thenApply(FunctionUtils.nullFn()); + if (dispatcher == null) { + return FutureUtils.completedVoidFuture(); + } else { + return dispatcher.onRemovedJobGraph(jobId); + } } // --------------------------------------------------------------- @@ -311,5 +310,7 @@ DispatcherService create( interface DispatcherService extends AutoCloseableAsync { DispatcherGateway getGateway(); + + CompletableFuture onRemovedJobGraph(JobID jobId); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImpl.java index facda4971a0d..324f711e2483 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImpl.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.dispatcher.runner; +import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.dispatcher.Dispatcher; import org.apache.flink.runtime.dispatcher.DispatcherGateway; @@ -38,6 +39,11 @@ public DispatcherGateway getGateway() { return dispatcherGateway; } + @Override + public CompletableFuture onRemovedJobGraph(JobID jobId) { + return dispatcher.onRemovedJobGraph(jobId); + } + @Override public CompletableFuture closeAsync() { return dispatcher.closeAsync(); 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 f51a7310f000..0bf6722a096b 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 @@ -594,6 +594,29 @@ public void testShutDownClusterShouldCompleteShutDownFuture() throws Exception { dispatcher.getShutDownFuture().get(); } + @Test + public void testOnRemovedJobGraphDoesNotCleanUpHAFiles() throws Exception { + final CompletableFuture removeJobGraphFuture = new CompletableFuture<>(); + final TestingJobGraphStore testingJobGraphStore = TestingJobGraphStore.newBuilder() + .setRemoveJobGraphConsumer(removeJobGraphFuture::complete) + .build(); + + dispatcher = new TestingDispatcherBuilder() + .setInitialJobGraphs(Collections.singleton(jobGraph)) + .setJobGraphWriter(testingJobGraphStore) + .build(); + dispatcher.start(); + + final CompletableFuture processFuture = dispatcher.onRemovedJobGraph(jobGraph.getJobID()); + + processFuture.join(); + + try { + removeJobGraphFuture.get(10L, TimeUnit.MILLISECONDS); + fail("onRemovedJobGraph should not remove the job from the JobGraphStore."); + } catch (TimeoutException expected) {} + } + private static final class BlockingJobManagerRunnerFactory extends TestingJobManagerRunnerFactory { @Nonnull diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java index 86994ce50a69..a7a6c1cbe52b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java @@ -252,21 +252,22 @@ public void closeAsync_duringJobRecovery_preventsDispatcherServiceCreation() thr } @Test - public void onRemovedJobGraph_cancelsRunningJob() throws Exception { + public void onRemovedJobGraph_terminatesRunningJob() throws Exception { jobGraphStore = TestingJobGraphStore.newBuilder() .setInitialJobGraphs(Collections.singleton(JOB_GRAPH)) .build(); - final CompletableFuture cancelJobFuture = new CompletableFuture<>(); - final TestingDispatcherGateway testingDispatcherGateway = new TestingDispatcherGateway.Builder() - .setCancelJobFunction( - jobToCancel -> { - cancelJobFuture.complete(jobToCancel); - return CompletableFuture.completedFuture(Acknowledge.get()); - }) + final CompletableFuture terminateJobFuture = new CompletableFuture<>(); + final TestingDispatcherService testingDispatcherService = TestingDispatcherService.newBuilder() + .setOnRemovedJobGraphFunction(jobID -> { + terminateJobFuture.complete(jobID); + return FutureUtils.completedVoidFuture(); + }) .build(); - dispatcherServiceFactory = createDispatcherServiceFactoryFor(testingDispatcherGateway); + dispatcherServiceFactory = TestingDispatcherServiceFactory.newBuilder() + .setCreateFunction((dispatcherId, jobGraphs, jobGraphWriter) -> testingDispatcherService) + .build(); try (final DispatcherLeaderProcessImpl dispatcherLeaderProcess = createDispatcherLeaderProcess()) { dispatcherLeaderProcess.start(); @@ -278,7 +279,36 @@ public void onRemovedJobGraph_cancelsRunningJob() throws Exception { jobGraphStore.removeJobGraph(JOB_GRAPH.getJobID()); dispatcherLeaderProcess.onRemovedJobGraph(JOB_GRAPH.getJobID()); - assertThat(cancelJobFuture.get(), is(JOB_GRAPH.getJobID())); + assertThat(terminateJobFuture.get(), is(JOB_GRAPH.getJobID())); + } + } + + @Test + public void onRemovedJobGraph_failingRemovalCall_failsFatally() throws Exception { + final FlinkException testException = new FlinkException("Test exception"); + + final TestingDispatcherService testingDispatcherService = TestingDispatcherService.newBuilder() + .setOnRemovedJobGraphFunction(jobID -> FutureUtils.completedExceptionally(testException)) + .build(); + + dispatcherServiceFactory = TestingDispatcherServiceFactory.newBuilder() + .setCreateFunction((dispatcherId, jobGraphs, jobGraphWriter) -> testingDispatcherService) + .build(); + + try (final DispatcherLeaderProcessImpl dispatcherLeaderProcess = createDispatcherLeaderProcess()) { + dispatcherLeaderProcess.start(); + + // wait for the dispatcher process to be created + dispatcherLeaderProcess.getDispatcherGateway().get(); + + // now notify the dispatcher service + dispatcherLeaderProcess.onRemovedJobGraph(JOB_GRAPH.getJobID()); + + final Throwable fatalError = fatalErrorHandler.getErrorFuture().join(); + + assertTrue(ExceptionUtils.findThrowable(fatalError, cause -> cause.equals(testException)).isPresent()); + + fatalErrorHandler.clearError(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherService.java index 9e9ba7d24737..6981a1067a70 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherService.java @@ -18,11 +18,13 @@ package org.apache.flink.runtime.dispatcher.runner; +import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.webmonitor.TestingDispatcherGateway; import java.util.concurrent.CompletableFuture; +import java.util.function.Function; import java.util.function.Supplier; class TestingDispatcherService implements DispatcherLeaderProcessImpl.DispatcherService { @@ -31,13 +33,19 @@ class TestingDispatcherService implements DispatcherLeaderProcessImpl.Dispatcher private final Supplier> terminationFutureSupplier; + private final Function> onRemovedJobGraphFunction; + private final DispatcherGateway dispatcherGateway; private CompletableFuture terminationFuture; - private TestingDispatcherService(Supplier> terminationFutureSupplier, DispatcherGateway dispatcherGateway) { - this.terminationFutureSupplier = terminationFutureSupplier; - this.dispatcherGateway = dispatcherGateway; + private TestingDispatcherService( + Supplier> terminationFutureSupplier, + Function> onRemovedJobGraphFunction, + DispatcherGateway dispatcherGateway) { + this.terminationFutureSupplier = terminationFutureSupplier; + this.onRemovedJobGraphFunction = onRemovedJobGraphFunction; + this.dispatcherGateway = dispatcherGateway; } @Override @@ -45,6 +53,11 @@ public DispatcherGateway getGateway() { return dispatcherGateway; } + @Override + public CompletableFuture onRemovedJobGraph(JobID jobId) { + return onRemovedJobGraphFunction.apply(jobId); + } + @Override public CompletableFuture closeAsync() { synchronized (lock) { @@ -64,6 +77,8 @@ public static class Builder { private Supplier> terminationFutureSupplier = FutureUtils::completedVoidFuture; + private Function> onRemovedJobGraphFunction = ignored -> FutureUtils.completedVoidFuture(); + private DispatcherGateway dispatcherGateway = new TestingDispatcherGateway.Builder().build(); private Builder() {} @@ -78,8 +93,13 @@ public Builder setDispatcherGateway(DispatcherGateway dispatcherGateway) { return this; } + public Builder setOnRemovedJobGraphFunction(Function> onRemovedJobGraphFunction) { + this.onRemovedJobGraphFunction = onRemovedJobGraphFunction; + return this; + } + public TestingDispatcherService build() { - return new TestingDispatcherService(terminationFutureSupplier, dispatcherGateway); + return new TestingDispatcherService(terminationFutureSupplier, onRemovedJobGraphFunction, dispatcherGateway); } } } From a805f1b299a452d924d0c91d9956f038f072c697 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 25 Sep 2019 15:49:15 +0200 Subject: [PATCH 275/746] [FLINK-11843] Forward shut down future from Dispatcher through to the DispatcherRunnerImplNG --- .../AbstractDispatcherLeaderProcess.java | 45 +++++++++- .../runner/DispatcherLeaderProcess.java | 3 + .../runner/DispatcherLeaderProcessImpl.java | 24 +---- .../DispatcherLeaderProcessImplFactory.java | 4 +- .../runner/DispatcherRunnerImplNG.java | 53 ++++++++--- .../runner/DispatcherServiceImpl.java | 8 +- .../runner/DispatcherServiceImplFactory.java | 4 +- .../runner/JobDispatcherLeaderProcess.java | 8 +- .../JobDispatcherLeaderProcessFactory.java | 4 +- ...DispatcherLeaderProcessFactoryFactory.java | 2 +- .../StoppedDispatcherLeaderProcess.java | 11 ++- .../runner/DispatcherRunnerImplNGTest.java | 90 ++++++++++++++++++- .../TestingDispatcherLeaderProcess.java | 20 ++++- .../runner/TestingDispatcherService.java | 33 +++++-- .../TestingDispatcherServiceFactory.java | 12 +-- 15 files changed, 253 insertions(+), 68 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java index 8bfa964f8e3f..1e754cd2f118 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java @@ -19,14 +19,21 @@ package org.apache.flink.runtime.dispatcher.runner; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.dispatcher.DispatcherGateway; +import org.apache.flink.runtime.dispatcher.DispatcherId; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmanager.JobGraphWriter; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.webmonitor.RestfulGateway; +import org.apache.flink.util.AutoCloseableAsync; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Collection; import java.util.Optional; import java.util.UUID; import java.util.concurrent.CompletableFuture; @@ -49,6 +56,8 @@ abstract class AbstractDispatcherLeaderProcess implements DispatcherLeaderProces private final CompletableFuture terminationFuture; + private final CompletableFuture shutDownFuture; + private State state; AbstractDispatcherLeaderProcess(UUID leaderSessionId, FatalErrorHandler fatalErrorHandler) { @@ -58,6 +67,7 @@ abstract class AbstractDispatcherLeaderProcess implements DispatcherLeaderProces this.dispatcherGatewayFuture = new CompletableFuture<>(); this.confirmLeaderSessionFuture = dispatcherGatewayFuture.thenApply(RestfulGateway::getAddress); this.terminationFuture = new CompletableFuture<>(); + this.shutDownFuture = new CompletableFuture<>(); this.state = State.CREATED; } @@ -97,6 +107,11 @@ public final CompletableFuture getConfirmLeaderSessionFuture() { return confirmLeaderSessionFuture; } + @Override + public CompletableFuture getShutDownFuture() { + return shutDownFuture; + } + @Override public final CompletableFuture closeAsync() { runIfStateIsNot( @@ -120,8 +135,15 @@ protected CompletableFuture onClose() { return FutureUtils.completedVoidFuture(); } - protected void completeDispatcherGatewayFuture(DispatcherGateway dispatcherGateway) { - dispatcherGatewayFuture.complete(dispatcherGateway); + protected final void completeDispatcherSetup(DispatcherService dispatcherService) { + runIfStateIs( + State.RUNNING, + () -> completeDispatcherSetupInternal(dispatcherService)); + } + + private void completeDispatcherSetupInternal(DispatcherService dispatcherService) { + dispatcherGatewayFuture.complete(dispatcherService.getGateway()); + FutureUtils.forward(dispatcherService.getShutDownFuture(), shutDownFuture); } final Optional supplyUnsynchronizedIfRunning(Supplier supplier) { @@ -182,4 +204,23 @@ protected enum State { RUNNING, STOPPED } + + // ------------------------------------------------------------ + // Internal classes + // ------------------------------------------------------------ + + interface DispatcherServiceFactory { + DispatcherService create( + DispatcherId fencingToken, + Collection recoveredJobs, + JobGraphWriter jobGraphWriter); + } + + interface DispatcherService extends AutoCloseableAsync { + DispatcherGateway getGateway(); + + CompletableFuture onRemovedJobGraph(JobID jobId); + + CompletableFuture getShutDownFuture(); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcess.java index 6d5c29bddb43..db7b4a5f0f79 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcess.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.dispatcher.runner; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.dispatcher.Dispatcher; import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.util.AutoCloseableAsync; @@ -37,4 +38,6 @@ interface DispatcherLeaderProcess extends AutoCloseableAsync { CompletableFuture getDispatcherGateway(); CompletableFuture getConfirmLeaderSessionFuture(); + + CompletableFuture getShutDownFuture(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java index a3965d289873..0795a11d0f71 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java @@ -26,10 +26,8 @@ import org.apache.flink.runtime.dispatcher.DispatcherId; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.JobGraphStore; -import org.apache.flink.runtime.jobmanager.JobGraphWriter; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcUtils; -import org.apache.flink.util.AutoCloseableAsync; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.Preconditions; @@ -105,11 +103,14 @@ private void createDispatcherIfRunning(Collection jobGraphs) { } private void createDispatcher(Collection jobGraphs) { + Preconditions.checkState(dispatcher == null); + dispatcher = dispatcherFactory.create( DispatcherId.fromUuid(getLeaderSessionId()), jobGraphs, jobGraphStore); - completeDispatcherGatewayFuture(dispatcher.getGateway()); + + completeDispatcherSetup(dispatcher); } private CompletableFuture> recoverJobsAsync() { @@ -296,21 +297,4 @@ public static DispatcherLeaderProcessImpl create( ioExecutor, fatalErrorHandler); } - - // ------------------------------------------------------------ - // Internal classes - // ------------------------------------------------------------ - - interface DispatcherServiceFactory { - DispatcherService create( - DispatcherId fencingToken, - Collection recoveredJobs, - JobGraphWriter jobGraphWriter); - } - - interface DispatcherService extends AutoCloseableAsync { - DispatcherGateway getGateway(); - - CompletableFuture onRemovedJobGraph(JobID jobId); - } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplFactory.java index 085f290d37f3..80c5b7822245 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplFactory.java @@ -29,13 +29,13 @@ */ class DispatcherLeaderProcessImplFactory implements DispatcherLeaderProcessFactory { - private final DispatcherLeaderProcessImpl.DispatcherServiceFactory dispatcherServiceFactory; + private final AbstractDispatcherLeaderProcess.DispatcherServiceFactory dispatcherServiceFactory; private final JobGraphStoreFactory jobGraphStoreFactory; private final Executor ioExecutor; private final FatalErrorHandler fatalErrorHandler; DispatcherLeaderProcessImplFactory( - DispatcherLeaderProcessImpl.DispatcherServiceFactory dispatcherServiceFactory, + AbstractDispatcherLeaderProcess.DispatcherServiceFactory dispatcherServiceFactory, JobGraphStoreFactory jobGraphStoreFactory, Executor ioExecutor, FatalErrorHandler fatalErrorHandler) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNG.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNG.java index 16f79657d49d..c0555ac15384 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNG.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNG.java @@ -156,18 +156,6 @@ private void startNewDispatcherLeaderProcess(UUID leaderSessionID) { private void createAndAssignNewDispatcherLeaderProcess(UUID leaderSessionID) { dispatcherLeaderProcess = createNewDispatcherLeaderProcess(leaderSessionID); - - final CompletableFuture newDispatcherGatewayFuture = dispatcherLeaderProcess.getDispatcherGateway(); - FutureUtils.forward(newDispatcherGatewayFuture, dispatcherGatewayFuture); - dispatcherGatewayFuture = newDispatcherGatewayFuture; - - FutureUtils.assertNoException( - dispatcherLeaderProcess.getConfirmLeaderSessionFuture().thenAccept( - leaderAddress -> { - if (leaderElectionService.hasLeadership(leaderSessionID)) { - leaderElectionService.confirmLeadership(leaderSessionID, leaderAddress); - } - })); } private void stopDispatcherLeaderProcess() { @@ -180,7 +168,46 @@ private void stopDispatcherLeaderProcess() { private DispatcherLeaderProcess createNewDispatcherLeaderProcess(UUID leaderSessionID) { LOG.debug("Create new {} with leader session id {}.", DispatcherLeaderProcess.class.getSimpleName(), leaderSessionID); - return dispatcherLeaderProcessFactory.create(leaderSessionID); + + final DispatcherLeaderProcess newDispatcherLeaderProcess = dispatcherLeaderProcessFactory.create(leaderSessionID); + + forwardDispatcherGatewayFuture(newDispatcherLeaderProcess); + forwardShutDownFuture(newDispatcherLeaderProcess); + forwardConfirmLeaderSessionFuture(leaderSessionID, newDispatcherLeaderProcess); + + return newDispatcherLeaderProcess; + } + + private void forwardDispatcherGatewayFuture(DispatcherLeaderProcess newDispatcherLeaderProcess) { + final CompletableFuture newDispatcherGatewayFuture = newDispatcherLeaderProcess.getDispatcherGateway(); + FutureUtils.forward(newDispatcherGatewayFuture, dispatcherGatewayFuture); + dispatcherGatewayFuture = newDispatcherGatewayFuture; + } + + private void forwardShutDownFuture(DispatcherLeaderProcess newDispatcherLeaderProcess) { + newDispatcherLeaderProcess.getShutDownFuture().whenComplete( + (applicationStatus, throwable) -> { + synchronized (lock) { + // ignore if no longer running or if leader processes is no longer valid + if (isRunning && this.dispatcherLeaderProcess == newDispatcherLeaderProcess) { + if (throwable != null) { + shutDownFuture.completeExceptionally(throwable); + } else { + shutDownFuture.complete(applicationStatus); + } + } + } + }); + } + + private void forwardConfirmLeaderSessionFuture(UUID leaderSessionID, DispatcherLeaderProcess newDispatcherLeaderProcess) { + FutureUtils.assertNoException( + newDispatcherLeaderProcess.getConfirmLeaderSessionFuture().thenAccept( + leaderAddress -> { + if (leaderElectionService.hasLeadership(leaderSessionID)) { + leaderElectionService.confirmLeadership(leaderSessionID, leaderAddress); + } + })); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImpl.java index 324f711e2483..8a9e0e129b0c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImpl.java @@ -19,12 +19,13 @@ package org.apache.flink.runtime.dispatcher.runner; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.dispatcher.Dispatcher; import org.apache.flink.runtime.dispatcher.DispatcherGateway; import java.util.concurrent.CompletableFuture; -class DispatcherServiceImpl implements DispatcherLeaderProcessImpl.DispatcherService { +class DispatcherServiceImpl implements AbstractDispatcherLeaderProcess.DispatcherService { private final Dispatcher dispatcher; private final DispatcherGateway dispatcherGateway; @@ -44,6 +45,11 @@ public CompletableFuture onRemovedJobGraph(JobID jobId) { return dispatcher.onRemovedJobGraph(jobId); } + @Override + public CompletableFuture getShutDownFuture() { + return dispatcher.getShutDownFuture(); + } + @Override public CompletableFuture closeAsync() { return dispatcher.closeAsync(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.java index d538f32ab5d0..fb74e2a2ffee 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.java @@ -33,7 +33,7 @@ /** * Factory for the {@link DispatcherServiceImpl}. */ -class DispatcherServiceImplFactory implements DispatcherLeaderProcessImpl.DispatcherServiceFactory { +class DispatcherServiceImplFactory implements AbstractDispatcherLeaderProcess.DispatcherServiceFactory { private final DispatcherFactory dispatcherFactory; @@ -51,7 +51,7 @@ class DispatcherServiceImplFactory implements DispatcherLeaderProcessImpl.Dispat } @Override - public DispatcherLeaderProcessImpl.DispatcherService create( + public AbstractDispatcherLeaderProcess.DispatcherService create( DispatcherId fencingToken, Collection recoveredJobs, JobGraphWriter jobGraphWriter) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java index fb3f8aa282f2..248ec8a12b26 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java @@ -33,15 +33,15 @@ */ public class JobDispatcherLeaderProcess extends AbstractDispatcherLeaderProcess { - private final DispatcherLeaderProcessImpl.DispatcherServiceFactory dispatcherServiceFactory; + private final DispatcherServiceFactory dispatcherServiceFactory; private final JobGraph jobGraph; - private DispatcherLeaderProcessImpl.DispatcherService dispatcherService; + private DispatcherService dispatcherService; JobDispatcherLeaderProcess( UUID leaderSessionId, - DispatcherLeaderProcessImpl.DispatcherServiceFactory dispatcherServiceFactory, + DispatcherServiceFactory dispatcherServiceFactory, JobGraph jobGraph, FatalErrorHandler fatalErrorHandler) { super(leaderSessionId, fatalErrorHandler); @@ -56,7 +56,7 @@ protected void onStart() { Collections.singleton(jobGraph), ThrowingJobGraphWriter.INSTANCE); - completeDispatcherGatewayFuture(dispatcherService.getGateway()); + completeDispatcherSetup(dispatcherService); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactory.java index db8f4e97c6f7..085c54f5d233 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactory.java @@ -27,14 +27,14 @@ * Factory for the {@link JobDispatcherLeaderProcess}. */ public class JobDispatcherLeaderProcessFactory implements DispatcherLeaderProcessFactory { - private final DispatcherLeaderProcessImpl.DispatcherServiceFactory dispatcherServiceFactory; + private final AbstractDispatcherLeaderProcess.DispatcherServiceFactory dispatcherServiceFactory; private final JobGraph jobGraph; private final FatalErrorHandler fatalErrorHandler; JobDispatcherLeaderProcessFactory( - DispatcherLeaderProcessImpl.DispatcherServiceFactory dispatcherServiceFactory, + AbstractDispatcherLeaderProcess.DispatcherServiceFactory dispatcherServiceFactory, JobGraph jobGraph, FatalErrorHandler fatalErrorHandler) { this.dispatcherServiceFactory = dispatcherServiceFactory; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactoryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactoryFactory.java index 975b6ff91e1a..5bfd21eddd6b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactoryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactoryFactory.java @@ -44,7 +44,7 @@ public DispatcherLeaderProcessFactory createFactory( RpcService rpcService, PartialDispatcherServices partialDispatcherServices, FatalErrorHandler fatalErrorHandler) { - final DispatcherLeaderProcessImpl.DispatcherServiceFactory dispatcherServiceFactory = new DispatcherServiceImplFactory( + final AbstractDispatcherLeaderProcess.DispatcherServiceFactory dispatcherServiceFactory = new DispatcherServiceImplFactory( dispatcherFactory, rpcService, partialDispatcherServices); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StoppedDispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StoppedDispatcherLeaderProcess.java index d63102b965fe..51a8b41383cf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StoppedDispatcherLeaderProcess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StoppedDispatcherLeaderProcess.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.dispatcher.runner; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.dispatcher.DispatcherGateway; import java.util.UUID; @@ -32,7 +33,8 @@ public enum StoppedDispatcherLeaderProcess implements DispatcherLeaderProcess { private static final CompletableFuture TERMINATION_FUTURE = CompletableFuture.completedFuture(null); private static final UUID LEADER_SESSION_ID = new UUID(0L, 0L); - private static final CompletableFuture NEVER_COMPLETED_FUTURE = new CompletableFuture<>(); + private static final CompletableFuture NEVER_COMPLETED_LEADER_SESSION_FUTURE = new CompletableFuture<>(); + private static final CompletableFuture NEVER_COMPLETED_SHUTDOWN_FUTURE = new CompletableFuture<>(); @Override public void start() { @@ -51,7 +53,12 @@ public CompletableFuture getDispatcherGateway() { @Override public CompletableFuture getConfirmLeaderSessionFuture() { - return NEVER_COMPLETED_FUTURE; + return NEVER_COMPLETED_LEADER_SESSION_FUTURE; + } + + @Override + public CompletableFuture getShutDownFuture() { + return NEVER_COMPLETED_SHUTDOWN_FUTURE; } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGTest.java index d8fd33ecc583..b1ceb6ae9efb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGTest.java @@ -39,6 +39,7 @@ import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.fail; /** @@ -71,15 +72,14 @@ public void teardown() throws Exception { } @Test - public void closeAsync_withUncompletedShutDownFuture_completesShutDownFuture() throws Exception { + public void closeAsync_doesNotCompleteUncompletedShutDownFuture() throws Exception { final DispatcherRunnerImplNG dispatcherRunner = createDispatcherRunner(); final CompletableFuture terminationFuture = dispatcherRunner.closeAsync(); terminationFuture.get(); final CompletableFuture shutDownFuture = dispatcherRunner.getShutDownFuture(); - assertThat(shutDownFuture.isDone(), is(true)); - assertThat(shutDownFuture.get(), is(ApplicationStatus.UNKNOWN)); + assertThat(shutDownFuture.isDone(), is(false)); } @Test @@ -136,6 +136,90 @@ public void getDispatcherGateway_withChangingLeaders_returnsLeadingDispatcherGat } } + @Test + public void getShutDownFuture_whileRunning_forwardsDispatcherLeaderProcessShutDownRequest() throws Exception { + final UUID leaderSessionId = UUID.randomUUID(); + final CompletableFuture shutDownFuture = new CompletableFuture<>(); + final TestingDispatcherLeaderProcess testingDispatcherLeaderProcess = TestingDispatcherLeaderProcess.newBuilder(leaderSessionId) + .setShutDownFuture(shutDownFuture) + .build(); + testingDispatcherLeaderProcessFactory = TestingDispatcherLeaderProcessFactory.from(testingDispatcherLeaderProcess); + + try (final DispatcherRunnerImplNG dispatcherRunner = createDispatcherRunner()) { + testingLeaderElectionService.isLeader(leaderSessionId); + + final CompletableFuture dispatcherShutDownFuture = dispatcherRunner.getShutDownFuture(); + + assertFalse(dispatcherShutDownFuture.isDone()); + + final ApplicationStatus finalApplicationStatus = ApplicationStatus.UNKNOWN; + shutDownFuture.complete(finalApplicationStatus); + + assertThat(dispatcherShutDownFuture.get(), is(finalApplicationStatus)); + } + } + + @Test + public void getShutDownFuture_afterClose_ignoresDispatcherLeaderProcessShutDownRequest() throws Exception { + final UUID leaderSessionId = UUID.randomUUID(); + final CompletableFuture shutDownFuture = new CompletableFuture<>(); + final TestingDispatcherLeaderProcess testingDispatcherLeaderProcess = TestingDispatcherLeaderProcess.newBuilder(leaderSessionId) + .setShutDownFuture(shutDownFuture) + .build(); + testingDispatcherLeaderProcessFactory = TestingDispatcherLeaderProcessFactory.from(testingDispatcherLeaderProcess); + + try (final DispatcherRunnerImplNG dispatcherRunner = createDispatcherRunner()) { + testingLeaderElectionService.isLeader(leaderSessionId); + + final CompletableFuture dispatcherShutDownFuture = dispatcherRunner.getShutDownFuture(); + + assertFalse(dispatcherShutDownFuture.isDone()); + + dispatcherRunner.closeAsync(); + + final ApplicationStatus finalApplicationStatus = ApplicationStatus.UNKNOWN; + shutDownFuture.complete(finalApplicationStatus); + + try { + dispatcherShutDownFuture.get(10L, TimeUnit.MILLISECONDS); + fail("The dispatcher runner should no longer react to the dispatcher leader process's shut down request if it has been terminated."); + } catch (TimeoutException expected) {} + } + } + + @Test + public void getShutDownFuture_newLeader_ignoresOldDispatcherLeaderProcessShutDownRequest() throws Exception { + final UUID firstLeaderSessionId = UUID.randomUUID(); + final UUID secondLeaderSessionId = UUID.randomUUID(); + final CompletableFuture shutDownFuture = new CompletableFuture<>(); + final TestingDispatcherLeaderProcess firstTestingDispatcherLeaderProcess = TestingDispatcherLeaderProcess.newBuilder(firstLeaderSessionId) + .setShutDownFuture(shutDownFuture) + .build(); + final TestingDispatcherLeaderProcess secondTestingDispatcherLeaderProcess = TestingDispatcherLeaderProcess.newBuilder(secondLeaderSessionId) + .build(); + testingDispatcherLeaderProcessFactory = TestingDispatcherLeaderProcessFactory.from( + firstTestingDispatcherLeaderProcess, + secondTestingDispatcherLeaderProcess); + + try (final DispatcherRunnerImplNG dispatcherRunner = createDispatcherRunner()) { + testingLeaderElectionService.isLeader(firstLeaderSessionId); + + final CompletableFuture dispatcherShutDownFuture = dispatcherRunner.getShutDownFuture(); + + assertFalse(dispatcherShutDownFuture.isDone()); + + testingLeaderElectionService.isLeader(secondLeaderSessionId); + + final ApplicationStatus finalApplicationStatus = ApplicationStatus.UNKNOWN; + shutDownFuture.complete(finalApplicationStatus); + + try { + dispatcherShutDownFuture.get(10L, TimeUnit.MILLISECONDS); + fail("The dispatcher runner should no longer react to the dispatcher leader process's shut down request if it has been terminated."); + } catch (TimeoutException expected) {} + } + } + @Test public void revokeLeadership_withExistingLeader_stopsLeaderProcess() throws Exception { final UUID leaderSessionId = UUID.randomUUID(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherLeaderProcess.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherLeaderProcess.java index 7f4d51ae3adc..8bea433fd933 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherLeaderProcess.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherLeaderProcess.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.dispatcher.runner; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.dispatcher.DispatcherGateway; import java.util.UUID; @@ -32,6 +33,7 @@ class TestingDispatcherLeaderProcess implements DispatcherLeaderProcess { private final Consumer startConsumer; private final Supplier> closeAsyncSupplier; private final CompletableFuture confirmLeaderSessionFuture; + private final CompletableFuture shutDownFuture; private CompletableFuture terminationFuture = null; @@ -40,12 +42,14 @@ private TestingDispatcherLeaderProcess( CompletableFuture dispatcherGatewayFuture, Consumer startConsumer, Supplier> closeAsyncSupplier, - CompletableFuture confirmLeaderSessionFuture) { + CompletableFuture confirmLeaderSessionFuture, + CompletableFuture shutDownFuture) { this.leaderSessionId = leaderSessionId; this.dispatcherGatewayFuture = dispatcherGatewayFuture; this.startConsumer = startConsumer; this.closeAsyncSupplier = closeAsyncSupplier; this.confirmLeaderSessionFuture = confirmLeaderSessionFuture; + this.shutDownFuture = shutDownFuture; } @Override @@ -68,6 +72,11 @@ public CompletableFuture getConfirmLeaderSessionFuture() { return confirmLeaderSessionFuture; } + @Override + public CompletableFuture getShutDownFuture() { + return shutDownFuture; + } + @Override public CompletableFuture closeAsync() { if (terminationFuture == null) { @@ -91,6 +100,7 @@ public static class Builder { private Supplier> closeAsyncSupplier = () -> CompletableFuture.completedFuture(null); private CompletableFuture confirmLeaderSessionFuture = CompletableFuture.completedFuture("Unknown address"); + private CompletableFuture shutDownFuture = new CompletableFuture<>(); private Builder(UUID leaderSessionId) { this.leaderSessionId = leaderSessionId; @@ -116,13 +126,19 @@ public Builder setConfirmLeaderSessionFuture(CompletableFuture confirmLe return this; } + public Builder setShutDownFuture(CompletableFuture shutDownFuture) { + this.shutDownFuture = shutDownFuture; + return this; + } + public TestingDispatcherLeaderProcess build() { return new TestingDispatcherLeaderProcess( leaderSessionId, dispatcherGatewayFuture, startConsumer, closeAsyncSupplier, - confirmLeaderSessionFuture); + confirmLeaderSessionFuture, + shutDownFuture); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherService.java index 6981a1067a70..0c9fdfc45c23 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherService.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.dispatcher.runner; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.webmonitor.TestingDispatcherGateway; @@ -27,7 +28,7 @@ import java.util.function.Function; import java.util.function.Supplier; -class TestingDispatcherService implements DispatcherLeaderProcessImpl.DispatcherService { +class TestingDispatcherService implements AbstractDispatcherLeaderProcess.DispatcherService { private final Object lock = new Object(); @@ -37,15 +38,19 @@ class TestingDispatcherService implements DispatcherLeaderProcessImpl.Dispatcher private final DispatcherGateway dispatcherGateway; + private final CompletableFuture shutDownFuture; + private CompletableFuture terminationFuture; private TestingDispatcherService( - Supplier> terminationFutureSupplier, - Function> onRemovedJobGraphFunction, - DispatcherGateway dispatcherGateway) { - this.terminationFutureSupplier = terminationFutureSupplier; - this.onRemovedJobGraphFunction = onRemovedJobGraphFunction; - this.dispatcherGateway = dispatcherGateway; + Supplier> terminationFutureSupplier, + Function> onRemovedJobGraphFunction, + DispatcherGateway dispatcherGateway, + CompletableFuture shutDownFuture) { + this.terminationFutureSupplier = terminationFutureSupplier; + this.onRemovedJobGraphFunction = onRemovedJobGraphFunction; + this.dispatcherGateway = dispatcherGateway; + this.shutDownFuture = shutDownFuture; } @Override @@ -58,6 +63,11 @@ public CompletableFuture onRemovedJobGraph(JobID jobId) { return onRemovedJobGraphFunction.apply(jobId); } + @Override + public CompletableFuture getShutDownFuture() { + return shutDownFuture; + } + @Override public CompletableFuture closeAsync() { synchronized (lock) { @@ -81,6 +91,8 @@ public static class Builder { private DispatcherGateway dispatcherGateway = new TestingDispatcherGateway.Builder().build(); + private CompletableFuture shutDownFuture = new CompletableFuture<>(); + private Builder() {} public Builder setTerminationFutureSupplier(Supplier> terminationFutureSupplier) { @@ -98,8 +110,13 @@ public Builder setOnRemovedJobGraphFunction(Function shutDownFuture) { + this.shutDownFuture = shutDownFuture; + return this; + } + public TestingDispatcherService build() { - return new TestingDispatcherService(terminationFutureSupplier, onRemovedJobGraphFunction, dispatcherGateway); + return new TestingDispatcherService(terminationFutureSupplier, onRemovedJobGraphFunction, dispatcherGateway, shutDownFuture); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java index 4f9b766e3bab..1462c1cea6ab 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java @@ -25,15 +25,15 @@ import java.util.Collection; -class TestingDispatcherServiceFactory implements DispatcherLeaderProcessImpl.DispatcherServiceFactory { - private final TriFunction, JobGraphWriter, DispatcherLeaderProcessImpl.DispatcherService> createFunction; +class TestingDispatcherServiceFactory implements AbstractDispatcherLeaderProcess.DispatcherServiceFactory { + private final TriFunction, JobGraphWriter, AbstractDispatcherLeaderProcess.DispatcherService> createFunction; - private TestingDispatcherServiceFactory(TriFunction, JobGraphWriter, DispatcherLeaderProcessImpl.DispatcherService> createFunction) { + private TestingDispatcherServiceFactory(TriFunction, JobGraphWriter, AbstractDispatcherLeaderProcess.DispatcherService> createFunction) { this.createFunction = createFunction; } @Override - public DispatcherLeaderProcessImpl.DispatcherService create( + public AbstractDispatcherLeaderProcess.DispatcherService create( DispatcherId fencingToken, Collection recoveredJobs, JobGraphWriter jobGraphWriter) { @@ -45,11 +45,11 @@ public static Builder newBuilder() { } public static class Builder { - private TriFunction, JobGraphWriter, DispatcherLeaderProcessImpl.DispatcherService> createFunction = (ignoredA, ignoredB, ignoredC) -> TestingDispatcherService.newBuilder().build(); + private TriFunction, JobGraphWriter, AbstractDispatcherLeaderProcess.DispatcherService> createFunction = (ignoredA, ignoredB, ignoredC) -> TestingDispatcherService.newBuilder().build(); private Builder() {} - Builder setCreateFunction(TriFunction, JobGraphWriter, DispatcherLeaderProcessImpl.DispatcherService> createFunction) { + Builder setCreateFunction(TriFunction, JobGraphWriter, AbstractDispatcherLeaderProcess.DispatcherService> createFunction) { this.createFunction = createFunction; return this; } From a5438ca696d560a83352444086cd8f0bb1e92324 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 25 Sep 2019 15:57:28 +0200 Subject: [PATCH 276/746] [FLINK-11843] Move DispatcherService into AbstractDispatcherLeaderProcess --- .../AbstractDispatcherLeaderProcess.java | 40 ++++++++++++++++--- .../runner/DispatcherLeaderProcessImpl.java | 39 ++++-------------- .../runner/JobDispatcherLeaderProcess.java | 15 +------ 3 files changed, 42 insertions(+), 52 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java index 1e754cd2f118..ecfcee56e209 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java @@ -29,10 +29,13 @@ import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.webmonitor.RestfulGateway; import org.apache.flink.util.AutoCloseableAsync; +import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.util.Collection; import java.util.Optional; import java.util.UUID; @@ -60,6 +63,9 @@ abstract class AbstractDispatcherLeaderProcess implements DispatcherLeaderProces private State state; + @Nullable + private DispatcherService dispatcherService; + AbstractDispatcherLeaderProcess(UUID leaderSessionId, FatalErrorHandler fatalErrorHandler) { this.leaderSessionId = leaderSessionId; this.fatalErrorHandler = fatalErrorHandler; @@ -112,6 +118,10 @@ public CompletableFuture getShutDownFuture() { return shutDownFuture; } + protected final Optional getDispatcherService() { + return Optional.ofNullable(dispatcherService); + } + @Override public final CompletableFuture closeAsync() { runIfStateIsNot( @@ -122,28 +132,46 @@ public final CompletableFuture closeAsync() { } private void closeInternal() { + log.info("Stopping {}.", getClass().getSimpleName()); + + final CompletableFuture dispatcherServiceTerminationFuture = closeDispatcherService(); + + final CompletableFuture onCloseTerminationFuture = FutureUtils.composeAfterwards( + dispatcherServiceTerminationFuture, + this::onClose); + FutureUtils.forward( - onClose(), - terminationFuture); + onCloseTerminationFuture, + this.terminationFuture); state = State.STOPPED; } + private CompletableFuture closeDispatcherService() { + if (dispatcherService != null) { + return dispatcherService.closeAsync(); + } else { + return FutureUtils.completedVoidFuture(); + } + } + protected abstract void onStart(); protected CompletableFuture onClose() { return FutureUtils.completedVoidFuture(); } - protected final void completeDispatcherSetup(DispatcherService dispatcherService) { + final void completeDispatcherSetup(DispatcherService dispatcherService) { runIfStateIs( State.RUNNING, () -> completeDispatcherSetupInternal(dispatcherService)); } - private void completeDispatcherSetupInternal(DispatcherService dispatcherService) { - dispatcherGatewayFuture.complete(dispatcherService.getGateway()); - FutureUtils.forward(dispatcherService.getShutDownFuture(), shutDownFuture); + private void completeDispatcherSetupInternal(DispatcherService createdDispatcherService) { + Preconditions.checkState(dispatcherService == null, "The DispatcherService can only be set once."); + dispatcherService = createdDispatcherService; + dispatcherGatewayFuture.complete(createdDispatcherService.getGateway()); + FutureUtils.forward(createdDispatcherService.getShutDownFuture(), shutDownFuture); } final Optional supplyUnsynchronizedIfRunning(Supplier supplier) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java index 0795a11d0f71..2da4fa85edd3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java @@ -33,8 +33,6 @@ import org.apache.flink.util.Preconditions; import org.apache.flink.util.function.FunctionUtils; -import javax.annotation.Nullable; - import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -56,9 +54,6 @@ public class DispatcherLeaderProcessImpl extends AbstractDispatcherLeaderProces private final Executor ioExecutor; - @Nullable - private DispatcherService dispatcher; - private CompletableFuture onGoingRecoveryOperation = FutureUtils.completedVoidFuture(); private DispatcherLeaderProcessImpl( @@ -72,8 +67,6 @@ private DispatcherLeaderProcessImpl( this.dispatcherFactory = dispatcherFactory; this.jobGraphStore = jobGraphStore; this.ioExecutor = ioExecutor; - - this.dispatcher = null; } @Override @@ -103,14 +96,13 @@ private void createDispatcherIfRunning(Collection jobGraphs) { } private void createDispatcher(Collection jobGraphs) { - Preconditions.checkState(dispatcher == null); - dispatcher = dispatcherFactory.create( + final DispatcherService dispatcherService = dispatcherFactory.create( DispatcherId.fromUuid(getLeaderSessionId()), jobGraphs, jobGraphStore); - completeDispatcherSetup(dispatcher); + completeDispatcherSetup(dispatcherService); } private CompletableFuture> recoverJobsAsync() { @@ -161,31 +153,17 @@ private JobGraph recoverJob(JobID jobId) { @Override protected CompletableFuture onClose() { - log.info("Stopping {}.", getClass().getSimpleName()); - final CompletableFuture dispatcherTerminationFuture; - - if (dispatcher != null) { - dispatcherTerminationFuture = dispatcher.closeAsync(); - } else { - dispatcherTerminationFuture = FutureUtils.completedVoidFuture(); - } - - return FutureUtils.runAfterwardsAsync( - dispatcherTerminationFuture, + return CompletableFuture.runAsync( this::stopServices, ioExecutor); } - private void stopServices() throws Exception { - Exception exception = null; - + private void stopServices() { try { jobGraphStore.stop(); } catch (Exception e) { - exception = e; + ExceptionUtils.rethrow(e); } - - ExceptionUtils.tryRethrowException(exception); } // ------------------------------------------------------------ @@ -273,11 +251,8 @@ private Optional> removeJobGraphIfRunning(JobID jobId) { } private CompletableFuture removeJobGraph(JobID jobId) { - if (dispatcher == null) { - return FutureUtils.completedVoidFuture(); - } else { - return dispatcher.onRemovedJobGraph(jobId); - } + return getDispatcherService().map(dispatcherService -> dispatcherService.onRemovedJobGraph(jobId)) + .orElseGet(FutureUtils::completedVoidFuture); } // --------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java index 248ec8a12b26..b0edcef0ffad 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.dispatcher.runner; -import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.dispatcher.DispatcherId; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.ThrowingJobGraphWriter; @@ -26,7 +25,6 @@ import java.util.Collections; import java.util.UUID; -import java.util.concurrent.CompletableFuture; /** * {@link DispatcherLeaderProcess} implementation for the per-job mode. @@ -37,8 +35,6 @@ public class JobDispatcherLeaderProcess extends AbstractDispatcherLeaderProcess private final JobGraph jobGraph; - private DispatcherService dispatcherService; - JobDispatcherLeaderProcess( UUID leaderSessionId, DispatcherServiceFactory dispatcherServiceFactory, @@ -51,20 +47,11 @@ public class JobDispatcherLeaderProcess extends AbstractDispatcherLeaderProcess @Override protected void onStart() { - dispatcherService = dispatcherServiceFactory.create( + final DispatcherService dispatcherService = dispatcherServiceFactory.create( DispatcherId.fromUuid(getLeaderSessionId()), Collections.singleton(jobGraph), ThrowingJobGraphWriter.INSTANCE); completeDispatcherSetup(dispatcherService); } - - @Override - protected CompletableFuture onClose() { - if (dispatcherService != null) { - return dispatcherService.closeAsync(); - } else { - return FutureUtils.completedVoidFuture(); - } - } } From cdd0c29102a0bdc3a36ff8ece1b032a7b10fabf8 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 25 Sep 2019 16:41:18 +0200 Subject: [PATCH 277/746] [FLINK-11843] Remove old DispatcherRunnerImpl and DispatcherRunnerFactoryImpl --- .../runner/DispatcherRunnerFactoryImpl.java | 56 ----------- .../runner/DispatcherRunnerImpl.java | 94 ------------------- .../runner/DispatcherRunnerImplTest.java | 2 +- .../ZooKeeperDispatcherRunnerImplTest.java | 2 +- 4 files changed, 2 insertions(+), 152 deletions(-) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerFactoryImpl.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java 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 deleted file mode 100644 index 8acfbf1a49bf..000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerFactoryImpl.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.dispatcher.runner; - -import org.apache.flink.runtime.dispatcher.DispatcherFactory; -import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; -import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobGraphStore; -import org.apache.flink.runtime.dispatcher.StandaloneDispatcher; -import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; -import org.apache.flink.runtime.leaderelection.LeaderElectionService; -import org.apache.flink.runtime.rpc.FatalErrorHandler; -import org.apache.flink.runtime.rpc.RpcService; - -import java.util.concurrent.Executor; - -/** - * Factory which creates a {@link DispatcherRunnerImpl} which runs a {@link StandaloneDispatcher}. - */ -public class DispatcherRunnerFactoryImpl implements DispatcherRunnerFactory { - - private final DispatcherFactory dispatcherFactory; - - public DispatcherRunnerFactoryImpl(DispatcherFactory dispatcherFactory) { - this.dispatcherFactory = dispatcherFactory; - } - - @Override - public DispatcherRunnerImpl createDispatcherRunner( - LeaderElectionService leaderElectionService, - FatalErrorHandler fatalErrorHandler, - JobGraphStoreFactory jobGraphStoreFactory, - Executor ioExecutor, - RpcService rpcService, - PartialDispatcherServices partialDispatcherServices) throws Exception { - return new DispatcherRunnerImpl( - dispatcherFactory, - rpcService, - PartialDispatcherServicesWithJobGraphStore.from(partialDispatcherServices, jobGraphStoreFactory.create())); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.java deleted file mode 100644 index dead9b5cc4a5..000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImpl.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.dispatcher.runner; - -import org.apache.flink.runtime.clusterframework.ApplicationStatus; -import org.apache.flink.runtime.concurrent.FutureUtils; -import org.apache.flink.runtime.dispatcher.Dispatcher; -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.PartialDispatcherServicesWithJobGraphStore; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; -import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.webmonitor.retriever.LeaderRetriever; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.concurrent.CompletableFuture; - -/** - * Runner responsible for executing a {@link Dispatcher} or a subclass thereof. - */ -class DispatcherRunnerImpl implements DispatcherRunner { - - private final Dispatcher dispatcher; - - private final LeaderRetrievalService leaderRetrievalService; - - private final LeaderRetriever leaderRetriever; - - DispatcherRunnerImpl( - DispatcherFactory dispatcherFactory, - RpcService rpcService, - PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { - this.dispatcher = dispatcherFactory.createDispatcher( - rpcService, - DispatcherId.generate(), - Collections.emptyList(), - partialDispatcherServicesWithJobGraphStore); - this.leaderRetrievalService = partialDispatcherServicesWithJobGraphStore.getHighAvailabilityServices().getDispatcherLeaderRetriever(); - this.leaderRetriever = new LeaderRetriever(); - - leaderRetrievalService.start(leaderRetriever); - dispatcher.start(); - } - - @Override - public CompletableFuture getDispatcherGateway() { - return leaderRetriever.getLeaderFuture().thenApply(ignored -> dispatcher.getSelfGateway(DispatcherGateway.class)); - } - - @Override - public CompletableFuture closeAsync() { - Exception exception = null; - - try { - leaderRetrievalService.stop(); - } catch (Exception e) { - exception = e; - } - - Collection> terminationFutures = new ArrayList<>(); - - terminationFutures.add(dispatcher.closeAsync()); - - if (exception != null) { - terminationFutures.add(FutureUtils.completedExceptionally(exception)); - } - - return FutureUtils.completeAll(terminationFutures); - } - - @Override - public CompletableFuture getShutDownFuture() { - return dispatcher.getShutDownFuture(); - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java index 3aa13d2917d2..2383a360f6cb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java @@ -69,7 +69,7 @@ import static org.junit.Assert.assertThat; /** - * Tests for the {@link DispatcherRunnerImpl}. + * Tests for the {@link DispatcherRunnerImplNG}. */ public class DispatcherRunnerImplTest extends TestLogger { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDispatcherRunnerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDispatcherRunnerImplTest.java index 0c30b19a2179..68f074efe6ee 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDispatcherRunnerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDispatcherRunnerImplTest.java @@ -75,7 +75,7 @@ import static org.junit.Assert.assertThat; /** - * Tests for the interaction between the {@link DispatcherRunnerImpl} and ZooKeeper. + * Tests for the interaction between the {@link DispatcherRunnerImplNG} and ZooKeeper. */ public class ZooKeeperDispatcherRunnerImplTest extends TestLogger { From 1f2969357c441e24b71daef83d21563da9a93bb4 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 26 Sep 2019 13:17:06 +0200 Subject: [PATCH 278/746] [FLINK-11843] Rename DispatcherRunnerImplNG into DefaultDispatcherRunner Rename XImplNG implementations into DefaultX. Rename DispatcherLeaderProcessImpl into SessionDispatcherLeaderProcess. --- ...plNG.java => DefaultDispatcherRunner.java} | 6 +-- ...va => DefaultDispatcherRunnerFactory.java} | 18 ++++----- ...mpl.java => DefaultDispatcherService.java} | 8 ++-- ...a => DefaultDispatcherServiceFactory.java} | 8 ++-- ...DispatcherLeaderProcessFactoryFactory.java | 4 +- ...va => SessionDispatcherLeaderProcess.java} | 8 ++-- ...essionDispatcherLeaderProcessFactory.java} | 8 ++-- ...DispatcherLeaderProcessFactoryFactory.java | 6 +-- .../StoppedDispatcherLeaderProcess.java | 2 +- ...atcherResourceManagerComponentFactory.java | 6 +-- ...ava => DefaultDispatcherRunnerITCase.java} | 10 ++--- ....java => DefaultDispatcherRunnerTest.java} | 30 +++++++-------- .../runner/DispatcherRunnerImplTest.java | 4 +- ...> SessionDispatcherLeaderProcessTest.java} | 38 +++++++++---------- ...ZooKeeperDefaultDispatcherRunnerTest.java} | 10 ++--- .../minicluster/TestingMiniCluster.java | 4 +- 16 files changed, 85 insertions(+), 85 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/{DispatcherRunnerImplNG.java => DefaultDispatcherRunner.java} (97%) rename flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/{DispatcherRunnerImplNGFactory.java => DefaultDispatcherRunnerFactory.java} (79%) rename flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/{DispatcherServiceImpl.java => DefaultDispatcherService.java} (86%) rename flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/{DispatcherServiceImplFactory.java => DefaultDispatcherServiceFactory.java} (90%) rename flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/{DispatcherLeaderProcessImpl.java => SessionDispatcherLeaderProcess.java} (96%) rename flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/{DispatcherLeaderProcessImplFactory.java => SessionDispatcherLeaderProcessFactory.java} (88%) rename flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/{DispatcherRunnerImplNGITCase.java => DefaultDispatcherRunnerITCase.java} (92%) rename flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/{DispatcherRunnerImplNGTest.java => DefaultDispatcherRunnerTest.java} (93%) rename flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/{DispatcherLeaderProcessImplTest.java => SessionDispatcherLeaderProcessTest.java} (90%) rename flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/{ZooKeeperDispatcherRunnerImplTest.java => ZooKeeperDefaultDispatcherRunnerTest.java} (96%) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNG.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java similarity index 97% rename from flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNG.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java index c0555ac15384..92faf4d1d794 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNG.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java @@ -37,9 +37,9 @@ * Runner for the {@link org.apache.flink.runtime.dispatcher.Dispatcher} which is responsible for the * leader election. */ -public class DispatcherRunnerImplNG implements DispatcherRunner, LeaderContender { +public class DefaultDispatcherRunner implements DispatcherRunner, LeaderContender { - private static final Logger LOG = LoggerFactory.getLogger(DispatcherRunnerImplNG.class); + private static final Logger LOG = LoggerFactory.getLogger(DefaultDispatcherRunner.class); private final Object lock = new Object(); @@ -61,7 +61,7 @@ public class DispatcherRunnerImplNG implements DispatcherRunner, LeaderContender private CompletableFuture dispatcherGatewayFuture; - DispatcherRunnerImplNG( + DefaultDispatcherRunner( LeaderElectionService leaderElectionService, FatalErrorHandler fatalErrorHandler, DispatcherLeaderProcessFactory dispatcherLeaderProcessFactory) throws Exception { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerFactory.java similarity index 79% rename from flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGFactory.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerFactory.java index 5ca79f0fcd0e..3e4027f98362 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerFactory.java @@ -29,18 +29,18 @@ import java.util.concurrent.Executor; /** - * {@link DispatcherRunnerFactory} implementation which creates {@link DispatcherRunnerImplNG} + * {@link DispatcherRunnerFactory} implementation which creates {@link DefaultDispatcherRunner} * instances. */ -public class DispatcherRunnerImplNGFactory implements DispatcherRunnerFactory { +public class DefaultDispatcherRunnerFactory implements DispatcherRunnerFactory { private final DispatcherLeaderProcessFactoryFactory dispatcherLeaderProcessFactoryFactory; - private DispatcherRunnerImplNGFactory(DispatcherLeaderProcessFactoryFactory dispatcherLeaderProcessFactoryFactory) { + private DefaultDispatcherRunnerFactory(DispatcherLeaderProcessFactoryFactory dispatcherLeaderProcessFactoryFactory) { this.dispatcherLeaderProcessFactoryFactory = dispatcherLeaderProcessFactoryFactory; } @Override - public DispatcherRunnerImplNG createDispatcherRunner( + public DefaultDispatcherRunner createDispatcherRunner( LeaderElectionService leaderElectionService, FatalErrorHandler fatalErrorHandler, JobGraphStoreFactory jobGraphStoreFactory, @@ -55,19 +55,19 @@ public DispatcherRunnerImplNG createDispatcherRunner( partialDispatcherServices, fatalErrorHandler); - return new DispatcherRunnerImplNG( + return new DefaultDispatcherRunner( leaderElectionService, fatalErrorHandler, dispatcherLeaderProcessFactory); } - public static DispatcherRunnerImplNGFactory createSessionRunner(DispatcherFactory dispatcherFactory) { - return new DispatcherRunnerImplNGFactory( + public static DefaultDispatcherRunnerFactory createSessionRunner(DispatcherFactory dispatcherFactory) { + return new DefaultDispatcherRunnerFactory( SessionDispatcherLeaderProcessFactoryFactory.create(dispatcherFactory)); } - public static DispatcherRunnerImplNGFactory createJobRunner(JobGraphRetriever jobGraphRetriever) { - return new DispatcherRunnerImplNGFactory( + public static DefaultDispatcherRunnerFactory createJobRunner(JobGraphRetriever jobGraphRetriever) { + return new DefaultDispatcherRunnerFactory( JobDispatcherLeaderProcessFactoryFactory.create(jobGraphRetriever)); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherService.java similarity index 86% rename from flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImpl.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherService.java index 8a9e0e129b0c..4af987b056a8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherService.java @@ -25,12 +25,12 @@ import java.util.concurrent.CompletableFuture; -class DispatcherServiceImpl implements AbstractDispatcherLeaderProcess.DispatcherService { +class DefaultDispatcherService implements AbstractDispatcherLeaderProcess.DispatcherService { private final Dispatcher dispatcher; private final DispatcherGateway dispatcherGateway; - private DispatcherServiceImpl(Dispatcher dispatcher) { + private DefaultDispatcherService(Dispatcher dispatcher) { this.dispatcher = dispatcher; this.dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); } @@ -55,7 +55,7 @@ public CompletableFuture closeAsync() { return dispatcher.closeAsync(); } - public static DispatcherServiceImpl from(Dispatcher dispatcher) { - return new DispatcherServiceImpl(dispatcher); + public static DefaultDispatcherService from(Dispatcher dispatcher) { + return new DefaultDispatcherService(dispatcher); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherServiceFactory.java similarity index 90% rename from flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherServiceFactory.java index fb74e2a2ffee..dd2c78611dfc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherServiceImplFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherServiceFactory.java @@ -31,9 +31,9 @@ import java.util.Collection; /** - * Factory for the {@link DispatcherServiceImpl}. + * Factory for the {@link DefaultDispatcherService}. */ -class DispatcherServiceImplFactory implements AbstractDispatcherLeaderProcess.DispatcherServiceFactory { +class DefaultDispatcherServiceFactory implements AbstractDispatcherLeaderProcess.DispatcherServiceFactory { private final DispatcherFactory dispatcherFactory; @@ -41,7 +41,7 @@ class DispatcherServiceImplFactory implements AbstractDispatcherLeaderProcess.Di private final PartialDispatcherServices partialDispatcherServices; - DispatcherServiceImplFactory( + DefaultDispatcherServiceFactory( DispatcherFactory dispatcherFactory, RpcService rpcService, PartialDispatcherServices partialDispatcherServices) { @@ -68,6 +68,6 @@ public AbstractDispatcherLeaderProcess.DispatcherService create( dispatcher.start(); - return DispatcherServiceImpl.from(dispatcher); + return DefaultDispatcherService.from(dispatcher); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java index 137cf42846eb..bde0ab16110f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java @@ -57,13 +57,13 @@ public DispatcherLeaderProcessFactory createFactory( throw new FlinkRuntimeException("Could not retrieve the JobGraph.", e); } - final DispatcherServiceImplFactory dispatcherServiceImplFactory = new DispatcherServiceImplFactory( + final DefaultDispatcherServiceFactory defaultDispatcherServiceFactory = new DefaultDispatcherServiceFactory( JobDispatcherFactory.INSTANCE, rpcService, partialDispatcherServices); return new JobDispatcherLeaderProcessFactory( - dispatcherServiceImplFactory, + defaultDispatcherServiceFactory, jobGraph, fatalErrorHandler); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java similarity index 96% rename from flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java index 2da4fa85edd3..4c22b22f52eb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java @@ -46,7 +46,7 @@ * Process which encapsulates the job recovery logic and life cycle management of a * {@link Dispatcher}. */ -public class DispatcherLeaderProcessImpl extends AbstractDispatcherLeaderProcess implements JobGraphStore.JobGraphListener { +public class SessionDispatcherLeaderProcess extends AbstractDispatcherLeaderProcess implements JobGraphStore.JobGraphListener { private final DispatcherServiceFactory dispatcherFactory; @@ -56,7 +56,7 @@ public class DispatcherLeaderProcessImpl extends AbstractDispatcherLeaderProces private CompletableFuture onGoingRecoveryOperation = FutureUtils.completedVoidFuture(); - private DispatcherLeaderProcessImpl( + private SessionDispatcherLeaderProcess( UUID leaderSessionId, DispatcherServiceFactory dispatcherFactory, JobGraphStore jobGraphStore, @@ -259,13 +259,13 @@ private CompletableFuture removeJobGraph(JobID jobId) { // Factory methods // --------------------------------------------------------------- - public static DispatcherLeaderProcessImpl create( + public static SessionDispatcherLeaderProcess create( UUID leaderSessionId, DispatcherServiceFactory dispatcherFactory, JobGraphStore jobGraphStore, Executor ioExecutor, FatalErrorHandler fatalErrorHandler) { - return new DispatcherLeaderProcessImpl( + return new SessionDispatcherLeaderProcess( leaderSessionId, dispatcherFactory, jobGraphStore, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactory.java similarity index 88% rename from flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplFactory.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactory.java index 80c5b7822245..2e1161d13883 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactory.java @@ -25,16 +25,16 @@ import java.util.concurrent.Executor; /** - * Factory for the {@link DispatcherLeaderProcessImpl}. + * Factory for the {@link SessionDispatcherLeaderProcess}. */ -class DispatcherLeaderProcessImplFactory implements DispatcherLeaderProcessFactory { +class SessionDispatcherLeaderProcessFactory implements DispatcherLeaderProcessFactory { private final AbstractDispatcherLeaderProcess.DispatcherServiceFactory dispatcherServiceFactory; private final JobGraphStoreFactory jobGraphStoreFactory; private final Executor ioExecutor; private final FatalErrorHandler fatalErrorHandler; - DispatcherLeaderProcessImplFactory( + SessionDispatcherLeaderProcessFactory( AbstractDispatcherLeaderProcess.DispatcherServiceFactory dispatcherServiceFactory, JobGraphStoreFactory jobGraphStoreFactory, Executor ioExecutor, @@ -47,7 +47,7 @@ class DispatcherLeaderProcessImplFactory implements DispatcherLeaderProcessFacto @Override public DispatcherLeaderProcess create(UUID leaderSessionID) { - return DispatcherLeaderProcessImpl.create( + return SessionDispatcherLeaderProcess.create( leaderSessionID, dispatcherServiceFactory, jobGraphStoreFactory.create(), diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactoryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactoryFactory.java index 5bfd21eddd6b..ddbd81df9c49 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactoryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactoryFactory.java @@ -27,7 +27,7 @@ import java.util.concurrent.Executor; /** - * Factory for the {@link DispatcherLeaderProcessImplFactory}. + * Factory for the {@link SessionDispatcherLeaderProcessFactory}. */ public class SessionDispatcherLeaderProcessFactoryFactory implements DispatcherLeaderProcessFactoryFactory { @@ -44,12 +44,12 @@ public DispatcherLeaderProcessFactory createFactory( RpcService rpcService, PartialDispatcherServices partialDispatcherServices, FatalErrorHandler fatalErrorHandler) { - final AbstractDispatcherLeaderProcess.DispatcherServiceFactory dispatcherServiceFactory = new DispatcherServiceImplFactory( + final AbstractDispatcherLeaderProcess.DispatcherServiceFactory dispatcherServiceFactory = new DefaultDispatcherServiceFactory( dispatcherFactory, rpcService, partialDispatcherServices); - return new DispatcherLeaderProcessImplFactory( + return new SessionDispatcherLeaderProcessFactory( dispatcherServiceFactory, jobGraphStoreFactory, ioExecutor, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StoppedDispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StoppedDispatcherLeaderProcess.java index 51a8b41383cf..f4a5a7cff07f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StoppedDispatcherLeaderProcess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StoppedDispatcherLeaderProcess.java @@ -26,7 +26,7 @@ /** * {@link DispatcherLeaderProcess} implementation which is stopped. This class - * is useful as the initial state of the {@link DispatcherRunnerImplNG}. + * is useful as the initial state of the {@link DefaultDispatcherRunner}. */ public enum StoppedDispatcherLeaderProcess implements DispatcherLeaderProcess { INSTANCE; 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 0650a13f2d00..59fe1b016717 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 @@ -31,9 +31,9 @@ import org.apache.flink.runtime.dispatcher.HistoryServerArchivist; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; import org.apache.flink.runtime.dispatcher.SessionDispatcherFactory; +import org.apache.flink.runtime.dispatcher.runner.DefaultDispatcherRunnerFactory; import org.apache.flink.runtime.dispatcher.runner.DispatcherRunner; import org.apache.flink.runtime.dispatcher.runner.DispatcherRunnerFactory; -import org.apache.flink.runtime.dispatcher.runner.DispatcherRunnerImplNGFactory; import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; @@ -265,7 +265,7 @@ public DispatcherResourceManagerComponent create( public static DefaultDispatcherResourceManagerComponentFactory createSessionComponentFactory( ResourceManagerFactory resourceManagerFactory) { return new DefaultDispatcherResourceManagerComponentFactory( - DispatcherRunnerImplNGFactory.createSessionRunner(SessionDispatcherFactory.INSTANCE), + DefaultDispatcherRunnerFactory.createSessionRunner(SessionDispatcherFactory.INSTANCE), resourceManagerFactory, SessionRestEndpointFactory.INSTANCE); } @@ -274,7 +274,7 @@ public static DefaultDispatcherResourceManagerComponentFactory createJobComponen ResourceManagerFactory resourceManagerFactory, JobGraphRetriever jobGraphRetriever) { return new DefaultDispatcherResourceManagerComponentFactory( - DispatcherRunnerImplNGFactory.createJobRunner(jobGraphRetriever), + DefaultDispatcherRunnerFactory.createJobRunner(jobGraphRetriever), resourceManagerFactory, JobRestEndpointFactory.INSTANCE); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java similarity index 92% rename from flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGITCase.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java index 5f73e6c6436a..8617996370be 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java @@ -55,9 +55,9 @@ import static org.junit.Assert.assertThat; /** - * Integration tests for the {@link DispatcherRunnerImplNG}. + * Integration tests for the {@link DefaultDispatcherRunner}. */ -public class DispatcherRunnerImplNGITCase extends TestLogger { +public class DefaultDispatcherRunnerITCase extends TestLogger { private static final Time TIMEOUT = Time.seconds(10L); @@ -108,7 +108,7 @@ public void teardown() throws Exception { @Test public void leaderChange_afterJobSubmission_recoversSubmittedJob() throws Exception { - try (final DispatcherRunnerImplNG dispatcherRunner = createDispatcherRunner()) { + try (final DefaultDispatcherRunner dispatcherRunner = createDispatcherRunner()) { final UUID firstLeaderSessionId = UUID.randomUUID(); dispatcherLeaderElectionService.isLeader(firstLeaderSessionId); @@ -140,8 +140,8 @@ private static JobGraph createJobGraph() { return testJob; } - private DispatcherRunnerImplNG createDispatcherRunner() throws Exception { - final DispatcherRunnerImplNGFactory runnerFactory = DispatcherRunnerImplNGFactory.createSessionRunner(SessionDispatcherWithUUIDFactory.INSTANCE); + private DefaultDispatcherRunner createDispatcherRunner() throws Exception { + final DefaultDispatcherRunnerFactory runnerFactory = DefaultDispatcherRunnerFactory.createSessionRunner(SessionDispatcherWithUUIDFactory.INSTANCE); return runnerFactory.createDispatcherRunner( dispatcherLeaderElectionService, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerTest.java similarity index 93% rename from flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerTest.java index b1ceb6ae9efb..48c95aef00e3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplNGTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerTest.java @@ -43,9 +43,9 @@ import static org.junit.Assert.fail; /** - * Tests for the {@link DispatcherRunnerImplNG}. + * Tests for the {@link DefaultDispatcherRunner}. */ -public class DispatcherRunnerImplNGTest extends TestLogger { +public class DefaultDispatcherRunnerTest extends TestLogger { private TestingLeaderElectionService testingLeaderElectionService; private TestingFatalErrorHandler testingFatalErrorHandler; @@ -73,7 +73,7 @@ public void teardown() throws Exception { @Test public void closeAsync_doesNotCompleteUncompletedShutDownFuture() throws Exception { - final DispatcherRunnerImplNG dispatcherRunner = createDispatcherRunner(); + final DefaultDispatcherRunner dispatcherRunner = createDispatcherRunner(); final CompletableFuture terminationFuture = dispatcherRunner.closeAsync(); terminationFuture.get(); @@ -91,7 +91,7 @@ public void getDispatcherGateway_beforeDispatcherLeaderProcessCompletes_returnsD .build(); testingDispatcherLeaderProcessFactory = TestingDispatcherLeaderProcessFactory.from(testingDispatcherLeaderProcess); - try (final DispatcherRunnerImplNG dispatcherRunner = createDispatcherRunner()) { + try (final DefaultDispatcherRunner dispatcherRunner = createDispatcherRunner()) { final CompletableFuture dispatcherGatewayFuture = dispatcherRunner.getDispatcherGateway(); @@ -121,7 +121,7 @@ public void getDispatcherGateway_withChangingLeaders_returnsLeadingDispatcherGat firstDispatcherLeaderProcess, secondDispatcherLeaderProcess); - try (final DispatcherRunnerImplNG dispatcherRunner = createDispatcherRunner()) { + try (final DefaultDispatcherRunner dispatcherRunner = createDispatcherRunner()) { testingLeaderElectionService.isLeader(firstLeaderSessionId); final CompletableFuture firstDispatcherGatewayFuture = dispatcherRunner.getDispatcherGateway(); @@ -145,7 +145,7 @@ public void getShutDownFuture_whileRunning_forwardsDispatcherLeaderProcessShutDo .build(); testingDispatcherLeaderProcessFactory = TestingDispatcherLeaderProcessFactory.from(testingDispatcherLeaderProcess); - try (final DispatcherRunnerImplNG dispatcherRunner = createDispatcherRunner()) { + try (final DefaultDispatcherRunner dispatcherRunner = createDispatcherRunner()) { testingLeaderElectionService.isLeader(leaderSessionId); final CompletableFuture dispatcherShutDownFuture = dispatcherRunner.getShutDownFuture(); @@ -168,7 +168,7 @@ public void getShutDownFuture_afterClose_ignoresDispatcherLeaderProcessShutDownR .build(); testingDispatcherLeaderProcessFactory = TestingDispatcherLeaderProcessFactory.from(testingDispatcherLeaderProcess); - try (final DispatcherRunnerImplNG dispatcherRunner = createDispatcherRunner()) { + try (final DefaultDispatcherRunner dispatcherRunner = createDispatcherRunner()) { testingLeaderElectionService.isLeader(leaderSessionId); final CompletableFuture dispatcherShutDownFuture = dispatcherRunner.getShutDownFuture(); @@ -201,7 +201,7 @@ public void getShutDownFuture_newLeader_ignoresOldDispatcherLeaderProcessShutDow firstTestingDispatcherLeaderProcess, secondTestingDispatcherLeaderProcess); - try (final DispatcherRunnerImplNG dispatcherRunner = createDispatcherRunner()) { + try (final DefaultDispatcherRunner dispatcherRunner = createDispatcherRunner()) { testingLeaderElectionService.isLeader(firstLeaderSessionId); final CompletableFuture dispatcherShutDownFuture = dispatcherRunner.getShutDownFuture(); @@ -235,7 +235,7 @@ public void revokeLeadership_withExistingLeader_stopsLeaderProcess() throws Exce return FutureUtils.completedVoidFuture(); }) .build()); - try (final DispatcherRunnerImplNG dispatcherRunner = createDispatcherRunner()) { + try (final DefaultDispatcherRunner dispatcherRunner = createDispatcherRunner()) { testingLeaderElectionService.isLeader(leaderSessionId); // wait until the leader process has been started @@ -260,7 +260,7 @@ public void grantLeadership_withExistingLeader_waitsForTerminationOfFirstLeader( firstTestingDispatcherLeaderProcess.asTestingDispatcherLeaderProcess(), secondTestingDispatcherLeaderProcess.asTestingDispatcherLeaderProcess()); - try (final DispatcherRunnerImplNG dispatcherRunner = createDispatcherRunner()) { + try (final DefaultDispatcherRunner dispatcherRunner = createDispatcherRunner()) { testingLeaderElectionService.isLeader(firstLeaderSessionId); assertThat(firstTestingDispatcherLeaderProcess.isStarted(), is(true)); @@ -278,7 +278,7 @@ public void grantLeadership_withExistingLeader_waitsForTerminationOfFirstLeader( public void grantLeadership_validLeader_confirmsLeaderSession() throws Exception { final UUID leaderSessionId = UUID.randomUUID(); - try (final DispatcherRunnerImplNG dispatcherRunner = createDispatcherRunner()) { + try (final DefaultDispatcherRunner dispatcherRunner = createDispatcherRunner()) { testingLeaderElectionService.isLeader(leaderSessionId); final CompletableFuture confirmationFuture = testingLeaderElectionService.getConfirmationFuture(); @@ -298,7 +298,7 @@ public void grantLeadership_oldLeader_doesNotConfirmLeaderSession() throws Excep testingDispatcherLeaderProcessFactory = TestingDispatcherLeaderProcessFactory.from(testingDispatcherLeaderProcess); - try (final DispatcherRunnerImplNG dispatcherRunner = createDispatcherRunner()) { + try (final DefaultDispatcherRunner dispatcherRunner = createDispatcherRunner()) { testingLeaderElectionService.isLeader(leaderSessionId); testingLeaderElectionService.notLeader(); @@ -339,7 +339,7 @@ public void grantLeadership_multipleLeaderChanges_lastDispatcherLeaderProcessWai secondDispatcherLeaderProcess, thirdDispatcherLeaderProcess); - final DispatcherRunnerImplNG dispatcherRunner = createDispatcherRunner(); + final DefaultDispatcherRunner dispatcherRunner = createDispatcherRunner(); try { testingLeaderElectionService.isLeader(firstLeaderSession); @@ -406,8 +406,8 @@ private TestingDispatcherGateway createDispatcherGateway(UUID leaderSessionId) { .build(); } - private DispatcherRunnerImplNG createDispatcherRunner() throws Exception { - return new DispatcherRunnerImplNG( + private DefaultDispatcherRunner createDispatcherRunner() throws Exception { + return new DefaultDispatcherRunner( testingLeaderElectionService, testingFatalErrorHandler, testingDispatcherLeaderProcessFactory); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java index 2383a360f6cb..e94a96c49b5c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java @@ -69,7 +69,7 @@ import static org.junit.Assert.assertThat; /** - * Tests for the {@link DispatcherRunnerImplNG}. + * Tests for the {@link DefaultDispatcherRunner}. */ public class DispatcherRunnerImplTest extends TestLogger { @@ -124,7 +124,7 @@ public void testJobRecoveryUnderLeaderChange() throws Exception { final TestingJobManagerRunnerFactory jobManagerRunnerFactory = new TestingJobManagerRunnerFactory(1); - final DispatcherRunnerImplNGFactory dispatcherRunnerFactory = DispatcherRunnerImplNGFactory.createSessionRunner(new TestingDispatcherFactory(jobManagerRunnerFactory)); + final DefaultDispatcherRunnerFactory dispatcherRunnerFactory = DefaultDispatcherRunnerFactory.createSessionRunner(new TestingDispatcherFactory(jobManagerRunnerFactory)); try (final DispatcherRunner dispatcherRunner = createDispatcherRunner( rpcService, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java similarity index 90% rename from flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java index a7a6c1cbe52b..89a7fd8a4ab9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java @@ -62,9 +62,9 @@ import static org.junit.Assert.fail; /** - * Tests for the {@link DispatcherLeaderProcessImpl}. + * Tests for the {@link SessionDispatcherLeaderProcess}. */ -public class DispatcherLeaderProcessImplTest extends TestLogger { +public class SessionDispatcherLeaderProcessTest extends TestLogger { private static final JobGraph JOB_GRAPH = new JobGraph("JobGraph"); @@ -107,12 +107,12 @@ public static void teardownClass() { @Test public void start_afterClose_doesNotHaveAnEffect() throws Exception { - final DispatcherLeaderProcessImpl dispatcherLeaderProcess = createDispatcherLeaderProcess(); + final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess(); dispatcherLeaderProcess.close(); dispatcherLeaderProcess.start(); - assertThat(dispatcherLeaderProcess.getState(), is(DispatcherLeaderProcessImpl.State.STOPPED)); + assertThat(dispatcherLeaderProcess.getState(), is(SessionDispatcherLeaderProcess.State.STOPPED)); } @Test @@ -131,9 +131,9 @@ public void start_triggersJobGraphRecoveryAndDispatcherServiceCreation() throws ) .build(); - try (final DispatcherLeaderProcessImpl dispatcherLeaderProcess = createDispatcherLeaderProcess()) { + try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { dispatcherLeaderProcess.start(); - assertThat(dispatcherLeaderProcess.getState(), is(DispatcherLeaderProcessImpl.State.RUNNING)); + assertThat(dispatcherLeaderProcess.getState(), is(SessionDispatcherLeaderProcess.State.RUNNING)); final Collection recoveredJobGraphs = recoveredJobGraphsFuture.get(); @@ -160,7 +160,7 @@ public void closeAsync_stopsJobGraphStoreAndDispatcher() throws Exception { .build()) .build(); - try (final DispatcherLeaderProcessImpl dispatcherLeaderProcess = createDispatcherLeaderProcess()) { + try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { dispatcherLeaderProcess.start(); // wait for the creation of the DispatcherService @@ -200,7 +200,7 @@ public void confirmLeaderSessionFuture_completesAfterDispatcherServiceHasBeenSta })) .build(); - try (final DispatcherLeaderProcessImpl dispatcherLeaderProcess = createDispatcherLeaderProcess()) { + try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { final CompletableFuture confirmLeaderSessionFuture = dispatcherLeaderProcess.getConfirmLeaderSessionFuture(); dispatcherLeaderProcess.start(); @@ -235,7 +235,7 @@ public void closeAsync_duringJobRecovery_preventsDispatcherServiceCreation() thr }) .build(); - try (final DispatcherLeaderProcessImpl dispatcherLeaderProcess = createDispatcherLeaderProcess()) { + try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { dispatcherLeaderProcess.start(); jobRecoveryStarted.await(); @@ -269,7 +269,7 @@ public void onRemovedJobGraph_terminatesRunningJob() throws Exception { .setCreateFunction((dispatcherId, jobGraphs, jobGraphWriter) -> testingDispatcherService) .build(); - try (final DispatcherLeaderProcessImpl dispatcherLeaderProcess = createDispatcherLeaderProcess()) { + try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { dispatcherLeaderProcess.start(); // wait for the dispatcher process to be created @@ -295,7 +295,7 @@ public void onRemovedJobGraph_failingRemovalCall_failsFatally() throws Exception .setCreateFunction((dispatcherId, jobGraphs, jobGraphWriter) -> testingDispatcherService) .build(); - try (final DispatcherLeaderProcessImpl dispatcherLeaderProcess = createDispatcherLeaderProcess()) { + try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { dispatcherLeaderProcess.start(); // wait for the dispatcher process to be created @@ -325,7 +325,7 @@ public void onAddedJobGraph_submitsRecoveredJob() throws Exception { dispatcherServiceFactory = createDispatcherServiceFactoryFor(testingDispatcherGateway); - try (final DispatcherLeaderProcessImpl dispatcherLeaderProcess = createDispatcherLeaderProcess()) { + try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { dispatcherLeaderProcess.start(); // wait first for the dispatcher service to be created @@ -351,7 +351,7 @@ public void onAddedJobGraph_ifNotRunning_isBeingIgnored() throws Exception { }) .build(); - try (final DispatcherLeaderProcessImpl dispatcherLeaderProcess = createDispatcherLeaderProcess()) { + try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { dispatcherLeaderProcess.start(); // wait until the process has started the dispatcher @@ -381,7 +381,7 @@ public void onAddedJobGraph_failingRecovery_propagatesTheFailure() throws Except }) .build(); - try (final DispatcherLeaderProcessImpl dispatcherLeaderProcess = createDispatcherLeaderProcess()) { + try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { dispatcherLeaderProcess.start(); // wait first for the dispatcher service to be created @@ -394,7 +394,7 @@ public void onAddedJobGraph_failingRecovery_propagatesTheFailure() throws Except final Throwable throwable = errorFuture.get(); Assert.assertThat(ExceptionUtils.findThrowable(throwable, expectedFailure::equals).isPresent(), Is.is(true)); - assertThat(dispatcherLeaderProcess.getState(), is(DispatcherLeaderProcessImpl.State.STOPPED)); + assertThat(dispatcherLeaderProcess.getState(), is(SessionDispatcherLeaderProcess.State.STOPPED)); fatalErrorHandler.clearError(); } @@ -428,7 +428,7 @@ public void recoverJobs_withJobIdRecoveryFailure_failsFatally() throws Exception } private void runJobRecoveryFailureTest(FlinkException testException) throws Exception { - try (final DispatcherLeaderProcessImpl dispatcherLeaderProcess = createDispatcherLeaderProcess()) { + try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { dispatcherLeaderProcess.start(); // we expect that a fatal error occurred @@ -479,7 +479,7 @@ private void runOnAddedJobGraphTest(TestingDispatcherGateway dispatcherGateway, }) .build(); - try (final DispatcherLeaderProcessImpl dispatcherLeaderProcess = createDispatcherLeaderProcess()) { + try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { dispatcherLeaderProcess.start(); dispatcherLeaderProcess.getDispatcherGateway().get(); @@ -506,8 +506,8 @@ private TestingDispatcherServiceFactory createDispatcherServiceFactoryFor(Testin .build(); } - private DispatcherLeaderProcessImpl createDispatcherLeaderProcess() { - return DispatcherLeaderProcessImpl.create( + private SessionDispatcherLeaderProcess createDispatcherLeaderProcess() { + return SessionDispatcherLeaderProcess.create( leaderSessionId, dispatcherServiceFactory, jobGraphStore, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDispatcherRunnerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java similarity index 96% rename from flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDispatcherRunnerImplTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java index 68f074efe6ee..284d98deff21 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDispatcherRunnerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java @@ -75,11 +75,11 @@ import static org.junit.Assert.assertThat; /** - * Tests for the interaction between the {@link DispatcherRunnerImplNG} and ZooKeeper. + * Tests for the interaction between the {@link DefaultDispatcherRunner} and ZooKeeper. */ -public class ZooKeeperDispatcherRunnerImplTest extends TestLogger { +public class ZooKeeperDefaultDispatcherRunnerTest extends TestLogger { - private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperDispatcherRunnerImplTest.class); + private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperDefaultDispatcherRunnerTest.class); private static final Time TESTING_TIMEOUT = Time.seconds(10L); @@ -156,14 +156,14 @@ public void testResourceCleanupUnderLeadershipChange() throws Exception { final JobGraph jobGraph = createJobGraphWithBlobs(); - final DispatcherRunnerImplNGFactory dispatcherRunnerImplNGFactory = DispatcherRunnerImplNGFactory.createSessionRunner(SessionDispatcherFactory.INSTANCE); + final DefaultDispatcherRunnerFactory defaultDispatcherRunnerFactory = DefaultDispatcherRunnerFactory.createSessionRunner(SessionDispatcherFactory.INSTANCE); try (final DispatcherRunner dispatcherRunner = createDispatcherRunner( rpcService, dispatcherLeaderElectionService, () -> createZooKeeperJobGraphStore(client), partialDispatcherServices, - dispatcherRunnerImplNGFactory)) { + defaultDispatcherRunnerFactory)) { // initial run DispatcherGateway dispatcherGateway = grantLeadership(dispatcherLeaderElectionService, dispatcherLeaderRetriever, dispatcherRunner); 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 b0b4ffa40fcb..4f2bd2fcada6 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 @@ -22,7 +22,7 @@ import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.dispatcher.MemoryArchivedExecutionGraphStore; -import org.apache.flink.runtime.dispatcher.runner.DispatcherRunnerImplNGFactory; +import org.apache.flink.runtime.dispatcher.runner.DefaultDispatcherRunnerFactory; import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponent; import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponentFactory; import org.apache.flink.runtime.entrypoint.component.TestingDefaultDispatcherResourceManagerComponentFactory; @@ -131,7 +131,7 @@ public CompletableFuture getDispatcherGatewayFuture() { private DispatcherResourceManagerComponentFactory createTestingDispatcherResourceManagerComponentFactory() { return TestingDefaultDispatcherResourceManagerComponentFactory.createSessionComponentFactory( - DispatcherRunnerImplNGFactory.createSessionRunner(SessionDispatcherWithUUIDFactory.INSTANCE), + DefaultDispatcherRunnerFactory.createSessionRunner(SessionDispatcherWithUUIDFactory.INSTANCE), StandaloneResourceManagerWithUUIDFactory.INSTANCE); } } From 78583463e840699d62b844a82a89ca9a4a706eba Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 26 Sep 2019 13:31:06 +0200 Subject: [PATCH 279/746] [FLINK-11843] Move DispatcherRunnerImplTest#testJobRecoveryUnderLeaderChange to DefaultDispatcherRunnerITCase The DispatcherRunnerImplTest#testJobRecoveryUnderLeaderChange has been moved to the DefaultDispatcherRunnerITCase#leaderChange_withBlockingJobManagerTermination_doesNotAffectNewLeader. --- .../DispatcherResourceCleanupTest.java | 2 +- .../runtime/dispatcher/DispatcherTest.java | 2 +- .../runner/DefaultDispatcherRunnerITCase.java | 88 +++++++- .../runner/DispatcherRunnerImplTest.java | 206 ------------------ 4 files changed, 86 insertions(+), 212 deletions(-) delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java 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 36110be2fb7e..6b137be6dcf5 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 @@ -204,11 +204,11 @@ private void startDispatcher(JobManagerRunnerFactory jobManagerRunnerFactory) th () -> CompletableFuture.completedFuture(resourceManagerGateway), blobServer, heartbeatServices, - UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), archivedExecutionGraphStore, fatalErrorHandler, VoidHistoryServerArchivist.INSTANCE, null, + UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), jobGraphWriter, jobManagerRunnerFactory)); 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 0bf6722a096b..f136a8cdfb6e 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 @@ -254,11 +254,11 @@ TestingDispatcher build() throws Exception { () -> CompletableFuture.completedFuture(resourceManagerGateway), blobServer, heartbeatServices, - UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), archivedExecutionGraphStore, fatalErrorHandler, VoidHistoryServerArchivist.INSTANCE, null, + UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), jobGraphWriter, jobManagerRunnerFactory)); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java index 8617996370be..eb6fc5d19756 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java @@ -21,18 +21,29 @@ 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.dispatcher.Dispatcher; +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.DispatcherServices; +import org.apache.flink.runtime.dispatcher.JobManagerRunnerFactory; import org.apache.flink.runtime.dispatcher.MemoryArchivedExecutionGraphStore; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobGraphStore; +import org.apache.flink.runtime.dispatcher.SingleJobJobGraphStore; +import org.apache.flink.runtime.dispatcher.StandaloneDispatcher; +import org.apache.flink.runtime.dispatcher.TestingJobManagerRunnerFactory; import org.apache.flink.runtime.dispatcher.VoidHistoryServerArchivist; import org.apache.flink.runtime.heartbeat.TestingHeartbeatServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServicesBuilder; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobmanager.JobGraphStore; +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.minicluster.SessionDispatcherWithUUIDFactory; +import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.TestingRpcServiceResource; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testtasks.NoOpInvokable; @@ -45,13 +56,20 @@ import org.junit.Before; import org.junit.ClassRule; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; import java.util.Collection; import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; /** @@ -59,6 +77,8 @@ */ public class DefaultDispatcherRunnerITCase extends TestLogger { + private static final Logger LOG = LoggerFactory.getLogger(DefaultDispatcherRunnerITCase.class); + private static final Time TIMEOUT = Time.seconds(10L); private static final JobID TEST_JOB_ID = new JobID(); @@ -79,8 +99,11 @@ public class DefaultDispatcherRunnerITCase extends TestLogger { private PartialDispatcherServices partialDispatcherServices; + private DefaultDispatcherRunnerFactory dispatcherRunnerFactory; + @Before public void setup() { + dispatcherRunnerFactory = DefaultDispatcherRunnerFactory.createSessionRunner(SessionDispatcherWithUUIDFactory.INSTANCE); jobGraph = createJobGraph(); dispatcherLeaderElectionService = new TestingLeaderElectionService(); fatalErrorHandler = new TestingFatalErrorHandler(); @@ -92,7 +115,7 @@ public void setup() { CompletableFuture::new, blobServerResource.getBlobServer(), new TestingHeartbeatServices(), - UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), + UnregisteredMetricGroups::createUnregisteredJobManagerMetricGroup, new MemoryArchivedExecutionGraphStore(), fatalErrorHandler, VoidHistoryServerArchivist.INSTANCE, @@ -131,6 +154,65 @@ public void leaderChange_afterJobSubmission_recoversSubmittedJob() throws Except } } + /** + * See FLINK-11843. This is a probabilistic test which needs to be executed several times to fail. + */ + @Test + public void leaderChange_withBlockingJobManagerTermination_doesNotAffectNewLeader() throws Exception { + final TestingJobManagerRunnerFactory jobManagerRunnerFactory = new TestingJobManagerRunnerFactory(1); + dispatcherRunnerFactory = DefaultDispatcherRunnerFactory.createSessionRunner(new TestingDispatcherFactory(jobManagerRunnerFactory)); + jobGraphStore = new SingleJobJobGraphStore(jobGraph); + + try (final DispatcherRunner dispatcherRunner = createDispatcherRunner()) { + + // initial run + dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); + final TestingJobManagerRunner testingJobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); + + dispatcherLeaderElectionService.notLeader(); + + LOG.info("Re-grant leadership first time."); + dispatcherLeaderElectionService.isLeader(UUID.randomUUID()); + + // give the Dispatcher some time to recover jobs + Thread.sleep(1L); + + dispatcherLeaderElectionService.notLeader(); + + LOG.info("Re-grant leadership second time."); + final UUID leaderSessionId = UUID.randomUUID(); + final CompletableFuture leaderFuture = dispatcherLeaderElectionService.isLeader(leaderSessionId); + assertThat(leaderFuture.isDone(), is(false)); + + LOG.info("Complete the termination of the first job manager runner."); + testingJobManagerRunner.completeTerminationFuture(); + + assertThat(leaderFuture.get(TIMEOUT.toMilliseconds(), TimeUnit.MILLISECONDS), is(equalTo(leaderSessionId))); + } + } + + private static class TestingDispatcherFactory implements DispatcherFactory { + private final JobManagerRunnerFactory jobManagerRunnerFactory; + + private TestingDispatcherFactory(JobManagerRunnerFactory jobManagerRunnerFactory) { + this.jobManagerRunnerFactory = jobManagerRunnerFactory; + } + + @Override + public Dispatcher createDispatcher( + @Nonnull RpcService rpcService, + @Nonnull DispatcherId fencingToken, + @Nonnull Collection recoveredJobs, + @Nonnull PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { + return new StandaloneDispatcher( + rpcService, + generateEndpointIdWithUUID(), + fencingToken, + recoveredJobs, + DispatcherServices.from(partialDispatcherServicesWithJobGraphStore, jobManagerRunnerFactory)); + } + } + private static JobGraph createJobGraph() { final JobVertex testVertex = new JobVertex("testVertex"); testVertex.setInvokableClass(NoOpInvokable.class); @@ -141,9 +223,7 @@ private static JobGraph createJobGraph() { } private DefaultDispatcherRunner createDispatcherRunner() throws Exception { - final DefaultDispatcherRunnerFactory runnerFactory = DefaultDispatcherRunnerFactory.createSessionRunner(SessionDispatcherWithUUIDFactory.INSTANCE); - - return runnerFactory.createDispatcherRunner( + return dispatcherRunnerFactory.createDispatcherRunner( dispatcherLeaderElectionService, fatalErrorHandler, () -> jobGraphStore, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java deleted file mode 100644 index e94a96c49b5c..000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerImplTest.java +++ /dev/null @@ -1,206 +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.runner; - -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.blob.VoidBlobStore; -import org.apache.flink.runtime.dispatcher.Dispatcher; -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.DispatcherServices; -import org.apache.flink.runtime.dispatcher.JobManagerRunnerFactory; -import org.apache.flink.runtime.dispatcher.MemoryArchivedExecutionGraphStore; -import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; -import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobGraphStore; -import org.apache.flink.runtime.dispatcher.SingleJobJobGraphStore; -import org.apache.flink.runtime.dispatcher.StandaloneDispatcher; -import org.apache.flink.runtime.dispatcher.TestingJobManagerRunnerFactory; -import org.apache.flink.runtime.dispatcher.VoidHistoryServerArchivist; -import org.apache.flink.runtime.heartbeat.TestingHeartbeatServices; -import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; -import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServicesBuilder; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; -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.rpc.RpcService; -import org.apache.flink.runtime.rpc.TestingRpcService; -import org.apache.flink.runtime.rpc.TestingRpcServiceResource; -import org.apache.flink.runtime.testingUtils.TestingUtils; -import org.apache.flink.runtime.util.TestingFatalErrorHandler; -import org.apache.flink.util.TestLogger; - -import org.junit.After; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.annotation.Nonnull; - -import java.util.Collection; -import java.util.UUID; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; - -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.is; -import static org.junit.Assert.assertThat; - -/** - * Tests for the {@link DefaultDispatcherRunner}. - */ -public class DispatcherRunnerImplTest extends TestLogger { - - private static final Logger LOG = LoggerFactory.getLogger(DispatcherRunnerImplTest.class); - - private static final Time TESTING_TIMEOUT = Time.seconds(10L); - - @ClassRule - public static TestingRpcServiceResource testingRpcServiceResource = new TestingRpcServiceResource(); - - private TestingFatalErrorHandler fatalErrorHandler; - - @Before - public void setup() { - fatalErrorHandler = new TestingFatalErrorHandler(); - } - - @After - public void teardown() throws Exception { - if (fatalErrorHandler != null) { - fatalErrorHandler.rethrowError(); - } - } - - /** - * See FLINK-11843. This is a probabilistic test which needs to be executed several times to fail. - */ - @Test - public void testJobRecoveryUnderLeaderChange() throws Exception { - final TestingRpcService rpcService = testingRpcServiceResource.getTestingRpcService(); - final TestingLeaderElectionService dispatcherLeaderElectionService = new TestingLeaderElectionService(); - final Configuration configuration = new Configuration(); - - final JobGraph jobGraph = new JobGraph(); - - try (final BlobServer blobServer = new BlobServer(configuration, new VoidBlobStore()); - final TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServicesBuilder() - .setDispatcherLeaderElectionService(dispatcherLeaderElectionService) - .build()) { - - final PartialDispatcherServices partialDispatcherServices = new PartialDispatcherServices( - configuration, - highAvailabilityServices, - CompletableFuture::new, - blobServer, - new TestingHeartbeatServices(), - UnregisteredMetricGroups::createUnregisteredJobManagerMetricGroup, - new MemoryArchivedExecutionGraphStore(), - fatalErrorHandler, - VoidHistoryServerArchivist.INSTANCE, - null); - - final TestingJobManagerRunnerFactory jobManagerRunnerFactory = new TestingJobManagerRunnerFactory(1); - - final DefaultDispatcherRunnerFactory dispatcherRunnerFactory = DefaultDispatcherRunnerFactory.createSessionRunner(new TestingDispatcherFactory(jobManagerRunnerFactory)); - - try (final DispatcherRunner dispatcherRunner = createDispatcherRunner( - rpcService, - dispatcherLeaderElectionService, - partialDispatcherServices, - dispatcherRunnerFactory, - () -> new SingleJobJobGraphStore(jobGraph))) { - // initial run - grantLeadership(dispatcherLeaderElectionService, dispatcherRunner); - final TestingJobManagerRunner testingJobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); - - dispatcherLeaderElectionService.notLeader(); - - LOG.info("Re-grant leadership first time."); - dispatcherLeaderElectionService.isLeader(UUID.randomUUID()); - - // give the Dispatcher some time to recover jobs - Thread.sleep(1L); - - dispatcherLeaderElectionService.notLeader(); - - LOG.info("Re-grant leadership second time."); - final UUID leaderSessionId = UUID.randomUUID(); - final CompletableFuture leaderFuture = dispatcherLeaderElectionService.isLeader(leaderSessionId); - - LOG.info("Complete the termination of the first job manager runner."); - testingJobManagerRunner.completeTerminationFuture(); - - assertThat(leaderFuture.get(TESTING_TIMEOUT.toMilliseconds(), TimeUnit.MILLISECONDS), is(equalTo(leaderSessionId))); - } - } - } - - private DispatcherRunner createDispatcherRunner( - TestingRpcService rpcService, - TestingLeaderElectionService dispatcherLeaderElectionService, - PartialDispatcherServices partialDispatcherServices, - DispatcherRunnerFactory dispatcherRunnerFactory, - JobGraphStoreFactory jobGraphStoreFactory) throws Exception { - return dispatcherRunnerFactory.createDispatcherRunner( - dispatcherLeaderElectionService, - fatalErrorHandler, - jobGraphStoreFactory, - TestingUtils.defaultExecutor(), - rpcService, - partialDispatcherServices); - } - - private DispatcherGateway grantLeadership( - TestingLeaderElectionService dispatcherLeaderElectionService, - DispatcherRunner dispatcherRunner) throws InterruptedException, java.util.concurrent.ExecutionException { - final UUID leaderSessionId = UUID.randomUUID(); - dispatcherLeaderElectionService.isLeader(leaderSessionId).get(); - - return dispatcherRunner.getDispatcherGateway().get(); - } - - private static class TestingDispatcherFactory implements DispatcherFactory { - private final JobManagerRunnerFactory jobManagerRunnerFactory; - - private TestingDispatcherFactory(JobManagerRunnerFactory jobManagerRunnerFactory) { - this.jobManagerRunnerFactory = jobManagerRunnerFactory; - } - - @Override - public Dispatcher createDispatcher( - @Nonnull RpcService rpcService, - @Nonnull DispatcherId fencingToken, - @Nonnull Collection recoveredJobs, - @Nonnull PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { - return new StandaloneDispatcher( - rpcService, - generateEndpointIdWithUUID(), - fencingToken, - recoveredJobs, - DispatcherServices.from(partialDispatcherServicesWithJobGraphStore, jobManagerRunnerFactory)); - } - } -} From 717b77c6651c37d0f48524a9767f19025ba23491 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 19 Oct 2019 16:25:30 +0200 Subject: [PATCH 280/746] [FLINK-11843] Various cleanups for DefaultDispatcherRunner and tests --- .../runner/DefaultDispatcherRunner.java | 32 ++++++----- .../DefaultDispatcherRunnerFactory.java | 4 +- .../StoppedDispatcherLeaderProcess.java | 13 ++--- .../runner/DefaultDispatcherRunnerITCase.java | 24 +++++--- .../runner/DefaultDispatcherRunnerTest.java | 57 +++++++++---------- 5 files changed, 68 insertions(+), 62 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java index 92faf4d1d794..13ea88016977 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java @@ -37,7 +37,7 @@ * Runner for the {@link org.apache.flink.runtime.dispatcher.Dispatcher} which is responsible for the * leader election. */ -public class DefaultDispatcherRunner implements DispatcherRunner, LeaderContender { +public final class DefaultDispatcherRunner implements DispatcherRunner, LeaderContender { private static final Logger LOG = LoggerFactory.getLogger(DefaultDispatcherRunner.class); @@ -53,7 +53,7 @@ public class DefaultDispatcherRunner implements DispatcherRunner, LeaderContende private final CompletableFuture shutDownFuture; - private boolean isRunning; + private boolean running; private DispatcherLeaderProcess dispatcherLeaderProcess; @@ -61,7 +61,7 @@ public class DefaultDispatcherRunner implements DispatcherRunner, LeaderContende private CompletableFuture dispatcherGatewayFuture; - DefaultDispatcherRunner( + private DefaultDispatcherRunner( LeaderElectionService leaderElectionService, FatalErrorHandler fatalErrorHandler, DispatcherLeaderProcessFactory dispatcherLeaderProcessFactory) throws Exception { @@ -71,7 +71,7 @@ public class DefaultDispatcherRunner implements DispatcherRunner, LeaderContende this.terminationFuture = new CompletableFuture<>(); this.shutDownFuture = new CompletableFuture<>(); - this.isRunning = true; + this.running = true; this.dispatcherLeaderProcess = StoppedDispatcherLeaderProcess.INSTANCE; this.previousDispatcherLeaderProcessTerminationFuture = CompletableFuture.completedFuture(null); this.dispatcherGatewayFuture = new CompletableFuture<>(); @@ -100,10 +100,10 @@ public CompletableFuture getShutDownFuture() { @Override public CompletableFuture closeAsync() { synchronized (lock) { - if (!isRunning) { + if (!running) { return terminationFuture; } else { - isRunning = false; + running = false; } } @@ -147,17 +147,13 @@ public void grantLeadership(UUID leaderSessionID) { private void startNewDispatcherLeaderProcess(UUID leaderSessionID) { stopDispatcherLeaderProcess(); - createAndAssignNewDispatcherLeaderProcess(leaderSessionID); + dispatcherLeaderProcess = createNewDispatcherLeaderProcess(leaderSessionID); final DispatcherLeaderProcess newDispatcherLeaderProcess = dispatcherLeaderProcess; FutureUtils.assertNoException( previousDispatcherLeaderProcessTerminationFuture.thenRun(newDispatcherLeaderProcess::start)); } - private void createAndAssignNewDispatcherLeaderProcess(UUID leaderSessionID) { - dispatcherLeaderProcess = createNewDispatcherLeaderProcess(leaderSessionID); - } - private void stopDispatcherLeaderProcess() { final CompletableFuture terminationFuture = dispatcherLeaderProcess.closeAsync(); previousDispatcherLeaderProcessTerminationFuture = FutureUtils.completeAll( @@ -189,7 +185,7 @@ private void forwardShutDownFuture(DispatcherLeaderProcess newDispatcherLeaderPr (applicationStatus, throwable) -> { synchronized (lock) { // ignore if no longer running or if leader processes is no longer valid - if (isRunning && this.dispatcherLeaderProcess == newDispatcherLeaderProcess) { + if (running && this.dispatcherLeaderProcess == newDispatcherLeaderProcess) { if (throwable != null) { shutDownFuture.completeExceptionally(throwable); } else { @@ -217,7 +213,7 @@ public void revokeLeadership() { private void runActionIfRunning(Runnable runnable) { synchronized (lock) { - if (isRunning) { + if (running) { runnable.run(); } else { LOG.debug("Ignoring action because {} has already been stopped.", getClass().getSimpleName()); @@ -232,4 +228,14 @@ public void handleError(Exception exception) { String.format("Exception during leader election of %s occurred.", getClass().getSimpleName()), exception)); } + + public static DispatcherRunner create( + LeaderElectionService leaderElectionService, + FatalErrorHandler fatalErrorHandler, + DispatcherLeaderProcessFactory dispatcherLeaderProcessFactory) throws Exception { + return new DefaultDispatcherRunner( + leaderElectionService, + fatalErrorHandler, + dispatcherLeaderProcessFactory); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerFactory.java index 3e4027f98362..84ba3f49f570 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerFactory.java @@ -40,7 +40,7 @@ private DefaultDispatcherRunnerFactory(DispatcherLeaderProcessFactoryFactory dis } @Override - public DefaultDispatcherRunner createDispatcherRunner( + public DispatcherRunner createDispatcherRunner( LeaderElectionService leaderElectionService, FatalErrorHandler fatalErrorHandler, JobGraphStoreFactory jobGraphStoreFactory, @@ -55,7 +55,7 @@ public DefaultDispatcherRunner createDispatcherRunner( partialDispatcherServices, fatalErrorHandler); - return new DefaultDispatcherRunner( + return DefaultDispatcherRunner.create( leaderElectionService, fatalErrorHandler, dispatcherLeaderProcessFactory); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StoppedDispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StoppedDispatcherLeaderProcess.java index f4a5a7cff07f..a84eb0a50dac 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StoppedDispatcherLeaderProcess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StoppedDispatcherLeaderProcess.java @@ -32,33 +32,30 @@ public enum StoppedDispatcherLeaderProcess implements DispatcherLeaderProcess { INSTANCE; private static final CompletableFuture TERMINATION_FUTURE = CompletableFuture.completedFuture(null); - private static final UUID LEADER_SESSION_ID = new UUID(0L, 0L); - private static final CompletableFuture NEVER_COMPLETED_LEADER_SESSION_FUTURE = new CompletableFuture<>(); - private static final CompletableFuture NEVER_COMPLETED_SHUTDOWN_FUTURE = new CompletableFuture<>(); @Override public void start() { - + throw new UnsupportedOperationException("This method should never be called."); } @Override public UUID getLeaderSessionId() { - return LEADER_SESSION_ID; + throw new UnsupportedOperationException("This method should never be called."); } @Override public CompletableFuture getDispatcherGateway() { - return null; + throw new UnsupportedOperationException("This method should never be called."); } @Override public CompletableFuture getConfirmLeaderSessionFuture() { - return NEVER_COMPLETED_LEADER_SESSION_FUTURE; + throw new UnsupportedOperationException("This method should never be called."); } @Override public CompletableFuture getShutDownFuture() { - return NEVER_COMPLETED_SHUTDOWN_FUTURE; + throw new UnsupportedOperationException("This method should never be called."); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java index eb6fc5d19756..98be0e154df2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java @@ -49,6 +49,7 @@ import org.apache.flink.runtime.testtasks.NoOpInvokable; import org.apache.flink.runtime.testutils.TestingJobGraphStore; import org.apache.flink.runtime.util.BlobServerResource; +import org.apache.flink.runtime.util.LeaderConnectionInfo; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.util.TestLogger; @@ -131,29 +132,34 @@ public void teardown() throws Exception { @Test public void leaderChange_afterJobSubmission_recoversSubmittedJob() throws Exception { - try (final DefaultDispatcherRunner dispatcherRunner = createDispatcherRunner()) { + try (final DispatcherRunner dispatcherRunner = createDispatcherRunner()) { final UUID firstLeaderSessionId = UUID.randomUUID(); - dispatcherLeaderElectionService.isLeader(firstLeaderSessionId); - - final DispatcherGateway firstDispatcherGateway = dispatcherRunner.getDispatcherGateway().get(); + final DispatcherGateway firstDispatcherGateway = electLeaderAndRetrieveGateway(firstLeaderSessionId); firstDispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); dispatcherLeaderElectionService.notLeader(); final UUID secondLeaderSessionId = UUID.randomUUID(); - dispatcherLeaderElectionService.isLeader(secondLeaderSessionId).get(); - - final DispatcherGateway secondDispatcherGateway = dispatcherRunner.getDispatcherGateway().get(); + final DispatcherGateway secondDispatcherGateway = electLeaderAndRetrieveGateway(secondLeaderSessionId); final Collection jobIds = secondDispatcherGateway.listJobs(TIMEOUT).get(); - assertThat(jobIds, hasSize(1)); assertThat(jobIds, contains(jobGraph.getJobID())); } } + private DispatcherGateway electLeaderAndRetrieveGateway(UUID firstLeaderSessionId) throws InterruptedException, java.util.concurrent.ExecutionException { + dispatcherLeaderElectionService.isLeader(firstLeaderSessionId); + final LeaderConnectionInfo leaderConnectionInfo = dispatcherLeaderElectionService.getConfirmationFuture().get(); + + return rpcServiceResource.getTestingRpcService().connect( + leaderConnectionInfo.getAddress(), + DispatcherId.fromUuid(leaderConnectionInfo.getLeaderSessionId()), + DispatcherGateway.class).get(); + } + /** * See FLINK-11843. This is a probabilistic test which needs to be executed several times to fail. */ @@ -222,7 +228,7 @@ private static JobGraph createJobGraph() { return testJob; } - private DefaultDispatcherRunner createDispatcherRunner() throws Exception { + private DispatcherRunner createDispatcherRunner() throws Exception { return dispatcherRunnerFactory.createDispatcherRunner( dispatcherLeaderElectionService, fatalErrorHandler, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerTest.java index 48c95aef00e3..bfc5f80c4475 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerTest.java @@ -18,9 +18,9 @@ package org.apache.flink.runtime.dispatcher.runner; +import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.concurrent.FutureUtils; -import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.dispatcher.DispatcherId; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.util.LeaderConnectionInfo; @@ -73,7 +73,7 @@ public void teardown() throws Exception { @Test public void closeAsync_doesNotCompleteUncompletedShutDownFuture() throws Exception { - final DefaultDispatcherRunner dispatcherRunner = createDispatcherRunner(); + final DispatcherRunner dispatcherRunner = createDispatcherRunner(); final CompletableFuture terminationFuture = dispatcherRunner.closeAsync(); terminationFuture.get(); @@ -91,7 +91,7 @@ public void getDispatcherGateway_beforeDispatcherLeaderProcessCompletes_returnsD .build(); testingDispatcherLeaderProcessFactory = TestingDispatcherLeaderProcessFactory.from(testingDispatcherLeaderProcess); - try (final DefaultDispatcherRunner dispatcherRunner = createDispatcherRunner()) { + try (final DispatcherRunner dispatcherRunner = createDispatcherRunner()) { final CompletableFuture dispatcherGatewayFuture = dispatcherRunner.getDispatcherGateway(); @@ -121,7 +121,7 @@ public void getDispatcherGateway_withChangingLeaders_returnsLeadingDispatcherGat firstDispatcherLeaderProcess, secondDispatcherLeaderProcess); - try (final DefaultDispatcherRunner dispatcherRunner = createDispatcherRunner()) { + try (final DispatcherRunner dispatcherRunner = createDispatcherRunner()) { testingLeaderElectionService.isLeader(firstLeaderSessionId); final CompletableFuture firstDispatcherGatewayFuture = dispatcherRunner.getDispatcherGateway(); @@ -145,7 +145,7 @@ public void getShutDownFuture_whileRunning_forwardsDispatcherLeaderProcessShutDo .build(); testingDispatcherLeaderProcessFactory = TestingDispatcherLeaderProcessFactory.from(testingDispatcherLeaderProcess); - try (final DefaultDispatcherRunner dispatcherRunner = createDispatcherRunner()) { + try (final DispatcherRunner dispatcherRunner = createDispatcherRunner()) { testingLeaderElectionService.isLeader(leaderSessionId); final CompletableFuture dispatcherShutDownFuture = dispatcherRunner.getShutDownFuture(); @@ -168,7 +168,7 @@ public void getShutDownFuture_afterClose_ignoresDispatcherLeaderProcessShutDownR .build(); testingDispatcherLeaderProcessFactory = TestingDispatcherLeaderProcessFactory.from(testingDispatcherLeaderProcess); - try (final DefaultDispatcherRunner dispatcherRunner = createDispatcherRunner()) { + try (final DispatcherRunner dispatcherRunner = createDispatcherRunner()) { testingLeaderElectionService.isLeader(leaderSessionId); final CompletableFuture dispatcherShutDownFuture = dispatcherRunner.getShutDownFuture(); @@ -201,7 +201,7 @@ public void getShutDownFuture_newLeader_ignoresOldDispatcherLeaderProcessShutDow firstTestingDispatcherLeaderProcess, secondTestingDispatcherLeaderProcess); - try (final DefaultDispatcherRunner dispatcherRunner = createDispatcherRunner()) { + try (final DispatcherRunner dispatcherRunner = createDispatcherRunner()) { testingLeaderElectionService.isLeader(firstLeaderSessionId); final CompletableFuture dispatcherShutDownFuture = dispatcherRunner.getShutDownFuture(); @@ -213,10 +213,7 @@ public void getShutDownFuture_newLeader_ignoresOldDispatcherLeaderProcessShutDow final ApplicationStatus finalApplicationStatus = ApplicationStatus.UNKNOWN; shutDownFuture.complete(finalApplicationStatus); - try { - dispatcherShutDownFuture.get(10L, TimeUnit.MILLISECONDS); - fail("The dispatcher runner should no longer react to the dispatcher leader process's shut down request if it has been terminated."); - } catch (TimeoutException expected) {} + assertFalse(dispatcherShutDownFuture.isDone()); } } @@ -224,27 +221,27 @@ public void getShutDownFuture_newLeader_ignoresOldDispatcherLeaderProcessShutDow public void revokeLeadership_withExistingLeader_stopsLeaderProcess() throws Exception { final UUID leaderSessionId = UUID.randomUUID(); - final CompletableFuture startFuture = new CompletableFuture<>(); - final CompletableFuture stopFuture = new CompletableFuture<>(); + final OneShotLatch startLatch = new OneShotLatch(); + final OneShotLatch stopLatch = new OneShotLatch(); testingDispatcherLeaderProcessFactory = TestingDispatcherLeaderProcessFactory.from( TestingDispatcherLeaderProcess.newBuilder(leaderSessionId) - .setStartConsumer(startFuture::complete) + .setStartConsumer(ignored -> startLatch.trigger()) .setCloseAsyncSupplier( () -> { - stopFuture.complete(null); + stopLatch.trigger(); return FutureUtils.completedVoidFuture(); }) .build()); - try (final DefaultDispatcherRunner dispatcherRunner = createDispatcherRunner()) { + try (final DispatcherRunner dispatcherRunner = createDispatcherRunner()) { testingLeaderElectionService.isLeader(leaderSessionId); // wait until the leader process has been started - startFuture.get(); + startLatch.await(); testingLeaderElectionService.notLeader(); // verify that the leader gets stopped - stopFuture.get(); + stopLatch.await(); } } @@ -253,14 +250,14 @@ public void grantLeadership_withExistingLeader_waitsForTerminationOfFirstLeader( final UUID firstLeaderSessionId = UUID.randomUUID(); final UUID secondLeaderSessionId = UUID.randomUUID(); - final StartStopTestingDispatcherLeaderProcess firstTestingDispatcherLeaderProcess = StartStopTestingDispatcherLeaderProcess.create(firstLeaderSessionId); - final StartStopTestingDispatcherLeaderProcess secondTestingDispatcherLeaderProcess = StartStopTestingDispatcherLeaderProcess.create(secondLeaderSessionId); + final StartStopDispatcherLeaderProcess firstTestingDispatcherLeaderProcess = StartStopDispatcherLeaderProcess.create(firstLeaderSessionId); + final StartStopDispatcherLeaderProcess secondTestingDispatcherLeaderProcess = StartStopDispatcherLeaderProcess.create(secondLeaderSessionId); testingDispatcherLeaderProcessFactory = TestingDispatcherLeaderProcessFactory.from( firstTestingDispatcherLeaderProcess.asTestingDispatcherLeaderProcess(), secondTestingDispatcherLeaderProcess.asTestingDispatcherLeaderProcess()); - try (final DefaultDispatcherRunner dispatcherRunner = createDispatcherRunner()) { + try (final DispatcherRunner dispatcherRunner = createDispatcherRunner()) { testingLeaderElectionService.isLeader(firstLeaderSessionId); assertThat(firstTestingDispatcherLeaderProcess.isStarted(), is(true)); @@ -278,7 +275,7 @@ public void grantLeadership_withExistingLeader_waitsForTerminationOfFirstLeader( public void grantLeadership_validLeader_confirmsLeaderSession() throws Exception { final UUID leaderSessionId = UUID.randomUUID(); - try (final DefaultDispatcherRunner dispatcherRunner = createDispatcherRunner()) { + try (final DispatcherRunner dispatcherRunner = createDispatcherRunner()) { testingLeaderElectionService.isLeader(leaderSessionId); final CompletableFuture confirmationFuture = testingLeaderElectionService.getConfirmationFuture(); @@ -298,7 +295,7 @@ public void grantLeadership_oldLeader_doesNotConfirmLeaderSession() throws Excep testingDispatcherLeaderProcessFactory = TestingDispatcherLeaderProcessFactory.from(testingDispatcherLeaderProcess); - try (final DefaultDispatcherRunner dispatcherRunner = createDispatcherRunner()) { + try (final DispatcherRunner dispatcherRunner = createDispatcherRunner()) { testingLeaderElectionService.isLeader(leaderSessionId); testingLeaderElectionService.notLeader(); @@ -339,7 +336,7 @@ public void grantLeadership_multipleLeaderChanges_lastDispatcherLeaderProcessWai secondDispatcherLeaderProcess, thirdDispatcherLeaderProcess); - final DefaultDispatcherRunner dispatcherRunner = createDispatcherRunner(); + final DispatcherRunner dispatcherRunner = createDispatcherRunner(); try { testingLeaderElectionService.isLeader(firstLeaderSession); @@ -361,13 +358,13 @@ public void grantLeadership_multipleLeaderChanges_lastDispatcherLeaderProcessWai } } - private static class StartStopTestingDispatcherLeaderProcess { + private static final class StartStopDispatcherLeaderProcess { private final TestingDispatcherLeaderProcess testingDispatcherLeaderProcess; private final CompletableFuture startFuture; private final CompletableFuture terminationFuture; - private StartStopTestingDispatcherLeaderProcess( + private StartStopDispatcherLeaderProcess( TestingDispatcherLeaderProcess testingDispatcherLeaderProcess, CompletableFuture startFuture, CompletableFuture terminationFuture) { @@ -388,7 +385,7 @@ private void terminateProcess() { terminationFuture.complete(null); } - private static StartStopTestingDispatcherLeaderProcess create(UUID leaderSessionId) { + private static StartStopDispatcherLeaderProcess create(UUID leaderSessionId) { final CompletableFuture processStartFuture = new CompletableFuture<>(); final CompletableFuture processTerminationFuture = new CompletableFuture<>(); final TestingDispatcherLeaderProcess dispatcherLeaderProcess = TestingDispatcherLeaderProcess.newBuilder(leaderSessionId) @@ -396,7 +393,7 @@ private static StartStopTestingDispatcherLeaderProcess create(UUID leaderSession .setCloseAsyncSupplier(() -> processTerminationFuture) .build(); - return new StartStopTestingDispatcherLeaderProcess(dispatcherLeaderProcess, processStartFuture, processTerminationFuture); + return new StartStopDispatcherLeaderProcess(dispatcherLeaderProcess, processStartFuture, processTerminationFuture); } } @@ -406,8 +403,8 @@ private TestingDispatcherGateway createDispatcherGateway(UUID leaderSessionId) { .build(); } - private DefaultDispatcherRunner createDispatcherRunner() throws Exception { - return new DefaultDispatcherRunner( + private DispatcherRunner createDispatcherRunner() throws Exception { + return DefaultDispatcherRunner.create( testingLeaderElectionService, testingFatalErrorHandler, testingDispatcherLeaderProcessFactory); From 06e79c8eb577b61e1ebaac3dc97d98057c431d92 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 21 Oct 2019 17:10:06 +0200 Subject: [PATCH 281/746] [FLINK-11843] Introduce DispatcherRunnerLeaderElectionLifecycleManager The DispatcherRunnerLeaderElectionLifecycleManager is responsible for stopping the LeaderElectionService. --- .../runner/DefaultDispatcherRunner.java | 34 ++------- ...rRunnerLeaderElectionLifecycleManager.java | 72 +++++++++++++++++++ 2 files changed, 76 insertions(+), 30 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerLeaderElectionLifecycleManager.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java index 13ea88016977..4e224b989234 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java @@ -64,7 +64,7 @@ public final class DefaultDispatcherRunner implements DispatcherRunner, LeaderCo private DefaultDispatcherRunner( LeaderElectionService leaderElectionService, FatalErrorHandler fatalErrorHandler, - DispatcherLeaderProcessFactory dispatcherLeaderProcessFactory) throws Exception { + DispatcherLeaderProcessFactory dispatcherLeaderProcessFactory) { this.leaderElectionService = leaderElectionService; this.fatalErrorHandler = fatalErrorHandler; this.dispatcherLeaderProcessFactory = dispatcherLeaderProcessFactory; @@ -75,14 +75,6 @@ private DefaultDispatcherRunner( this.dispatcherLeaderProcess = StoppedDispatcherLeaderProcess.INSTANCE; this.previousDispatcherLeaderProcessTerminationFuture = CompletableFuture.completedFuture(null); this.dispatcherGatewayFuture = new CompletableFuture<>(); - - startDispatcherRunner(leaderElectionService); - } - - private void startDispatcherRunner(LeaderElectionService leaderElectionService) throws Exception { - LOG.info("Starting {}.", getClass().getName()); - - leaderElectionService.start(this); } @Override @@ -108,33 +100,14 @@ public CompletableFuture closeAsync() { } stopDispatcherLeaderProcess(); - final CompletableFuture servicesTerminationFuture = stopServices(); FutureUtils.forward( - FutureUtils.completeAll( - Arrays.asList( - previousDispatcherLeaderProcessTerminationFuture, - servicesTerminationFuture)), + previousDispatcherLeaderProcessTerminationFuture, terminationFuture); return terminationFuture; } - private CompletableFuture stopServices() { - Exception exception = null; - - try { - leaderElectionService.stop(); - } catch (Exception e) { - exception = e; - } - if (exception == null) { - return CompletableFuture.completedFuture(null); - } else { - return FutureUtils.completedExceptionally(exception); - } - } - // --------------------------------------------------------------- // Leader election // --------------------------------------------------------------- @@ -233,9 +206,10 @@ public static DispatcherRunner create( LeaderElectionService leaderElectionService, FatalErrorHandler fatalErrorHandler, DispatcherLeaderProcessFactory dispatcherLeaderProcessFactory) throws Exception { - return new DefaultDispatcherRunner( + final DefaultDispatcherRunner dispatcherRunner = new DefaultDispatcherRunner( leaderElectionService, fatalErrorHandler, dispatcherLeaderProcessFactory); + return DispatcherRunnerLeaderElectionLifecycleManager.createFor(dispatcherRunner, leaderElectionService); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerLeaderElectionLifecycleManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerLeaderElectionLifecycleManager.java new file mode 100644 index 000000000000..eea819cfb4dd --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerLeaderElectionLifecycleManager.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.runtime.dispatcher.runner; + +import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; +import org.apache.flink.runtime.leaderelection.LeaderContender; +import org.apache.flink.runtime.leaderelection.LeaderElectionService; + +import java.util.Arrays; +import java.util.concurrent.CompletableFuture; + +final class DispatcherRunnerLeaderElectionLifecycleManager implements DispatcherRunner { + private final T dispatcherRunner; + private final LeaderElectionService leaderElectionService; + + private DispatcherRunnerLeaderElectionLifecycleManager(T dispatcherRunner, LeaderElectionService leaderElectionService) throws Exception { + this.dispatcherRunner = dispatcherRunner; + this.leaderElectionService = leaderElectionService; + + leaderElectionService.start(dispatcherRunner); + } + + @Override + public CompletableFuture getDispatcherGateway() { + return dispatcherRunner.getDispatcherGateway(); + } + + @Override + public CompletableFuture getShutDownFuture() { + return dispatcherRunner.getShutDownFuture(); + } + + @Override + public CompletableFuture closeAsync() { + final CompletableFuture servicesTerminationFuture = stopServices(); + final CompletableFuture dispatcherRunnerTerminationFuture = dispatcherRunner.closeAsync(); + + return FutureUtils.completeAll(Arrays.asList(servicesTerminationFuture, dispatcherRunnerTerminationFuture)); + } + + private CompletableFuture stopServices() { + try { + leaderElectionService.stop(); + } catch (Exception e) { + return FutureUtils.completedExceptionally(e); + } + + return FutureUtils.completedVoidFuture(); + } + + public static DispatcherRunner createFor(T dispatcherRunner, LeaderElectionService leaderElectionService) throws Exception { + return new DispatcherRunnerLeaderElectionLifecycleManager<>(dispatcherRunner, leaderElectionService); + } +} From e3f202698f562001a3d8e0779042688b64fc1a16 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 21 Oct 2019 17:38:47 +0200 Subject: [PATCH 282/746] [FLINK-11665] Various ZooKeeperDefaultDispatcherRunnerTest clean ups --- .../ZooKeeperDefaultDispatcherRunnerTest.java | 21 ++++++++++--------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java index 284d98deff21..5fcc8866b2a2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java @@ -27,6 +27,7 @@ import org.apache.flink.runtime.blob.PermanentBlobKey; import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.dispatcher.DispatcherGateway; +import org.apache.flink.runtime.dispatcher.DispatcherId; import org.apache.flink.runtime.dispatcher.MemoryArchivedExecutionGraphStore; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; import org.apache.flink.runtime.dispatcher.SessionDispatcherFactory; @@ -42,13 +43,13 @@ import org.apache.flink.runtime.jobmanager.ZooKeeperJobGraphStore; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; -import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.rpc.TestingRpcServiceResource; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testtasks.NoOpInvokable; import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.runtime.util.LeaderConnectionInfo; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.runtime.util.ZooKeeperUtils; import org.apache.flink.runtime.zookeeper.ZooKeeperResource; @@ -132,13 +133,11 @@ public void teardown() throws Exception { public void testResourceCleanupUnderLeadershipChange() throws Exception { final TestingRpcService rpcService = testingRpcServiceResource.getTestingRpcService(); final TestingLeaderElectionService dispatcherLeaderElectionService = new TestingLeaderElectionService(); - final SettableLeaderRetrievalService dispatcherLeaderRetriever = new SettableLeaderRetrievalService(); final CuratorFramework client = ZooKeeperUtils.startCuratorFramework(configuration); try (final TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServicesBuilder() .setRunningJobsRegistry(new ZooKeeperRunningJobsRegistry(client, configuration)) .setDispatcherLeaderElectionService(dispatcherLeaderElectionService) - .setDispatcherLeaderRetriever(dispatcherLeaderRetriever) .setJobMasterLeaderRetrieverFunction(jobId -> ZooKeeperUtils.createLeaderRetrievalService(client, configuration)) .build()) { @@ -166,7 +165,7 @@ public void testResourceCleanupUnderLeadershipChange() throws Exception { defaultDispatcherRunnerFactory)) { // initial run - DispatcherGateway dispatcherGateway = grantLeadership(dispatcherLeaderElectionService, dispatcherLeaderRetriever, dispatcherRunner); + DispatcherGateway dispatcherGateway = grantLeadership(dispatcherLeaderElectionService); LOG.info("Initial job submission {}.", jobGraph.getJobID()); dispatcherGateway.submitJob(jobGraph, TESTING_TIMEOUT).get(); @@ -175,7 +174,7 @@ public void testResourceCleanupUnderLeadershipChange() throws Exception { // recovering submitted jobs LOG.info("Re-grant leadership first time."); - dispatcherGateway = grantLeadership(dispatcherLeaderElectionService, dispatcherLeaderRetriever, dispatcherRunner); + dispatcherGateway = grantLeadership(dispatcherLeaderElectionService); LOG.info("Cancel recovered job {}.", jobGraph.getJobID()); // cancellation of the job should remove everything @@ -224,13 +223,15 @@ private ZooKeeperJobGraphStore createZooKeeperJobGraphStore(CuratorFramework cli } } - private DispatcherGateway grantLeadership(TestingLeaderElectionService dispatcherLeaderElectionService, SettableLeaderRetrievalService dispatcherLeaderRetriever, DispatcherRunner dispatcherRunner) throws InterruptedException, java.util.concurrent.ExecutionException { + private DispatcherGateway grantLeadership(TestingLeaderElectionService dispatcherLeaderElectionService) throws InterruptedException, java.util.concurrent.ExecutionException { final UUID leaderSessionId = UUID.randomUUID(); - dispatcherLeaderElectionService.isLeader(leaderSessionId).get(); - // TODO: Remove once runner properly works - dispatcherLeaderRetriever.notifyListener("foobar", leaderSessionId); + dispatcherLeaderElectionService.isLeader(leaderSessionId); + final LeaderConnectionInfo leaderConnectionInfo = dispatcherLeaderElectionService.getConfirmationFuture().get(); - return dispatcherRunner.getDispatcherGateway().get(); + return testingRpcServiceResource.getTestingRpcService().connect( + leaderConnectionInfo.getAddress(), + DispatcherId.fromUuid(leaderSessionId), + DispatcherGateway.class).get(); } private JobGraph createJobGraphWithBlobs() throws IOException { From 7fcb64bf76d01aa5311fd9288e58790fadabaebb Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 21 Oct 2019 18:46:30 +0200 Subject: [PATCH 283/746] [FLINK-11843] Remove unused dispatcher runner accessor from DispatcherResourceManagerComponent --- .../component/DispatcherResourceManagerComponent.java | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponent.java index f8c43f68333c..4d1b1f9ac0d1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DispatcherResourceManagerComponent.java @@ -94,16 +94,6 @@ public final CompletableFuture getShutDownFuture() { return shutDownFuture; } - @Nonnull - public DispatcherRunner getDispatcherRunner() { - return dispatcherRunner; - } - - @Nonnull - public WebMonitorEndpoint getWebMonitorEndpoint() { - return webMonitorEndpoint; - } - /** * Deregister the Flink application from the resource management system by signalling * the {@link ResourceManager}. From 401f5f3970c6e969f3749ee72a4c57cb3f905af2 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 21 Oct 2019 18:49:14 +0200 Subject: [PATCH 284/746] [FLINK-11843] Remove unnecessary getDispatcherGateway method from DispatcherRunner --- .../runner/DefaultDispatcherRunner.java | 18 ------- .../dispatcher/runner/DispatcherRunner.java | 9 ---- ...rRunnerLeaderElectionLifecycleManager.java | 6 --- .../runner/DefaultDispatcherRunnerTest.java | 54 ------------------- 4 files changed, 87 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java index 4e224b989234..701023e6f908 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java @@ -20,7 +20,6 @@ import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.concurrent.FutureUtils; -import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.leaderelection.LeaderContender; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.rpc.FatalErrorHandler; @@ -59,8 +58,6 @@ public final class DefaultDispatcherRunner implements DispatcherRunner, LeaderCo private CompletableFuture previousDispatcherLeaderProcessTerminationFuture; - private CompletableFuture dispatcherGatewayFuture; - private DefaultDispatcherRunner( LeaderElectionService leaderElectionService, FatalErrorHandler fatalErrorHandler, @@ -74,14 +71,6 @@ private DefaultDispatcherRunner( this.running = true; this.dispatcherLeaderProcess = StoppedDispatcherLeaderProcess.INSTANCE; this.previousDispatcherLeaderProcessTerminationFuture = CompletableFuture.completedFuture(null); - this.dispatcherGatewayFuture = new CompletableFuture<>(); - } - - @Override - public CompletableFuture getDispatcherGateway() { - synchronized (lock) { - return dispatcherGatewayFuture; - } } @Override @@ -140,19 +129,12 @@ private DispatcherLeaderProcess createNewDispatcherLeaderProcess(UUID leaderSess final DispatcherLeaderProcess newDispatcherLeaderProcess = dispatcherLeaderProcessFactory.create(leaderSessionID); - forwardDispatcherGatewayFuture(newDispatcherLeaderProcess); forwardShutDownFuture(newDispatcherLeaderProcess); forwardConfirmLeaderSessionFuture(leaderSessionID, newDispatcherLeaderProcess); return newDispatcherLeaderProcess; } - private void forwardDispatcherGatewayFuture(DispatcherLeaderProcess newDispatcherLeaderProcess) { - final CompletableFuture newDispatcherGatewayFuture = newDispatcherLeaderProcess.getDispatcherGateway(); - FutureUtils.forward(newDispatcherGatewayFuture, dispatcherGatewayFuture); - dispatcherGatewayFuture = newDispatcherGatewayFuture; - } - private void forwardShutDownFuture(DispatcherLeaderProcess newDispatcherLeaderProcess) { newDispatcherLeaderProcess.getShutDownFuture().whenComplete( (applicationStatus, throwable) -> { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunner.java index c049ecbc8e52..63ed60a71f66 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunner.java @@ -20,7 +20,6 @@ import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.dispatcher.Dispatcher; -import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.util.AutoCloseableAsync; import java.util.concurrent.CompletableFuture; @@ -30,14 +29,6 @@ */ public interface DispatcherRunner extends AutoCloseableAsync { - /** - * Return a future which is completed once the {@link Dispatcher} gains - * leadership. - * - * @return Future which is completed with the leader's gateway - */ - CompletableFuture getDispatcherGateway(); - /** * Return shut down future of this runner. The shut down future is being * completed with the final {@link ApplicationStatus} once the runner wants diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerLeaderElectionLifecycleManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerLeaderElectionLifecycleManager.java index eea819cfb4dd..89815f8cb280 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerLeaderElectionLifecycleManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerLeaderElectionLifecycleManager.java @@ -20,7 +20,6 @@ import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.concurrent.FutureUtils; -import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.leaderelection.LeaderContender; import org.apache.flink.runtime.leaderelection.LeaderElectionService; @@ -38,11 +37,6 @@ private DispatcherRunnerLeaderElectionLifecycleManager(T dispatcherRunner, Leade leaderElectionService.start(dispatcherRunner); } - @Override - public CompletableFuture getDispatcherGateway() { - return dispatcherRunner.getDispatcherGateway(); - } - @Override public CompletableFuture getShutDownFuture() { return dispatcherRunner.getShutDownFuture(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerTest.java index bfc5f80c4475..6fd1173dbea8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerTest.java @@ -82,60 +82,6 @@ public void closeAsync_doesNotCompleteUncompletedShutDownFuture() throws Excepti assertThat(shutDownFuture.isDone(), is(false)); } - @Test - public void getDispatcherGateway_beforeDispatcherLeaderProcessCompletes_returnsDispatcherGateway() throws Exception { - final UUID leaderSessionId = UUID.randomUUID(); - final TestingDispatcherGateway expectedDispatcherGateway = createDispatcherGateway(leaderSessionId); - final TestingDispatcherLeaderProcess testingDispatcherLeaderProcess = TestingDispatcherLeaderProcess.newBuilder(leaderSessionId) - .setDispatcherGatewayFuture(CompletableFuture.completedFuture(expectedDispatcherGateway)) - .build(); - - testingDispatcherLeaderProcessFactory = TestingDispatcherLeaderProcessFactory.from(testingDispatcherLeaderProcess); - try (final DispatcherRunner dispatcherRunner = createDispatcherRunner()) { - - final CompletableFuture dispatcherGatewayFuture = dispatcherRunner.getDispatcherGateway(); - - assertThat(dispatcherGatewayFuture.isDone(), is(false)); - - testingLeaderElectionService.isLeader(leaderSessionId); - - assertThat(dispatcherGatewayFuture.get(), is(expectedDispatcherGateway)); - } - } - - @Test - public void getDispatcherGateway_withChangingLeaders_returnsLeadingDispatcherGateway() throws Exception { - final UUID firstLeaderSessionId = UUID.randomUUID(); - final UUID secondLeaderSessionId = UUID.randomUUID(); - final TestingDispatcherGateway firstDispatcherGateway = createDispatcherGateway(firstLeaderSessionId); - final TestingDispatcherGateway secondDispatcherGateway = createDispatcherGateway(secondLeaderSessionId); - - final TestingDispatcherLeaderProcess firstDispatcherLeaderProcess = TestingDispatcherLeaderProcess.newBuilder(firstLeaderSessionId) - .setDispatcherGatewayFuture(CompletableFuture.completedFuture(firstDispatcherGateway)) - .build(); - final TestingDispatcherLeaderProcess secondDispatcherLeaderProcess = TestingDispatcherLeaderProcess.newBuilder(secondLeaderSessionId) - .setDispatcherGatewayFuture(CompletableFuture.completedFuture(secondDispatcherGateway)) - .build(); - - testingDispatcherLeaderProcessFactory = TestingDispatcherLeaderProcessFactory.from( - firstDispatcherLeaderProcess, - secondDispatcherLeaderProcess); - - try (final DispatcherRunner dispatcherRunner = createDispatcherRunner()) { - testingLeaderElectionService.isLeader(firstLeaderSessionId); - - final CompletableFuture firstDispatcherGatewayFuture = dispatcherRunner.getDispatcherGateway(); - - testingLeaderElectionService.notLeader(); - testingLeaderElectionService.isLeader(secondLeaderSessionId); - - final CompletableFuture secondDispatcherGatewayFuture = dispatcherRunner.getDispatcherGateway(); - - assertThat(firstDispatcherGatewayFuture.get(), is(firstDispatcherGateway)); - assertThat(secondDispatcherGatewayFuture.get(), is(secondDispatcherGateway)); - } - } - @Test public void getShutDownFuture_whileRunning_forwardsDispatcherLeaderProcessShutDownRequest() throws Exception { final UUID leaderSessionId = UUID.randomUUID(); From 035c6a1a2e2786c76906bb84ee28652385e6b9dc Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 21 Oct 2019 18:51:58 +0200 Subject: [PATCH 285/746] [FLINK-11843] Rename DispatcherLeaderProcess#getConfirmLeaderSessionFuture into getLeaderAddressFuture --- .../runner/AbstractDispatcherLeaderProcess.java | 8 ++++---- .../dispatcher/runner/DefaultDispatcherRunner.java | 2 +- .../dispatcher/runner/DispatcherLeaderProcess.java | 2 +- .../dispatcher/runner/StoppedDispatcherLeaderProcess.java | 2 +- .../runner/SessionDispatcherLeaderProcessTest.java | 2 +- .../dispatcher/runner/TestingDispatcherLeaderProcess.java | 2 +- 6 files changed, 9 insertions(+), 9 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java index ecfcee56e209..4a9720e86eae 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java @@ -55,7 +55,7 @@ abstract class AbstractDispatcherLeaderProcess implements DispatcherLeaderProces private final CompletableFuture dispatcherGatewayFuture; - private final CompletableFuture confirmLeaderSessionFuture; + private final CompletableFuture leaderAddressFuture; private final CompletableFuture terminationFuture; @@ -71,7 +71,7 @@ abstract class AbstractDispatcherLeaderProcess implements DispatcherLeaderProces this.fatalErrorHandler = fatalErrorHandler; this.dispatcherGatewayFuture = new CompletableFuture<>(); - this.confirmLeaderSessionFuture = dispatcherGatewayFuture.thenApply(RestfulGateway::getAddress); + this.leaderAddressFuture = dispatcherGatewayFuture.thenApply(RestfulGateway::getAddress); this.terminationFuture = new CompletableFuture<>(); this.shutDownFuture = new CompletableFuture<>(); @@ -109,8 +109,8 @@ public final CompletableFuture getDispatcherGateway() { } @Override - public final CompletableFuture getConfirmLeaderSessionFuture() { - return confirmLeaderSessionFuture; + public final CompletableFuture getLeaderAddressFuture() { + return leaderAddressFuture; } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java index 701023e6f908..507bece4ed20 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunner.java @@ -153,7 +153,7 @@ private void forwardShutDownFuture(DispatcherLeaderProcess newDispatcherLeaderPr private void forwardConfirmLeaderSessionFuture(UUID leaderSessionID, DispatcherLeaderProcess newDispatcherLeaderProcess) { FutureUtils.assertNoException( - newDispatcherLeaderProcess.getConfirmLeaderSessionFuture().thenAccept( + newDispatcherLeaderProcess.getLeaderAddressFuture().thenAccept( leaderAddress -> { if (leaderElectionService.hasLeadership(leaderSessionID)) { leaderElectionService.confirmLeadership(leaderSessionID, leaderAddress); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcess.java index db7b4a5f0f79..0c604bf3d9b4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcess.java @@ -37,7 +37,7 @@ interface DispatcherLeaderProcess extends AutoCloseableAsync { CompletableFuture getDispatcherGateway(); - CompletableFuture getConfirmLeaderSessionFuture(); + CompletableFuture getLeaderAddressFuture(); CompletableFuture getShutDownFuture(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StoppedDispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StoppedDispatcherLeaderProcess.java index a84eb0a50dac..62031ed2b44e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StoppedDispatcherLeaderProcess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/StoppedDispatcherLeaderProcess.java @@ -49,7 +49,7 @@ public CompletableFuture getDispatcherGateway() { } @Override - public CompletableFuture getConfirmLeaderSessionFuture() { + public CompletableFuture getLeaderAddressFuture() { throw new UnsupportedOperationException("This method should never be called."); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java index 89a7fd8a4ab9..f03ec690a854 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java @@ -201,7 +201,7 @@ public void confirmLeaderSessionFuture_completesAfterDispatcherServiceHasBeenSta .build(); try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { - final CompletableFuture confirmLeaderSessionFuture = dispatcherLeaderProcess.getConfirmLeaderSessionFuture(); + final CompletableFuture confirmLeaderSessionFuture = dispatcherLeaderProcess.getLeaderAddressFuture(); dispatcherLeaderProcess.start(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherLeaderProcess.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherLeaderProcess.java index 8bea433fd933..6abbfb7bdf7f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherLeaderProcess.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherLeaderProcess.java @@ -68,7 +68,7 @@ public CompletableFuture getDispatcherGateway() { } @Override - public CompletableFuture getConfirmLeaderSessionFuture() { + public CompletableFuture getLeaderAddressFuture() { return confirmLeaderSessionFuture; } From 32a3e5a6a072b346827990813d4da4c54b086408 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 22 Oct 2019 11:21:40 +0200 Subject: [PATCH 286/746] [FLINK-11843] Various SessionDispatcherLeaderProcessTest clean ups --- .../SessionDispatcherLeaderProcessTest.java | 36 +++++++------------ 1 file changed, 13 insertions(+), 23 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java index f03ec690a854..5a2692835089 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java @@ -36,17 +36,14 @@ import org.apache.flink.util.function.ThrowingConsumer; import org.apache.flink.util.function.TriFunctionWithException; -import org.hamcrest.core.Is; import org.junit.After; import org.junit.AfterClass; -import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; import java.util.Collection; import java.util.Collections; -import java.util.Map; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; @@ -57,7 +54,6 @@ import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; -import static org.hamcrest.Matchers.hasSize; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -137,7 +133,6 @@ public void start_triggersJobGraphRecoveryAndDispatcherServiceCreation() throws final Collection recoveredJobGraphs = recoveredJobGraphsFuture.get(); - assertThat(recoveredJobGraphs, hasSize(1)); assertThat(recoveredJobGraphs, containsInAnyOrder(JOB_GRAPH)); } } @@ -150,13 +145,9 @@ public void closeAsync_stopsJobGraphStoreAndDispatcher() throws Exception { .build(); final CompletableFuture dispatcherServiceTerminationFuture = new CompletableFuture<>(); - final OneShotLatch dispatcherServiceShutdownLatch = new OneShotLatch(); dispatcherServiceFactory = TestingDispatcherServiceFactory.newBuilder() .setCreateFunction((ignoredA, ignoredB, ignoredC) -> TestingDispatcherService.newBuilder() - .setTerminationFutureSupplier(() -> { - dispatcherServiceShutdownLatch.trigger(); - return dispatcherServiceTerminationFuture; - }) + .setTerminationFutureSupplier(() -> dispatcherServiceTerminationFuture) .build()) .build(); @@ -171,7 +162,6 @@ public void closeAsync_stopsJobGraphStoreAndDispatcher() throws Exception { assertThat(jobGraphStopFuture.isDone(), is(false)); assertThat(terminationFuture.isDone(), is(false)); - dispatcherServiceShutdownLatch.await(); dispatcherServiceTerminationFuture.complete(null); // verify that we shut down the JobGraphStore @@ -215,14 +205,14 @@ public void confirmLeaderSessionFuture_completesAfterDispatcherServiceHasBeenSta @Test public void closeAsync_duringJobRecovery_preventsDispatcherServiceCreation() throws Exception { - final OneShotLatch jobRecoveryStarted = new OneShotLatch(); - final OneShotLatch completeJobRecovery = new OneShotLatch(); - final OneShotLatch createDispatcherService = new OneShotLatch(); + final OneShotLatch jobRecoveryStartedLatch = new OneShotLatch(); + final OneShotLatch completeJobRecoveryLatch = new OneShotLatch(); + final OneShotLatch createDispatcherServiceLatch = new OneShotLatch(); this.jobGraphStore = TestingJobGraphStore.newBuilder() .setJobIdsFunction(storedJobs -> { - jobRecoveryStarted.trigger(); - completeJobRecovery.await(); + jobRecoveryStartedLatch.trigger(); + completeJobRecoveryLatch.await(); return storedJobs; }) .build(); @@ -230,7 +220,7 @@ public void closeAsync_duringJobRecovery_preventsDispatcherServiceCreation() thr this.dispatcherServiceFactory = TestingDispatcherServiceFactory.newBuilder() .setCreateFunction( (ignoredA, ignoredB, ignoredC) -> { - createDispatcherService.trigger(); + createDispatcherServiceLatch.trigger(); return TestingDispatcherService.newBuilder().build(); }) .build(); @@ -238,14 +228,14 @@ public void closeAsync_duringJobRecovery_preventsDispatcherServiceCreation() thr try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { dispatcherLeaderProcess.start(); - jobRecoveryStarted.await(); + jobRecoveryStartedLatch.await(); dispatcherLeaderProcess.closeAsync(); - completeJobRecovery.trigger(); + completeJobRecoveryLatch.trigger(); try { - createDispatcherService.await(10L, TimeUnit.MILLISECONDS); + createDispatcherServiceLatch.await(10L, TimeUnit.MILLISECONDS); fail("No dispatcher service should be created after the process has been stopped."); } catch (TimeoutException expected) {} } @@ -392,7 +382,7 @@ public void onAddedJobGraph_failingRecovery_propagatesTheFailure() throws Except final CompletableFuture errorFuture = fatalErrorHandler.getErrorFuture(); final Throwable throwable = errorFuture.get(); - Assert.assertThat(ExceptionUtils.findThrowable(throwable, expectedFailure::equals).isPresent(), Is.is(true)); + assertThat(ExceptionUtils.findThrowable(throwable, expectedFailure::equals).isPresent(), is(true)); assertThat(dispatcherLeaderProcess.getState(), is(SessionDispatcherLeaderProcess.State.STOPPED)); @@ -405,7 +395,7 @@ public void recoverJobs_withRecoveryFailure_failsFatally() throws Exception { final FlinkException testException = new FlinkException("Test exception"); jobGraphStore = TestingJobGraphStore.newBuilder() .setRecoverJobGraphFunction( - (JobID jobId, Map jobGraphs) -> { + (ignoredA, ignoredB) -> { throw testException; }) .setInitialJobGraphs(Collections.singleton(JOB_GRAPH)) @@ -433,7 +423,7 @@ private void runJobRecoveryFailureTest(FlinkException testException) throws Exce // we expect that a fatal error occurred final Throwable error = fatalErrorHandler.getErrorFuture().get(); - Assert.assertThat(ExceptionUtils.findThrowableWithMessage(error, testException.getMessage()).isPresent(), is(true)); + assertThat(ExceptionUtils.findThrowableWithMessage(error, testException.getMessage()).isPresent(), is(true)); fatalErrorHandler.clearError(); } From f387e50a2236ba49196ce0acfd8c9dfa68fa554e Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 22 Oct 2019 12:00:56 +0200 Subject: [PATCH 287/746] [hotfix] Remove Nonnull annotations from DispatcherFactory and sub classes --- .../flink/runtime/dispatcher/DispatcherFactory.java | 10 ++++------ .../runtime/dispatcher/JobDispatcherFactory.java | 10 ++++------ .../runner/DefaultDispatcherRunnerITCase.java | 11 ++++------- .../minicluster/SessionDispatcherWithUUIDFactory.java | 10 ++++------ 4 files changed, 16 insertions(+), 25 deletions(-) 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 508acf853f66..cd3c34daaab2 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 @@ -21,8 +21,6 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.rpc.RpcService; -import javax.annotation.Nonnull; - import java.util.Collection; import java.util.UUID; @@ -35,10 +33,10 @@ public interface DispatcherFactory { * Create a {@link Dispatcher}. */ Dispatcher createDispatcher( - @Nonnull RpcService rpcService, - @Nonnull DispatcherId fencingToken, - @Nonnull Collection recoveredJobs, - @Nonnull PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception; + RpcService rpcService, + DispatcherId fencingToken, + Collection recoveredJobs, + PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception; default String generateEndpointIdWithUUID() { return getEndpointId() + UUID.randomUUID(); 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 784bb42710d9..a8cafc8d7f86 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 @@ -25,8 +25,6 @@ import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables; -import javax.annotation.Nonnull; - import java.util.Collection; import static org.apache.flink.runtime.entrypoint.ClusterEntrypoint.EXECUTION_MODE; @@ -39,10 +37,10 @@ public enum JobDispatcherFactory implements DispatcherFactory { @Override public MiniDispatcher createDispatcher( - @Nonnull RpcService rpcService, - @Nonnull DispatcherId fencingToken, - @Nonnull Collection recoveredJobs, - @Nonnull PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { + RpcService rpcService, + DispatcherId fencingToken, + Collection recoveredJobs, + PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { final JobGraph jobGraph = Iterables.getOnlyElement(recoveredJobs); final Configuration configuration = partialDispatcherServicesWithJobGraphStore.getConfiguration(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java index 98be0e154df2..85b88241544c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java @@ -60,8 +60,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.annotation.Nonnull; - import java.util.Collection; import java.util.UUID; import java.util.concurrent.CompletableFuture; @@ -69,7 +67,6 @@ import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; @@ -206,10 +203,10 @@ private TestingDispatcherFactory(JobManagerRunnerFactory jobManagerRunnerFactory @Override public Dispatcher createDispatcher( - @Nonnull RpcService rpcService, - @Nonnull DispatcherId fencingToken, - @Nonnull Collection recoveredJobs, - @Nonnull PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { + RpcService rpcService, + DispatcherId fencingToken, + Collection recoveredJobs, + PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { return new StandaloneDispatcher( rpcService, generateEndpointIdWithUUID(), 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 c77147c03b06..994dbf06c545 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 @@ -27,8 +27,6 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.rpc.RpcService; -import javax.annotation.Nonnull; - import java.util.Collection; /** @@ -40,10 +38,10 @@ public enum SessionDispatcherWithUUIDFactory implements DispatcherFactory { @Override public StandaloneDispatcher createDispatcher( - @Nonnull RpcService rpcService, - @Nonnull DispatcherId fencingToken, - @Nonnull Collection recoveredJobs, - @Nonnull PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { + RpcService rpcService, + DispatcherId fencingToken, + Collection recoveredJobs, + PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) throws Exception { // create the default dispatcher return new StandaloneDispatcher( rpcService, From 1c2097e716b50f8f306648761ebccea79e0794f3 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 23 Oct 2019 19:26:34 +0200 Subject: [PATCH 288/746] [FLINK-11843] Rename DispatcherService into DispatcherGatewayService This closes #9832. --- .../AbstractDispatcherLeaderProcess.java | 16 ++++++++-------- ...va => DefaultDispatcherGatewayService.java} | 8 ++++---- ...efaultDispatcherGatewayServiceFactory.java} | 10 +++++----- .../runner/JobDispatcherLeaderProcess.java | 8 ++++---- .../JobDispatcherLeaderProcessFactory.java | 8 ++++---- ...bDispatcherLeaderProcessFactoryFactory.java | 2 +- .../runner/SessionDispatcherLeaderProcess.java | 10 +++++----- .../SessionDispatcherLeaderProcessFactory.java | 8 ++++---- ...nDispatcherLeaderProcessFactoryFactory.java | 4 ++-- .../SessionDispatcherLeaderProcessTest.java | 18 +++++++++--------- ...va => TestingDispatcherGatewayService.java} | 8 ++++---- .../TestingDispatcherServiceFactory.java | 12 ++++++------ 12 files changed, 56 insertions(+), 56 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/{DefaultDispatcherService.java => DefaultDispatcherGatewayService.java} (85%) rename flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/{DefaultDispatcherServiceFactory.java => DefaultDispatcherGatewayServiceFactory.java} (86%) rename flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/{TestingDispatcherService.java => TestingDispatcherGatewayService.java} (92%) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java index 4a9720e86eae..948d41e71c38 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java @@ -64,7 +64,7 @@ abstract class AbstractDispatcherLeaderProcess implements DispatcherLeaderProces private State state; @Nullable - private DispatcherService dispatcherService; + private DispatcherGatewayService dispatcherService; AbstractDispatcherLeaderProcess(UUID leaderSessionId, FatalErrorHandler fatalErrorHandler) { this.leaderSessionId = leaderSessionId; @@ -118,7 +118,7 @@ public CompletableFuture getShutDownFuture() { return shutDownFuture; } - protected final Optional getDispatcherService() { + protected final Optional getDispatcherService() { return Optional.ofNullable(dispatcherService); } @@ -161,14 +161,14 @@ protected CompletableFuture onClose() { return FutureUtils.completedVoidFuture(); } - final void completeDispatcherSetup(DispatcherService dispatcherService) { + final void completeDispatcherSetup(DispatcherGatewayService dispatcherService) { runIfStateIs( State.RUNNING, () -> completeDispatcherSetupInternal(dispatcherService)); } - private void completeDispatcherSetupInternal(DispatcherService createdDispatcherService) { - Preconditions.checkState(dispatcherService == null, "The DispatcherService can only be set once."); + private void completeDispatcherSetupInternal(DispatcherGatewayService createdDispatcherService) { + Preconditions.checkState(dispatcherService == null, "The DispatcherGatewayService can only be set once."); dispatcherService = createdDispatcherService; dispatcherGatewayFuture.complete(createdDispatcherService.getGateway()); FutureUtils.forward(createdDispatcherService.getShutDownFuture(), shutDownFuture); @@ -237,14 +237,14 @@ protected enum State { // Internal classes // ------------------------------------------------------------ - interface DispatcherServiceFactory { - DispatcherService create( + interface DispatcherGatewayServiceFactory { + DispatcherGatewayService create( DispatcherId fencingToken, Collection recoveredJobs, JobGraphWriter jobGraphWriter); } - interface DispatcherService extends AutoCloseableAsync { + interface DispatcherGatewayService extends AutoCloseableAsync { DispatcherGateway getGateway(); CompletableFuture onRemovedJobGraph(JobID jobId); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherGatewayService.java similarity index 85% rename from flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherService.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherGatewayService.java index 4af987b056a8..720c79f6ad67 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherGatewayService.java @@ -25,12 +25,12 @@ import java.util.concurrent.CompletableFuture; -class DefaultDispatcherService implements AbstractDispatcherLeaderProcess.DispatcherService { +class DefaultDispatcherGatewayService implements AbstractDispatcherLeaderProcess.DispatcherGatewayService { private final Dispatcher dispatcher; private final DispatcherGateway dispatcherGateway; - private DefaultDispatcherService(Dispatcher dispatcher) { + private DefaultDispatcherGatewayService(Dispatcher dispatcher) { this.dispatcher = dispatcher; this.dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); } @@ -55,7 +55,7 @@ public CompletableFuture closeAsync() { return dispatcher.closeAsync(); } - public static DefaultDispatcherService from(Dispatcher dispatcher) { - return new DefaultDispatcherService(dispatcher); + public static DefaultDispatcherGatewayService from(Dispatcher dispatcher) { + return new DefaultDispatcherGatewayService(dispatcher); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherServiceFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherGatewayServiceFactory.java similarity index 86% rename from flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherServiceFactory.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherGatewayServiceFactory.java index dd2c78611dfc..73a0bd022d83 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherServiceFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherGatewayServiceFactory.java @@ -31,9 +31,9 @@ import java.util.Collection; /** - * Factory for the {@link DefaultDispatcherService}. + * Factory for the {@link DefaultDispatcherGatewayService}. */ -class DefaultDispatcherServiceFactory implements AbstractDispatcherLeaderProcess.DispatcherServiceFactory { +class DefaultDispatcherGatewayServiceFactory implements AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory { private final DispatcherFactory dispatcherFactory; @@ -41,7 +41,7 @@ class DefaultDispatcherServiceFactory implements AbstractDispatcherLeaderProcess private final PartialDispatcherServices partialDispatcherServices; - DefaultDispatcherServiceFactory( + DefaultDispatcherGatewayServiceFactory( DispatcherFactory dispatcherFactory, RpcService rpcService, PartialDispatcherServices partialDispatcherServices) { @@ -51,7 +51,7 @@ class DefaultDispatcherServiceFactory implements AbstractDispatcherLeaderProcess } @Override - public AbstractDispatcherLeaderProcess.DispatcherService create( + public AbstractDispatcherLeaderProcess.DispatcherGatewayService create( DispatcherId fencingToken, Collection recoveredJobs, JobGraphWriter jobGraphWriter) { @@ -68,6 +68,6 @@ public AbstractDispatcherLeaderProcess.DispatcherService create( dispatcher.start(); - return DefaultDispatcherService.from(dispatcher); + return DefaultDispatcherGatewayService.from(dispatcher); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java index b0edcef0ffad..165035f62032 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java @@ -31,23 +31,23 @@ */ public class JobDispatcherLeaderProcess extends AbstractDispatcherLeaderProcess { - private final DispatcherServiceFactory dispatcherServiceFactory; + private final DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory; private final JobGraph jobGraph; JobDispatcherLeaderProcess( UUID leaderSessionId, - DispatcherServiceFactory dispatcherServiceFactory, + DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory, JobGraph jobGraph, FatalErrorHandler fatalErrorHandler) { super(leaderSessionId, fatalErrorHandler); this.jobGraph = jobGraph; - this.dispatcherServiceFactory = dispatcherServiceFactory; + this.dispatcherGatewayServiceFactory = dispatcherGatewayServiceFactory; } @Override protected void onStart() { - final DispatcherService dispatcherService = dispatcherServiceFactory.create( + final DispatcherGatewayService dispatcherService = dispatcherGatewayServiceFactory.create( DispatcherId.fromUuid(getLeaderSessionId()), Collections.singleton(jobGraph), ThrowingJobGraphWriter.INSTANCE); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactory.java index 085c54f5d233..f644ae14020e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactory.java @@ -27,23 +27,23 @@ * Factory for the {@link JobDispatcherLeaderProcess}. */ public class JobDispatcherLeaderProcessFactory implements DispatcherLeaderProcessFactory { - private final AbstractDispatcherLeaderProcess.DispatcherServiceFactory dispatcherServiceFactory; + private final AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory; private final JobGraph jobGraph; private final FatalErrorHandler fatalErrorHandler; JobDispatcherLeaderProcessFactory( - AbstractDispatcherLeaderProcess.DispatcherServiceFactory dispatcherServiceFactory, + AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory, JobGraph jobGraph, FatalErrorHandler fatalErrorHandler) { - this.dispatcherServiceFactory = dispatcherServiceFactory; + this.dispatcherGatewayServiceFactory = dispatcherGatewayServiceFactory; this.jobGraph = jobGraph; this.fatalErrorHandler = fatalErrorHandler; } @Override public DispatcherLeaderProcess create(UUID leaderSessionID) { - return new JobDispatcherLeaderProcess(leaderSessionID, dispatcherServiceFactory, jobGraph, fatalErrorHandler); + return new JobDispatcherLeaderProcess(leaderSessionID, dispatcherGatewayServiceFactory, jobGraph, fatalErrorHandler); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java index bde0ab16110f..0ecf906fcf85 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java @@ -57,7 +57,7 @@ public DispatcherLeaderProcessFactory createFactory( throw new FlinkRuntimeException("Could not retrieve the JobGraph.", e); } - final DefaultDispatcherServiceFactory defaultDispatcherServiceFactory = new DefaultDispatcherServiceFactory( + final DefaultDispatcherGatewayServiceFactory defaultDispatcherServiceFactory = new DefaultDispatcherGatewayServiceFactory( JobDispatcherFactory.INSTANCE, rpcService, partialDispatcherServices); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java index 4c22b22f52eb..29e3ab15927e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java @@ -48,7 +48,7 @@ */ public class SessionDispatcherLeaderProcess extends AbstractDispatcherLeaderProcess implements JobGraphStore.JobGraphListener { - private final DispatcherServiceFactory dispatcherFactory; + private final DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory; private final JobGraphStore jobGraphStore; @@ -58,13 +58,13 @@ public class SessionDispatcherLeaderProcess extends AbstractDispatcherLeaderProc private SessionDispatcherLeaderProcess( UUID leaderSessionId, - DispatcherServiceFactory dispatcherFactory, + DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory, JobGraphStore jobGraphStore, Executor ioExecutor, FatalErrorHandler fatalErrorHandler) { super(leaderSessionId, fatalErrorHandler); - this.dispatcherFactory = dispatcherFactory; + this.dispatcherGatewayServiceFactory = dispatcherGatewayServiceFactory; this.jobGraphStore = jobGraphStore; this.ioExecutor = ioExecutor; } @@ -97,7 +97,7 @@ private void createDispatcherIfRunning(Collection jobGraphs) { private void createDispatcher(Collection jobGraphs) { - final DispatcherService dispatcherService = dispatcherFactory.create( + final DispatcherGatewayService dispatcherService = dispatcherGatewayServiceFactory.create( DispatcherId.fromUuid(getLeaderSessionId()), jobGraphs, jobGraphStore); @@ -261,7 +261,7 @@ private CompletableFuture removeJobGraph(JobID jobId) { public static SessionDispatcherLeaderProcess create( UUID leaderSessionId, - DispatcherServiceFactory dispatcherFactory, + DispatcherGatewayServiceFactory dispatcherFactory, JobGraphStore jobGraphStore, Executor ioExecutor, FatalErrorHandler fatalErrorHandler) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactory.java index 2e1161d13883..6be827e92ddd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactory.java @@ -29,17 +29,17 @@ */ class SessionDispatcherLeaderProcessFactory implements DispatcherLeaderProcessFactory { - private final AbstractDispatcherLeaderProcess.DispatcherServiceFactory dispatcherServiceFactory; + private final AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory; private final JobGraphStoreFactory jobGraphStoreFactory; private final Executor ioExecutor; private final FatalErrorHandler fatalErrorHandler; SessionDispatcherLeaderProcessFactory( - AbstractDispatcherLeaderProcess.DispatcherServiceFactory dispatcherServiceFactory, + AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory, JobGraphStoreFactory jobGraphStoreFactory, Executor ioExecutor, FatalErrorHandler fatalErrorHandler) { - this.dispatcherServiceFactory = dispatcherServiceFactory; + this.dispatcherGatewayServiceFactory = dispatcherGatewayServiceFactory; this.jobGraphStoreFactory = jobGraphStoreFactory; this.ioExecutor = ioExecutor; this.fatalErrorHandler = fatalErrorHandler; @@ -49,7 +49,7 @@ class SessionDispatcherLeaderProcessFactory implements DispatcherLeaderProcessFa public DispatcherLeaderProcess create(UUID leaderSessionID) { return SessionDispatcherLeaderProcess.create( leaderSessionID, - dispatcherServiceFactory, + dispatcherGatewayServiceFactory, jobGraphStoreFactory.create(), ioExecutor, fatalErrorHandler); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactoryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactoryFactory.java index ddbd81df9c49..48f5f0990c20 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactoryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactoryFactory.java @@ -44,13 +44,13 @@ public DispatcherLeaderProcessFactory createFactory( RpcService rpcService, PartialDispatcherServices partialDispatcherServices, FatalErrorHandler fatalErrorHandler) { - final AbstractDispatcherLeaderProcess.DispatcherServiceFactory dispatcherServiceFactory = new DefaultDispatcherServiceFactory( + final AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory = new DefaultDispatcherGatewayServiceFactory( dispatcherFactory, rpcService, partialDispatcherServices); return new SessionDispatcherLeaderProcessFactory( - dispatcherServiceFactory, + dispatcherGatewayServiceFactory, jobGraphStoreFactory, ioExecutor, fatalErrorHandler); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java index 5a2692835089..0a3439969555 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java @@ -122,7 +122,7 @@ public void start_triggersJobGraphRecoveryAndDispatcherServiceCreation() throws .setCreateFunction( (fencingToken, recoveredJobGraphs, jobGraphStore) -> { recoveredJobGraphsFuture.complete(recoveredJobGraphs); - return TestingDispatcherService.newBuilder().build(); + return TestingDispatcherGatewayService.newBuilder().build(); } ) .build(); @@ -146,7 +146,7 @@ public void closeAsync_stopsJobGraphStoreAndDispatcher() throws Exception { final CompletableFuture dispatcherServiceTerminationFuture = new CompletableFuture<>(); dispatcherServiceFactory = TestingDispatcherServiceFactory.newBuilder() - .setCreateFunction((ignoredA, ignoredB, ignoredC) -> TestingDispatcherService.newBuilder() + .setCreateFunction((ignoredA, ignoredB, ignoredC) -> TestingDispatcherGatewayService.newBuilder() .setTerminationFutureSupplier(() -> dispatcherServiceTerminationFuture) .build()) .build(); @@ -154,7 +154,7 @@ public void closeAsync_stopsJobGraphStoreAndDispatcher() throws Exception { try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { dispatcherLeaderProcess.start(); - // wait for the creation of the DispatcherService + // wait for the creation of the DispatcherGatewayService dispatcherLeaderProcess.getDispatcherGateway().get(); final CompletableFuture terminationFuture = dispatcherLeaderProcess.closeAsync(); @@ -184,7 +184,7 @@ public void confirmLeaderSessionFuture_completesAfterDispatcherServiceHasBeenSta .setCreateFunction( TriFunctionWithException.unchecked((ignoredA, ignoredB, ignoredC) -> { createDispatcherServiceLatch.await(); - return TestingDispatcherService.newBuilder() + return TestingDispatcherGatewayService.newBuilder() .setDispatcherGateway(dispatcherGateway) .build(); })) @@ -221,7 +221,7 @@ public void closeAsync_duringJobRecovery_preventsDispatcherServiceCreation() thr .setCreateFunction( (ignoredA, ignoredB, ignoredC) -> { createDispatcherServiceLatch.trigger(); - return TestingDispatcherService.newBuilder().build(); + return TestingDispatcherGatewayService.newBuilder().build(); }) .build(); @@ -248,7 +248,7 @@ public void onRemovedJobGraph_terminatesRunningJob() throws Exception { .build(); final CompletableFuture terminateJobFuture = new CompletableFuture<>(); - final TestingDispatcherService testingDispatcherService = TestingDispatcherService.newBuilder() + final TestingDispatcherGatewayService testingDispatcherService = TestingDispatcherGatewayService.newBuilder() .setOnRemovedJobGraphFunction(jobID -> { terminateJobFuture.complete(jobID); return FutureUtils.completedVoidFuture(); @@ -277,7 +277,7 @@ public void onRemovedJobGraph_terminatesRunningJob() throws Exception { public void onRemovedJobGraph_failingRemovalCall_failsFatally() throws Exception { final FlinkException testException = new FlinkException("Test exception"); - final TestingDispatcherService testingDispatcherService = TestingDispatcherService.newBuilder() + final TestingDispatcherGatewayService testingDispatcherService = TestingDispatcherGatewayService.newBuilder() .setOnRemovedJobGraphFunction(jobID -> FutureUtils.completedExceptionally(testException)) .build(); @@ -463,7 +463,7 @@ private void runOnAddedJobGraphTest(TestingDispatcherGateway dispatcherGateway, .setCreateFunction((dispatcherId, jobGraphs, jobGraphWriter) -> { assertThat(jobGraphs, containsInAnyOrder(JOB_GRAPH)); - return TestingDispatcherService.newBuilder() + return TestingDispatcherGatewayService.newBuilder() .setDispatcherGateway(dispatcherGateway) .build(); }) @@ -490,7 +490,7 @@ private void verifyOnAddedJobGraphResultDidNotFail(TestingFatalErrorHandler fata private TestingDispatcherServiceFactory createDispatcherServiceFactoryFor(TestingDispatcherGateway testingDispatcherGateway) { return TestingDispatcherServiceFactory.newBuilder() .setCreateFunction( - (ignoredA, ignoredB, ignoredC) -> TestingDispatcherService.newBuilder() + (ignoredA, ignoredB, ignoredC) -> TestingDispatcherGatewayService.newBuilder() .setDispatcherGateway(testingDispatcherGateway) .build()) .build(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherGatewayService.java similarity index 92% rename from flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherService.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherGatewayService.java index 0c9fdfc45c23..ea999d82debf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherGatewayService.java @@ -28,7 +28,7 @@ import java.util.function.Function; import java.util.function.Supplier; -class TestingDispatcherService implements AbstractDispatcherLeaderProcess.DispatcherService { +class TestingDispatcherGatewayService implements AbstractDispatcherLeaderProcess.DispatcherGatewayService { private final Object lock = new Object(); @@ -42,7 +42,7 @@ class TestingDispatcherService implements AbstractDispatcherLeaderProcess.Dispat private CompletableFuture terminationFuture; - private TestingDispatcherService( + private TestingDispatcherGatewayService( Supplier> terminationFutureSupplier, Function> onRemovedJobGraphFunction, DispatcherGateway dispatcherGateway, @@ -115,8 +115,8 @@ public Builder setShutDownFuture(CompletableFuture shutDownFu return this; } - public TestingDispatcherService build() { - return new TestingDispatcherService(terminationFutureSupplier, onRemovedJobGraphFunction, dispatcherGateway, shutDownFuture); + public TestingDispatcherGatewayService build() { + return new TestingDispatcherGatewayService(terminationFutureSupplier, onRemovedJobGraphFunction, dispatcherGateway, shutDownFuture); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java index 1462c1cea6ab..0c4e0f0e1866 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java @@ -25,15 +25,15 @@ import java.util.Collection; -class TestingDispatcherServiceFactory implements AbstractDispatcherLeaderProcess.DispatcherServiceFactory { - private final TriFunction, JobGraphWriter, AbstractDispatcherLeaderProcess.DispatcherService> createFunction; +class TestingDispatcherServiceFactory implements AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory { + private final TriFunction, JobGraphWriter, AbstractDispatcherLeaderProcess.DispatcherGatewayService> createFunction; - private TestingDispatcherServiceFactory(TriFunction, JobGraphWriter, AbstractDispatcherLeaderProcess.DispatcherService> createFunction) { + private TestingDispatcherServiceFactory(TriFunction, JobGraphWriter, AbstractDispatcherLeaderProcess.DispatcherGatewayService> createFunction) { this.createFunction = createFunction; } @Override - public AbstractDispatcherLeaderProcess.DispatcherService create( + public AbstractDispatcherLeaderProcess.DispatcherGatewayService create( DispatcherId fencingToken, Collection recoveredJobs, JobGraphWriter jobGraphWriter) { @@ -45,11 +45,11 @@ public static Builder newBuilder() { } public static class Builder { - private TriFunction, JobGraphWriter, AbstractDispatcherLeaderProcess.DispatcherService> createFunction = (ignoredA, ignoredB, ignoredC) -> TestingDispatcherService.newBuilder().build(); + private TriFunction, JobGraphWriter, AbstractDispatcherLeaderProcess.DispatcherGatewayService> createFunction = (ignoredA, ignoredB, ignoredC) -> TestingDispatcherGatewayService.newBuilder().build(); private Builder() {} - Builder setCreateFunction(TriFunction, JobGraphWriter, AbstractDispatcherLeaderProcess.DispatcherService> createFunction) { + Builder setCreateFunction(TriFunction, JobGraphWriter, AbstractDispatcherLeaderProcess.DispatcherGatewayService> createFunction) { this.createFunction = createFunction; return this; } From ca29aa5c863c04366c9bcd330577562cd13fe277 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 25 Oct 2019 15:11:41 +0200 Subject: [PATCH 289/746] [hotfix] Remove exception suppression from Dispatcher#stopDispatcherServices --- .../java/org/apache/flink/runtime/dispatcher/Dispatcher.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 ffd9ab09ab9a..6fa9e7f35070 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 @@ -252,7 +252,7 @@ private void stopDispatcherServices() throws Exception { try { jobManagerSharedServices.shutdown(); } catch (Exception e) { - exception = ExceptionUtils.firstOrSuppressed(e, exception); + exception = e; } jobManagerMetricGroup.close(); From bd038c915180125bf444351ca075ffbd919d9e23 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Mon, 21 Oct 2019 17:05:29 +0800 Subject: [PATCH 290/746] [hotfix][core] Introduce IterableUtils It supports a neater way to convert an Iterable to a Stream. --- .../org/apache/flink/util/IterableUtils.java | 53 +++++++++++++++++++ .../apache/flink/util/IterableUtilsTest.java | 45 ++++++++++++++++ 2 files changed, 98 insertions(+) create mode 100644 flink-core/src/main/java/org/apache/flink/util/IterableUtils.java create mode 100644 flink-core/src/test/java/org/apache/flink/util/IterableUtilsTest.java diff --git a/flink-core/src/main/java/org/apache/flink/util/IterableUtils.java b/flink-core/src/main/java/org/apache/flink/util/IterableUtils.java new file mode 100644 index 000000000000..78fc1598dc09 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/util/IterableUtils.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.util; + +import java.util.Collection; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A collection of utilities that expand the usage of {@link Iterable}. + */ +public class IterableUtils { + + /** + * Convert the given {@link Iterable} to a {@link Stream}. + * + * @param iterable to convert to a stream + * @param type of the elements of the iterable + * @return stream converted from the given {@link Iterable} + */ + public static Stream toStream(Iterable iterable) { + checkNotNull(iterable); + + return iterable instanceof Collection ? + ((Collection) iterable).stream() : + StreamSupport.stream(iterable.spliterator(), false); + } + + // -------------------------------------------------------------------------------------------- + + /** + * Private default constructor to avoid instantiation. + */ + private IterableUtils() {} +} diff --git a/flink-core/src/test/java/org/apache/flink/util/IterableUtilsTest.java b/flink-core/src/test/java/org/apache/flink/util/IterableUtilsTest.java new file mode 100644 index 000000000000..5b6ce51936cf --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/util/IterableUtilsTest.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.util; + +import org.junit.Test; + +import java.util.ArrayDeque; +import java.util.Arrays; +import java.util.Queue; +import java.util.stream.Stream; + +import static org.junit.Assert.assertTrue; + +/** + * Tests for the {@link IterableUtils}. + */ +public class IterableUtilsTest extends TestLogger { + + private final Iterable testIterable = Arrays.asList(1, 8, 5, 3, 8); + + @Test + public void testToStream() { + Queue deque = new ArrayDeque<>(); + testIterable.forEach(deque::add); + + Stream stream = IterableUtils.toStream(testIterable); + assertTrue(stream.allMatch(value -> deque.poll().equals(value))); + } +} From 2e427c0899a3c4caea6b70677f098832f01b0a05 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Sat, 19 Oct 2019 01:08:12 +0800 Subject: [PATCH 291/746] [FLINK-14450][runtime] Refactor SchedulingTopology to extend base topology [FLINK-14450][runtime] Refactor SchedulingTopology to extend base topology (for review - Part2: implementations) [FLINK-14450][runtime] Refactor SchedulingTopology to extend base topology (for review - Part3: usages) This closes #9936. --- .../executiongraph/ExecutionGraph.java | 6 +- .../NotReleasingPartitionReleaseStrategy.java | 2 +- .../PartitionReleaseStrategy.java | 2 +- .../RegionPartitionReleaseStrategy.java | 23 +++--- .../IntermediateResultPartitionID.java | 7 +- .../runtime/scheduler/SchedulerBase.java | 4 +- .../DefaultSchedulingExecutionVertex.java | 28 ++++--- .../DefaultSchedulingResultPartition.java | 22 +++--- ...utionGraphToSchedulingTopologyAdapter.java | 30 ++++--- .../strategy/EagerSchedulingStrategy.java | 6 +- .../scheduler/strategy/ExecutionVertexID.java | 3 +- .../InputDependencyConstraintChecker.java | 41 +++++----- .../LazyFromSourcesSchedulingStrategy.java | 46 +++++------ .../strategy/SchedulingExecutionVertex.java | 28 +------ .../strategy/SchedulingResultPartition.java | 35 +-------- .../strategy/SchedulingStrategyFactory.java | 2 +- .../strategy/SchedulingTopology.java | 20 ++--- .../scheduler/DefaultSchedulerTest.java | 6 +- .../DefaultSchedulingExecutionVertexTest.java | 15 ++-- ...nGraphToSchedulingTopologyAdapterTest.java | 78 ++++++++++++++----- .../strategy/EagerSchedulingStrategyTest.java | 2 +- .../InputDependencyConstraintCheckerTest.java | 2 +- ...LazyFromSourcesSchedulingStrategyTest.java | 2 +- .../strategy/TestSchedulingStrategy.java | 8 +- .../TestingSchedulingExecutionVertex.java | 16 ++-- .../TestingSchedulingResultPartition.java | 19 ++--- .../strategy/TestingSchedulingTopology.java | 29 ++++--- 27 files changed, 247 insertions(+), 235 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index e61296613ee4..8bbf1448c1b2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -261,7 +261,7 @@ public class ExecutionGraph implements AccessExecutionGraph { private PartitionReleaseStrategy partitionReleaseStrategy; - private SchedulingTopology schedulingTopology; + private SchedulingTopology schedulingTopology; @Nullable private InternalFailuresListener internalTaskFailuresListener; @@ -1649,8 +1649,8 @@ private void releasePartitions(final List releasa } ResultPartitionID createResultPartitionId(final IntermediateResultPartitionID resultPartitionId) { - final SchedulingResultPartition schedulingResultPartition = schedulingTopology.getResultPartitionOrThrow(resultPartitionId); - final SchedulingExecutionVertex producer = schedulingResultPartition.getProducer(); + final SchedulingResultPartition schedulingResultPartition = schedulingTopology.getResultPartitionOrThrow(resultPartitionId); + final SchedulingExecutionVertex producer = schedulingResultPartition.getProducer(); final ExecutionVertexID producerId = producer.getId(); final JobVertexID jobVertexId = producerId.getJobVertexId(); final ExecutionJobVertex jobVertex = getJobVertex(jobVertexId); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/NotReleasingPartitionReleaseStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/NotReleasingPartitionReleaseStrategy.java index e38687048669..65d0c55dc73d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/NotReleasingPartitionReleaseStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/NotReleasingPartitionReleaseStrategy.java @@ -48,7 +48,7 @@ public void vertexUnfinished(final ExecutionVertexID executionVertexID) { public static class Factory implements PartitionReleaseStrategy.Factory { @Override - public PartitionReleaseStrategy createInstance(final SchedulingTopology schedulingStrategy, final FailoverTopology failoverTopology) { + public PartitionReleaseStrategy createInstance(final SchedulingTopology schedulingStrategy, final FailoverTopology failoverTopology) { return new NotReleasingPartitionReleaseStrategy(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/PartitionReleaseStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/PartitionReleaseStrategy.java index d7b317e10e74..fc99d7ce3b5e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/PartitionReleaseStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/PartitionReleaseStrategy.java @@ -53,6 +53,6 @@ public interface PartitionReleaseStrategy { * Factory for {@link PartitionReleaseStrategy}. */ interface Factory { - PartitionReleaseStrategy createInstance(SchedulingTopology schedulingStrategy, FailoverTopology failoverTopology); + PartitionReleaseStrategy createInstance(SchedulingTopology schedulingStrategy, FailoverTopology failoverTopology); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/RegionPartitionReleaseStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/RegionPartitionReleaseStrategy.java index b930e100e19a..b069d0a63ce8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/RegionPartitionReleaseStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/RegionPartitionReleaseStrategy.java @@ -27,6 +27,7 @@ import org.apache.flink.runtime.scheduler.strategy.SchedulingExecutionVertex; import org.apache.flink.runtime.scheduler.strategy.SchedulingResultPartition; import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology; +import org.apache.flink.util.IterableUtils; import java.util.Collection; import java.util.Collections; @@ -47,14 +48,14 @@ */ public class RegionPartitionReleaseStrategy implements PartitionReleaseStrategy { - private final SchedulingTopology schedulingTopology; + private final SchedulingTopology schedulingTopology; private final Map consumedBlockingPartitionsByRegion = new IdentityHashMap<>(); private final Map regionExecutionViewByVertex = new HashMap<>(); public RegionPartitionReleaseStrategy( - final SchedulingTopology schedulingTopology, + final SchedulingTopology schedulingTopology, final Set pipelinedRegions) { this.schedulingTopology = checkNotNull(schedulingTopology); @@ -86,23 +87,23 @@ private PipelinedRegionConsumedBlockingPartitions computeConsumedPartitionsOfVer } private Set findResultPartitionsOutsideOfRegion(final PipelinedRegion pipelinedRegion) { - final Set allConsumedPartitionsInRegion = pipelinedRegion + final Set> allConsumedPartitionsInRegion = pipelinedRegion .getExecutionVertexIds() .stream() .map(schedulingTopology::getVertexOrThrow) - .flatMap(schedulingExecutionVertex -> schedulingExecutionVertex.getConsumedResultPartitions().stream()) + .flatMap(vertex -> IterableUtils.toStream(vertex.getConsumedResults())) .collect(Collectors.toSet()); return filterResultPartitionsOutsideOfRegion(allConsumedPartitionsInRegion, pipelinedRegion); } private static Set filterResultPartitionsOutsideOfRegion( - final Collection resultPartitions, + final Collection> resultPartitions, final PipelinedRegion pipelinedRegion) { final Set result = new HashSet<>(); - for (final SchedulingResultPartition maybeOutsidePartition : resultPartitions) { - final SchedulingExecutionVertex producer = maybeOutsidePartition.getProducer(); + for (final SchedulingResultPartition maybeOutsidePartition : resultPartitions) { + final SchedulingExecutionVertex producer = maybeOutsidePartition.getProducer(); if (!pipelinedRegion.contains(producer.getId())) { result.add(maybeOutsidePartition.getId()); } @@ -158,10 +159,8 @@ private List filterReleasablePartitions(final Pip } private boolean areConsumerRegionsFinished(final IntermediateResultPartitionID resultPartitionId) { - final SchedulingResultPartition resultPartition = schedulingTopology.getResultPartitionOrThrow(resultPartitionId); - final Collection consumers = resultPartition.getConsumers(); - return consumers - .stream() + final SchedulingResultPartition resultPartition = schedulingTopology.getResultPartitionOrThrow(resultPartitionId); + return IterableUtils.toStream(resultPartition.getConsumers()) .map(SchedulingExecutionVertex::getId) .allMatch(this::isRegionOfVertexFinished); } @@ -178,7 +177,7 @@ public static class Factory implements PartitionReleaseStrategy.Factory { @Override public PartitionReleaseStrategy createInstance( - final SchedulingTopology schedulingStrategy, + final SchedulingTopology schedulingStrategy, final FailoverTopology failoverTopology) { final Set> distinctRegions = PipelinedRegionComputeUtil.computePipelinedRegions(failoverTopology); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateResultPartitionID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateResultPartitionID.java index 24d76c452cba..803c0003761b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateResultPartitionID.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateResultPartitionID.java @@ -18,11 +18,16 @@ package org.apache.flink.runtime.jobgraph; +import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; +import org.apache.flink.runtime.topology.ResultID; import org.apache.flink.util.AbstractID; import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; -public class IntermediateResultPartitionID extends AbstractID { +/** + * Id identifying {@link IntermediateResultPartition}. + */ +public class IntermediateResultPartitionID extends AbstractID implements ResultID { private static final long serialVersionUID = 1L; 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 65fcecf09daf..0ba5b092ab67 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 @@ -109,7 +109,7 @@ public abstract class SchedulerBase implements SchedulerNG { private final ExecutionGraph executionGraph; - private final SchedulingTopology schedulingTopology; + private final SchedulingTopology schedulingTopology; private final FailoverTopology failoverTopology; @@ -301,7 +301,7 @@ protected final FailoverTopology getFailoverTopology() { return failoverTopology; } - protected final SchedulingTopology getSchedulingTopology() { + protected final SchedulingTopology getSchedulingTopology() { return schedulingTopology; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingExecutionVertex.java index 92d804746543..b9a8e195f7cf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingExecutionVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingExecutionVertex.java @@ -22,11 +22,8 @@ import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; import org.apache.flink.runtime.scheduler.strategy.SchedulingExecutionVertex; -import org.apache.flink.runtime.scheduler.strategy.SchedulingResultPartition; import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; import java.util.List; import java.util.function.Supplier; @@ -35,13 +32,14 @@ /** * Default implementation of {@link SchedulingExecutionVertex}. */ -class DefaultSchedulingExecutionVertex implements SchedulingExecutionVertex { +class DefaultSchedulingExecutionVertex + implements SchedulingExecutionVertex { private final ExecutionVertexID executionVertexId; - private final List consumedPartitions; + private final List consumedResults; - private final List producedPartitions; + private final List producedResults; private final Supplier stateSupplier; @@ -49,13 +47,13 @@ class DefaultSchedulingExecutionVertex implements SchedulingExecutionVertex { DefaultSchedulingExecutionVertex( ExecutionVertexID executionVertexId, - List producedPartitions, + List producedPartitions, Supplier stateSupplier, InputDependencyConstraint constraint) { this.executionVertexId = checkNotNull(executionVertexId); - this.consumedPartitions = new ArrayList<>(); + this.consumedResults = new ArrayList<>(); this.stateSupplier = checkNotNull(stateSupplier); - this.producedPartitions = checkNotNull(producedPartitions); + this.producedResults = checkNotNull(producedPartitions); this.inputDependencyConstraint = checkNotNull(constraint); } @@ -70,13 +68,13 @@ public ExecutionState getState() { } @Override - public Collection getConsumedResultPartitions() { - return Collections.unmodifiableCollection(consumedPartitions); + public Iterable getConsumedResults() { + return consumedResults; } @Override - public Collection getProducedResultPartitions() { - return Collections.unmodifiableCollection(producedPartitions); + public Iterable getProducedResults() { + return producedResults; } @Override @@ -84,7 +82,7 @@ public InputDependencyConstraint getInputDependencyConstraint() { return inputDependencyConstraint; } - void addConsumedPartition(X partition) { - consumedPartitions.add(partition); + void addConsumedResult(DefaultSchedulingResultPartition result) { + consumedResults.add(result); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingResultPartition.java index 45a80dcddb0f..6f0ba419e26c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingResultPartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingResultPartition.java @@ -21,12 +21,9 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; -import org.apache.flink.runtime.scheduler.strategy.SchedulingExecutionVertex; import org.apache.flink.runtime.scheduler.strategy.SchedulingResultPartition; import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; import java.util.List; import static org.apache.flink.runtime.scheduler.strategy.SchedulingResultPartition.ResultPartitionState.DONE; @@ -37,7 +34,8 @@ /** * Default implementation of {@link SchedulingResultPartition}. */ -class DefaultSchedulingResultPartition implements SchedulingResultPartition { +class DefaultSchedulingResultPartition + implements SchedulingResultPartition { private final IntermediateResultPartitionID resultPartitionId; @@ -45,9 +43,9 @@ class DefaultSchedulingResultPartition implements SchedulingResultPartition { private final ResultPartitionType partitionType; - private SchedulingExecutionVertex producer; + private DefaultSchedulingExecutionVertex producer; - private final List consumers; + private final List consumers; DefaultSchedulingResultPartition( IntermediateResultPartitionID partitionId, @@ -70,7 +68,7 @@ public IntermediateDataSetID getResultId() { } @Override - public ResultPartitionType getPartitionType() { + public ResultPartitionType getResultType() { return partitionType; } @@ -87,20 +85,20 @@ public ResultPartitionState getState() { } @Override - public SchedulingExecutionVertex getProducer() { + public DefaultSchedulingExecutionVertex getProducer() { return producer; } @Override - public Collection getConsumers() { - return Collections.unmodifiableCollection(consumers); + public Iterable getConsumers() { + return consumers; } - void addConsumer(SchedulingExecutionVertex vertex) { + void addConsumer(DefaultSchedulingExecutionVertex vertex) { consumers.add(checkNotNull(vertex)); } - void setProducer(SchedulingExecutionVertex vertex) { + void setProducer(DefaultSchedulingExecutionVertex vertex) { producer = checkNotNull(vertex); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/ExecutionGraphToSchedulingTopologyAdapter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/ExecutionGraphToSchedulingTopologyAdapter.java index 94c835c5a578..048a537dfcae 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/ExecutionGraphToSchedulingTopologyAdapter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/ExecutionGraphToSchedulingTopologyAdapter.java @@ -21,18 +21,18 @@ import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.ExecutionEdge; import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.executiongraph.ExecutionVertex; import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; -import org.apache.flink.runtime.scheduler.strategy.SchedulingExecutionVertex; -import org.apache.flink.runtime.scheduler.strategy.SchedulingResultPartition; import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.function.Supplier; @@ -41,17 +41,24 @@ /** * Adapter of {@link ExecutionGraph} to {@link SchedulingTopology}. */ -public class ExecutionGraphToSchedulingTopologyAdapter implements SchedulingTopology { +public class ExecutionGraphToSchedulingTopologyAdapter + implements SchedulingTopology { + + private final boolean containsCoLocationConstraints; private final Map executionVerticesById; - private final List executionVerticesList; + private final List executionVerticesList; - private final Map resultPartitionsById; + private final Map resultPartitionsById; public ExecutionGraphToSchedulingTopologyAdapter(ExecutionGraph graph) { checkNotNull(graph, "execution graph can not be null"); + this.containsCoLocationConstraints = graph.getAllVertices().values().stream() + .map(ExecutionJobVertex::getCoLocationGroup) + .anyMatch(Objects::nonNull); + this.executionVerticesById = new HashMap<>(); this.executionVerticesList = new ArrayList<>(graph.getTotalNumberOfVertices()); Map tmpResultPartitionsById = new HashMap<>(); @@ -73,17 +80,22 @@ public ExecutionGraphToSchedulingTopologyAdapter(ExecutionGraph graph) { } @Override - public Iterable getVertices() { + public Iterable getVertices() { return executionVerticesList; } @Override - public Optional getVertex(ExecutionVertexID executionVertexId) { + public boolean containsCoLocationConstraints() { + return containsCoLocationConstraints; + } + + @Override + public Optional getVertex(ExecutionVertexID executionVertexId) { return Optional.ofNullable(executionVerticesById.get(executionVertexId)); } @Override - public Optional getResultPartition(IntermediateResultPartitionID intermediateResultPartitionId) { + public Optional getResultPartition(IntermediateResultPartitionID intermediateResultPartitionId) { return Optional.ofNullable(resultPartitionsById.get(intermediateResultPartitionId)); } @@ -128,7 +140,7 @@ private static void connectVerticesToConsumedPartitions( for (int index = 0; index < executionVertex.getNumberOfInputs(); index++) { for (ExecutionEdge edge : executionVertex.getInputEdges(index)) { DefaultSchedulingResultPartition partition = resultPartitions.get(edge.getSource().getPartitionId()); - schedulingVertex.addConsumedPartition(partition); + schedulingVertex.addConsumedResult(partition); partition.addConsumer(schedulingVertex); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/EagerSchedulingStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/EagerSchedulingStrategy.java index 954fff5b1e55..baee0edc815e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/EagerSchedulingStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/EagerSchedulingStrategy.java @@ -41,13 +41,13 @@ public class EagerSchedulingStrategy implements SchedulingStrategy { private final SchedulerOperations schedulerOperations; - private final SchedulingTopology schedulingTopology; + private final SchedulingTopology schedulingTopology; private final DeploymentOption deploymentOption = new DeploymentOption(false); public EagerSchedulingStrategy( SchedulerOperations schedulerOperations, - SchedulingTopology schedulingTopology) { + SchedulingTopology schedulingTopology) { this.schedulerOperations = checkNotNull(schedulerOperations); this.schedulingTopology = checkNotNull(schedulingTopology); } @@ -104,7 +104,7 @@ public static class Factory implements SchedulingStrategyFactory { @Override public SchedulingStrategy createInstance( SchedulerOperations schedulerOperations, - SchedulingTopology schedulingTopology, + SchedulingTopology schedulingTopology, JobGraph jobGraph) { return new EagerSchedulingStrategy(schedulerOperations, schedulingTopology); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/ExecutionVertexID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/ExecutionVertexID.java index 6e44e11cc87c..d1314f23397d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/ExecutionVertexID.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/ExecutionVertexID.java @@ -20,6 +20,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.topology.VertexID; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -27,7 +28,7 @@ /** * Id identifying {@link ExecutionVertex}. */ -public class ExecutionVertexID { +public class ExecutionVertexID implements VertexID { private final JobVertexID jobVertexId; private final int subtaskIndex; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/InputDependencyConstraintChecker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/InputDependencyConstraintChecker.java index ec0bf6fd172e..c5219125e231 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/InputDependencyConstraintChecker.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/InputDependencyConstraintChecker.java @@ -22,6 +22,9 @@ import org.apache.flink.runtime.jobgraph.IntermediateDataSet; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.util.IterableUtils; + +import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables; import java.util.ArrayList; import java.util.Collections; @@ -44,9 +47,9 @@ public class InputDependencyConstraintChecker { private final SchedulingIntermediateDataSetManager intermediateDataSetManager = new SchedulingIntermediateDataSetManager(); - public boolean check(final SchedulingExecutionVertex schedulingExecutionVertex) { + public boolean check(final SchedulingExecutionVertex schedulingExecutionVertex) { final InputDependencyConstraint inputConstraint = schedulingExecutionVertex.getInputDependencyConstraint(); - if (schedulingExecutionVertex.getConsumedResultPartitions().isEmpty() || ALL.equals(inputConstraint)) { + if (Iterables.isEmpty(schedulingExecutionVertex.getConsumedResults()) || ALL.equals(inputConstraint)) { return checkAll(schedulingExecutionVertex); } else if (ANY.equals(inputConstraint)) { return checkAny(schedulingExecutionVertex); @@ -55,32 +58,30 @@ public boolean check(final SchedulingExecutionVertex schedulingExecutionVertex) } } - List markSchedulingResultPartitionFinished(SchedulingResultPartition srp) { + List> markSchedulingResultPartitionFinished(SchedulingResultPartition srp) { return intermediateDataSetManager.markSchedulingResultPartitionFinished(srp); } - void resetSchedulingResultPartition(SchedulingResultPartition srp) { + void resetSchedulingResultPartition(SchedulingResultPartition srp) { intermediateDataSetManager.resetSchedulingResultPartition(srp); } - void addSchedulingResultPartition(SchedulingResultPartition srp) { + void addSchedulingResultPartition(SchedulingResultPartition srp) { intermediateDataSetManager.addSchedulingResultPartition(srp); } - private boolean checkAll(final SchedulingExecutionVertex schedulingExecutionVertex) { - return schedulingExecutionVertex.getConsumedResultPartitions() - .stream() + private boolean checkAll(final SchedulingExecutionVertex schedulingExecutionVertex) { + return IterableUtils.toStream(schedulingExecutionVertex.getConsumedResults()) .allMatch(this::partitionConsumable); } - private boolean checkAny(final SchedulingExecutionVertex schedulingExecutionVertex) { - return schedulingExecutionVertex.getConsumedResultPartitions() - .stream() + private boolean checkAny(final SchedulingExecutionVertex schedulingExecutionVertex) { + return IterableUtils.toStream(schedulingExecutionVertex.getConsumedResults()) .anyMatch(this::partitionConsumable); } - private boolean partitionConsumable(SchedulingResultPartition partition) { - if (BLOCKING.equals(partition.getPartitionType())) { + private boolean partitionConsumable(SchedulingResultPartition partition) { + if (BLOCKING.equals(partition.getResultType())) { return intermediateDataSetManager.allPartitionsFinished(partition); } else { SchedulingResultPartition.ResultPartitionState state = partition.getState(); @@ -92,7 +93,7 @@ private static class SchedulingIntermediateDataSetManager { private final Map intermediateDataSets = new HashMap<>(); - List markSchedulingResultPartitionFinished(SchedulingResultPartition srp) { + List> markSchedulingResultPartitionFinished(SchedulingResultPartition srp) { SchedulingIntermediateDataSet intermediateDataSet = getSchedulingIntermediateDataSet(srp.getResultId()); if (intermediateDataSet.markPartitionFinished(srp.getId())) { return intermediateDataSet.getSchedulingResultPartitions(); @@ -100,17 +101,17 @@ List markSchedulingResultPartitionFinished(Scheduling return Collections.emptyList(); } - void resetSchedulingResultPartition(SchedulingResultPartition srp) { + void resetSchedulingResultPartition(SchedulingResultPartition srp) { SchedulingIntermediateDataSet sid = getSchedulingIntermediateDataSet(srp.getResultId()); sid.resetPartition(srp.getId()); } - void addSchedulingResultPartition(SchedulingResultPartition srp) { + void addSchedulingResultPartition(SchedulingResultPartition srp) { SchedulingIntermediateDataSet sid = getOrCreateSchedulingIntermediateDataSetIfAbsent(srp.getResultId()); sid.addSchedulingResultPartition(srp); } - boolean allPartitionsFinished(SchedulingResultPartition srp) { + boolean allPartitionsFinished(SchedulingResultPartition srp) { SchedulingIntermediateDataSet sid = getSchedulingIntermediateDataSet(srp.getResultId()); return sid.allPartitionsFinished(); } @@ -146,7 +147,7 @@ private SchedulingIntermediateDataSet getSchedulingIntermediateDataSetInternal( */ private static class SchedulingIntermediateDataSet { - private final List partitions; + private final List> partitions; private final Set producingPartitionIds; @@ -168,12 +169,12 @@ boolean allPartitionsFinished() { return producingPartitionIds.isEmpty(); } - void addSchedulingResultPartition(SchedulingResultPartition partition) { + void addSchedulingResultPartition(SchedulingResultPartition partition) { partitions.add(partition); producingPartitionIds.add(partition.getId()); } - List getSchedulingResultPartitions() { + List> getSchedulingResultPartitions() { return Collections.unmodifiableList(partitions); } } 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 bc4ebb65c54a..adbdbcc63c98 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 @@ -24,8 +24,10 @@ import org.apache.flink.runtime.scheduler.DeploymentOption; import org.apache.flink.runtime.scheduler.ExecutionVertexDeploymentOption; import org.apache.flink.runtime.scheduler.SchedulerOperations; +import org.apache.flink.util.IterableUtils; + +import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables; -import java.util.Collection; import java.util.HashMap; import java.util.Map; import java.util.Set; @@ -42,11 +44,11 @@ */ public class LazyFromSourcesSchedulingStrategy implements SchedulingStrategy { - private static final Predicate IS_IN_CREATED_EXECUTION_STATE = schedulingExecutionVertex -> CREATED == schedulingExecutionVertex.getState(); + private static final Predicate> IS_IN_CREATED_EXECUTION_STATE = schedulingExecutionVertex -> CREATED == schedulingExecutionVertex.getState(); private final SchedulerOperations schedulerOperations; - private final SchedulingTopology schedulingTopology; + private final SchedulingTopology schedulingTopology; private final Map deploymentOptions; @@ -54,7 +56,7 @@ public class LazyFromSourcesSchedulingStrategy implements SchedulingStrategy { public LazyFromSourcesSchedulingStrategy( SchedulerOperations schedulerOperations, - SchedulingTopology schedulingTopology) { + SchedulingTopology schedulingTopology) { this.schedulerOperations = checkNotNull(schedulerOperations); this.schedulingTopology = checkNotNull(schedulingTopology); @@ -67,10 +69,10 @@ public void startScheduling() { final DeploymentOption updateOption = new DeploymentOption(true); final DeploymentOption nonUpdateOption = new DeploymentOption(false); - for (SchedulingExecutionVertex schedulingVertex : schedulingTopology.getVertices()) { + for (SchedulingExecutionVertex schedulingVertex : schedulingTopology.getVertices()) { DeploymentOption option = nonUpdateOption; - for (SchedulingResultPartition srp : schedulingVertex.getProducedResultPartitions()) { - if (srp.getPartitionType().isPipelined()) { + for (SchedulingResultPartition srp : schedulingVertex.getProducedResults()) { + if (srp.getResultType().isPipelined()) { option = updateOption; } inputConstraintChecker.addSchedulingResultPartition(srp); @@ -87,7 +89,7 @@ public void restartTasks(Set verticesToRestart) { verticesToRestart .stream() .map(schedulingTopology::getVertexOrThrow) - .flatMap(vertex -> vertex.getProducedResultPartitions().stream()) + .flatMap(vertex -> IterableUtils.toStream(vertex.getProducedResults())) .forEach(inputConstraintChecker::resetSchedulingResultPartition); allocateSlotsAndDeployExecutionVertexIds(verticesToRestart); @@ -99,12 +101,11 @@ public void onExecutionStateChange(ExecutionVertexID executionVertexId, Executio return; } - final Set verticesToSchedule = schedulingTopology.getVertexOrThrow(executionVertexId) - .getProducedResultPartitions() - .stream() - .filter(partition -> partition.getPartitionType().isBlocking()) + final Set> verticesToSchedule = IterableUtils + .toStream(schedulingTopology.getVertexOrThrow(executionVertexId).getProducedResults()) + .filter(partition -> partition.getResultType().isBlocking()) .flatMap(partition -> inputConstraintChecker.markSchedulingResultPartitionFinished(partition).stream()) - .flatMap(partition -> partition.getConsumers().stream()) + .flatMap(partition -> IterableUtils.toStream(partition.getConsumers())) .collect(Collectors.toSet()); allocateSlotsAndDeployExecutionVertices(verticesToSchedule); @@ -112,15 +113,15 @@ public void onExecutionStateChange(ExecutionVertexID executionVertexId, Executio @Override public void onPartitionConsumable(ExecutionVertexID executionVertexId, ResultPartitionID resultPartitionId) { - final SchedulingResultPartition resultPartition = schedulingTopology + final SchedulingResultPartition resultPartition = schedulingTopology .getResultPartitionOrThrow(resultPartitionId.getPartitionId()); - if (!resultPartition.getPartitionType().isPipelined()) { + if (!resultPartition.getResultType().isPipelined()) { return; } - final SchedulingExecutionVertex producerVertex = schedulingTopology.getVertexOrThrow(executionVertexId); - if (!producerVertex.getProducedResultPartitions().contains(resultPartition)) { + final SchedulingExecutionVertex producerVertex = schedulingTopology.getVertexOrThrow(executionVertexId); + if (!Iterables.contains(producerVertex.getProducedResults(), resultPartition)) { throw new IllegalStateException("partition " + resultPartitionId + " is not the produced partition of " + executionVertexId); } @@ -136,10 +137,11 @@ private void allocateSlotsAndDeployExecutionVertexIds(Set ver .collect(Collectors.toList())); } - private void allocateSlotsAndDeployExecutionVertices(final Collection schedulingExecutionVertices) { + private void allocateSlotsAndDeployExecutionVertices( + final Iterable> vertices) { + schedulerOperations.allocateSlotsAndDeploy( - schedulingExecutionVertices - .stream() + IterableUtils.toStream(vertices) .filter(isInputConstraintSatisfied().and(IS_IN_CREATED_EXECUTION_STATE)) .map(SchedulingExecutionVertex::getId) .map(executionVertexID -> new ExecutionVertexDeploymentOption( @@ -148,7 +150,7 @@ private void allocateSlotsAndDeployExecutionVertices(final Collection isInputConstraintSatisfied() { + private Predicate> isInputConstraintSatisfied() { return inputConstraintChecker::check; } @@ -166,7 +168,7 @@ public static class Factory implements SchedulingStrategyFactory { @Override public SchedulingStrategy createInstance( SchedulerOperations schedulerOperations, - SchedulingTopology schedulingTopology, + SchedulingTopology schedulingTopology, JobGraph jobGraph) { return new LazyFromSourcesSchedulingStrategy(schedulerOperations, schedulingTopology); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/SchedulingExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/SchedulingExecutionVertex.java index b4e1c2f06ec6..5c5cb31374fa 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/SchedulingExecutionVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/SchedulingExecutionVertex.java @@ -21,20 +21,14 @@ import org.apache.flink.api.common.InputDependencyConstraint; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.ExecutionVertex; - -import java.util.Collection; +import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.topology.Vertex; /** * Scheduling representation of {@link ExecutionVertex}. */ -public interface SchedulingExecutionVertex { - - /** - * Gets id of the execution vertex. - * - * @return id of the execution vertex - */ - ExecutionVertexID getId(); +public interface SchedulingExecutionVertex, R extends SchedulingResultPartition> + extends Vertex { /** * Gets the state of the execution vertex. @@ -43,20 +37,6 @@ public interface SchedulingExecutionVertex { */ ExecutionState getState(); - /** - * Get all consumed result partitions. - * - * @return collection of input partitions - */ - Collection getConsumedResultPartitions(); - - /** - * Get all produced result partitions. - * - * @return collection of output edges - */ - Collection getProducedResultPartitions(); - /** * Get {@link InputDependencyConstraint}. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/SchedulingResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/SchedulingResultPartition.java index 86ea8ba015a0..7a01e079ca37 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/SchedulingResultPartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/SchedulingResultPartition.java @@ -19,23 +19,15 @@ package org.apache.flink.runtime.scheduler.strategy; import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; -import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; - -import java.util.Collection; +import org.apache.flink.runtime.topology.Result; /** * Representation of {@link IntermediateResultPartition}. */ -public interface SchedulingResultPartition { - - /** - * Gets id of the result partition. - * - * @return id of the result partition - */ - IntermediateResultPartitionID getId(); +public interface SchedulingResultPartition, R extends SchedulingResultPartition> + extends Result { /** * Gets id of the intermediate result. @@ -44,13 +36,6 @@ public interface SchedulingResultPartition { */ IntermediateDataSetID getResultId(); - /** - * Gets the {@link ResultPartitionType}. - * - * @return result partition type - */ - ResultPartitionType getPartitionType(); - /** * Gets the {@link ResultPartitionState}. * @@ -58,20 +43,6 @@ public interface SchedulingResultPartition { */ ResultPartitionState getState(); - /** - * Gets the producer of this result partition. - * - * @return producer vertex of this result partition - */ - SchedulingExecutionVertex getProducer(); - - /** - * Gets the consumers of this result partition. - * - * @return Collection of consumer vertices of this result partition - */ - Collection getConsumers(); - /** * State of the result partition. */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/SchedulingStrategyFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/SchedulingStrategyFactory.java index 28b82577facf..28bca7f1c6a0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/SchedulingStrategyFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/SchedulingStrategyFactory.java @@ -28,6 +28,6 @@ public interface SchedulingStrategyFactory { SchedulingStrategy createInstance( SchedulerOperations schedulerOperations, - SchedulingTopology schedulingTopology, + SchedulingTopology schedulingTopology, JobGraph jobGraph); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/SchedulingTopology.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/SchedulingTopology.java index ce037a3617f3..ee6def912cdd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/SchedulingTopology.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/SchedulingTopology.java @@ -19,21 +19,15 @@ package org.apache.flink.runtime.scheduler.strategy; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.topology.Topology; import java.util.Optional; /** * Topology of {@link SchedulingExecutionVertex}. */ -public interface SchedulingTopology { - - /** - * Returns an iterable over all {@link SchedulingExecutionVertex} in topological - * sorted order. - * - * @return Iterable over all scheduling vertices in topological sorted order - */ - Iterable getVertices(); +public interface SchedulingTopology, R extends SchedulingResultPartition> + extends Topology { /** * Looks up the {@link SchedulingExecutionVertex} for the given {@link ExecutionVertexID}. @@ -41,7 +35,7 @@ public interface SchedulingTopology { * @param executionVertexId identifying the respective scheduling vertex * @return Optional containing the respective scheduling vertex or none if the vertex does not exist */ - Optional getVertex(ExecutionVertexID executionVertexId); + Optional getVertex(ExecutionVertexID executionVertexId); /** * Looks up the {@link SchedulingExecutionVertex} for the given {@link ExecutionVertexID}. @@ -50,7 +44,7 @@ public interface SchedulingTopology { * @return The respective scheduling vertex * @throws IllegalArgumentException If the vertex does not exist */ - default SchedulingExecutionVertex getVertexOrThrow(ExecutionVertexID executionVertexId) { + default V getVertexOrThrow(ExecutionVertexID executionVertexId) { return getVertex(executionVertexId).orElseThrow( () -> new IllegalArgumentException("can not find vertex: " + executionVertexId)); } @@ -61,7 +55,7 @@ default SchedulingExecutionVertex getVertexOrThrow(ExecutionVertexID executionVe * @param intermediateResultPartitionId identifying the respective scheduling result partition * @return Optional containing the respective scheduling result partition or none if the partition does not exist */ - Optional getResultPartition(IntermediateResultPartitionID intermediateResultPartitionId); + Optional getResultPartition(IntermediateResultPartitionID intermediateResultPartitionId); /** * Looks up the {@link SchedulingResultPartition} for the given {@link IntermediateResultPartitionID}. @@ -70,7 +64,7 @@ default SchedulingExecutionVertex getVertexOrThrow(ExecutionVertexID executionVe * @return The respective scheduling result partition * @throws IllegalArgumentException If the partition does not exist */ - default SchedulingResultPartition getResultPartitionOrThrow(IntermediateResultPartitionID intermediateResultPartitionId) { + default R getResultPartitionOrThrow(IntermediateResultPartitionID intermediateResultPartitionId) { return getResultPartition(intermediateResultPartitionId).orElseThrow( () -> new IllegalArgumentException("can not find partition: " + intermediateResultPartitionId)); } 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 97693dd367c2..66c449981fcd 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 @@ -294,11 +294,11 @@ public void vertexIsResetBeforeRestarted() throws Exception { final TestSchedulingStrategy.Factory schedulingStrategyFactory = new TestSchedulingStrategy.Factory(); final DefaultScheduler scheduler = createScheduler(jobGraph, schedulingStrategyFactory); final TestSchedulingStrategy schedulingStrategy = schedulingStrategyFactory.getLastCreatedSchedulingStrategy(); - final SchedulingTopology topology = schedulingStrategy.getSchedulingTopology(); + final SchedulingTopology topology = schedulingStrategy.getSchedulingTopology(); startScheduling(scheduler); - final SchedulingExecutionVertex onlySchedulingVertex = Iterables.getOnlyElement(topology.getVertices()); + final SchedulingExecutionVertex onlySchedulingVertex = Iterables.getOnlyElement(topology.getVertices()); schedulingStrategy.schedule(Collections.singleton(onlySchedulingVertex.getId())); final ArchivedExecutionVertex onlyExecutionVertex = Iterables.getOnlyElement(scheduler.requestJob().getAllExecutionVertices()); @@ -318,7 +318,7 @@ public void scheduleOnlyIfVertexIsCreated() throws Exception { final TestSchedulingStrategy.Factory schedulingStrategyFactory = new TestSchedulingStrategy.Factory(); final DefaultScheduler scheduler = createScheduler(jobGraph, schedulingStrategyFactory); final TestSchedulingStrategy schedulingStrategy = schedulingStrategyFactory.getLastCreatedSchedulingStrategy(); - final SchedulingTopology topology = schedulingStrategy.getSchedulingTopology(); + final SchedulingTopology topology = schedulingStrategy.getSchedulingTopology(); startScheduling(scheduler); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingExecutionVertexTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingExecutionVertexTest.java index 639548cec4fb..267610d985b4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingExecutionVertexTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingExecutionVertexTest.java @@ -24,6 +24,7 @@ import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; import org.apache.flink.runtime.scheduler.strategy.SchedulingResultPartition; +import org.apache.flink.util.IterableUtils; import org.apache.flink.util.TestLogger; import org.junit.Before; @@ -69,7 +70,7 @@ public void setUp() throws Exception { Collections.emptyList(), stateSupplier, ANY); - consumerVertex.addConsumedPartition(schedulingResultPartition); + consumerVertex.addConsumedResult(schedulingResultPartition); } @Test @@ -82,16 +83,20 @@ public void testGetExecutionState() { @Test public void testGetProducedResultPartitions() { - IntermediateResultPartitionID partitionIds1 = producerVertex - .getProducedResultPartitions().stream().findAny().map(SchedulingResultPartition::getId) + IntermediateResultPartitionID partitionIds1 = IterableUtils + .toStream(producerVertex.getProducedResults()) + .findAny() + .map(SchedulingResultPartition::getId) .orElseThrow(() -> new IllegalArgumentException("can not find result partition")); assertEquals(partitionIds1, intermediateResultPartitionId); } @Test public void testGetConsumedResultPartitions() { - IntermediateResultPartitionID partitionIds1 = consumerVertex - .getConsumedResultPartitions().stream().findAny().map(SchedulingResultPartition::getId) + IntermediateResultPartitionID partitionIds1 = IterableUtils + .toStream(consumerVertex.getConsumedResults()) + .findAny() + .map(SchedulingResultPartition::getId) .orElseThrow(() -> new IllegalArgumentException("can not find result partition")); assertEquals(partitionIds1, intermediateResultPartitionId); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/ExecutionGraphToSchedulingTopologyAdapterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/ExecutionGraphToSchedulingTopologyAdapterTest.java index ab72e2c209f3..9728501c57ed 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/ExecutionGraphToSchedulingTopologyAdapterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/ExecutionGraphToSchedulingTopologyAdapterTest.java @@ -28,12 +28,14 @@ import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; +import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; -import org.apache.flink.runtime.scheduler.strategy.SchedulingExecutionVertex; -import org.apache.flink.runtime.scheduler.strategy.SchedulingResultPartition; -import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology; +import org.apache.flink.util.IterableUtils; import org.apache.flink.util.TestLogger; +import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables; + import org.junit.Before; import org.junit.Test; @@ -51,6 +53,7 @@ import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.createNoOpVertex; import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.createSimpleTestGraph; import static org.apache.flink.runtime.io.network.partition.ResultPartitionType.BLOCKING; +import static org.apache.flink.runtime.io.network.partition.ResultPartitionType.PIPELINED; import static org.apache.flink.runtime.jobgraph.DistributionPattern.ALL_TO_ALL; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -97,7 +100,7 @@ public void testGetResultPartition() { for (ExecutionVertex vertex : executionGraph.getAllExecutionVertices()) { for (Map.Entry entry : vertex.getProducedPartitions().entrySet()) { IntermediateResultPartition partition = entry.getValue(); - SchedulingResultPartition schedulingResultPartition = adapter.getResultPartition(entry.getKey()) + DefaultSchedulingResultPartition schedulingResultPartition = adapter.getResultPartition(entry.getKey()) .orElseThrow(() -> new IllegalArgumentException("can not find partition " + entry.getKey())); assertPartitionEquals(partition, schedulingResultPartition); @@ -125,16 +128,52 @@ public void testResultPartitionOrThrow() { } } + @Test + public void testWithCoLocationConstraints() throws Exception { + ExecutionGraph executionGraph = createExecutionGraphWithCoLocationConstraint(); + adapter = new ExecutionGraphToSchedulingTopologyAdapter(executionGraph); + assertTrue(adapter.containsCoLocationConstraints()); + } + + @Test + public void testWithoutCoLocationConstraints() { + assertFalse(adapter.containsCoLocationConstraints()); + } + + private ExecutionGraph createExecutionGraphWithCoLocationConstraint() throws Exception { + JobVertex[] jobVertices = new JobVertex[2]; + int parallelism = 3; + jobVertices[0] = createNoOpVertex("v1", parallelism); + jobVertices[1] = createNoOpVertex("v2", parallelism); + jobVertices[1].connectNewDataSetAsInput(jobVertices[0], ALL_TO_ALL, PIPELINED); + + SlotSharingGroup slotSharingGroup = new SlotSharingGroup(); + jobVertices[0].setSlotSharingGroup(slotSharingGroup); + jobVertices[1].setSlotSharingGroup(slotSharingGroup); + + CoLocationGroup coLocationGroup = new CoLocationGroup(); + coLocationGroup.addVertex(jobVertices[0]); + coLocationGroup.addVertex(jobVertices[1]); + jobVertices[0].updateCoLocationGroup(coLocationGroup); + jobVertices[1].updateCoLocationGroup(coLocationGroup); + + return createSimpleTestGraph( + new JobID(), + taskManagerGateway, + triggeredRestartStrategy, + jobVertices); + } + private static void assertGraphEquals( ExecutionGraph originalGraph, - SchedulingTopology adaptedTopology) { + ExecutionGraphToSchedulingTopologyAdapter adaptedTopology) { Iterator originalVertices = originalGraph.getAllExecutionVertices().iterator(); - Iterator adaptedVertices = adaptedTopology.getVertices().iterator(); + Iterator adaptedVertices = adaptedTopology.getVertices().iterator(); while (originalVertices.hasNext()) { ExecutionVertex originalVertex = originalVertices.next(); - SchedulingExecutionVertex adaptedVertex = adaptedVertices.next(); + DefaultSchedulingExecutionVertex adaptedVertex = adaptedVertices.next(); assertVertexEquals(originalVertex, adaptedVertex); @@ -143,12 +182,12 @@ private static void assertGraphEquals( .flatMap(Arrays::stream) .map(ExecutionEdge::getSource) .collect(Collectors.toList()); - Collection adaptedConsumedPartitions = adaptedVertex.getConsumedResultPartitions(); + Iterable adaptedConsumedPartitions = adaptedVertex.getConsumedResults(); assertPartitionsEquals(originalConsumedPartitions, adaptedConsumedPartitions); Collection originalProducedPartitions = originalVertex.getProducedPartitions().values(); - Collection adaptedProducedPartitions = adaptedVertex.getProducedResultPartitions(); + Iterable adaptedProducedPartitions = adaptedVertex.getProducedResults(); assertPartitionsEquals(originalProducedPartitions, adaptedProducedPartitions); } @@ -157,13 +196,13 @@ private static void assertGraphEquals( } private static void assertPartitionsEquals( - Collection originalPartitions, - Collection adaptedPartitions) { + Iterable originalResultPartitions, + Iterable adaptedResultPartitions) { - assertEquals(originalPartitions.size(), adaptedPartitions.size()); + assertEquals(Iterables.size(originalResultPartitions), Iterables.size(adaptedResultPartitions)); - for (IntermediateResultPartition originalPartition : originalPartitions) { - SchedulingResultPartition adaptedPartition = adaptedPartitions.stream() + for (IntermediateResultPartition originalPartition : originalResultPartitions) { + DefaultSchedulingResultPartition adaptedPartition = IterableUtils.toStream(adaptedResultPartitions) .filter(adapted -> adapted.getId().equals(originalPartition.getPartitionId())) .findAny() .orElseThrow(() -> new AssertionError("Could not find matching adapted partition for " + originalPartition)); @@ -174,7 +213,7 @@ private static void assertPartitionsEquals( .flatMap(Collection::stream) .map(ExecutionEdge::getTarget) .collect(Collectors.toList()); - Collection adaptedConsumers = adaptedPartition.getConsumers(); + Iterable adaptedConsumers = adaptedPartition.getConsumers(); for (ExecutionVertex originalConsumer : originalConsumers) { // it is sufficient to verify that some vertex exists with the correct ID here, @@ -182,18 +221,18 @@ private static void assertPartitionsEquals( // this DOES rely on an implicit assumption that the vertices objects returned by the topology are // identical to those stored in the partition ExecutionVertexID originalId = originalConsumer.getID(); - assertTrue(adaptedConsumers.stream().anyMatch(adaptedConsumer -> adaptedConsumer.getId().equals(originalId))); + assertTrue(IterableUtils.toStream(adaptedConsumers).anyMatch(adaptedConsumer -> adaptedConsumer.getId().equals(originalId))); } } } private static void assertPartitionEquals( IntermediateResultPartition originalPartition, - SchedulingResultPartition adaptedPartition) { + DefaultSchedulingResultPartition adaptedPartition) { assertEquals(originalPartition.getPartitionId(), adaptedPartition.getId()); assertEquals(originalPartition.getIntermediateResult().getId(), adaptedPartition.getResultId()); - assertEquals(originalPartition.getResultType(), adaptedPartition.getPartitionType()); + assertEquals(originalPartition.getResultType(), adaptedPartition.getResultType()); assertVertexEquals( originalPartition.getProducer(), adaptedPartition.getProducer()); @@ -201,7 +240,8 @@ private static void assertPartitionEquals( private static void assertVertexEquals( ExecutionVertex originalVertex, - SchedulingExecutionVertex adaptedVertex) { + DefaultSchedulingExecutionVertex adaptedVertex) { + assertEquals( originalVertex.getID(), adaptedVertex.getId()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/EagerSchedulingStrategyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/EagerSchedulingStrategyTest.java index 364267c52efc..175aef96ed82 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/EagerSchedulingStrategyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/EagerSchedulingStrategyTest.java @@ -75,7 +75,7 @@ public void testStartScheduling() { Collection scheduledVertices = testingSchedulerOperations.getScheduledVertices().get(0); Collection scheduledVertexIDs = getExecutionVertexIdsFromDeployOptions(scheduledVertices); assertThat(scheduledVertexIDs, hasSize(5)); - for (SchedulingExecutionVertex schedulingExecutionVertex : testingSchedulingTopology.getVertices()) { + for (TestingSchedulingExecutionVertex schedulingExecutionVertex : testingSchedulingTopology.getVertices()) { assertThat(scheduledVertexIDs, hasItem(schedulingExecutionVertex.getId())); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/InputDependencyConstraintCheckerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/InputDependencyConstraintCheckerTest.java index a45961f27ad2..5db7dc528ef4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/InputDependencyConstraintCheckerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/InputDependencyConstraintCheckerTest.java @@ -242,7 +242,7 @@ private static InputDependencyConstraintChecker createInputDependencyConstraintC List partitions) { InputDependencyConstraintChecker inputChecker = new InputDependencyConstraintChecker(); - for (SchedulingResultPartition partition : partitions) { + for (SchedulingResultPartition partition : partitions) { inputChecker.addSchedulingResultPartition(partition); } return inputChecker; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/LazyFromSourcesSchedulingStrategyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/LazyFromSourcesSchedulingStrategyTest.java index 05e3ab09940b..11ee37cbd7d0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/LazyFromSourcesSchedulingStrategyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/LazyFromSourcesSchedulingStrategyTest.java @@ -253,7 +253,7 @@ public void testPipelinedPartitionConsumable() { final LazyFromSourcesSchedulingStrategy schedulingStrategy = startScheduling(testingSchedulingTopology); final TestingSchedulingExecutionVertex producer1 = producers.get(0); - final SchedulingResultPartition partition1 = producer1.getProducedResultPartitions().iterator().next(); + final TestingSchedulingResultPartition partition1 = producer1.getProducedResults().iterator().next(); schedulingStrategy.onExecutionStateChange(producer1.getId(), ExecutionState.RUNNING); schedulingStrategy.onPartitionConsumable(producer1.getId(), new ResultPartitionID(partition1.getId(), new ExecutionAttemptID())); 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 index eebec4fec184..b82060634ddd 100644 --- 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 @@ -39,7 +39,7 @@ public class TestSchedulingStrategy implements SchedulingStrategy { private final SchedulerOperations schedulerOperations; - private final SchedulingTopology schedulingTopology; + private final SchedulingTopology schedulingTopology; private final DeploymentOption deploymentOption = new DeploymentOption(false); @@ -47,7 +47,7 @@ public class TestSchedulingStrategy implements SchedulingStrategy { public TestSchedulingStrategy( final SchedulerOperations schedulerOperations, - final SchedulingTopology schedulingTopology) { + final SchedulingTopology schedulingTopology) { this.schedulerOperations = checkNotNull(schedulerOperations); this.schedulingTopology = checkNotNull(schedulingTopology); @@ -74,7 +74,7 @@ public void schedule(final Set verticesToSchedule) { allocateSlotsAndDeploy(verticesToSchedule); } - public SchedulingTopology getSchedulingTopology() { + public SchedulingTopology getSchedulingTopology() { return schedulingTopology; } @@ -108,7 +108,7 @@ public static class Factory implements SchedulingStrategyFactory { @Override public SchedulingStrategy createInstance( final SchedulerOperations schedulerOperations, - final SchedulingTopology schedulingTopology, + final SchedulingTopology schedulingTopology, final JobGraph jobGraph) { lastInstance = new TestSchedulingStrategy(schedulerOperations, schedulingTopology); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/TestingSchedulingExecutionVertex.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/TestingSchedulingExecutionVertex.java index 17f15c5a9d33..dcb0df54c99d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/TestingSchedulingExecutionVertex.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/TestingSchedulingExecutionVertex.java @@ -24,20 +24,20 @@ import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import static org.apache.flink.util.Preconditions.checkNotNull; /** * A simple scheduling execution vertex for testing purposes. */ -public class TestingSchedulingExecutionVertex implements SchedulingExecutionVertex { +public class TestingSchedulingExecutionVertex + implements SchedulingExecutionVertex { private final ExecutionVertexID executionVertexId; private final Collection consumedPartitions; - private final Collection producedPartitions; + private final Collection producedPartitions; private InputDependencyConstraint inputDependencyConstraint; @@ -74,13 +74,13 @@ public ExecutionState getState() { } @Override - public Collection getConsumedResultPartitions() { - return Collections.unmodifiableCollection(consumedPartitions); + public Iterable getConsumedResults() { + return consumedPartitions; } @Override - public Collection getProducedResultPartitions() { - return Collections.unmodifiableCollection(producedPartitions); + public Iterable getProducedResults() { + return producedPartitions; } @Override @@ -92,7 +92,7 @@ void addConsumedPartition(TestingSchedulingResultPartition partition) { consumedPartitions.add(partition); } - void addProducedPartition(SchedulingResultPartition partition) { + void addProducedPartition(TestingSchedulingResultPartition partition) { producedPartitions.add(partition); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/TestingSchedulingResultPartition.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/TestingSchedulingResultPartition.java index cb603195dea4..a878ebd33846 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/TestingSchedulingResultPartition.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/TestingSchedulingResultPartition.java @@ -24,23 +24,24 @@ import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import static org.apache.flink.util.Preconditions.checkNotNull; /** * A simple implementation of {@link SchedulingResultPartition} for testing. */ -public class TestingSchedulingResultPartition implements SchedulingResultPartition { +public class TestingSchedulingResultPartition + implements SchedulingResultPartition { + private final IntermediateDataSetID intermediateDataSetID; private final IntermediateResultPartitionID intermediateResultPartitionID; private final ResultPartitionType partitionType; - private SchedulingExecutionVertex producer; + private TestingSchedulingExecutionVertex producer; - private Collection consumers; + private Collection consumers; private ResultPartitionState state; @@ -63,7 +64,7 @@ public IntermediateDataSetID getResultId() { } @Override - public ResultPartitionType getPartitionType() { + public ResultPartitionType getResultType() { return partitionType; } @@ -73,16 +74,16 @@ public ResultPartitionState getState() { } @Override - public SchedulingExecutionVertex getProducer() { + public TestingSchedulingExecutionVertex getProducer() { return producer; } @Override - public Collection getConsumers() { - return Collections.unmodifiableCollection(consumers); + public Iterable getConsumers() { + return consumers; } - void addConsumer(SchedulingExecutionVertex consumer) { + void addConsumer(TestingSchedulingExecutionVertex consumer) { this.consumers.add(consumer); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/TestingSchedulingTopology.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/TestingSchedulingTopology.java index e0ea6c4e9497..c162ebf29dc9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/TestingSchedulingTopology.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/TestingSchedulingTopology.java @@ -27,7 +27,6 @@ import org.apache.flink.util.Preconditions; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -39,36 +38,42 @@ /** * A simple scheduling topology for testing purposes. */ -public class TestingSchedulingTopology implements SchedulingTopology { +public class TestingSchedulingTopology + implements SchedulingTopology { - private final Map schedulingExecutionVertices = new HashMap<>(); + private final Map schedulingExecutionVertices = new HashMap<>(); - private final Map schedulingResultPartitions = new HashMap<>(); + private final Map schedulingResultPartitions = new HashMap<>(); @Override - public Iterable getVertices() { + public Iterable getVertices() { return Collections.unmodifiableCollection(schedulingExecutionVertices.values()); } @Override - public Optional getVertex(ExecutionVertexID executionVertexId) { + public boolean containsCoLocationConstraints() { + return false; + } + + @Override + public Optional getVertex(ExecutionVertexID executionVertexId) { return Optional.ofNullable(schedulingExecutionVertices.get(executionVertexId)); } @Override - public Optional getResultPartition( + public Optional getResultPartition( IntermediateResultPartitionID intermediateResultPartitionId) { return Optional.of(schedulingResultPartitions.get(intermediateResultPartitionId)); } - void addSchedulingExecutionVertex(SchedulingExecutionVertex schedulingExecutionVertex) { + void addSchedulingExecutionVertex(TestingSchedulingExecutionVertex schedulingExecutionVertex) { schedulingExecutionVertices.put(schedulingExecutionVertex.getId(), schedulingExecutionVertex); - addSchedulingResultPartitions(schedulingExecutionVertex.getConsumedResultPartitions()); - addSchedulingResultPartitions(schedulingExecutionVertex.getProducedResultPartitions()); + addSchedulingResultPartitions(schedulingExecutionVertex.getConsumedResults()); + addSchedulingResultPartitions(schedulingExecutionVertex.getProducedResults()); } - private void addSchedulingResultPartitions(final Collection resultPartitions) { - for (SchedulingResultPartition schedulingResultPartition : resultPartitions) { + private void addSchedulingResultPartitions(final Iterable resultPartitions) { + for (TestingSchedulingResultPartition schedulingResultPartition : resultPartitions) { schedulingResultPartitions.put(schedulingResultPartition.getId(), schedulingResultPartition); } } From 72033165468df5d3d340e9dc837047483b3b066c Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Tue, 1 Oct 2019 21:35:29 +0800 Subject: [PATCH 292/746] [hotfix][runtime] Remove FailoverVertex#getExecutionVertexName which is not necessary. This is to simplify the topology interface. --- .../adapter/DefaultFailoverTopology.java | 4 +--- .../failover/adapter/DefaultFailoverVertex.java | 12 +----------- .../failover/flip1/FailoverVertex.java | 7 ------- .../flip1/PipelinedRegionComputeUtil.java | 4 ++-- .../adapter/DefaultFailoverTopologyTest.java | 4 ++-- .../failover/flip1/TestFailoverTopology.java | 16 ++-------------- 6 files changed, 8 insertions(+), 39 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverTopology.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverTopology.java index e27d2134276d..8818b24bdebb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverTopology.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverTopology.java @@ -54,9 +54,7 @@ public DefaultFailoverTopology(ExecutionGraph executionGraph) { this.failoverVertices = new ArrayList<>(); final Map failoverVertexMap = new IdentityHashMap<>(); for (ExecutionVertex vertex : executionGraph.getAllExecutionVertices()) { - final DefaultFailoverVertex failoverVertex = new DefaultFailoverVertex( - vertex.getID(), - vertex.getTaskNameWithSubtaskIndex()); + final DefaultFailoverVertex failoverVertex = new DefaultFailoverVertex(vertex.getID()); this.failoverVertices.add(failoverVertex); failoverVertexMap.put(vertex, failoverVertex); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverVertex.java index bf0072f81d44..dca7515aeabe 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverVertex.java @@ -34,18 +34,13 @@ class DefaultFailoverVertex implements FailoverVertex { private final ExecutionVertexID executionVertexID; - private final String executionVertexName; - private final List inputEdges; private final List outputEdges; - DefaultFailoverVertex( - ExecutionVertexID executionVertexID, - String executionVertexName) { + DefaultFailoverVertex(ExecutionVertexID executionVertexID) { this.executionVertexID = checkNotNull(executionVertexID); - this.executionVertexName = checkNotNull(executionVertexName); this.inputEdges = new ArrayList<>(); this.outputEdges = new ArrayList<>(); } @@ -55,11 +50,6 @@ public ExecutionVertexID getExecutionVertexID() { return executionVertexID; } - @Override - public String getExecutionVertexName() { - return executionVertexName; - } - @Override public Iterable getInputEdges() { return inputEdges; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverVertex.java index 7c1497a5511b..47970d77b789 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverVertex.java @@ -31,13 +31,6 @@ public interface FailoverVertex { */ ExecutionVertexID getExecutionVertexID(); - /** - * Returns the name of this vertex. - * - * @return name of this vertex - */ - String getExecutionVertexName(); - /** * Returns all input edges of this vertex. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/PipelinedRegionComputeUtil.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/PipelinedRegionComputeUtil.java index 8f19ed95686c..bfb4204384d1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/PipelinedRegionComputeUtil.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/PipelinedRegionComputeUtil.java @@ -74,9 +74,9 @@ public static Set> computePipelinedRegions(final FailoverTop final Set producerRegion = vertexToRegion.get(producerVertex); if (producerRegion == null) { - throw new IllegalStateException("Producer task " + producerVertex.getExecutionVertexName() + throw new IllegalStateException("Producer task " + producerVertex.getExecutionVertexID() + " failover region is null while calculating failover region for the consumer task " - + vertex.getExecutionVertexName() + ". This should be a failover region building bug."); + + vertex.getExecutionVertexID() + ". This should be a failover region building bug."); } // check if it is the same as the producer region, if so skip the merge diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverTopologyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverTopologyTest.java index 2dd39245c2ca..a09e8878f61a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverTopologyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverTopologyTest.java @@ -32,6 +32,7 @@ import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; import org.apache.flink.util.TestLogger; + import org.junit.Test; import java.util.Arrays; @@ -168,8 +169,7 @@ private static void assertVertexEquals(ExecutionVertex originalVertex, FailoverV private static boolean compareVertexInternalProperties(ExecutionVertex originalVertex, FailoverVertex adaptedVertex) { return originalVertex.getJobvertexId().equals(adaptedVertex.getExecutionVertexID().getJobVertexId()) && - originalVertex.getParallelSubtaskIndex() == adaptedVertex.getExecutionVertexID().getSubtaskIndex() && - originalVertex.getTaskNameWithSubtaskIndex().equals(adaptedVertex.getExecutionVertexName()); + originalVertex.getParallelSubtaskIndex() == adaptedVertex.getExecutionVertexID().getSubtaskIndex(); } private static void assertEdgesEquals( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/TestFailoverTopology.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/TestFailoverTopology.java index a1d688bff0ab..7ccda644ff55 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/TestFailoverTopology.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/TestFailoverTopology.java @@ -25,7 +25,6 @@ import java.util.ArrayList; import java.util.Collection; -import java.util.UUID; /** * A FailoverTopology implementation for tests. @@ -58,11 +57,9 @@ public static class TestFailoverVertex implements FailoverVertex { private final Collection inputEdges = new ArrayList<>(); private final Collection outputEdges = new ArrayList<>(); private final ExecutionVertexID id; - private final String name; - public TestFailoverVertex(ExecutionVertexID id, String name) { + public TestFailoverVertex(ExecutionVertexID id) { this.id = id; - this.name = name; } void addInputEdge(FailoverEdge edge) { @@ -77,11 +74,6 @@ public ExecutionVertexID getExecutionVertexID() { return id; } - @Override - public String getExecutionVertexName() { - return name; - } - @Override public Iterable getInputEdges() { return inputEdges::iterator; @@ -139,11 +131,7 @@ public static class Builder { private Collection vertices = new ArrayList<>(); public TestFailoverVertex newVertex() { - return newVertex(UUID.randomUUID().toString()); - } - - public TestFailoverVertex newVertex(String name) { - TestFailoverVertex testFailoverVertex = new TestFailoverVertex(new ExecutionVertexID(new JobVertexID(), 0), name); + TestFailoverVertex testFailoverVertex = new TestFailoverVertex(new ExecutionVertexID(new JobVertexID(), 0)); vertices.add(testFailoverVertex); return testFailoverVertex; } From b84337a1085a8bbc101ef9f9dcd698b60507ba30 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Mon, 21 Oct 2019 11:58:06 +0800 Subject: [PATCH 293/746] [FLINK-14451][runtime] Refactor FailoverTopology to extend base topology [FLINK-14451][runtime] Refactor FailoverTopology to extend base topology (for review - Part2: implementations) Also drops DefaultFailoverTopology and uses ExecutionGraphToSchedulingTopologyAdapter instead [FLINK-14451][runtime] Refactor FailoverTopology to extend base topology (for review - Part3: usages) This closes #9948. --- .../executiongraph/ExecutionGraph.java | 3 +- ...aptedRestartPipelinedRegionStrategyNG.java | 4 +- .../failover/adapter/DefaultFailoverEdge.java | 72 ------- .../adapter/DefaultFailoverTopology.java | 95 --------- .../adapter/DefaultFailoverVertex.java | 70 ------ .../flip1/ExecutionFailureHandler.java | 11 +- .../failover/flip1/FailoverRegion.java | 8 +- ...Edge.java => FailoverResultPartition.java} | 39 +--- .../failover/flip1/FailoverStrategy.java | 2 +- .../failover/flip1/FailoverTopology.java | 24 +-- .../failover/flip1/FailoverVertex.java | 29 +-- .../flip1/PipelinedRegionComputeUtil.java | 48 +++-- .../flip1/RestartPipelinedRegionStrategy.java | 47 +++-- .../NotReleasingPartitionReleaseStrategy.java | 5 +- .../PartitionReleaseStrategy.java | 2 +- .../RegionPartitionReleaseStrategy.java | 4 +- .../runtime/scheduler/SchedulerBase.java | 7 +- .../DefaultSchedulingExecutionVertex.java | 4 +- .../DefaultSchedulingResultPartition.java | 4 +- ...utionGraphToSchedulingTopologyAdapter.java | 4 +- .../adapter/DefaultFailoverTopologyTest.java | 199 ------------------ .../flip1/ExecutionFailureHandlerTest.java | 8 +- ...rtPipelinedRegionStrategyBuildingTest.java | 142 ++++++------- .../RestartPipelinedRegionStrategyTest.java | 174 +++++++-------- .../failover/flip1/TestFailoverTopology.java | 90 ++++---- 25 files changed, 315 insertions(+), 780 deletions(-) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverEdge.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverTopology.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverVertex.java rename flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/{FailoverEdge.java => FailoverResultPartition.java} (50%) delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverTopologyTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 8bbf1448c1b2..95e2844c1f1f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -49,7 +49,6 @@ import org.apache.flink.runtime.execution.SuppressRestartsException; import org.apache.flink.runtime.executiongraph.failover.FailoverStrategy; import org.apache.flink.runtime.executiongraph.failover.RestartAllStrategy; -import org.apache.flink.runtime.executiongraph.failover.adapter.DefaultFailoverTopology; import org.apache.flink.runtime.executiongraph.failover.flip1.ResultPartitionAvailabilityChecker; import org.apache.flink.runtime.executiongraph.failover.flip1.partitionrelease.NotReleasingPartitionReleaseStrategy; import org.apache.flink.runtime.executiongraph.failover.flip1.partitionrelease.PartitionReleaseStrategy; @@ -951,7 +950,7 @@ public void attachJobGraph(List topologiallySorted) throws JobExcepti schedulingTopology = new ExecutionGraphToSchedulingTopologyAdapter(this); partitionReleaseStrategy = partitionReleaseStrategyFactory.createInstance( schedulingTopology, - new DefaultFailoverTopology(this)); + new ExecutionGraphToSchedulingTopologyAdapter(this)); } public boolean isLegacyScheduling() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java index 2d211dc459da..a69801bc2821 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java @@ -28,7 +28,6 @@ import org.apache.flink.runtime.executiongraph.ExecutionVertex; import org.apache.flink.runtime.executiongraph.GlobalModVersionMismatch; import org.apache.flink.runtime.executiongraph.SchedulingUtils; -import org.apache.flink.runtime.executiongraph.failover.adapter.DefaultFailoverTopology; import org.apache.flink.runtime.executiongraph.failover.flip1.RestartPipelinedRegionStrategy; import org.apache.flink.runtime.executiongraph.restart.RestartCallback; import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; @@ -37,6 +36,7 @@ import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; import org.apache.flink.runtime.scheduler.ExecutionVertexVersion; import org.apache.flink.runtime.scheduler.ExecutionVertexVersioner; +import org.apache.flink.runtime.scheduler.adapter.ExecutionGraphToSchedulingTopologyAdapter; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; import org.apache.flink.util.ExceptionUtils; @@ -295,7 +295,7 @@ public void notifyNewVertices(final List newJobVerticesTopol // currently it's safe to add it here, as this method is invoked only once in production code. checkState(restartPipelinedRegionStrategy == null, "notifyNewVertices() must be called only once"); this.restartPipelinedRegionStrategy = new RestartPipelinedRegionStrategy( - new DefaultFailoverTopology(executionGraph), executionGraph.getResultPartitionAvailabilityChecker()); + new ExecutionGraphToSchedulingTopologyAdapter(executionGraph), executionGraph.getResultPartitionAvailabilityChecker()); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverEdge.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverEdge.java deleted file mode 100644 index f1094b5cb7fd..000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverEdge.java +++ /dev/null @@ -1,72 +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.adapter; - -import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverEdge; -import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverVertex; -import org.apache.flink.runtime.io.network.partition.ResultPartitionType; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * Default implementation of {@link FailoverEdge}. - */ -class DefaultFailoverEdge implements FailoverEdge { - - private final IntermediateResultPartitionID resultPartitionId; - - private final ResultPartitionType partitionType; - - private final DefaultFailoverVertex sourceVertex; - - private final DefaultFailoverVertex targetVertex; - - DefaultFailoverEdge( - IntermediateResultPartitionID partitionId, - ResultPartitionType partitionType, - DefaultFailoverVertex sourceVertex, - DefaultFailoverVertex targetVertex) { - - this.resultPartitionId = checkNotNull(partitionId); - this.partitionType = checkNotNull(partitionType); - this.sourceVertex = checkNotNull(sourceVertex); - this.targetVertex = checkNotNull(targetVertex); - } - - @Override - public IntermediateResultPartitionID getResultPartitionID() { - return resultPartitionId; - } - - @Override - public ResultPartitionType getResultPartitionType() { - return partitionType; - } - - @Override - public FailoverVertex getSourceVertex() { - return sourceVertex; - } - - @Override - public FailoverVertex getTargetVertex() { - return targetVertex; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverTopology.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverTopology.java deleted file mode 100644 index 8818b24bdebb..000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverTopology.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.executiongraph.failover.adapter; - -import org.apache.flink.runtime.executiongraph.ExecutionGraph; -import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; -import org.apache.flink.runtime.executiongraph.ExecutionVertex; -import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; -import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverTopology; -import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverVertex; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.IdentityHashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * Default implementation of {@link FailoverTopology} which is an adaptor of {@link ExecutionGraph}. - */ -public class DefaultFailoverTopology implements FailoverTopology { - - private final boolean containsCoLocationConstraints; - - private final List failoverVertices; - - public DefaultFailoverTopology(ExecutionGraph executionGraph) { - checkNotNull(executionGraph); - - this.containsCoLocationConstraints = executionGraph.getAllVertices().values().stream() - .map(ExecutionJobVertex::getCoLocationGroup) - .anyMatch(Objects::nonNull); - - // generate vertices - this.failoverVertices = new ArrayList<>(); - final Map failoverVertexMap = new IdentityHashMap<>(); - for (ExecutionVertex vertex : executionGraph.getAllExecutionVertices()) { - final DefaultFailoverVertex failoverVertex = new DefaultFailoverVertex(vertex.getID()); - this.failoverVertices.add(failoverVertex); - failoverVertexMap.put(vertex, failoverVertex); - } - - // generate edges - connectVerticesWithEdges(failoverVertexMap); - } - - private void connectVerticesWithEdges(Map failoverVertexMap) { - for (ExecutionVertex vertex : failoverVertexMap.keySet()) { - final DefaultFailoverVertex failoverVertex = failoverVertexMap.get(vertex); - vertex.getProducedPartitions().values().stream() - .map(IntermediateResultPartition::getConsumers) - .flatMap(Collection::stream) - .flatMap(Collection::stream) - .forEach(e -> { - final DefaultFailoverVertex consumerFailoverVertex = failoverVertexMap.get(e.getTarget()); - final DefaultFailoverEdge failoverEdge = new DefaultFailoverEdge( - e.getSource().getPartitionId(), - e.getSource().getResultType(), - failoverVertex, - consumerFailoverVertex); - failoverVertex.addOutputEdge(failoverEdge); - consumerFailoverVertex.addInputEdge(failoverEdge); - }); - } - } - - @Override - public Iterable getFailoverVertices() { - return failoverVertices; - } - - @Override - public boolean containsCoLocationConstraints() { - return containsCoLocationConstraints; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverVertex.java deleted file mode 100644 index dca7515aeabe..000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverVertex.java +++ /dev/null @@ -1,70 +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.adapter; - -import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverEdge; -import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverVertex; -import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; - -import java.util.ArrayList; -import java.util.List; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * Default implementation of {@link FailoverVertex}. - */ -class DefaultFailoverVertex implements FailoverVertex { - - private final ExecutionVertexID executionVertexID; - - private final List inputEdges; - - private final List outputEdges; - - DefaultFailoverVertex(ExecutionVertexID executionVertexID) { - - this.executionVertexID = checkNotNull(executionVertexID); - this.inputEdges = new ArrayList<>(); - this.outputEdges = new ArrayList<>(); - } - - @Override - public ExecutionVertexID getExecutionVertexID() { - return executionVertexID; - } - - @Override - public Iterable getInputEdges() { - return inputEdges; - } - - @Override - public Iterable getOutputEdges() { - return outputEdges; - } - - void addInputEdge(DefaultFailoverEdge edge) { - inputEdges.add(edge); - } - - void addOutputEdge(DefaultFailoverEdge edge) { - outputEdges.add(edge); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandler.java index 882fc13e9a06..3a6753ef5fb3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandler.java @@ -22,11 +22,11 @@ import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; import org.apache.flink.runtime.throwable.ThrowableClassifier; import org.apache.flink.runtime.throwable.ThrowableType; +import org.apache.flink.util.IterableUtils; import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; -import java.util.stream.StreamSupport; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -36,7 +36,7 @@ */ public class ExecutionFailureHandler { - private final FailoverTopology failoverTopology; + private final FailoverTopology failoverTopology; /** Strategy to judge which tasks should be restarted. */ private final FailoverStrategy failoverStrategy; @@ -52,7 +52,7 @@ public class ExecutionFailureHandler { * @param restartBackoffTimeStrategy helps to decide whether to restart failed tasks and the restarting delay */ public ExecutionFailureHandler( - FailoverTopology failoverTopology, + FailoverTopology failoverTopology, FailoverStrategy failoverStrategy, RestartBackoffTimeStrategy restartBackoffTimeStrategy) { @@ -83,9 +83,8 @@ public FailureHandlingResult getFailureHandlingResult(ExecutionVertexID failedTa public FailureHandlingResult getGlobalFailureHandlingResult(final Throwable cause) { return handleFailure( cause, - StreamSupport - .stream(failoverTopology.getFailoverVertices().spliterator(), false) - .map(FailoverVertex::getExecutionVertexID) + IterableUtils.toStream(failoverTopology.getVertices()) + .map(FailoverVertex::getId) .collect(Collectors.toSet())); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverRegion.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverRegion.java index 9a013df03b1c..86b1f6455ad9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverRegion.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverRegion.java @@ -34,17 +34,17 @@ public class FailoverRegion { private final Set executionVertexIDs; /** All vertices in this region. */ - private final Set executionVertices; + private final Set> executionVertices; /** * Creates a new failover region containing a set of vertices. * * @param executionVertices to be contained in this region */ - public FailoverRegion(Set executionVertices) { + public FailoverRegion(Set> executionVertices) { this.executionVertices = checkNotNull(executionVertices); this.executionVertexIDs = new HashSet<>(); - executionVertices.forEach(v -> this.executionVertexIDs.add(v.getExecutionVertexID())); + executionVertices.forEach(v -> this.executionVertexIDs.add(v.getId())); } /** @@ -61,7 +61,7 @@ public Set getAllExecutionVertexIDs() { * * @return all vertices in this region */ - public Set getAllExecutionVertices() { + public Set> getAllExecutionVertices() { return executionVertices; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverEdge.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverResultPartition.java similarity index 50% rename from flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverEdge.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverResultPartition.java index 73dac88d0fdf..39f243516810 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverEdge.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverResultPartition.java @@ -17,41 +17,14 @@ package org.apache.flink.runtime.executiongraph.failover.flip1; -import org.apache.flink.runtime.io.network.partition.ResultPartitionType; +import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.topology.Result; /** - * A connection between {@link FailoverVertex FailoverVertices}. - * - *

producer -> ResultPartition -> consumer + * Represents a {@link IntermediateResultPartition} produced by a {@link FailoverVertex}. */ -public interface FailoverEdge { - - /** - * Returns the ID of the result partition that the source produces. - * - * @return ID of the result partition that the source produces - */ - IntermediateResultPartitionID getResultPartitionID(); - - /** - * Returns the {@link ResultPartitionType} of the produced result partition. - * - * @return type of the produced result partition - */ - ResultPartitionType getResultPartitionType(); - - /** - * Returns the source vertex, i.e., the producer of the result partition. - * - * @return source vertex - */ - FailoverVertex getSourceVertex(); - - /** - * Returns the target vertex, i.e., the consumer of the result partition. - * - * @return target vertex - */ - FailoverVertex getTargetVertex(); +public interface FailoverResultPartition, R extends FailoverResultPartition> + extends Result { } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverStrategy.java index 0bd0c01b0abe..23d4f7f92504 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverStrategy.java @@ -50,6 +50,6 @@ interface Factory { * @param topology of the graph to failover * @return The instantiated failover strategy. */ - FailoverStrategy create(FailoverTopology topology); + FailoverStrategy create(FailoverTopology topology); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverTopology.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverTopology.java index 18f48fcb8189..b9d0b6f2815f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverTopology.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverTopology.java @@ -17,23 +17,13 @@ package org.apache.flink.runtime.executiongraph.failover.flip1; +import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.topology.Topology; + /** - * Represents a topology. + * Represents a topology for failover. */ -public interface FailoverTopology { - - /** - * Returns an iterable over all vertices, topologically sorted. - * - * @return topologically sorted iterable over all vertices - */ - Iterable getFailoverVertices(); - - /** - * Returns whether the topology contains co-location constraints. - * Co-location constraints are currently used for iterations. - * - * @return whether the topology contains co-location constraints - */ - boolean containsCoLocationConstraints(); +public interface FailoverTopology, R extends FailoverResultPartition> + extends Topology { } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverVertex.java index 47970d77b789..62f2eb7ef405 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverVertex.java @@ -17,31 +17,14 @@ package org.apache.flink.runtime.executiongraph.failover.flip1; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.topology.Vertex; /** - * Represents an ExecutionVertex. + * Represents an {@link ExecutionVertex}. */ -public interface FailoverVertex { - - /** - * Returns the ID of this vertex. - * - * @return ID of this vertex - */ - ExecutionVertexID getExecutionVertexID(); - - /** - * Returns all input edges of this vertex. - * - * @return input edges of this vertex - */ - Iterable getInputEdges(); - - /** - * Returns all output edges of this vertex. - * - * @return output edges of this vertex - */ - Iterable getOutputEdges(); +public interface FailoverVertex, R extends FailoverResultPartition> + extends Vertex { } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/PipelinedRegionComputeUtil.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/PipelinedRegionComputeUtil.java index bfb4204384d1..516176263ada 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/PipelinedRegionComputeUtil.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/PipelinedRegionComputeUtil.java @@ -40,43 +40,43 @@ public final class PipelinedRegionComputeUtil { private static final Logger LOG = LoggerFactory.getLogger(PipelinedRegionComputeUtil.class); - public static Set toPipelinedRegionsSet(final Set> distinctRegions) { + public static Set toPipelinedRegionsSet(final Set>> distinctRegions) { return distinctRegions.stream() .map(toExecutionVertexIdSet()) .map(PipelinedRegion::from) .collect(Collectors.toSet()); } - private static Function, Set> toExecutionVertexIdSet() { + private static Function>, Set> toExecutionVertexIdSet() { return failoverVertices -> failoverVertices.stream() - .map(FailoverVertex::getExecutionVertexID) + .map(FailoverVertex::getId) .collect(Collectors.toSet()); } - public static Set> computePipelinedRegions(final FailoverTopology topology) { + public static Set>> computePipelinedRegions(final FailoverTopology topology) { // currently we let a job with co-location constraints fail as one region // putting co-located vertices in the same region with each other can be a future improvement if (topology.containsCoLocationConstraints()) { return uniqueRegions(buildOneRegionForAllVertices(topology)); } - final Map> vertexToRegion = new IdentityHashMap<>(); + final Map, Set>> vertexToRegion = new IdentityHashMap<>(); // iterate all the vertices which are topologically sorted - for (FailoverVertex vertex : topology.getFailoverVertices()) { - Set currentRegion = new HashSet<>(1); + for (FailoverVertex vertex : topology.getVertices()) { + Set> currentRegion = new HashSet<>(1); currentRegion.add(vertex); vertexToRegion.put(vertex, currentRegion); - for (FailoverEdge inputEdge : vertex.getInputEdges()) { - if (inputEdge.getResultPartitionType().isPipelined()) { - final FailoverVertex producerVertex = inputEdge.getSourceVertex(); - final Set producerRegion = vertexToRegion.get(producerVertex); + for (FailoverResultPartition consumedResult : vertex.getConsumedResults()) { + if (consumedResult.getResultType().isPipelined()) { + final FailoverVertex producerVertex = consumedResult.getProducer(); + final Set> producerRegion = vertexToRegion.get(producerVertex); if (producerRegion == null) { - throw new IllegalStateException("Producer task " + producerVertex.getExecutionVertexID() + throw new IllegalStateException("Producer task " + producerVertex.getId() + " failover region is null while calculating failover region for the consumer task " - + vertex.getExecutionVertexID() + ". This should be a failover region building bug."); + + vertex.getId() + ". This should be a failover region building bug."); } // check if it is the same as the producer region, if so skip the merge @@ -84,8 +84,8 @@ public static Set> computePipelinedRegions(final FailoverTop if (currentRegion != producerRegion) { // merge current region and producer region // merge the smaller region into the larger one to reduce the cost - final Set smallerSet; - final Set largerSet; + final Set> smallerSet; + final Set> largerSet; if (currentRegion.size() < producerRegion.size()) { smallerSet = currentRegion; largerSet = producerRegion; @@ -93,7 +93,7 @@ public static Set> computePipelinedRegions(final FailoverTop smallerSet = producerRegion; largerSet = currentRegion; } - for (FailoverVertex v : smallerSet) { + for (FailoverVertex v : smallerSet) { vertexToRegion.put(v, largerSet); } largerSet.addAll(smallerSet); @@ -106,22 +106,26 @@ public static Set> computePipelinedRegions(final FailoverTop return uniqueRegions(vertexToRegion); } - private static Map> buildOneRegionForAllVertices(final FailoverTopology topology) { + private static Map, Set>> buildOneRegionForAllVertices( + final FailoverTopology topology) { + LOG.warn("Cannot decompose the topology into individual failover regions due to use of " + "Co-Location constraints (iterations). Job will fail over as one holistic unit."); - final Map> vertexToRegion = new IdentityHashMap<>(); + final Map, Set>> vertexToRegion = new IdentityHashMap<>(); - final Set allVertices = new HashSet<>(); - for (FailoverVertex vertex : topology.getFailoverVertices()) { + final Set> allVertices = new HashSet<>(); + for (FailoverVertex vertex : topology.getVertices()) { allVertices.add(vertex); vertexToRegion.put(vertex, allVertices); } return vertexToRegion; } - private static Set> uniqueRegions(final Map> vertexToRegion) { - final Set> distinctRegions = Collections.newSetFromMap(new IdentityHashMap<>()); + private static Set>> uniqueRegions( + final Map, Set>> vertexToRegion) { + + final Set>> distinctRegions = Collections.newSetFromMap(new IdentityHashMap<>()); distinctRegions.addAll(vertexToRegion.values()); return distinctRegions; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionStrategy.java index 941b0497bb49..a6ac8f0696ea 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionStrategy.java @@ -47,8 +47,8 @@ public class RestartPipelinedRegionStrategy implements FailoverStrategy { /** The log object used for debugging. */ private static final Logger LOG = LoggerFactory.getLogger(RestartPipelinedRegionStrategy.class); - /** The topology containing info about all the vertices and edges. */ - private final FailoverTopology topology; + /** The topology containing info about all the vertices and result partitions. */ + private final FailoverTopology topology; /** All failover regions. */ private final Set regions; @@ -63,21 +63,21 @@ public class RestartPipelinedRegionStrategy implements FailoverStrategy { * Creates a new failover strategy to restart pipelined regions that works on the given topology. * The result partitions are always considered to be available if no data consumption error happens. * - * @param topology containing info about all the vertices and edges + * @param topology containing info about all the vertices and result partitions */ @VisibleForTesting - public RestartPipelinedRegionStrategy(FailoverTopology topology) { + public RestartPipelinedRegionStrategy(FailoverTopology topology) { this(topology, resultPartitionID -> true); } /** * Creates a new failover strategy to restart pipelined regions that works on the given topology. * - * @param topology containing info about all the vertices and edges + * @param topology containing info about all the vertices and result partitions * @param resultPartitionAvailabilityChecker helps to query result partition availability */ public RestartPipelinedRegionStrategy( - FailoverTopology topology, + FailoverTopology topology, ResultPartitionAvailabilityChecker resultPartitionAvailabilityChecker) { this.topology = checkNotNull(topology); @@ -95,15 +95,15 @@ public RestartPipelinedRegionStrategy( // ------------------------------------------------------------------------ private void buildFailoverRegions() { - final Set> distinctRegions = PipelinedRegionComputeUtil.computePipelinedRegions(topology); + final Set>> distinctRegions = PipelinedRegionComputeUtil.computePipelinedRegions(topology); // creating all the failover regions and register them - for (Set regionVertices : distinctRegions) { + for (Set> regionVertices : distinctRegions) { LOG.debug("Creating a failover region with {} vertices.", regionVertices.size()); final FailoverRegion failoverRegion = new FailoverRegion(regionVertices); regions.add(failoverRegion); - for (FailoverVertex vertex : regionVertices) { - vertexToRegionMap.put(vertex.getExecutionVertexID(), failoverRegion); + for (FailoverVertex vertex : regionVertices) { + vertexToRegionMap.put(vertex.getId(), failoverRegion); } } @@ -186,10 +186,10 @@ private Set getRegionsToRestart(FailoverRegion failedRegion) { regionsToRestart.add(regionToRestart); // if a needed input result partition is not available, its producer region is involved - for (FailoverVertex vertex : regionToRestart.getAllExecutionVertices()) { - for (FailoverEdge inEdge : vertex.getInputEdges()) { - if (!resultPartitionAvailabilityChecker.isAvailable(inEdge.getResultPartitionID())) { - FailoverRegion producerRegion = vertexToRegionMap.get(inEdge.getSourceVertex().getExecutionVertexID()); + for (FailoverVertex vertex : regionToRestart.getAllExecutionVertices()) { + for (FailoverResultPartition consumedPartition : vertex.getConsumedResults()) { + if (!resultPartitionAvailabilityChecker.isAvailable(consumedPartition.getId())) { + FailoverRegion producerRegion = vertexToRegionMap.get(consumedPartition.getProducer().getId()); if (!visitedRegions.contains(producerRegion)) { visitedRegions.add(producerRegion); regionsToVisit.add(producerRegion); @@ -199,12 +199,14 @@ private Set getRegionsToRestart(FailoverRegion failedRegion) { } // all consumer regions of an involved region should be involved - for (FailoverVertex vertex : regionToRestart.getAllExecutionVertices()) { - for (FailoverEdge outEdge : vertex.getOutputEdges()) { - FailoverRegion consumerRegion = vertexToRegionMap.get(outEdge.getTargetVertex().getExecutionVertexID()); - if (!visitedRegions.contains(consumerRegion)) { - visitedRegions.add(consumerRegion); - regionsToVisit.add(consumerRegion); + for (FailoverVertex vertex : regionToRestart.getAllExecutionVertices()) { + for (FailoverResultPartition producedPartition : vertex.getProducedResults()) { + for (FailoverVertex consumerVertex : producedPartition.getConsumers()) { + FailoverRegion consumerRegion = vertexToRegionMap.get(consumerVertex.getId()); + if (!visitedRegions.contains(consumerRegion)) { + visitedRegions.add(consumerRegion); + regionsToVisit.add(consumerRegion); + } } } } @@ -258,10 +260,13 @@ public void removeResultPartitionFromFailedState(IntermediateResultPartitionID r } } + /** + * The factory to instantiate {@link RestartPipelinedRegionStrategy}. + */ public static class Factory implements FailoverStrategy.Factory { @Override - public FailoverStrategy create(final FailoverTopology topology) { + public FailoverStrategy create(final FailoverTopology topology) { return new RestartPipelinedRegionStrategy(topology); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/NotReleasingPartitionReleaseStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/NotReleasingPartitionReleaseStrategy.java index 65d0c55dc73d..fd8efe35306a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/NotReleasingPartitionReleaseStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/NotReleasingPartitionReleaseStrategy.java @@ -48,7 +48,10 @@ public void vertexUnfinished(final ExecutionVertexID executionVertexID) { public static class Factory implements PartitionReleaseStrategy.Factory { @Override - public PartitionReleaseStrategy createInstance(final SchedulingTopology schedulingStrategy, final FailoverTopology failoverTopology) { + public PartitionReleaseStrategy createInstance( + final SchedulingTopology schedulingStrategy, + final FailoverTopology failoverTopology) { + return new NotReleasingPartitionReleaseStrategy(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/PartitionReleaseStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/PartitionReleaseStrategy.java index fc99d7ce3b5e..461b93a2d76f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/PartitionReleaseStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/PartitionReleaseStrategy.java @@ -53,6 +53,6 @@ public interface PartitionReleaseStrategy { * Factory for {@link PartitionReleaseStrategy}. */ interface Factory { - PartitionReleaseStrategy createInstance(SchedulingTopology schedulingStrategy, FailoverTopology failoverTopology); + PartitionReleaseStrategy createInstance(SchedulingTopology schedulingStrategy, FailoverTopology failoverTopology); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/RegionPartitionReleaseStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/RegionPartitionReleaseStrategy.java index b069d0a63ce8..7c452ddfbb73 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/RegionPartitionReleaseStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/RegionPartitionReleaseStrategy.java @@ -178,9 +178,9 @@ public static class Factory implements PartitionReleaseStrategy.Factory { @Override public PartitionReleaseStrategy createInstance( final SchedulingTopology schedulingStrategy, - final FailoverTopology failoverTopology) { + final FailoverTopology failoverTopology) { - final Set> distinctRegions = PipelinedRegionComputeUtil.computePipelinedRegions(failoverTopology); + final Set>> distinctRegions = PipelinedRegionComputeUtil.computePipelinedRegions(failoverTopology); return new RegionPartitionReleaseStrategy( schedulingStrategy, PipelinedRegionComputeUtil.toPipelinedRegionsSet(distinctRegions)); 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 0ba5b092ab67..3f24f5cfbda8 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 @@ -48,7 +48,6 @@ 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; @@ -111,7 +110,7 @@ public abstract class SchedulerBase implements SchedulerNG { private final SchedulingTopology schedulingTopology; - private final FailoverTopology failoverTopology; + private final FailoverTopology failoverTopology; private final InputsLocationsRetriever inputsLocationsRetriever; @@ -187,7 +186,7 @@ public SchedulerBase( this.executionGraph = createAndRestoreExecutionGraph(jobManagerJobMetricGroup, checkNotNull(shuffleMaster), checkNotNull(partitionTracker)); this.schedulingTopology = new ExecutionGraphToSchedulingTopologyAdapter(executionGraph); - this.failoverTopology = new DefaultFailoverTopology(executionGraph); + this.failoverTopology = new ExecutionGraphToSchedulingTopologyAdapter(executionGraph); this.inputsLocationsRetriever = new ExecutionGraphToInputsLocationsRetrieverAdapter(executionGraph); } @@ -297,7 +296,7 @@ protected void failJob(Throwable cause) { executionGraph.failJob(cause); } - protected final FailoverTopology getFailoverTopology() { + protected final FailoverTopology getFailoverTopology() { return failoverTopology; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingExecutionVertex.java index b9a8e195f7cf..25246cc73374 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingExecutionVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingExecutionVertex.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.InputDependencyConstraint; import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverVertex; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; import org.apache.flink.runtime.scheduler.strategy.SchedulingExecutionVertex; @@ -33,7 +34,8 @@ * Default implementation of {@link SchedulingExecutionVertex}. */ class DefaultSchedulingExecutionVertex - implements SchedulingExecutionVertex { + implements SchedulingExecutionVertex, + FailoverVertex { private final ExecutionVertexID executionVertexId; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingResultPartition.java index 6f0ba419e26c..ebaf95ddd6f5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingResultPartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingResultPartition.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.scheduler.adapter; +import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverResultPartition; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; @@ -35,7 +36,8 @@ * Default implementation of {@link SchedulingResultPartition}. */ class DefaultSchedulingResultPartition - implements SchedulingResultPartition { + implements SchedulingResultPartition, + FailoverResultPartition { private final IntermediateResultPartitionID resultPartitionId; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/ExecutionGraphToSchedulingTopologyAdapter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/ExecutionGraphToSchedulingTopologyAdapter.java index 048a537dfcae..d0bf99427cb0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/ExecutionGraphToSchedulingTopologyAdapter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/ExecutionGraphToSchedulingTopologyAdapter.java @@ -24,6 +24,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.executiongraph.ExecutionVertex; import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; +import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverTopology; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology; @@ -42,7 +43,8 @@ * Adapter of {@link ExecutionGraph} to {@link SchedulingTopology}. */ public class ExecutionGraphToSchedulingTopologyAdapter - implements SchedulingTopology { + implements SchedulingTopology, + FailoverTopology { private final boolean containsCoLocationConstraints; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverTopologyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverTopologyTest.java deleted file mode 100644 index a09e8878f61a..000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/adapter/DefaultFailoverTopologyTest.java +++ /dev/null @@ -1,199 +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.adapter; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.runtime.executiongraph.ExecutionEdge; -import org.apache.flink.runtime.executiongraph.ExecutionGraph; -import org.apache.flink.runtime.executiongraph.ExecutionVertex; -import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; -import org.apache.flink.runtime.executiongraph.TestRestartStrategy; -import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverEdge; -import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverTopology; -import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverVertex; -import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; -import org.apache.flink.runtime.jobgraph.JobVertex; -import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; -import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; -import org.apache.flink.util.TestLogger; - -import org.junit.Test; - -import java.util.Arrays; -import java.util.Collection; -import java.util.List; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import java.util.stream.StreamSupport; - -import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.createNoOpVertex; -import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.createSimpleTestGraph; -import static org.apache.flink.runtime.io.network.partition.ResultPartitionType.BLOCKING; -import static org.apache.flink.runtime.io.network.partition.ResultPartitionType.PIPELINED; -import static org.apache.flink.runtime.jobgraph.DistributionPattern.ALL_TO_ALL; -import static org.apache.flink.runtime.jobgraph.DistributionPattern.POINTWISE; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -/** - * Unit tests for {@link DefaultFailoverTopology}. - */ -public class DefaultFailoverTopologyTest extends TestLogger { - - private final SimpleAckingTaskManagerGateway taskManagerGateway = new SimpleAckingTaskManagerGateway(); - - private final TestRestartStrategy triggeredRestartStrategy = TestRestartStrategy.manuallyTriggered(); - - /** - * Tests that the generated failover topology is strictly matched with the given ExecutionGraph. - */ - @Test - public void testTopology() throws Exception { - ExecutionGraph executionGraph = createExecutionGraph(); - DefaultFailoverTopology adapter = new DefaultFailoverTopology(executionGraph); - assertGraphEquals(executionGraph, adapter); - } - - /** - * Tests the case that the graph has collocation constraints. - */ - @Test - public void testWithCollocationConstraints() throws Exception { - ExecutionGraph executionGraph = createExecutionGraph(true); - DefaultFailoverTopology adapter = new DefaultFailoverTopology(executionGraph); - assertTrue(adapter.containsCoLocationConstraints()); - } - - /** - * Tests the case that the graph has no collocation constraint. - */ - @Test - public void testWithoutCollocationConstraints() throws Exception { - ExecutionGraph executionGraph = createExecutionGraph(false); - DefaultFailoverTopology adapter = new DefaultFailoverTopology(executionGraph); - assertFalse(adapter.containsCoLocationConstraints()); - } - - private ExecutionGraph createExecutionGraph() throws Exception { - return createExecutionGraph(false); - } - - private ExecutionGraph createExecutionGraph(boolean addCollocationConstraints) throws Exception { - JobVertex[] jobVertices = new JobVertex[3]; - int parallelism = 3; - jobVertices[0] = createNoOpVertex("v1", parallelism); - jobVertices[1] = createNoOpVertex("v2", parallelism); - jobVertices[2] = createNoOpVertex("v3", parallelism); - jobVertices[1].connectNewDataSetAsInput(jobVertices[0], ALL_TO_ALL, BLOCKING); - jobVertices[2].connectNewDataSetAsInput(jobVertices[1], POINTWISE, PIPELINED); - - if (addCollocationConstraints) { - SlotSharingGroup slotSharingGroup = new SlotSharingGroup(); - jobVertices[1].setSlotSharingGroup(slotSharingGroup); - jobVertices[2].setSlotSharingGroup(slotSharingGroup); - - CoLocationGroup coLocationGroup = new CoLocationGroup(); - coLocationGroup.addVertex(jobVertices[1]); - coLocationGroup.addVertex(jobVertices[2]); - jobVertices[1].updateCoLocationGroup(coLocationGroup); - jobVertices[2].updateCoLocationGroup(coLocationGroup); - } - - return createSimpleTestGraph( - new JobID(), - taskManagerGateway, - triggeredRestartStrategy, - jobVertices); - } - - private static void assertGraphEquals(ExecutionGraph originalGraph, FailoverTopology adaptedTopology) { - List originalVertices = StreamSupport.stream( - originalGraph.getAllExecutionVertices().spliterator(), - false).collect(Collectors.toList()); - List adaptedVertices = StreamSupport.stream( - adaptedTopology.getFailoverVertices().spliterator(), - false).collect(Collectors.toList()); - - assertEquals(originalVertices.size(), adaptedVertices.size()); - - // the vertices should be topologically sorted, i.e. in the same order - for (int i = 0; i < originalVertices.size(); i++) { - ExecutionVertex originalVertex = originalVertices.get(i); - FailoverVertex adaptedVertex = adaptedVertices.get(i); - assertVertexEquals(originalVertex, adaptedVertex); - } - } - - private static void assertVertexEquals(ExecutionVertex originalVertex, FailoverVertex adaptedVertex) { - // compare vertex internal properties - assertTrue(compareVertexInternalProperties(originalVertex, adaptedVertex)); - - // compare input edges - List originalInputEdges = IntStream.range(0, originalVertex.getNumberOfInputs()) - .mapToObj(originalVertex::getInputEdges) - .flatMap(Arrays::stream) - .collect(Collectors.toList()); - List adaptedInputEdges = StreamSupport.stream( - adaptedVertex.getInputEdges().spliterator(), - false).collect(Collectors.toList()); - assertEdgesEquals(originalInputEdges, adaptedInputEdges); - - // compare output edges - List originalOutputEdges = originalVertex.getProducedPartitions().values().stream() - .map(IntermediateResultPartition::getConsumers) - .flatMap(Collection::stream) - .flatMap(Collection::stream) - .collect(Collectors.toList()); - List adaptedOutputEdges = StreamSupport.stream( - adaptedVertex.getOutputEdges().spliterator(), - false).collect(Collectors.toList()); - assertEdgesEquals(originalOutputEdges, adaptedOutputEdges); - } - - private static boolean compareVertexInternalProperties(ExecutionVertex originalVertex, FailoverVertex adaptedVertex) { - return originalVertex.getJobvertexId().equals(adaptedVertex.getExecutionVertexID().getJobVertexId()) && - originalVertex.getParallelSubtaskIndex() == adaptedVertex.getExecutionVertexID().getSubtaskIndex(); - } - - private static void assertEdgesEquals( - Collection originalEdges, - Collection adaptedEdges) { - - assertEquals(originalEdges.size(), adaptedEdges.size()); - - for (ExecutionEdge originalEdge : originalEdges) { - List matchedAdaptedEdges = adaptedEdges.stream() - .filter(adapted -> compareEdge(originalEdge, adapted)) - .collect(Collectors.toList()); - - // there should be exactly 1 matched edge - // this ensures originalEdges and adaptedEdges elements have a one-to-one mapping - // it also helps to ensure there is no duplicated edge - assertEquals(1, matchedAdaptedEdges.size()); - } - } - - private static boolean compareEdge(ExecutionEdge originalEdge, FailoverEdge adaptedEdge) { - return originalEdge.getSource().getPartitionId().equals(adaptedEdge.getResultPartitionID()) && - originalEdge.getSource().getResultType().equals(adaptedEdge.getResultPartitionType()) && - compareVertexInternalProperties(originalEdge.getSource().getProducer(), adaptedEdge.getSourceVertex()) && - compareVertexInternalProperties(originalEdge.getTarget(), adaptedEdge.getTargetVertex()); - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandlerTest.java index 56d0c188a401..bf4feec3df1f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandlerTest.java @@ -21,6 +21,7 @@ import org.apache.flink.runtime.execution.SuppressRestartsException; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.util.IterableUtils; import org.apache.flink.util.TestLogger; import org.junit.Before; @@ -29,7 +30,6 @@ import java.util.Collections; import java.util.Set; import java.util.stream.Collectors; -import java.util.stream.StreamSupport; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -44,7 +44,7 @@ public class ExecutionFailureHandlerTest extends TestLogger { private static final long RESTART_DELAY_MS = 1234L; - private FailoverTopology failoverTopology; + private FailoverTopology failoverTopology; private TestFailoverStrategy failoverStrategy; @@ -170,8 +170,8 @@ public void testGlobalFailureHandling() { new Exception("test failure")); assertEquals( - StreamSupport.stream(failoverTopology.getFailoverVertices().spliterator(), false) - .map(FailoverVertex::getExecutionVertexID) + IterableUtils.toStream(failoverTopology.getVertices()) + .map(FailoverVertex::getId) .collect(Collectors.toSet()), result.getVerticesToRestart()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionStrategyBuildingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionStrategyBuildingTest.java index 106322222ad9..bd501c72504b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionStrategyBuildingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionStrategyBuildingTest.java @@ -51,13 +51,13 @@ public void testIndividualVertices() throws Exception { TestFailoverTopology.TestFailoverVertex v2 = topologyBuilder.newVertex(); TestFailoverTopology.TestFailoverVertex v3 = topologyBuilder.newVertex(); - FailoverTopology topology = topologyBuilder.build(); + TestFailoverTopology topology = topologyBuilder.build(); RestartPipelinedRegionStrategy strategy = new RestartPipelinedRegionStrategy(topology); - FailoverRegion r1 = strategy.getFailoverRegion(v1.getExecutionVertexID()); - FailoverRegion r2 = strategy.getFailoverRegion(v2.getExecutionVertexID()); - FailoverRegion r3 = strategy.getFailoverRegion(v3.getExecutionVertexID()); + FailoverRegion r1 = strategy.getFailoverRegion(v1.getId()); + FailoverRegion r2 = strategy.getFailoverRegion(v2.getId()); + FailoverRegion r3 = strategy.getFailoverRegion(v3.getId()); assertDistinctRegions(r1, r2, r3); } @@ -89,16 +89,16 @@ public void testEmbarrassinglyParallelCase() throws Exception { .connect(va2, vb2, ResultPartitionType.PIPELINED) .connect(va3, vb3, ResultPartitionType.PIPELINED); - FailoverTopology topology = topologyBuilder.build(); + TestFailoverTopology topology = topologyBuilder.build(); RestartPipelinedRegionStrategy strategy = new RestartPipelinedRegionStrategy(topology); - FailoverRegion ra1 = strategy.getFailoverRegion(va1.getExecutionVertexID()); - FailoverRegion ra2 = strategy.getFailoverRegion(va2.getExecutionVertexID()); - FailoverRegion ra3 = strategy.getFailoverRegion(va3.getExecutionVertexID()); - FailoverRegion rb1 = strategy.getFailoverRegion(vb1.getExecutionVertexID()); - FailoverRegion rb2 = strategy.getFailoverRegion(vb2.getExecutionVertexID()); - FailoverRegion rb3 = strategy.getFailoverRegion(vb3.getExecutionVertexID()); + FailoverRegion ra1 = strategy.getFailoverRegion(va1.getId()); + FailoverRegion ra2 = strategy.getFailoverRegion(va2.getId()); + FailoverRegion ra3 = strategy.getFailoverRegion(va3.getId()); + FailoverRegion rb1 = strategy.getFailoverRegion(vb1.getId()); + FailoverRegion rb2 = strategy.getFailoverRegion(vb2.getId()); + FailoverRegion rb3 = strategy.getFailoverRegion(vb3.getId()); assertSameRegion(ra1, rb1); assertSameRegion(ra2, rb2); @@ -138,16 +138,16 @@ public void testOneComponentViaTwoExchanges() throws Exception { .connect(vb2, vc1, ResultPartitionType.PIPELINED) .connect(vb2, vc2, ResultPartitionType.PIPELINED); - FailoverTopology topology = topologyBuilder.build(); + TestFailoverTopology topology = topologyBuilder.build(); RestartPipelinedRegionStrategy strategy = new RestartPipelinedRegionStrategy(topology); - FailoverRegion ra1 = strategy.getFailoverRegion(va1.getExecutionVertexID()); - FailoverRegion ra2 = strategy.getFailoverRegion(va2.getExecutionVertexID()); - FailoverRegion rb1 = strategy.getFailoverRegion(vb1.getExecutionVertexID()); - FailoverRegion rb2 = strategy.getFailoverRegion(vb2.getExecutionVertexID()); - FailoverRegion rc1 = strategy.getFailoverRegion(vc1.getExecutionVertexID()); - FailoverRegion rc2 = strategy.getFailoverRegion(vc2.getExecutionVertexID()); + FailoverRegion ra1 = strategy.getFailoverRegion(va1.getId()); + FailoverRegion ra2 = strategy.getFailoverRegion(va2.getId()); + FailoverRegion rb1 = strategy.getFailoverRegion(vb1.getId()); + FailoverRegion rb2 = strategy.getFailoverRegion(vb2.getId()); + FailoverRegion rc1 = strategy.getFailoverRegion(vc1.getId()); + FailoverRegion rc2 = strategy.getFailoverRegion(vc2.getId()); assertSameRegion(ra1, ra2, rb1, rb2, rc1, rc2); } @@ -186,17 +186,17 @@ public void testOneComponentViaCascadeOfJoins() throws Exception { .connect(v5, v7, ResultPartitionType.PIPELINED) .connect(v6, v7, ResultPartitionType.PIPELINED); - FailoverTopology topology = topologyBuilder.build(); + TestFailoverTopology topology = topologyBuilder.build(); RestartPipelinedRegionStrategy strategy = new RestartPipelinedRegionStrategy(topology); - FailoverRegion r1 = strategy.getFailoverRegion(v1.getExecutionVertexID()); - FailoverRegion r2 = strategy.getFailoverRegion(v2.getExecutionVertexID()); - FailoverRegion r3 = strategy.getFailoverRegion(v3.getExecutionVertexID()); - FailoverRegion r4 = strategy.getFailoverRegion(v4.getExecutionVertexID()); - FailoverRegion r5 = strategy.getFailoverRegion(v5.getExecutionVertexID()); - FailoverRegion r6 = strategy.getFailoverRegion(v6.getExecutionVertexID()); - FailoverRegion r7 = strategy.getFailoverRegion(v7.getExecutionVertexID()); + FailoverRegion r1 = strategy.getFailoverRegion(v1.getId()); + FailoverRegion r2 = strategy.getFailoverRegion(v2.getId()); + FailoverRegion r3 = strategy.getFailoverRegion(v3.getId()); + FailoverRegion r4 = strategy.getFailoverRegion(v4.getId()); + FailoverRegion r5 = strategy.getFailoverRegion(v5.getId()); + FailoverRegion r6 = strategy.getFailoverRegion(v6.getId()); + FailoverRegion r7 = strategy.getFailoverRegion(v7.getId()); assertSameRegion(r1, r2, r3, r4, r5, r6, r7); } @@ -235,17 +235,17 @@ public void testOneComponentInstanceFromOneSource() throws Exception { .connect(v3, v6, ResultPartitionType.PIPELINED) .connect(v3, v7, ResultPartitionType.PIPELINED); - FailoverTopology topology = topologyBuilder.build(); + TestFailoverTopology topology = topologyBuilder.build(); RestartPipelinedRegionStrategy strategy = new RestartPipelinedRegionStrategy(topology); - FailoverRegion r1 = strategy.getFailoverRegion(v1.getExecutionVertexID()); - FailoverRegion r2 = strategy.getFailoverRegion(v2.getExecutionVertexID()); - FailoverRegion r3 = strategy.getFailoverRegion(v3.getExecutionVertexID()); - FailoverRegion r4 = strategy.getFailoverRegion(v4.getExecutionVertexID()); - FailoverRegion r5 = strategy.getFailoverRegion(v5.getExecutionVertexID()); - FailoverRegion r6 = strategy.getFailoverRegion(v6.getExecutionVertexID()); - FailoverRegion r7 = strategy.getFailoverRegion(v7.getExecutionVertexID()); + FailoverRegion r1 = strategy.getFailoverRegion(v1.getId()); + FailoverRegion r2 = strategy.getFailoverRegion(v2.getId()); + FailoverRegion r3 = strategy.getFailoverRegion(v3.getId()); + FailoverRegion r4 = strategy.getFailoverRegion(v4.getId()); + FailoverRegion r5 = strategy.getFailoverRegion(v5.getId()); + FailoverRegion r6 = strategy.getFailoverRegion(v6.getId()); + FailoverRegion r7 = strategy.getFailoverRegion(v7.getId()); assertSameRegion(r1, r2, r3, r4, r5, r6, r7); } @@ -281,16 +281,16 @@ public void testTwoComponentsViaBlockingExchange() throws Exception { .connect(vb1, vc1, ResultPartitionType.BLOCKING) .connect(vb2, vc2, ResultPartitionType.BLOCKING); - FailoverTopology topology = topologyBuilder.build(); + TestFailoverTopology topology = topologyBuilder.build(); RestartPipelinedRegionStrategy strategy = new RestartPipelinedRegionStrategy(topology); - FailoverRegion ra1 = strategy.getFailoverRegion(va1.getExecutionVertexID()); - FailoverRegion ra2 = strategy.getFailoverRegion(va2.getExecutionVertexID()); - FailoverRegion rb1 = strategy.getFailoverRegion(vb1.getExecutionVertexID()); - FailoverRegion rb2 = strategy.getFailoverRegion(vb2.getExecutionVertexID()); - FailoverRegion rc1 = strategy.getFailoverRegion(vc1.getExecutionVertexID()); - FailoverRegion rc2 = strategy.getFailoverRegion(vc2.getExecutionVertexID()); + FailoverRegion ra1 = strategy.getFailoverRegion(va1.getId()); + FailoverRegion ra2 = strategy.getFailoverRegion(va2.getId()); + FailoverRegion rb1 = strategy.getFailoverRegion(vb1.getId()); + FailoverRegion rb2 = strategy.getFailoverRegion(vb2.getId()); + FailoverRegion rc1 = strategy.getFailoverRegion(vc1.getId()); + FailoverRegion rc2 = strategy.getFailoverRegion(vc2.getId()); assertSameRegion(ra1, ra2, rb1, rb2); @@ -330,16 +330,16 @@ public void testTwoComponentsViaBlockingExchange2() throws Exception { .connect(vb2, vc1, ResultPartitionType.BLOCKING) .connect(vb2, vc2, ResultPartitionType.BLOCKING); - FailoverTopology topology = topologyBuilder.build(); + TestFailoverTopology topology = topologyBuilder.build(); RestartPipelinedRegionStrategy strategy = new RestartPipelinedRegionStrategy(topology); - FailoverRegion ra1 = strategy.getFailoverRegion(va1.getExecutionVertexID()); - FailoverRegion ra2 = strategy.getFailoverRegion(va2.getExecutionVertexID()); - FailoverRegion rb1 = strategy.getFailoverRegion(vb1.getExecutionVertexID()); - FailoverRegion rb2 = strategy.getFailoverRegion(vb2.getExecutionVertexID()); - FailoverRegion rc1 = strategy.getFailoverRegion(vc1.getExecutionVertexID()); - FailoverRegion rc2 = strategy.getFailoverRegion(vc2.getExecutionVertexID()); + FailoverRegion ra1 = strategy.getFailoverRegion(va1.getId()); + FailoverRegion ra2 = strategy.getFailoverRegion(va2.getId()); + FailoverRegion rb1 = strategy.getFailoverRegion(vb1.getId()); + FailoverRegion rb2 = strategy.getFailoverRegion(vb2.getId()); + FailoverRegion rc1 = strategy.getFailoverRegion(vc1.getId()); + FailoverRegion rc2 = strategy.getFailoverRegion(vc2.getId()); assertSameRegion(ra1, ra2, rb1, rb2); @@ -384,17 +384,17 @@ public void testMultipleComponentsViaCascadeOfJoins() throws Exception { .connect(v5, v7, ResultPartitionType.BLOCKING) .connect(v6, v7, ResultPartitionType.BLOCKING); - FailoverTopology topology = topologyBuilder.build(); + TestFailoverTopology topology = topologyBuilder.build(); RestartPipelinedRegionStrategy strategy = new RestartPipelinedRegionStrategy(topology); - FailoverRegion r1 = strategy.getFailoverRegion(v1.getExecutionVertexID()); - FailoverRegion r2 = strategy.getFailoverRegion(v2.getExecutionVertexID()); - FailoverRegion r3 = strategy.getFailoverRegion(v3.getExecutionVertexID()); - FailoverRegion r4 = strategy.getFailoverRegion(v4.getExecutionVertexID()); - FailoverRegion r5 = strategy.getFailoverRegion(v5.getExecutionVertexID()); - FailoverRegion r6 = strategy.getFailoverRegion(v6.getExecutionVertexID()); - FailoverRegion r7 = strategy.getFailoverRegion(v7.getExecutionVertexID()); + FailoverRegion r1 = strategy.getFailoverRegion(v1.getId()); + FailoverRegion r2 = strategy.getFailoverRegion(v2.getId()); + FailoverRegion r3 = strategy.getFailoverRegion(v3.getId()); + FailoverRegion r4 = strategy.getFailoverRegion(v4.getId()); + FailoverRegion r5 = strategy.getFailoverRegion(v5.getId()); + FailoverRegion r6 = strategy.getFailoverRegion(v6.getId()); + FailoverRegion r7 = strategy.getFailoverRegion(v7.getId()); assertSameRegion(r1, r2, r5); assertSameRegion(r3, r4, r6); @@ -430,14 +430,14 @@ public void testDiamondWithMixedPipelinedAndBlockingExchanges() throws Exception .connect(v2, v4, ResultPartitionType.PIPELINED) .connect(v3, v4, ResultPartitionType.PIPELINED); - FailoverTopology topology = topologyBuilder.build(); + TestFailoverTopology topology = topologyBuilder.build(); RestartPipelinedRegionStrategy strategy = new RestartPipelinedRegionStrategy(topology); - FailoverRegion r1 = strategy.getFailoverRegion(v1.getExecutionVertexID()); - FailoverRegion r2 = strategy.getFailoverRegion(v2.getExecutionVertexID()); - FailoverRegion r3 = strategy.getFailoverRegion(v3.getExecutionVertexID()); - FailoverRegion r4 = strategy.getFailoverRegion(v4.getExecutionVertexID()); + FailoverRegion r1 = strategy.getFailoverRegion(v1.getId()); + FailoverRegion r2 = strategy.getFailoverRegion(v2.getId()); + FailoverRegion r3 = strategy.getFailoverRegion(v3.getId()); + FailoverRegion r4 = strategy.getFailoverRegion(v4.getId()); assertSameRegion(r1, r2, r3, r4); } @@ -472,14 +472,14 @@ public void testBlockingAllToAllTopologyWithCoLocation() throws Exception { .connect(va2, vb2, ResultPartitionType.BLOCKING); topologyBuilder.setContainsCoLocationConstraints(true); - FailoverTopology topology = topologyBuilder.build(); + TestFailoverTopology topology = topologyBuilder.build(); RestartPipelinedRegionStrategy strategy = new RestartPipelinedRegionStrategy(topology); - FailoverRegion ra1 = strategy.getFailoverRegion(va1.getExecutionVertexID()); - FailoverRegion ra2 = strategy.getFailoverRegion(va2.getExecutionVertexID()); - FailoverRegion rb1 = strategy.getFailoverRegion(vb1.getExecutionVertexID()); - FailoverRegion rb2 = strategy.getFailoverRegion(vb2.getExecutionVertexID()); + FailoverRegion ra1 = strategy.getFailoverRegion(va1.getId()); + FailoverRegion ra2 = strategy.getFailoverRegion(va2.getId()); + FailoverRegion rb1 = strategy.getFailoverRegion(vb1.getId()); + FailoverRegion rb2 = strategy.getFailoverRegion(vb2.getId()); assertSameRegion(ra1, ra2, rb1, rb2); } @@ -508,14 +508,14 @@ public void testPipelinedOneToOneTopologyWithCoLocation() throws Exception { .connect(va2, vb2, ResultPartitionType.PIPELINED); topologyBuilder.setContainsCoLocationConstraints(true); - FailoverTopology topology = topologyBuilder.build(); + TestFailoverTopology topology = topologyBuilder.build(); RestartPipelinedRegionStrategy strategy = new RestartPipelinedRegionStrategy(topology); - FailoverRegion ra1 = strategy.getFailoverRegion(va1.getExecutionVertexID()); - FailoverRegion ra2 = strategy.getFailoverRegion(va2.getExecutionVertexID()); - FailoverRegion rb1 = strategy.getFailoverRegion(vb1.getExecutionVertexID()); - FailoverRegion rb2 = strategy.getFailoverRegion(vb2.getExecutionVertexID()); + FailoverRegion ra1 = strategy.getFailoverRegion(va1.getId()); + FailoverRegion ra2 = strategy.getFailoverRegion(va2.getId()); + FailoverRegion rb1 = strategy.getFailoverRegion(vb1.getId()); + FailoverRegion rb2 = strategy.getFailoverRegion(vb2.getId()); assertSameRegion(ra1, ra2, rb1, rb2); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionStrategyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionStrategyTest.java index 5a9c844d6ed4..531bf86a0dd2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionStrategyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionStrategyTest.java @@ -74,50 +74,50 @@ public void testRegionFailoverForRegionInternalErrors() throws Exception { topologyBuilder.connect(v2, v5, ResultPartitionType.BLOCKING); topologyBuilder.connect(v3, v6, ResultPartitionType.BLOCKING); - FailoverTopology topology = topologyBuilder.build(); + TestFailoverTopology topology = topologyBuilder.build(); RestartPipelinedRegionStrategy strategy = new RestartPipelinedRegionStrategy(topology); // when v1 fails, {v1,v4,v5} should be restarted HashSet expectedResult = new HashSet<>(); - expectedResult.add(v1.getExecutionVertexID()); - expectedResult.add(v4.getExecutionVertexID()); - expectedResult.add(v5.getExecutionVertexID()); + expectedResult.add(v1.getId()); + expectedResult.add(v4.getId()); + expectedResult.add(v5.getId()); assertEquals(expectedResult, - strategy.getTasksNeedingRestart(v1.getExecutionVertexID(), new Exception("Test failure"))); + strategy.getTasksNeedingRestart(v1.getId(), new Exception("Test failure"))); // when v2 fails, {v2,v4,v5} should be restarted expectedResult.clear(); - expectedResult.add(v2.getExecutionVertexID()); - expectedResult.add(v4.getExecutionVertexID()); - expectedResult.add(v5.getExecutionVertexID()); + expectedResult.add(v2.getId()); + expectedResult.add(v4.getId()); + expectedResult.add(v5.getId()); assertEquals(expectedResult, - strategy.getTasksNeedingRestart(v2.getExecutionVertexID(), new Exception("Test failure"))); + strategy.getTasksNeedingRestart(v2.getId(), new Exception("Test failure"))); // when v3 fails, {v3,v6} should be restarted expectedResult.clear(); - expectedResult.add(v3.getExecutionVertexID()); - expectedResult.add(v6.getExecutionVertexID()); + expectedResult.add(v3.getId()); + expectedResult.add(v6.getId()); assertEquals(expectedResult, - strategy.getTasksNeedingRestart(v3.getExecutionVertexID(), new Exception("Test failure"))); + strategy.getTasksNeedingRestart(v3.getId(), new Exception("Test failure"))); // when v4 fails, {v4} should be restarted expectedResult.clear(); - expectedResult.add(v4.getExecutionVertexID()); + expectedResult.add(v4.getId()); assertEquals(expectedResult, - strategy.getTasksNeedingRestart(v4.getExecutionVertexID(), new Exception("Test failure"))); + strategy.getTasksNeedingRestart(v4.getId(), new Exception("Test failure"))); // when v5 fails, {v5} should be restarted expectedResult.clear(); - expectedResult.add(v5.getExecutionVertexID()); + expectedResult.add(v5.getId()); assertEquals(expectedResult, - strategy.getTasksNeedingRestart(v5.getExecutionVertexID(), new Exception("Test failure"))); + strategy.getTasksNeedingRestart(v5.getId(), new Exception("Test failure"))); // when v6 fails, {v6} should be restarted expectedResult.clear(); - expectedResult.add(v6.getExecutionVertexID()); + expectedResult.add(v6.getId()); assertEquals(expectedResult, - strategy.getTasksNeedingRestart(v6.getExecutionVertexID(), new Exception("Test failure"))); + strategy.getTasksNeedingRestart(v6.getId(), new Exception("Test failure"))); } /** @@ -154,72 +154,72 @@ public void testRegionFailoverForDataConsumptionErrors() throws Exception { topologyBuilder.connect(v2, v5, ResultPartitionType.BLOCKING); topologyBuilder.connect(v3, v6, ResultPartitionType.BLOCKING); - FailoverTopology topology = topologyBuilder.build(); + TestFailoverTopology topology = topologyBuilder.build(); RestartPipelinedRegionStrategy strategy = new RestartPipelinedRegionStrategy(topology); // when v4 fails to consume data from v1, {v1,v4,v5} should be restarted HashSet expectedResult = new HashSet<>(); - Iterator v4InputEdgeIterator = v4.getInputEdges().iterator(); - expectedResult.add(v1.getExecutionVertexID()); - expectedResult.add(v4.getExecutionVertexID()); - expectedResult.add(v5.getExecutionVertexID()); + Iterator v4InputEdgeIterator = v4.getConsumedResults().iterator(); + expectedResult.add(v1.getId()); + expectedResult.add(v4.getId()); + expectedResult.add(v5.getId()); assertEquals(expectedResult, - strategy.getTasksNeedingRestart(v4.getExecutionVertexID(), + strategy.getTasksNeedingRestart(v4.getId(), new PartitionConnectionException( new ResultPartitionID( - v4InputEdgeIterator.next().getResultPartitionID(), + v4InputEdgeIterator.next().getId(), new ExecutionAttemptID()), new Exception("Test failure")))); // when v4 fails to consume data from v2, {v2,v4,v5} should be restarted expectedResult.clear(); - expectedResult.add(v2.getExecutionVertexID()); - expectedResult.add(v4.getExecutionVertexID()); - expectedResult.add(v5.getExecutionVertexID()); + expectedResult.add(v2.getId()); + expectedResult.add(v4.getId()); + expectedResult.add(v5.getId()); assertEquals(expectedResult, - strategy.getTasksNeedingRestart(v4.getExecutionVertexID(), + strategy.getTasksNeedingRestart(v4.getId(), new PartitionNotFoundException( new ResultPartitionID( - v4InputEdgeIterator.next().getResultPartitionID(), + v4InputEdgeIterator.next().getId(), new ExecutionAttemptID())))); // when v5 fails to consume data from v1, {v1,v4,v5} should be restarted expectedResult.clear(); - Iterator v5InputEdgeIterator = v5.getInputEdges().iterator(); - expectedResult.add(v1.getExecutionVertexID()); - expectedResult.add(v4.getExecutionVertexID()); - expectedResult.add(v5.getExecutionVertexID()); + Iterator v5InputEdgeIterator = v5.getConsumedResults().iterator(); + expectedResult.add(v1.getId()); + expectedResult.add(v4.getId()); + expectedResult.add(v5.getId()); assertEquals(expectedResult, - strategy.getTasksNeedingRestart(v5.getExecutionVertexID(), + strategy.getTasksNeedingRestart(v5.getId(), new PartitionConnectionException( new ResultPartitionID( - v5InputEdgeIterator.next().getResultPartitionID(), + v5InputEdgeIterator.next().getId(), new ExecutionAttemptID()), new Exception("Test failure")))); // when v5 fails to consume data from v2, {v2,v4,v5} should be restarted expectedResult.clear(); - expectedResult.add(v2.getExecutionVertexID()); - expectedResult.add(v4.getExecutionVertexID()); - expectedResult.add(v5.getExecutionVertexID()); + expectedResult.add(v2.getId()); + expectedResult.add(v4.getId()); + expectedResult.add(v5.getId()); assertEquals(expectedResult, - strategy.getTasksNeedingRestart(v5.getExecutionVertexID(), + strategy.getTasksNeedingRestart(v5.getId(), new PartitionNotFoundException( new ResultPartitionID( - v5InputEdgeIterator.next().getResultPartitionID(), + v5InputEdgeIterator.next().getId(), new ExecutionAttemptID())))); // when v6 fails to consume data from v3, {v3,v6} should be restarted expectedResult.clear(); - Iterator v6InputEdgeIterator = v6.getInputEdges().iterator(); - expectedResult.add(v3.getExecutionVertexID()); - expectedResult.add(v6.getExecutionVertexID()); + Iterator v6InputEdgeIterator = v6.getConsumedResults().iterator(); + expectedResult.add(v3.getId()); + expectedResult.add(v6.getId()); assertEquals(expectedResult, - strategy.getTasksNeedingRestart(v6.getExecutionVertexID(), + strategy.getTasksNeedingRestart(v6.getId(), new PartitionConnectionException( new ResultPartitionID( - v6InputEdgeIterator.next().getResultPartitionID(), + v6InputEdgeIterator.next().getId(), new ExecutionAttemptID()), new Exception("Test failure")))); } @@ -249,13 +249,13 @@ public void testRegionFailoverForVariousResultPartitionAvailabilityCombinations( topologyBuilder.connect(v1, v3, ResultPartitionType.BLOCKING); topologyBuilder.connect(v2, v3, ResultPartitionType.BLOCKING); - FailoverTopology topology = topologyBuilder.build(); + TestFailoverTopology topology = topologyBuilder.build(); TestResultPartitionAvailabilityChecker availabilityChecker = new TestResultPartitionAvailabilityChecker(); RestartPipelinedRegionStrategy strategy = new RestartPipelinedRegionStrategy(topology, availabilityChecker); - IntermediateResultPartitionID rp1ID = v1.getOutputEdges().iterator().next().getResultPartitionID(); - IntermediateResultPartitionID rp2ID = v2.getOutputEdges().iterator().next().getResultPartitionID(); + IntermediateResultPartitionID rp1ID = v1.getProducedResults().iterator().next().getId(); + IntermediateResultPartitionID rp2ID = v2.getProducedResults().iterator().next().getId(); // ------------------------------------------------- // Combination1: (rp1 == available, rp == available) @@ -264,18 +264,18 @@ public void testRegionFailoverForVariousResultPartitionAvailabilityCombinations( // when v1 fails, {v1,v3} should be restarted assertThat( - strategy.getTasksNeedingRestart(v1.getExecutionVertexID(), new Exception("Test failure")), - containsInAnyOrder(v1.getExecutionVertexID(), v3.getExecutionVertexID())); + strategy.getTasksNeedingRestart(v1.getId(), new Exception("Test failure")), + containsInAnyOrder(v1.getId(), v3.getId())); // when v2 fails, {v2,v3} should be restarted assertThat( - strategy.getTasksNeedingRestart(v2.getExecutionVertexID(), new Exception("Test failure")), - containsInAnyOrder(v2.getExecutionVertexID(), v3.getExecutionVertexID())); + strategy.getTasksNeedingRestart(v2.getId(), new Exception("Test failure")), + containsInAnyOrder(v2.getId(), v3.getId())); // when v3 fails, {v3} should be restarted assertThat( - strategy.getTasksNeedingRestart(v3.getExecutionVertexID(), new Exception("Test failure")), - containsInAnyOrder(v3.getExecutionVertexID())); + strategy.getTasksNeedingRestart(v3.getId(), new Exception("Test failure")), + containsInAnyOrder(v3.getId())); // ------------------------------------------------- // Combination2: (rp1 == unavailable, rp == available) @@ -285,18 +285,18 @@ public void testRegionFailoverForVariousResultPartitionAvailabilityCombinations( // when v1 fails, {v1,v3} should be restarted assertThat( - strategy.getTasksNeedingRestart(v1.getExecutionVertexID(), new Exception("Test failure")), - containsInAnyOrder(v1.getExecutionVertexID(), v3.getExecutionVertexID())); + strategy.getTasksNeedingRestart(v1.getId(), new Exception("Test failure")), + containsInAnyOrder(v1.getId(), v3.getId())); // when v2 fails, {v1,v2,v3} should be restarted assertThat( - strategy.getTasksNeedingRestart(v2.getExecutionVertexID(), new Exception("Test failure")), - containsInAnyOrder(v1.getExecutionVertexID(), v2.getExecutionVertexID(), v3.getExecutionVertexID())); + strategy.getTasksNeedingRestart(v2.getId(), new Exception("Test failure")), + containsInAnyOrder(v1.getId(), v2.getId(), v3.getId())); // when v3 fails, {v1,v3} should be restarted assertThat( - strategy.getTasksNeedingRestart(v3.getExecutionVertexID(), new Exception("Test failure")), - containsInAnyOrder(v1.getExecutionVertexID(), v3.getExecutionVertexID())); + strategy.getTasksNeedingRestart(v3.getId(), new Exception("Test failure")), + containsInAnyOrder(v1.getId(), v3.getId())); // ------------------------------------------------- // Combination3: (rp1 == available, rp == unavailable) @@ -306,18 +306,18 @@ public void testRegionFailoverForVariousResultPartitionAvailabilityCombinations( // when v1 fails, {v1,v2,v3} should be restarted assertThat( - strategy.getTasksNeedingRestart(v1.getExecutionVertexID(), new Exception("Test failure")), - containsInAnyOrder(v1.getExecutionVertexID(), v2.getExecutionVertexID(), v3.getExecutionVertexID())); + strategy.getTasksNeedingRestart(v1.getId(), new Exception("Test failure")), + containsInAnyOrder(v1.getId(), v2.getId(), v3.getId())); // when v2 fails, {v2,v3} should be restarted assertThat( - strategy.getTasksNeedingRestart(v2.getExecutionVertexID(), new Exception("Test failure")), - containsInAnyOrder(v2.getExecutionVertexID(), v3.getExecutionVertexID())); + strategy.getTasksNeedingRestart(v2.getId(), new Exception("Test failure")), + containsInAnyOrder(v2.getId(), v3.getId())); // when v3 fails, {v2,v3} should be restarted assertThat( - strategy.getTasksNeedingRestart(v3.getExecutionVertexID(), new Exception("Test failure")), - containsInAnyOrder(v2.getExecutionVertexID(), v3.getExecutionVertexID())); + strategy.getTasksNeedingRestart(v3.getId(), new Exception("Test failure")), + containsInAnyOrder(v2.getId(), v3.getId())); // ------------------------------------------------- // Combination4: (rp1 == unavailable, rp == unavailable) @@ -328,18 +328,18 @@ public void testRegionFailoverForVariousResultPartitionAvailabilityCombinations( // when v1 fails, {v1,v2,v3} should be restarted assertThat( - strategy.getTasksNeedingRestart(v1.getExecutionVertexID(), new Exception("Test failure")), - containsInAnyOrder(v1.getExecutionVertexID(), v2.getExecutionVertexID(), v3.getExecutionVertexID())); + strategy.getTasksNeedingRestart(v1.getId(), new Exception("Test failure")), + containsInAnyOrder(v1.getId(), v2.getId(), v3.getId())); // when v2 fails, {v1,v2,v3} should be restarted assertThat( - strategy.getTasksNeedingRestart(v2.getExecutionVertexID(), new Exception("Test failure")), - containsInAnyOrder(v1.getExecutionVertexID(), v2.getExecutionVertexID(), v3.getExecutionVertexID())); + strategy.getTasksNeedingRestart(v2.getId(), new Exception("Test failure")), + containsInAnyOrder(v1.getId(), v2.getId(), v3.getId())); // when v3 fails, {v1,v2,v3} should be restarted assertThat( - strategy.getTasksNeedingRestart(v3.getExecutionVertexID(), new Exception("Test failure")), - containsInAnyOrder(v1.getExecutionVertexID(), v2.getExecutionVertexID(), v3.getExecutionVertexID())); + strategy.getTasksNeedingRestart(v3.getId(), new Exception("Test failure")), + containsInAnyOrder(v1.getId(), v2.getId(), v3.getId())); } /** @@ -370,32 +370,32 @@ public void testRegionFailoverForMultipleVerticesRegions() throws Exception { topologyBuilder.connect(v4, v5, ResultPartitionType.BLOCKING); topologyBuilder.connect(v5, v6, ResultPartitionType.PIPELINED); - FailoverTopology topology = topologyBuilder.build(); + TestFailoverTopology topology = topologyBuilder.build(); RestartPipelinedRegionStrategy strategy = new RestartPipelinedRegionStrategy(topology); // when v3 fails due to internal error, {v3,v4,v5,v6} should be restarted HashSet expectedResult = new HashSet<>(); - expectedResult.add(v3.getExecutionVertexID()); - expectedResult.add(v4.getExecutionVertexID()); - expectedResult.add(v5.getExecutionVertexID()); - expectedResult.add(v6.getExecutionVertexID()); + expectedResult.add(v3.getId()); + expectedResult.add(v4.getId()); + expectedResult.add(v5.getId()); + expectedResult.add(v6.getId()); assertEquals(expectedResult, - strategy.getTasksNeedingRestart(v3.getExecutionVertexID(), new Exception("Test failure"))); + strategy.getTasksNeedingRestart(v3.getId(), new Exception("Test failure"))); // when v3 fails to consume from v2, {v1,v2,v3,v4,v5,v6} should be restarted expectedResult.clear(); - expectedResult.add(v1.getExecutionVertexID()); - expectedResult.add(v2.getExecutionVertexID()); - expectedResult.add(v3.getExecutionVertexID()); - expectedResult.add(v4.getExecutionVertexID()); - expectedResult.add(v5.getExecutionVertexID()); - expectedResult.add(v6.getExecutionVertexID()); + expectedResult.add(v1.getId()); + expectedResult.add(v2.getId()); + expectedResult.add(v3.getId()); + expectedResult.add(v4.getId()); + expectedResult.add(v5.getId()); + expectedResult.add(v6.getId()); assertEquals(expectedResult, - strategy.getTasksNeedingRestart(v3.getExecutionVertexID(), + strategy.getTasksNeedingRestart(v3.getId(), new PartitionConnectionException( new ResultPartitionID( - v3.getInputEdges().iterator().next().getResultPartitionID(), + v3.getConsumedResults().iterator().next().getId(), new ExecutionAttemptID()), new Exception("Test failure")))); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/TestFailoverTopology.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/TestFailoverTopology.java index 7ccda644ff55..f20ee91d7698 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/TestFailoverTopology.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/TestFailoverTopology.java @@ -25,23 +25,26 @@ import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; +import java.util.List; /** - * A FailoverTopology implementation for tests. + * A {@link FailoverTopology} implementation for tests. */ -public class TestFailoverTopology implements FailoverTopology { +public class TestFailoverTopology + implements FailoverTopology { - private final Collection vertices; + private final Collection vertices; private final boolean containsCoLocationConstraints; - public TestFailoverTopology(Collection vertices, boolean containsCoLocationConstraints) { + public TestFailoverTopology(Collection vertices, boolean containsCoLocationConstraints) { this.vertices = vertices; this.containsCoLocationConstraints = containsCoLocationConstraints; } @Override - public Iterable getFailoverVertices() { - return vertices::iterator; + public Iterable getVertices() { + return vertices; } @Override @@ -50,76 +53,83 @@ public boolean containsCoLocationConstraints() { } /** - * A FailoverVertex implementation for tests. + * A {@link FailoverVertex} implementation for tests. */ - public static class TestFailoverVertex implements FailoverVertex { + public static class TestFailoverVertex + implements FailoverVertex { - private final Collection inputEdges = new ArrayList<>(); - private final Collection outputEdges = new ArrayList<>(); + private final Collection consumedPartitions = new ArrayList<>(); + private final Collection producedPartitions = new ArrayList<>(); private final ExecutionVertexID id; public TestFailoverVertex(ExecutionVertexID id) { this.id = id; } - void addInputEdge(FailoverEdge edge) { - inputEdges.add(edge); + void addConsumedPartition(TestFailoverResultPartition partition) { + consumedPartitions.add(partition); } - void addOuputEdge(FailoverEdge edge) { - outputEdges.add(edge); + void addProducedPartition(TestFailoverResultPartition partition) { + producedPartitions.add(partition); } - public ExecutionVertexID getExecutionVertexID() { + public ExecutionVertexID getId() { return id; } @Override - public Iterable getInputEdges() { - return inputEdges::iterator; + public Iterable getConsumedResults() { + return consumedPartitions; } @Override - public Iterable getOutputEdges() { - return outputEdges::iterator; + public Iterable getProducedResults() { + return producedPartitions; } } /** - * A FailoverEdge implementation for tests. + * A {@link FailoverResultPartition} implementation for tests. */ - public static class TestFailoverEdge implements FailoverEdge { + public static class TestFailoverResultPartition + implements FailoverResultPartition { private final IntermediateResultPartitionID resultPartitionID; private final ResultPartitionType resultPartitionType; - private final FailoverVertex sourceVertex; - private final FailoverVertex targetVertex; + private final TestFailoverVertex producer; + private final List consumers; + + public TestFailoverResultPartition( + IntermediateResultPartitionID resultPartitionID, + ResultPartitionType resultPartitionType, + TestFailoverVertex producer, + TestFailoverVertex consumer) { - public TestFailoverEdge(IntermediateResultPartitionID resultPartitionID, ResultPartitionType resultPartitionType, FailoverVertex sourceVertex, FailoverVertex targetVertex) { this.resultPartitionID = resultPartitionID; this.resultPartitionType = resultPartitionType; - this.sourceVertex = sourceVertex; - this.targetVertex = targetVertex; + this.producer = producer; + this.consumers = Collections.singletonList(consumer); } @Override - public IntermediateResultPartitionID getResultPartitionID() { + public IntermediateResultPartitionID getId() { return resultPartitionID; } @Override - public ResultPartitionType getResultPartitionType() { + public ResultPartitionType getResultType() { return resultPartitionType; } @Override - public FailoverVertex getSourceVertex() { - return sourceVertex; + public TestFailoverVertex getProducer() { + return producer; } @Override - public FailoverVertex getTargetVertex() { - return targetVertex; + public Iterable getConsumers() { + return consumers; } } @@ -128,7 +138,7 @@ public FailoverVertex getTargetVertex() { */ public static class Builder { private boolean containsCoLocationConstraints = false; - private Collection vertices = new ArrayList<>(); + private Collection vertices = new ArrayList<>(); public TestFailoverVertex newVertex() { TestFailoverVertex testFailoverVertex = new TestFailoverVertex(new ExecutionVertexID(new JobVertexID(), 0)); @@ -137,17 +147,17 @@ public TestFailoverVertex newVertex() { } public Builder connect(TestFailoverVertex source, TestFailoverVertex target, ResultPartitionType partitionType) { - FailoverEdge edge = new TestFailoverEdge(new IntermediateResultPartitionID(), partitionType, source, target); - source.addOuputEdge(edge); - target.addInputEdge(edge); + TestFailoverResultPartition partition = new TestFailoverResultPartition(new IntermediateResultPartitionID(), partitionType, source, target); + source.addProducedPartition(partition); + target.addConsumedPartition(partition); return this; } public Builder connect(TestFailoverVertex source, TestFailoverVertex target, ResultPartitionType partitionType, IntermediateResultPartitionID partitionID) { - FailoverEdge edge = new TestFailoverEdge(partitionID, partitionType, source, target); - source.addOuputEdge(edge); - target.addInputEdge(edge); + TestFailoverResultPartition partition = new TestFailoverResultPartition(partitionID, partitionType, source, target); + source.addProducedPartition(partition); + target.addConsumedPartition(partition); return this; } @@ -157,7 +167,7 @@ public Builder setContainsCoLocationConstraints(boolean containsCoLocationConstr return this; } - public FailoverTopology build() { + public TestFailoverTopology build() { return new TestFailoverTopology(vertices, containsCoLocationConstraints); } } From 6ef33b55831ce45b8f9040370d4fa27fdd361591 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Mon, 21 Oct 2019 18:23:54 +0800 Subject: [PATCH 294/746] [hotfix][runtime] Rename ExecutionGraphToSchedulingTopologyAdapter to DefaultExecutionTopology ExecutionGraphToSchedulingTopologyAdapter now is the default implementation of both SchedulingTopology and FailoverTopology. --- .../executiongraph/ExecutionGraph.java | 6 +-- ...aptedRestartPipelinedRegionStrategyNG.java | 4 +- .../runtime/scheduler/SchedulerBase.java | 6 +-- ...ter.java => DefaultExecutionTopology.java} | 51 +++++++++---------- ...ertex.java => DefaultExecutionVertex.java} | 23 ++++----- ...ition.java => DefaultResultPartition.java} | 21 ++++---- ...java => DefaultExecutionTopologyTest.java} | 32 ++++++------ ...t.java => DefaultExecutionVertexTest.java} | 14 ++--- ...t.java => DefaultResultPartitionTest.java} | 10 ++-- 9 files changed, 82 insertions(+), 85 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/{ExecutionGraphToSchedulingTopologyAdapter.java => DefaultExecutionTopology.java} (65%) rename flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/{DefaultSchedulingExecutionVertex.java => DefaultExecutionVertex.java} (74%) rename flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/{DefaultSchedulingResultPartition.java => DefaultResultPartition.java} (80%) rename flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/{ExecutionGraphToSchedulingTopologyAdapterTest.java => DefaultExecutionTopologyTest.java} (87%) rename flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/{DefaultSchedulingExecutionVertexTest.java => DefaultExecutionVertexTest.java} (88%) rename flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/{DefaultSchedulingResultPartitionTest.java => DefaultResultPartitionTest.java} (90%) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 95e2844c1f1f..a1a70c4a3f4e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -69,7 +69,7 @@ import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; import org.apache.flink.runtime.query.KvStateLocationRegistry; import org.apache.flink.runtime.scheduler.InternalFailuresListener; -import org.apache.flink.runtime.scheduler.adapter.ExecutionGraphToSchedulingTopologyAdapter; +import org.apache.flink.runtime.scheduler.adapter.DefaultExecutionTopology; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; import org.apache.flink.runtime.scheduler.strategy.SchedulingExecutionVertex; import org.apache.flink.runtime.scheduler.strategy.SchedulingResultPartition; @@ -947,10 +947,10 @@ public void attachJobGraph(List topologiallySorted) throws JobExcepti failoverStrategy.notifyNewVertices(newExecJobVertices); - schedulingTopology = new ExecutionGraphToSchedulingTopologyAdapter(this); + schedulingTopology = new DefaultExecutionTopology(this); partitionReleaseStrategy = partitionReleaseStrategyFactory.createInstance( schedulingTopology, - new ExecutionGraphToSchedulingTopologyAdapter(this)); + new DefaultExecutionTopology(this)); } public boolean isLegacyScheduling() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java index a69801bc2821..8f02bc966075 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java @@ -36,7 +36,7 @@ import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; import org.apache.flink.runtime.scheduler.ExecutionVertexVersion; import org.apache.flink.runtime.scheduler.ExecutionVertexVersioner; -import org.apache.flink.runtime.scheduler.adapter.ExecutionGraphToSchedulingTopologyAdapter; +import org.apache.flink.runtime.scheduler.adapter.DefaultExecutionTopology; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; import org.apache.flink.util.ExceptionUtils; @@ -295,7 +295,7 @@ public void notifyNewVertices(final List newJobVerticesTopol // currently it's safe to add it here, as this method is invoked only once in production code. checkState(restartPipelinedRegionStrategy == null, "notifyNewVertices() must be called only once"); this.restartPipelinedRegionStrategy = new RestartPipelinedRegionStrategy( - new ExecutionGraphToSchedulingTopologyAdapter(executionGraph), executionGraph.getResultPartitionAvailabilityChecker()); + new DefaultExecutionTopology(executionGraph), executionGraph.getResultPartitionAvailabilityChecker()); } @Override 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 3f24f5cfbda8..3aaf24b4249e 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 @@ -72,7 +72,7 @@ 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.adapter.DefaultExecutionTopology; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology; import org.apache.flink.runtime.shuffle.ShuffleMaster; @@ -185,8 +185,8 @@ public SchedulerBase( this.slotRequestTimeout = checkNotNull(slotRequestTimeout); this.executionGraph = createAndRestoreExecutionGraph(jobManagerJobMetricGroup, checkNotNull(shuffleMaster), checkNotNull(partitionTracker)); - this.schedulingTopology = new ExecutionGraphToSchedulingTopologyAdapter(executionGraph); - this.failoverTopology = new ExecutionGraphToSchedulingTopologyAdapter(executionGraph); + this.schedulingTopology = new DefaultExecutionTopology(executionGraph); + this.failoverTopology = new DefaultExecutionTopology(executionGraph); this.inputsLocationsRetriever = new ExecutionGraphToInputsLocationsRetrieverAdapter(executionGraph); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/ExecutionGraphToSchedulingTopologyAdapter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultExecutionTopology.java similarity index 65% rename from flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/ExecutionGraphToSchedulingTopologyAdapter.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultExecutionTopology.java index d0bf99427cb0..c3d61a2c34f4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/ExecutionGraphToSchedulingTopologyAdapter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultExecutionTopology.java @@ -40,21 +40,20 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** - * Adapter of {@link ExecutionGraph} to {@link SchedulingTopology}. + * Adapter of {@link ExecutionGraph} to {@link SchedulingTopology} and {@link FailoverTopology}. */ -public class ExecutionGraphToSchedulingTopologyAdapter - implements SchedulingTopology, - FailoverTopology { +public class DefaultExecutionTopology implements SchedulingTopology, + FailoverTopology { private final boolean containsCoLocationConstraints; - private final Map executionVerticesById; + private final Map executionVerticesById; - private final List executionVerticesList; + private final List executionVerticesList; - private final Map resultPartitionsById; + private final Map resultPartitionsById; - public ExecutionGraphToSchedulingTopologyAdapter(ExecutionGraph graph) { + public DefaultExecutionTopology(ExecutionGraph graph) { checkNotNull(graph, "execution graph can not be null"); this.containsCoLocationConstraints = graph.getAllVertices().values().stream() @@ -63,15 +62,15 @@ public ExecutionGraphToSchedulingTopologyAdapter(ExecutionGraph graph) { this.executionVerticesById = new HashMap<>(); this.executionVerticesList = new ArrayList<>(graph.getTotalNumberOfVertices()); - Map tmpResultPartitionsById = new HashMap<>(); - Map executionVertexMap = new HashMap<>(); + Map tmpResultPartitionsById = new HashMap<>(); + Map executionVertexMap = new HashMap<>(); for (ExecutionVertex vertex : graph.getAllExecutionVertices()) { - List producedPartitions = generateProducedSchedulingResultPartition(vertex.getProducedPartitions()); + List producedPartitions = generateProducedSchedulingResultPartition(vertex.getProducedPartitions()); producedPartitions.forEach(partition -> tmpResultPartitionsById.put(partition.getId(), partition)); - DefaultSchedulingExecutionVertex schedulingVertex = generateSchedulingExecutionVertex(vertex, producedPartitions); + DefaultExecutionVertex schedulingVertex = generateSchedulingExecutionVertex(vertex, producedPartitions); this.executionVerticesById.put(schedulingVertex.getId(), schedulingVertex); this.executionVerticesList.add(schedulingVertex); executionVertexMap.put(vertex, schedulingVertex); @@ -82,7 +81,7 @@ public ExecutionGraphToSchedulingTopologyAdapter(ExecutionGraph graph) { } @Override - public Iterable getVertices() { + public Iterable getVertices() { return executionVerticesList; } @@ -92,23 +91,23 @@ public boolean containsCoLocationConstraints() { } @Override - public Optional getVertex(ExecutionVertexID executionVertexId) { + public Optional getVertex(ExecutionVertexID executionVertexId) { return Optional.ofNullable(executionVerticesById.get(executionVertexId)); } @Override - public Optional getResultPartition(IntermediateResultPartitionID intermediateResultPartitionId) { + public Optional getResultPartition(IntermediateResultPartitionID intermediateResultPartitionId) { return Optional.ofNullable(resultPartitionsById.get(intermediateResultPartitionId)); } - private static List generateProducedSchedulingResultPartition( + private static List generateProducedSchedulingResultPartition( Map producedIntermediatePartitions) { - List producedSchedulingPartitions = new ArrayList<>(producedIntermediatePartitions.size()); + List producedSchedulingPartitions = new ArrayList<>(producedIntermediatePartitions.size()); producedIntermediatePartitions.values().forEach( irp -> producedSchedulingPartitions.add( - new DefaultSchedulingResultPartition( + new DefaultResultPartition( irp.getPartitionId(), irp.getIntermediateResult().getId(), irp.getResultType()))); @@ -116,11 +115,11 @@ private static List generateProducedScheduling return producedSchedulingPartitions; } - private static DefaultSchedulingExecutionVertex generateSchedulingExecutionVertex( + private static DefaultExecutionVertex generateSchedulingExecutionVertex( ExecutionVertex vertex, - List producedPartitions) { + List producedPartitions) { - DefaultSchedulingExecutionVertex schedulingVertex = new DefaultSchedulingExecutionVertex( + DefaultExecutionVertex schedulingVertex = new DefaultExecutionVertex( vertex.getID(), producedPartitions, new ExecutionStateSupplier(vertex), @@ -132,16 +131,16 @@ private static DefaultSchedulingExecutionVertex generateSchedulingExecutionVerte } private static void connectVerticesToConsumedPartitions( - Map executionVertexMap, - Map resultPartitions) { + Map executionVertexMap, + Map resultPartitions) { - for (Map.Entry mapEntry : executionVertexMap.entrySet()) { - final DefaultSchedulingExecutionVertex schedulingVertex = mapEntry.getValue(); + for (Map.Entry mapEntry : executionVertexMap.entrySet()) { + final DefaultExecutionVertex schedulingVertex = mapEntry.getValue(); final ExecutionVertex executionVertex = mapEntry.getKey(); for (int index = 0; index < executionVertex.getNumberOfInputs(); index++) { for (ExecutionEdge edge : executionVertex.getInputEdges(index)) { - DefaultSchedulingResultPartition partition = resultPartitions.get(edge.getSource().getPartitionId()); + DefaultResultPartition partition = resultPartitions.get(edge.getSource().getPartitionId()); schedulingVertex.addConsumedResult(partition); partition.addConsumer(schedulingVertex); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultExecutionVertex.java similarity index 74% rename from flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingExecutionVertex.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultExecutionVertex.java index 25246cc73374..aaf48994be79 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingExecutionVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultExecutionVertex.java @@ -31,25 +31,24 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** - * Default implementation of {@link SchedulingExecutionVertex}. + * Default implementation of {@link SchedulingExecutionVertex} and {@link FailoverVertex}. */ -class DefaultSchedulingExecutionVertex - implements SchedulingExecutionVertex, - FailoverVertex { +class DefaultExecutionVertex implements SchedulingExecutionVertex, + FailoverVertex { private final ExecutionVertexID executionVertexId; - private final List consumedResults; + private final List consumedResults; - private final List producedResults; + private final List producedResults; private final Supplier stateSupplier; private final InputDependencyConstraint inputDependencyConstraint; - DefaultSchedulingExecutionVertex( + DefaultExecutionVertex( ExecutionVertexID executionVertexId, - List producedPartitions, + List producedPartitions, Supplier stateSupplier, InputDependencyConstraint constraint) { this.executionVertexId = checkNotNull(executionVertexId); @@ -70,12 +69,12 @@ public ExecutionState getState() { } @Override - public Iterable getConsumedResults() { + public Iterable getConsumedResults() { return consumedResults; } @Override - public Iterable getProducedResults() { + public Iterable getProducedResults() { return producedResults; } @@ -84,7 +83,7 @@ public InputDependencyConstraint getInputDependencyConstraint() { return inputDependencyConstraint; } - void addConsumedResult(DefaultSchedulingResultPartition result) { - consumedResults.add(result); + void addConsumedResult(DefaultResultPartition result) { + consumedResults.add(result); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultResultPartition.java similarity index 80% rename from flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingResultPartition.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultResultPartition.java index ebaf95ddd6f5..5c0db898e181 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingResultPartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adapter/DefaultResultPartition.java @@ -33,11 +33,10 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** - * Default implementation of {@link SchedulingResultPartition}. + * Default implementation of {@link SchedulingResultPartition} and {@link FailoverResultPartition}. */ -class DefaultSchedulingResultPartition - implements SchedulingResultPartition, - FailoverResultPartition { +class DefaultResultPartition implements SchedulingResultPartition, + FailoverResultPartition { private final IntermediateResultPartitionID resultPartitionId; @@ -45,11 +44,11 @@ class DefaultSchedulingResultPartition private final ResultPartitionType partitionType; - private DefaultSchedulingExecutionVertex producer; + private DefaultExecutionVertex producer; - private final List consumers; + private final List consumers; - DefaultSchedulingResultPartition( + DefaultResultPartition( IntermediateResultPartitionID partitionId, IntermediateDataSetID intermediateDataSetId, ResultPartitionType partitionType) { @@ -87,20 +86,20 @@ public ResultPartitionState getState() { } @Override - public DefaultSchedulingExecutionVertex getProducer() { + public DefaultExecutionVertex getProducer() { return producer; } @Override - public Iterable getConsumers() { + public Iterable getConsumers() { return consumers; } - void addConsumer(DefaultSchedulingExecutionVertex vertex) { + void addConsumer(DefaultExecutionVertex vertex) { consumers.add(checkNotNull(vertex)); } - void setProducer(DefaultSchedulingExecutionVertex vertex) { + void setProducer(DefaultExecutionVertex vertex) { producer = checkNotNull(vertex); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/ExecutionGraphToSchedulingTopologyAdapterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultExecutionTopologyTest.java similarity index 87% rename from flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/ExecutionGraphToSchedulingTopologyAdapterTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultExecutionTopologyTest.java index 9728501c57ed..57b34c7e31a9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/ExecutionGraphToSchedulingTopologyAdapterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultExecutionTopologyTest.java @@ -60,9 +60,9 @@ import static org.junit.Assert.fail; /** - * Unit tests for {@link ExecutionGraphToSchedulingTopologyAdapter}. + * Unit tests for {@link DefaultExecutionTopology}. */ -public class ExecutionGraphToSchedulingTopologyAdapterTest extends TestLogger { +public class DefaultExecutionTopologyTest extends TestLogger { private final SimpleAckingTaskManagerGateway taskManagerGateway = new SimpleAckingTaskManagerGateway(); @@ -70,7 +70,7 @@ public class ExecutionGraphToSchedulingTopologyAdapterTest extends TestLogger { private ExecutionGraph executionGraph; - private ExecutionGraphToSchedulingTopologyAdapter adapter; + private DefaultExecutionTopology adapter; @Before public void setUp() throws Exception { @@ -86,7 +86,7 @@ public void setUp() throws Exception { taskManagerGateway, triggeredRestartStrategy, jobVertices); - adapter = new ExecutionGraphToSchedulingTopologyAdapter(executionGraph); + adapter = new DefaultExecutionTopology(executionGraph); } @Test @@ -100,7 +100,7 @@ public void testGetResultPartition() { for (ExecutionVertex vertex : executionGraph.getAllExecutionVertices()) { for (Map.Entry entry : vertex.getProducedPartitions().entrySet()) { IntermediateResultPartition partition = entry.getValue(); - DefaultSchedulingResultPartition schedulingResultPartition = adapter.getResultPartition(entry.getKey()) + DefaultResultPartition schedulingResultPartition = adapter.getResultPartition(entry.getKey()) .orElseThrow(() -> new IllegalArgumentException("can not find partition " + entry.getKey())); assertPartitionEquals(partition, schedulingResultPartition); @@ -131,7 +131,7 @@ public void testResultPartitionOrThrow() { @Test public void testWithCoLocationConstraints() throws Exception { ExecutionGraph executionGraph = createExecutionGraphWithCoLocationConstraint(); - adapter = new ExecutionGraphToSchedulingTopologyAdapter(executionGraph); + adapter = new DefaultExecutionTopology(executionGraph); assertTrue(adapter.containsCoLocationConstraints()); } @@ -166,14 +166,14 @@ private ExecutionGraph createExecutionGraphWithCoLocationConstraint() throws Exc private static void assertGraphEquals( ExecutionGraph originalGraph, - ExecutionGraphToSchedulingTopologyAdapter adaptedTopology) { + DefaultExecutionTopology adaptedTopology) { Iterator originalVertices = originalGraph.getAllExecutionVertices().iterator(); - Iterator adaptedVertices = adaptedTopology.getVertices().iterator(); + Iterator adaptedVertices = adaptedTopology.getVertices().iterator(); while (originalVertices.hasNext()) { ExecutionVertex originalVertex = originalVertices.next(); - DefaultSchedulingExecutionVertex adaptedVertex = adaptedVertices.next(); + DefaultExecutionVertex adaptedVertex = adaptedVertices.next(); assertVertexEquals(originalVertex, adaptedVertex); @@ -182,12 +182,12 @@ private static void assertGraphEquals( .flatMap(Arrays::stream) .map(ExecutionEdge::getSource) .collect(Collectors.toList()); - Iterable adaptedConsumedPartitions = adaptedVertex.getConsumedResults(); + Iterable adaptedConsumedPartitions = adaptedVertex.getConsumedResults(); assertPartitionsEquals(originalConsumedPartitions, adaptedConsumedPartitions); Collection originalProducedPartitions = originalVertex.getProducedPartitions().values(); - Iterable adaptedProducedPartitions = adaptedVertex.getProducedResults(); + Iterable adaptedProducedPartitions = adaptedVertex.getProducedResults(); assertPartitionsEquals(originalProducedPartitions, adaptedProducedPartitions); } @@ -197,12 +197,12 @@ private static void assertGraphEquals( private static void assertPartitionsEquals( Iterable originalResultPartitions, - Iterable adaptedResultPartitions) { + Iterable adaptedResultPartitions) { assertEquals(Iterables.size(originalResultPartitions), Iterables.size(adaptedResultPartitions)); for (IntermediateResultPartition originalPartition : originalResultPartitions) { - DefaultSchedulingResultPartition adaptedPartition = IterableUtils.toStream(adaptedResultPartitions) + DefaultResultPartition adaptedPartition = IterableUtils.toStream(adaptedResultPartitions) .filter(adapted -> adapted.getId().equals(originalPartition.getPartitionId())) .findAny() .orElseThrow(() -> new AssertionError("Could not find matching adapted partition for " + originalPartition)); @@ -213,7 +213,7 @@ private static void assertPartitionsEquals( .flatMap(Collection::stream) .map(ExecutionEdge::getTarget) .collect(Collectors.toList()); - Iterable adaptedConsumers = adaptedPartition.getConsumers(); + Iterable adaptedConsumers = adaptedPartition.getConsumers(); for (ExecutionVertex originalConsumer : originalConsumers) { // it is sufficient to verify that some vertex exists with the correct ID here, @@ -228,7 +228,7 @@ private static void assertPartitionsEquals( private static void assertPartitionEquals( IntermediateResultPartition originalPartition, - DefaultSchedulingResultPartition adaptedPartition) { + DefaultResultPartition adaptedPartition) { assertEquals(originalPartition.getPartitionId(), adaptedPartition.getId()); assertEquals(originalPartition.getIntermediateResult().getId(), adaptedPartition.getResultId()); @@ -240,7 +240,7 @@ private static void assertPartitionEquals( private static void assertVertexEquals( ExecutionVertex originalVertex, - DefaultSchedulingExecutionVertex adaptedVertex) { + DefaultExecutionVertex adaptedVertex) { assertEquals( originalVertex.getID(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingExecutionVertexTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultExecutionVertexTest.java similarity index 88% rename from flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingExecutionVertexTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultExecutionVertexTest.java index 267610d985b4..22702ba8acf5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingExecutionVertexTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultExecutionVertexTest.java @@ -38,15 +38,15 @@ import static org.junit.Assert.assertEquals; /** - * Unit tests for {@link DefaultSchedulingExecutionVertex}. + * Unit tests for {@link DefaultExecutionVertex}. */ -public class DefaultSchedulingExecutionVertexTest extends TestLogger { +public class DefaultExecutionVertexTest extends TestLogger { private final TestExecutionStateSupplier stateSupplier = new TestExecutionStateSupplier(); - private DefaultSchedulingExecutionVertex producerVertex; + private DefaultExecutionVertex producerVertex; - private DefaultSchedulingExecutionVertex consumerVertex; + private DefaultExecutionVertex consumerVertex; private IntermediateResultPartitionID intermediateResultPartitionId; @@ -55,17 +55,17 @@ public void setUp() throws Exception { intermediateResultPartitionId = new IntermediateResultPartitionID(); - DefaultSchedulingResultPartition schedulingResultPartition = new DefaultSchedulingResultPartition( + DefaultResultPartition schedulingResultPartition = new DefaultResultPartition( intermediateResultPartitionId, new IntermediateDataSetID(), BLOCKING); - producerVertex = new DefaultSchedulingExecutionVertex( + producerVertex = new DefaultExecutionVertex( new ExecutionVertexID(new JobVertexID(), 0), Collections.singletonList(schedulingResultPartition), stateSupplier, ANY); schedulingResultPartition.setProducer(producerVertex); - consumerVertex = new DefaultSchedulingExecutionVertex( + consumerVertex = new DefaultExecutionVertex( new ExecutionVertexID(new JobVertexID(), 0), Collections.emptyList(), stateSupplier, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingResultPartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultResultPartitionTest.java similarity index 90% rename from flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingResultPartitionTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultResultPartitionTest.java index 93ab29f961cb..3fc307ef3f45 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingResultPartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultResultPartitionTest.java @@ -40,25 +40,25 @@ import static org.junit.Assert.assertEquals; /** - * Unit tests for {@link DefaultSchedulingResultPartition}. + * Unit tests for {@link DefaultResultPartition}. */ -public class DefaultSchedulingResultPartitionTest extends TestLogger { +public class DefaultResultPartitionTest extends TestLogger { private static final TestExecutionStateSupplier stateProvider = new TestExecutionStateSupplier(); private final IntermediateResultPartitionID resultPartitionId = new IntermediateResultPartitionID(); private final IntermediateDataSetID intermediateResultId = new IntermediateDataSetID(); - private DefaultSchedulingResultPartition resultPartition; + private DefaultResultPartition resultPartition; @Before public void setUp() { - resultPartition = new DefaultSchedulingResultPartition( + resultPartition = new DefaultResultPartition( resultPartitionId, intermediateResultId, BLOCKING); - DefaultSchedulingExecutionVertex producerVertex = new DefaultSchedulingExecutionVertex( + DefaultExecutionVertex producerVertex = new DefaultExecutionVertex( new ExecutionVertexID(new JobVertexID(), 0), Collections.singletonList(resultPartition), stateProvider, From b1034c0ed951348fdcea4ad8bc1e33707087d8f3 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Mon, 21 Oct 2019 18:33:23 +0800 Subject: [PATCH 295/746] [FLINK-14452][runtime] Keep only one execution topology in scheduler This closes #9954. --- .../executiongraph/ExecutionGraph.java | 22 ++++++++++++++----- ...aptedRestartPipelinedRegionStrategyNG.java | 4 ++-- .../runtime/scheduler/SchedulerBase.java | 6 ++--- 3 files changed, 22 insertions(+), 10 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index a1a70c4a3f4e..272cc6a55b53 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -49,6 +49,7 @@ import org.apache.flink.runtime.execution.SuppressRestartsException; import org.apache.flink.runtime.executiongraph.failover.FailoverStrategy; import org.apache.flink.runtime.executiongraph.failover.RestartAllStrategy; +import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverTopology; import org.apache.flink.runtime.executiongraph.failover.flip1.ResultPartitionAvailabilityChecker; import org.apache.flink.runtime.executiongraph.failover.flip1.partitionrelease.NotReleasingPartitionReleaseStrategy; import org.apache.flink.runtime.executiongraph.failover.flip1.partitionrelease.PartitionReleaseStrategy; @@ -260,7 +261,7 @@ public class ExecutionGraph implements AccessExecutionGraph { private PartitionReleaseStrategy partitionReleaseStrategy; - private SchedulingTopology schedulingTopology; + private DefaultExecutionTopology executionTopology; @Nullable private InternalFailuresListener internalTaskFailuresListener; @@ -542,6 +543,14 @@ public boolean isQueuedSchedulingAllowed() { return this.allowQueuedScheduling; } + public SchedulingTopology getSchedulingTopology() { + return executionTopology; + } + + public FailoverTopology getFailoverTopology() { + return executionTopology; + } + public ScheduleMode getScheduleMode() { return scheduleMode; } @@ -945,12 +954,14 @@ public void attachJobGraph(List topologiallySorted) throws JobExcepti newExecJobVertices.add(ejv); } + // the topology assigning should happen before notifying new vertices to failoverStrategy + executionTopology = new DefaultExecutionTopology(this); + failoverStrategy.notifyNewVertices(newExecJobVertices); - schedulingTopology = new DefaultExecutionTopology(this); partitionReleaseStrategy = partitionReleaseStrategyFactory.createInstance( - schedulingTopology, - new DefaultExecutionTopology(this)); + getSchedulingTopology(), + getFailoverTopology()); } public boolean isLegacyScheduling() { @@ -1648,7 +1659,8 @@ private void releasePartitions(final List releasa } ResultPartitionID createResultPartitionId(final IntermediateResultPartitionID resultPartitionId) { - final SchedulingResultPartition schedulingResultPartition = schedulingTopology.getResultPartitionOrThrow(resultPartitionId); + final SchedulingResultPartition schedulingResultPartition = + getSchedulingTopology().getResultPartitionOrThrow(resultPartitionId); final SchedulingExecutionVertex producer = schedulingResultPartition.getProducer(); final ExecutionVertexID producerId = producer.getId(); final JobVertexID jobVertexId = producerId.getJobVertexId(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java index 8f02bc966075..c73114e32c30 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java @@ -36,7 +36,6 @@ import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; import org.apache.flink.runtime.scheduler.ExecutionVertexVersion; import org.apache.flink.runtime.scheduler.ExecutionVertexVersioner; -import org.apache.flink.runtime.scheduler.adapter.DefaultExecutionTopology; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; import org.apache.flink.util.ExceptionUtils; @@ -295,7 +294,8 @@ public void notifyNewVertices(final List newJobVerticesTopol // currently it's safe to add it here, as this method is invoked only once in production code. checkState(restartPipelinedRegionStrategy == null, "notifyNewVertices() must be called only once"); this.restartPipelinedRegionStrategy = new RestartPipelinedRegionStrategy( - new DefaultExecutionTopology(executionGraph), executionGraph.getResultPartitionAvailabilityChecker()); + executionGraph.getFailoverTopology(), + executionGraph.getResultPartitionAvailabilityChecker()); } @Override 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 3aaf24b4249e..1a231daac482 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 @@ -72,7 +72,6 @@ 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.DefaultExecutionTopology; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology; import org.apache.flink.runtime.shuffle.ShuffleMaster; @@ -185,8 +184,9 @@ public SchedulerBase( this.slotRequestTimeout = checkNotNull(slotRequestTimeout); this.executionGraph = createAndRestoreExecutionGraph(jobManagerJobMetricGroup, checkNotNull(shuffleMaster), checkNotNull(partitionTracker)); - this.schedulingTopology = new DefaultExecutionTopology(executionGraph); - this.failoverTopology = new DefaultExecutionTopology(executionGraph); + this.schedulingTopology = executionGraph.getSchedulingTopology(); + this.failoverTopology = executionGraph.getFailoverTopology(); + this.inputsLocationsRetriever = new ExecutionGraphToInputsLocationsRetrieverAdapter(executionGraph); } From 08f31cc5faa1e55e46b86b0348ec09caf81fefa5 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Tue, 22 Oct 2019 16:20:33 +0800 Subject: [PATCH 296/746] [FLINK-14453][runtime] Support building pipelined regions from base topology This closes #9967. --- .../failover/flip1/FailoverRegion.java | 2 +- .../flip1/PipelinedRegionComputeUtil.java | 47 ++++++++++--------- .../flip1/RestartPipelinedRegionStrategy.java | 5 +- .../RegionPartitionReleaseStrategy.java | 3 +- 4 files changed, 32 insertions(+), 25 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverRegion.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverRegion.java index 86b1f6455ad9..2e416a4fe866 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverRegion.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverRegion.java @@ -41,7 +41,7 @@ public class FailoverRegion { * * @param executionVertices to be contained in this region */ - public FailoverRegion(Set> executionVertices) { + public FailoverRegion(Set> executionVertices) { this.executionVertices = checkNotNull(executionVertices); this.executionVertexIDs = new HashSet<>(); executionVertices.forEach(v -> this.executionVertexIDs.add(v.getId())); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/PipelinedRegionComputeUtil.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/PipelinedRegionComputeUtil.java index 516176263ada..5dde31a3ff38 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/PipelinedRegionComputeUtil.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/PipelinedRegionComputeUtil.java @@ -21,6 +21,9 @@ import org.apache.flink.runtime.executiongraph.failover.flip1.partitionrelease.PipelinedRegion; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.topology.Result; +import org.apache.flink.runtime.topology.Topology; +import org.apache.flink.runtime.topology.Vertex; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,38 +43,42 @@ public final class PipelinedRegionComputeUtil { private static final Logger LOG = LoggerFactory.getLogger(PipelinedRegionComputeUtil.class); - public static Set toPipelinedRegionsSet(final Set>> distinctRegions) { + public static Set toPipelinedRegionsSet( + final Set>> distinctRegions) { + return distinctRegions.stream() .map(toExecutionVertexIdSet()) .map(PipelinedRegion::from) .collect(Collectors.toSet()); } - private static Function>, Set> toExecutionVertexIdSet() { + private static Function>, Set> toExecutionVertexIdSet() { return failoverVertices -> failoverVertices.stream() .map(FailoverVertex::getId) .collect(Collectors.toSet()); } - public static Set>> computePipelinedRegions(final FailoverTopology topology) { + public static , R extends Result> Set> computePipelinedRegions( + final Topology topology) { + // currently we let a job with co-location constraints fail as one region // putting co-located vertices in the same region with each other can be a future improvement if (topology.containsCoLocationConstraints()) { return uniqueRegions(buildOneRegionForAllVertices(topology)); } - final Map, Set>> vertexToRegion = new IdentityHashMap<>(); + final Map> vertexToRegion = new IdentityHashMap<>(); // iterate all the vertices which are topologically sorted - for (FailoverVertex vertex : topology.getVertices()) { - Set> currentRegion = new HashSet<>(1); + for (V vertex : topology.getVertices()) { + Set currentRegion = new HashSet<>(1); currentRegion.add(vertex); vertexToRegion.put(vertex, currentRegion); - for (FailoverResultPartition consumedResult : vertex.getConsumedResults()) { + for (R consumedResult : vertex.getConsumedResults()) { if (consumedResult.getResultType().isPipelined()) { - final FailoverVertex producerVertex = consumedResult.getProducer(); - final Set> producerRegion = vertexToRegion.get(producerVertex); + final V producerVertex = consumedResult.getProducer(); + final Set producerRegion = vertexToRegion.get(producerVertex); if (producerRegion == null) { throw new IllegalStateException("Producer task " + producerVertex.getId() @@ -84,8 +91,8 @@ public static Set toPipelinedRegionsSet(final Set> smallerSet; - final Set> largerSet; + final Set smallerSet; + final Set largerSet; if (currentRegion.size() < producerRegion.size()) { smallerSet = currentRegion; largerSet = producerRegion; @@ -93,7 +100,7 @@ public static Set toPipelinedRegionsSet(final Set v : smallerSet) { + for (V v : smallerSet) { vertexToRegion.put(v, largerSet); } largerSet.addAll(smallerSet); @@ -106,26 +113,24 @@ public static Set toPipelinedRegionsSet(final Set, Set>> buildOneRegionForAllVertices( - final FailoverTopology topology) { + private static > Map> buildOneRegionForAllVertices( + final Topology topology) { LOG.warn("Cannot decompose the topology into individual failover regions due to use of " + "Co-Location constraints (iterations). Job will fail over as one holistic unit."); - final Map, Set>> vertexToRegion = new IdentityHashMap<>(); + final Map> vertexToRegion = new IdentityHashMap<>(); - final Set> allVertices = new HashSet<>(); - for (FailoverVertex vertex : topology.getVertices()) { + final Set allVertices = new HashSet<>(); + for (V vertex : topology.getVertices()) { allVertices.add(vertex); vertexToRegion.put(vertex, allVertices); } return vertexToRegion; } - private static Set>> uniqueRegions( - final Map, Set>> vertexToRegion) { - - final Set>> distinctRegions = Collections.newSetFromMap(new IdentityHashMap<>()); + private static > Set> uniqueRegions(final Map> vertexToRegion) { + final Set> distinctRegions = Collections.newSetFromMap(new IdentityHashMap<>()); distinctRegions.addAll(vertexToRegion.values()); return distinctRegions; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionStrategy.java index a6ac8f0696ea..e118f7af1ea4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionStrategy.java @@ -95,10 +95,11 @@ public RestartPipelinedRegionStrategy( // ------------------------------------------------------------------------ private void buildFailoverRegions() { - final Set>> distinctRegions = PipelinedRegionComputeUtil.computePipelinedRegions(topology); + final Set>> distinctRegions = + PipelinedRegionComputeUtil.computePipelinedRegions(topology); // creating all the failover regions and register them - for (Set> regionVertices : distinctRegions) { + for (Set> regionVertices : distinctRegions) { LOG.debug("Creating a failover region with {} vertices.", regionVertices.size()); final FailoverRegion failoverRegion = new FailoverRegion(regionVertices); regions.add(failoverRegion); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/RegionPartitionReleaseStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/RegionPartitionReleaseStrategy.java index 7c452ddfbb73..f65f67ae3742 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/RegionPartitionReleaseStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/RegionPartitionReleaseStrategy.java @@ -180,7 +180,8 @@ public PartitionReleaseStrategy createInstance( final SchedulingTopology schedulingStrategy, final FailoverTopology failoverTopology) { - final Set>> distinctRegions = PipelinedRegionComputeUtil.computePipelinedRegions(failoverTopology); + final Set>> distinctRegions = + PipelinedRegionComputeUtil.computePipelinedRegions(failoverTopology); return new RegionPartitionReleaseStrategy( schedulingStrategy, PipelinedRegionComputeUtil.toPipelinedRegionsSet(distinctRegions)); From 101f3f833c1add1a997bcb575618b65f0b88ed0f Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Tue, 22 Oct 2019 16:52:45 +0800 Subject: [PATCH 297/746] [hotfix][runtime] Remove FailoverTopology dependency from PartitionReleaseStrategy --- .../flink/runtime/executiongraph/ExecutionGraph.java | 4 +--- .../failover/flip1/PipelinedRegionComputeUtil.java | 7 ++++--- .../NotReleasingPartitionReleaseStrategy.java | 6 +----- .../partitionrelease/PartitionReleaseStrategy.java | 3 +-- .../RegionPartitionReleaseStrategy.java | 11 ++++------- 5 files changed, 11 insertions(+), 20 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 272cc6a55b53..dab62df761fd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -959,9 +959,7 @@ public void attachJobGraph(List topologiallySorted) throws JobExcepti failoverStrategy.notifyNewVertices(newExecJobVertices); - partitionReleaseStrategy = partitionReleaseStrategyFactory.createInstance( - getSchedulingTopology(), - getFailoverTopology()); + partitionReleaseStrategy = partitionReleaseStrategyFactory.createInstance(getSchedulingTopology()); } public boolean isLegacyScheduling() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/PipelinedRegionComputeUtil.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/PipelinedRegionComputeUtil.java index 5dde31a3ff38..856116b21bb1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/PipelinedRegionComputeUtil.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/PipelinedRegionComputeUtil.java @@ -21,6 +21,7 @@ import org.apache.flink.runtime.executiongraph.failover.flip1.partitionrelease.PipelinedRegion; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.runtime.scheduler.strategy.SchedulingExecutionVertex; import org.apache.flink.runtime.topology.Result; import org.apache.flink.runtime.topology.Topology; import org.apache.flink.runtime.topology.Vertex; @@ -44,7 +45,7 @@ public final class PipelinedRegionComputeUtil { private static final Logger LOG = LoggerFactory.getLogger(PipelinedRegionComputeUtil.class); public static Set toPipelinedRegionsSet( - final Set>> distinctRegions) { + final Set>> distinctRegions) { return distinctRegions.stream() .map(toExecutionVertexIdSet()) @@ -52,9 +53,9 @@ public static Set toPipelinedRegionsSet( .collect(Collectors.toSet()); } - private static Function>, Set> toExecutionVertexIdSet() { + private static Function>, Set> toExecutionVertexIdSet() { return failoverVertices -> failoverVertices.stream() - .map(FailoverVertex::getId) + .map(SchedulingExecutionVertex::getId) .collect(Collectors.toSet()); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/NotReleasingPartitionReleaseStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/NotReleasingPartitionReleaseStrategy.java index fd8efe35306a..019a59544720 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/NotReleasingPartitionReleaseStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/NotReleasingPartitionReleaseStrategy.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.executiongraph.failover.flip1.partitionrelease; -import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverTopology; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology; @@ -48,10 +47,7 @@ public void vertexUnfinished(final ExecutionVertexID executionVertexID) { public static class Factory implements PartitionReleaseStrategy.Factory { @Override - public PartitionReleaseStrategy createInstance( - final SchedulingTopology schedulingStrategy, - final FailoverTopology failoverTopology) { - + public PartitionReleaseStrategy createInstance(final SchedulingTopology schedulingStrategy) { return new NotReleasingPartitionReleaseStrategy(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/PartitionReleaseStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/PartitionReleaseStrategy.java index 461b93a2d76f..7002ced6e1c6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/PartitionReleaseStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/PartitionReleaseStrategy.java @@ -20,7 +20,6 @@ package org.apache.flink.runtime.executiongraph.failover.flip1.partitionrelease; import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; -import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverTopology; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; import org.apache.flink.runtime.scheduler.strategy.SchedulingTopology; @@ -53,6 +52,6 @@ public interface PartitionReleaseStrategy { * Factory for {@link PartitionReleaseStrategy}. */ interface Factory { - PartitionReleaseStrategy createInstance(SchedulingTopology schedulingStrategy, FailoverTopology failoverTopology); + PartitionReleaseStrategy createInstance(SchedulingTopology schedulingStrategy); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/RegionPartitionReleaseStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/RegionPartitionReleaseStrategy.java index f65f67ae3742..a05a24a407bc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/RegionPartitionReleaseStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/partitionrelease/RegionPartitionReleaseStrategy.java @@ -19,8 +19,6 @@ package org.apache.flink.runtime.executiongraph.failover.flip1.partitionrelease; -import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverTopology; -import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverVertex; import org.apache.flink.runtime.executiongraph.failover.flip1.PipelinedRegionComputeUtil; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; @@ -176,12 +174,11 @@ private boolean isRegionOfVertexFinished(final ExecutionVertexID executionVertex public static class Factory implements PartitionReleaseStrategy.Factory { @Override - public PartitionReleaseStrategy createInstance( - final SchedulingTopology schedulingStrategy, - final FailoverTopology failoverTopology) { + public PartitionReleaseStrategy createInstance(final SchedulingTopology schedulingStrategy) { + + final Set>> distinctRegions = + PipelinedRegionComputeUtil.computePipelinedRegions(schedulingStrategy); - final Set>> distinctRegions = - PipelinedRegionComputeUtil.computePipelinedRegions(failoverTopology); return new RegionPartitionReleaseStrategy( schedulingStrategy, PipelinedRegionComputeUtil.toPipelinedRegionsSet(distinctRegions)); From ba2233b9220311db17b531862d4588c544671a24 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 24 Oct 2019 11:44:07 +0200 Subject: [PATCH 298/746] [FLINK-14518][coordination] Generalize TE->RM heartbeat payload Introduce an encapsulating class for the heartbeat payload to simplify future additions to the payload. --- .../resourcemanager/ResourceManager.java | 13 +++--- .../ResourceManagerGateway.java | 5 ++- .../runtime/taskexecutor/TaskExecutor.java | 20 ++++----- .../TaskExecutorHeartbeatPayload.java | 45 +++++++++++++++++++ .../utils/TestingResourceManagerGateway.java | 5 ++- 5 files changed, 68 insertions(+), 20 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorHeartbeatPayload.java 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 161aef8ebd1a..c89a88d63c10 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 @@ -65,6 +65,7 @@ import org.apache.flink.runtime.taskexecutor.FileType; import org.apache.flink.runtime.taskexecutor.SlotReport; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; +import org.apache.flink.runtime.taskexecutor.TaskExecutorHeartbeatPayload; import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess; import org.apache.flink.runtime.taskexecutor.TaskManagerServices; import org.apache.flink.util.ExceptionUtils; @@ -139,7 +140,7 @@ public abstract class ResourceManager private LeaderElectionService leaderElectionService; /** The heartbeat manager with task managers. */ - private HeartbeatManager taskManagerHeartbeatManager; + private HeartbeatManager taskManagerHeartbeatManager; /** The heartbeat manager with job managers. */ private HeartbeatManager jobManagerHeartbeatManager; @@ -404,8 +405,8 @@ public CompletableFuture sendSlotReport(ResourceID taskManagerResou } @Override - public void heartbeatFromTaskManager(final ResourceID resourceID, final SlotReport slotReport) { - taskManagerHeartbeatManager.receiveHeartbeat(resourceID, slotReport); + public void heartbeatFromTaskManager(final ResourceID resourceID, final TaskExecutorHeartbeatPayload heartbeatPayload) { + taskManagerHeartbeatManager.receiveHeartbeat(resourceID, heartbeatPayload); } @Override @@ -1129,7 +1130,7 @@ public void handleError(Throwable error) { } } - private class TaskManagerHeartbeatListener implements HeartbeatListener { + private class TaskManagerHeartbeatListener implements HeartbeatListener { @Override public void notifyHeartbeatTimeout(final ResourceID resourceID) { @@ -1142,7 +1143,7 @@ public void notifyHeartbeatTimeout(final ResourceID resourceID) { } @Override - public void reportPayload(final ResourceID resourceID, final SlotReport payload) { + public void reportPayload(final ResourceID resourceID, final TaskExecutorHeartbeatPayload payload) { validateRunsInMainThread(); final WorkerRegistration workerRegistration = taskExecutors.get(resourceID); @@ -1151,7 +1152,7 @@ public void reportPayload(final ResourceID resourceID, final SlotReport payload) } else { InstanceID instanceId = workerRegistration.getInstanceID(); - slotManager.reportSlotStatus(instanceId, payload); + slotManager.reportSlotStatus(instanceId, payload.getSlotReport()); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java index dcb4d5095ce0..4c72e6802b03 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java @@ -40,6 +40,7 @@ import org.apache.flink.runtime.taskexecutor.FileType; import org.apache.flink.runtime.taskexecutor.SlotReport; import org.apache.flink.runtime.taskexecutor.TaskExecutor; +import org.apache.flink.runtime.taskexecutor.TaskExecutorHeartbeatPayload; import javax.annotation.Nullable; @@ -150,9 +151,9 @@ void notifySlotAvailable( * Sends the heartbeat to resource manager from task manager * * @param heartbeatOrigin unique id of the task manager - * @param slotReport Current slot allocation on the originating TaskManager + * @param heartbeatPayload payload from the originating TaskManager */ - void heartbeatFromTaskManager(final ResourceID heartbeatOrigin, final SlotReport slotReport); + void heartbeatFromTaskManager(final ResourceID heartbeatOrigin, final TaskExecutorHeartbeatPayload heartbeatPayload); /** * Sends the heartbeat to resource manager from job manager 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 f0db4cd82b2d..8146cd5db250 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 @@ -208,7 +208,7 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { private final HeartbeatManager jobManagerHeartbeatManager; /** The heartbeat manager for resource manager in the task manager. */ - private final HeartbeatManager resourceManagerHeartbeatManager; + private final HeartbeatManager resourceManagerHeartbeatManager; private final PartitionTable partitionTable; @@ -281,7 +281,7 @@ public TaskExecutor( this.resourceManagerHeartbeatManager = createResourceManagerHeartbeatManager(heartbeatServices, resourceId); } - private HeartbeatManager createResourceManagerHeartbeatManager(HeartbeatServices heartbeatServices, ResourceID resourceId) { + private HeartbeatManager createResourceManagerHeartbeatManager(HeartbeatServices heartbeatServices, ResourceID resourceId) { return heartbeatServices.createHeartbeatManager( resourceId, new ResourceManagerHeartbeatListener(), @@ -1026,14 +1026,14 @@ private void establishResourceManagerConnection( }, getMainThreadExecutor()); // monitor the resource manager as heartbeat target - resourceManagerHeartbeatManager.monitorTarget(resourceManagerResourceId, new HeartbeatTarget() { + resourceManagerHeartbeatManager.monitorTarget(resourceManagerResourceId, new HeartbeatTarget() { @Override - public void receiveHeartbeat(ResourceID resourceID, SlotReport slotReport) { - resourceManagerGateway.heartbeatFromTaskManager(resourceID, slotReport); + public void receiveHeartbeat(ResourceID resourceID, TaskExecutorHeartbeatPayload heartbeatPayload) { + resourceManagerGateway.heartbeatFromTaskManager(resourceID, heartbeatPayload); } @Override - public void requestHeartbeat(ResourceID resourceID, SlotReport slotReport) { + public void requestHeartbeat(ResourceID resourceID, TaskExecutorHeartbeatPayload heartbeatPayload) { // the TaskManager won't send heartbeat requests to the ResourceManager } }); @@ -1634,7 +1634,7 @@ TaskExecutorToResourceManagerConnection getResourceManagerConnection() { } @VisibleForTesting - HeartbeatManager getResourceManagerHeartbeatManager() { + HeartbeatManager getResourceManagerHeartbeatManager() { return resourceManagerHeartbeatManager; } @@ -1816,7 +1816,7 @@ public AccumulatorReport retrievePayload(ResourceID resourceID) { } } - private class ResourceManagerHeartbeatListener implements HeartbeatListener { + private class ResourceManagerHeartbeatListener implements HeartbeatListener { @Override public void notifyHeartbeatTimeout(final ResourceID resourceId) { @@ -1838,9 +1838,9 @@ public void reportPayload(ResourceID resourceID, Void payload) { } @Override - public SlotReport retrievePayload(ResourceID resourceID) { + public TaskExecutorHeartbeatPayload retrievePayload(ResourceID resourceID) { validateRunsInMainThread(); - return taskSlotTable.createSlotReport(getResourceID()); + return new TaskExecutorHeartbeatPayload(taskSlotTable.createSlotReport(getResourceID())); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorHeartbeatPayload.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorHeartbeatPayload.java new file mode 100644 index 000000000000..c53b08e93d59 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorHeartbeatPayload.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.runtime.taskexecutor; + +import java.io.Serializable; + +/** + * Payload for heartbeats sent from the TaskExecutor to the ResourceManager. + */ +public class TaskExecutorHeartbeatPayload implements Serializable { + + private static final long serialVersionUID = -4556838854992435612L; + + private final SlotReport slotReport; + + public TaskExecutorHeartbeatPayload(SlotReport slotReport) { + this.slotReport = slotReport; + } + + public SlotReport getSlotReport() { + return slotReport; + } + + @Override + public String toString() { + return "TaskExecutorHeartbeatPayload{" + + "slotReport=" + slotReport + + '}'; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/utils/TestingResourceManagerGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/utils/TestingResourceManagerGateway.java index 721911103d2e..4a4e2b90ac52 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/utils/TestingResourceManagerGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/utils/TestingResourceManagerGateway.java @@ -43,6 +43,7 @@ import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerInfo; import org.apache.flink.runtime.taskexecutor.FileType; import org.apache.flink.runtime.taskexecutor.SlotReport; +import org.apache.flink.runtime.taskexecutor.TaskExecutorHeartbeatPayload; import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess; import org.apache.flink.util.Preconditions; @@ -246,11 +247,11 @@ public CompletableFuture getNumberOfRegisteredTaskManagers() { } @Override - public void heartbeatFromTaskManager(ResourceID heartbeatOrigin, SlotReport slotReport) { + public void heartbeatFromTaskManager(ResourceID heartbeatOrigin, TaskExecutorHeartbeatPayload heartbeatPayload) { final BiConsumer currentTaskExecutorHeartbeatConsumer = taskExecutorHeartbeatConsumer; if (currentTaskExecutorHeartbeatConsumer != null) { - currentTaskExecutorHeartbeatConsumer.accept(heartbeatOrigin, slotReport); + currentTaskExecutorHeartbeatConsumer.accept(heartbeatOrigin, heartbeatPayload.getSlotReport()); } } From a8619c50119bf32a117a48369d679e8590fb940c Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 18 Oct 2019 16:36:48 +0200 Subject: [PATCH 299/746] [hotfix][tests] Fix TestingShuffleMaster usage --- .../runtime/io/network/partition/PartitionTrackerImplTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTrackerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTrackerImplTest.java index 63e4f449da5c..01140ada0f92 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTrackerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTrackerImplTest.java @@ -216,7 +216,7 @@ public void testStopTrackingIssuesNoReleaseCalls() { final Queue>> taskExecutorReleaseCalls = new ArrayBlockingQueue<>(4); final PartitionTracker partitionTracker = new PartitionTrackerImpl( new JobID(), - new TestingShuffleMaster(), + shuffleMaster, resourceId -> Optional.of(createTaskExecutorGateway(resourceId, taskExecutorReleaseCalls)) ); From bebd27d62612e27307956a1a13c74e6111907075 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 18 Oct 2019 15:42:20 +0200 Subject: [PATCH 300/746] [FLINK-14475][coordination] Adjust TaskExecutor interface to accept promotions --- .../flink/runtime/executiongraph/Execution.java | 4 ++-- .../io/network/partition/PartitionTrackerImpl.java | 7 ++++--- .../jobmanager/slots/TaskManagerGateway.java | 4 ++-- .../runtime/jobmaster/RpcTaskManagerGateway.java | 7 ++++--- .../flink/runtime/taskexecutor/TaskExecutor.java | 6 +++--- .../runtime/taskexecutor/TaskExecutorGateway.java | 10 +++++----- .../ExecutionPartitionLifecycleTest.java | 3 ++- .../utils/SimpleAckingTaskManagerGateway.java | 3 ++- .../network/partition/PartitionTrackerImplTest.java | 2 +- .../flink/runtime/jobmaster/JobMasterTest.java | 2 +- .../TaskExecutorPartitionLifecycleTest.java | 8 ++++---- .../taskexecutor/TestingTaskExecutorGateway.java | 13 +++++++------ .../TestingTaskExecutorGatewayBuilder.java | 13 +++++++------ 13 files changed, 44 insertions(+), 38 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 d9cdf5a9e41b..595fb2ce9d7b 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 @@ -1385,12 +1385,12 @@ private void sendReleaseIntermediateResultPartitionsRpcCall() { final ShuffleMaster shuffleMaster = getVertex().getExecutionGraph().getShuffleMaster(); - Collection partitionIds = producedPartitions.values().stream() + Set partitionIds = producedPartitions.values().stream() .filter(resultPartitionDeploymentDescriptor -> resultPartitionDeploymentDescriptor.getPartitionType().isPipelined()) .map(ResultPartitionDeploymentDescriptor::getShuffleDescriptor) .peek(shuffleMaster::releasePartitionExternally) .map(ShuffleDescriptor::getResultPartitionID) - .collect(Collectors.toList()); + .collect(Collectors.toSet()); if (!partitionIds.isEmpty()) { // TODO For some tests this could be a problem when querying too early if all resources were released diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionTrackerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionTrackerImpl.java index 2e7f4216b701..092047b317b5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionTrackerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionTrackerImpl.java @@ -31,6 +31,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.stream.Collectors; import static java.util.stream.Collectors.toList; @@ -158,17 +159,17 @@ private void internalReleasePartitionsOnTaskExecutor( ResourceID potentialPartitionLocation, Collection partitionDeploymentDescriptors) { - final List partitionsRequiringRpcReleaseCalls = partitionDeploymentDescriptors.stream() + final Set partitionsRequiringRpcReleaseCalls = partitionDeploymentDescriptors.stream() .map(ResultPartitionDeploymentDescriptor::getShuffleDescriptor) .filter(descriptor -> descriptor.storesLocalResourcesOn().isPresent()) .map(ShuffleDescriptor::getResultPartitionID) - .collect(Collectors.toList()); + .collect(Collectors.toSet()); if (!partitionsRequiringRpcReleaseCalls.isEmpty()) { taskExecutorGatewayLookup .lookup(potentialPartitionLocation) .ifPresent(taskExecutorGateway -> - taskExecutorGateway.releasePartitions(jobId, partitionsRequiringRpcReleaseCalls)); + taskExecutorGateway.releaseOrPromotePartitions(jobId, partitionsRequiringRpcReleaseCalls, Collections.emptySet())); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/TaskManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/TaskManagerGateway.java index 77b04600c179..6e51b1e1aa22 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/TaskManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/TaskManagerGateway.java @@ -30,7 +30,7 @@ import org.apache.flink.runtime.messages.StackTraceSampleResponse; import org.apache.flink.runtime.rpc.RpcTimeout; -import java.util.Collection; +import java.util.Set; import java.util.concurrent.CompletableFuture; /** @@ -105,7 +105,7 @@ CompletableFuture updatePartitions( * @param jobId id of the job that the partitions belong to * @param partitionIds partition ids to release */ - void releasePartitions(JobID jobId, Collection partitionIds); + void releasePartitions(JobID jobId, Set partitionIds); /** * Notify the given task about a completed checkpoint. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/RpcTaskManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/RpcTaskManagerGateway.java index 1ef64164e4f9..96265431e5d1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/RpcTaskManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/RpcTaskManagerGateway.java @@ -32,7 +32,8 @@ import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.apache.flink.util.Preconditions; -import java.util.Collection; +import java.util.Collections; +import java.util.Set; import java.util.concurrent.CompletableFuture; /** @@ -88,8 +89,8 @@ public CompletableFuture updatePartitions(ExecutionAttemptID execut } @Override - public void releasePartitions(JobID jobId, Collection partitionIds) { - taskExecutorGateway.releasePartitions(jobId, partitionIds); + public void releasePartitions(JobID jobId, Set partitionIds) { + taskExecutorGateway.releaseOrPromotePartitions(jobId, partitionIds, Collections.emptySet()); } @Override 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 8146cd5db250..5f22ef62125c 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 @@ -725,10 +725,10 @@ public CompletableFuture updatePartitions( } @Override - public void releasePartitions(JobID jobId, Collection partitionIds) { + public void releaseOrPromotePartitions(JobID jobId, Set partitionToRelease, Set partitionsToPromote) { try { - partitionTable.stopTrackingPartitions(jobId, partitionIds); - shuffleEnvironment.releasePartitionsLocally(partitionIds); + partitionTable.stopTrackingPartitions(jobId, partitionToRelease); + shuffleEnvironment.releasePartitionsLocally(partitionToRelease); closeJobManagerConnectionIfNoAllocatedResources(jobId); } catch (Throwable t) { // TODO: Do we still need this catch branch? diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java index 75290d33fd50..9e50ada6cad0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java @@ -40,7 +40,7 @@ import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.types.SerializableOptional; -import java.util.Collection; +import java.util.Set; import java.util.concurrent.CompletableFuture; /** @@ -102,12 +102,12 @@ CompletableFuture updatePartitions( @RpcTimeout Time timeout); /** - * Batch release intermediate result partitions. - * + * Batch release/promote intermediate result partitions. * @param jobId id of the job that the partitions belong to - * @param partitionIds partition ids to release + * @param partitionToRelease partition ids to release + * @param partitionsToPromote partitions ids to promote */ - void releasePartitions(JobID jobId, Collection partitionIds); + void releaseOrPromotePartitions(JobID jobId, Set partitionToRelease, Set partitionsToPromote); /** * Trigger the checkpoint for the given task. The checkpoint is identified by the checkpoint ID diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionPartitionLifecycleTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionPartitionLifecycleTest.java index efa9ad4b1cc1..058857c36c29 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionPartitionLifecycleTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionPartitionLifecycleTest.java @@ -77,6 +77,7 @@ import java.util.concurrent.CompletableFuture; import java.util.function.Consumer; +import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -129,7 +130,7 @@ private void testPartitionReleaseOnStateTransitionsAfterRunning(Consumer> releasePartitionsCall = releasePartitionsCallFuture.get(); assertEquals(jobId, releasePartitionsCall.f0); - assertEquals(Collections.singletonList(descriptor.getShuffleDescriptor().getResultPartitionID()), releasePartitionsCall.f1); + assertThat(releasePartitionsCall.f1, contains(descriptor.getShuffleDescriptor().getResultPartitionID())); assertEquals(1, testingShuffleMaster.externallyReleasedPartitions.size()); assertEquals(descriptor.getShuffleDescriptor(), testingShuffleMaster.externallyReleasedPartitions.poll()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleAckingTaskManagerGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleAckingTaskManagerGateway.java index 02e148647188..0d07f3d31199 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleAckingTaskManagerGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleAckingTaskManagerGateway.java @@ -32,6 +32,7 @@ import org.apache.flink.runtime.messages.StackTraceSampleResponse; import java.util.Collection; +import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.function.BiConsumer; @@ -104,7 +105,7 @@ public CompletableFuture updatePartitions(ExecutionAttemptID execut } @Override - public void releasePartitions(JobID jobId, Collection partitionIds) { + public void releasePartitions(JobID jobId, Set partitionIds) { releasePartitionsConsumer.accept(jobId, partitionIds); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTrackerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTrackerImplTest.java index 01140ada0f92..37a359590cbf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTrackerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTrackerImplTest.java @@ -270,7 +270,7 @@ public Optional storesLocalResourcesOn() { private static TaskExecutorGateway createTaskExecutorGateway(ResourceID taskExecutorId, Collection>> releaseCalls) { return new TestingTaskExecutorGatewayBuilder() - .setReleasePartitionsConsumer((jobId, partitionIds) -> releaseCalls.add(Tuple3.of(taskExecutorId, jobId, partitionIds))) + .setReleaseOrPromotePartitionsConsumer((jobId, partitionToRelease, partitionsToPromote) -> releaseCalls.add(Tuple3.of(taskExecutorId, jobId, partitionToRelease))) .createTestingTaskExecutorGateway(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java index dfce78159605..4bcfd2cb6965 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java @@ -1924,7 +1924,7 @@ public void testPartitionReleaseOnJobTermination() throws Exception { final CompletableFuture>> releasePartitionsFuture = new CompletableFuture<>(); final CompletableFuture disconnectTaskExecutorFuture = new CompletableFuture<>(); final TestingTaskExecutorGateway testingTaskExecutorGateway = new TestingTaskExecutorGatewayBuilder() - .setReleasePartitionsConsumer((jobId, partitions) -> releasePartitionsFuture.complete(Tuple2.of(jobId, partitions))) + .setReleaseOrPromotePartitionsConsumer((jobId, partitionsToRelease, partitionsToPromote) -> releasePartitionsFuture.complete(Tuple2.of(jobId, partitionsToRelease))) .setDisconnectJobManagerConsumer((jobID, throwable) -> disconnectTaskExecutorFuture.complete(jobID)) .setSubmitTaskConsumer((tdd, ignored) -> { taskDeploymentDescriptorFuture.complete(tdd); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java index 6b6b9e4be869..e6ed4ae4b5b1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java @@ -169,7 +169,7 @@ public void testConnectionTerminationAfterExternalRelease() throws Exception { final CompletableFuture> firstReleasePartitionsCallFuture = new CompletableFuture<>(); runInTaskExecutorThreadAndWait(taskExecutor, () -> shuffleEnvironment.releasePartitionsLocallyFuture = firstReleasePartitionsCallFuture); - taskExecutorGateway.releasePartitions(jobId, Collections.singletonList(new ResultPartitionID())); + taskExecutorGateway.releaseOrPromotePartitions(jobId, Collections.singleton(new ResultPartitionID()), Collections.emptySet()); // at this point we only know that the TE has entered releasePartitions; we cannot be certain whether it // has already checked whether it should disconnect or not @@ -183,7 +183,7 @@ public void testConnectionTerminationAfterExternalRelease() throws Exception { runInTaskExecutorThreadAndWait(taskExecutor, () -> shuffleEnvironment.releasePartitionsLocallyFuture = secondReleasePartitionsCallFuture); // the TM should check whether partitions are still stored, and afterwards terminate the connection - taskExecutorGateway.releasePartitions(jobId, Collections.singletonList(resultPartitionId)); + taskExecutorGateway.releaseOrPromotePartitions(jobId, Collections.singleton(resultPartitionId), Collections.emptySet()); disconnectFuture.get(); } finally { @@ -210,7 +210,7 @@ public void testPipelinedPartitionNotReleasedAfterDisconnect() throws Exception @Test public void testBlockingPartitionReleaseAfterReleaseCall() throws Exception { testPartitionRelease( - (jobId, partitionId, taskExecutorGateway) -> taskExecutorGateway.releasePartitions(jobId, Collections.singletonList(partitionId)), + (jobId, partitionId, taskExecutorGateway) -> taskExecutorGateway.releaseOrPromotePartitions(jobId, Collections.singleton(partitionId), Collections.emptySet()), true, ResultPartitionType.BLOCKING); } @@ -218,7 +218,7 @@ public void testBlockingPartitionReleaseAfterReleaseCall() throws Exception { @Test public void testPipelinedPartitionReleaseAfterReleaseCall() throws Exception { testPartitionRelease( - (jobId, partitionId, taskExecutorGateway) -> taskExecutorGateway.releasePartitions(jobId, Collections.singletonList(partitionId)), + (jobId, partitionId, taskExecutorGateway) -> taskExecutorGateway.releaseOrPromotePartitions(jobId, Collections.singleton(partitionId), Collections.emptySet()), true, ResultPartitionType.PIPELINED); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGateway.java index 3aa00e6c7df7..ac8e3f088c52 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGateway.java @@ -38,8 +38,9 @@ import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.types.SerializableOptional; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.function.TriConsumer; -import java.util.Collection; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.function.BiConsumer; import java.util.function.BiFunction; @@ -74,7 +75,7 @@ public class TestingTaskExecutorGateway implements TaskExecutorGateway { private final Supplier> canBeReleasedSupplier; - private final BiConsumer> releasePartitionsConsumer; + private final TriConsumer, Set> releaseOrPromotePartitionsConsumer; TestingTaskExecutorGateway( String address, @@ -88,7 +89,7 @@ public class TestingTaskExecutorGateway implements TaskExecutorGateway { Consumer disconnectResourceManagerConsumer, Function> cancelTaskFunction, Supplier> canBeReleasedSupplier, - BiConsumer> releasePartitionsConsumer) { + TriConsumer, Set> releaseOrPromotePartitionsConsumer) { this.address = Preconditions.checkNotNull(address); this.hostname = Preconditions.checkNotNull(hostname); this.heartbeatJobManagerConsumer = Preconditions.checkNotNull(heartbeatJobManagerConsumer); @@ -100,7 +101,7 @@ public class TestingTaskExecutorGateway implements TaskExecutorGateway { this.disconnectResourceManagerConsumer = disconnectResourceManagerConsumer; this.cancelTaskFunction = cancelTaskFunction; this.canBeReleasedSupplier = canBeReleasedSupplier; - this.releasePartitionsConsumer = releasePartitionsConsumer; + this.releaseOrPromotePartitionsConsumer = releaseOrPromotePartitionsConsumer; } @Override @@ -130,8 +131,8 @@ public CompletableFuture updatePartitions(ExecutionAttemptID execut } @Override - public void releasePartitions(JobID jobId, Collection partitionIds) { - releasePartitionsConsumer.accept(jobId, partitionIds); + public void releaseOrPromotePartitions(JobID jobId, Set partitionToRelease, Set partitionsToPromote) { + releaseOrPromotePartitionsConsumer.accept(jobId, partitionToRelease, partitionsToPromote); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGatewayBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGatewayBuilder.java index c176ff8df4e4..9702fe59ae78 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGatewayBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGatewayBuilder.java @@ -30,8 +30,9 @@ import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.resourcemanager.ResourceManagerId; +import org.apache.flink.util.function.TriConsumer; -import java.util.Collection; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.function.BiConsumer; import java.util.function.BiFunction; @@ -52,7 +53,7 @@ public class TestingTaskExecutorGatewayBuilder { private static final Consumer NOOP_HEARTBEAT_RESOURCE_MANAGER_CONSUMER = ignored -> {}; private static final Consumer NOOP_DISCONNECT_RESOURCE_MANAGER_CONSUMER = ignored -> {}; private static final Function> NOOP_CANCEL_TASK_FUNCTION = ignored -> CompletableFuture.completedFuture(Acknowledge.get()); - private static final BiConsumer> NOOP_RELEASE_PARTITIONS_CONSUMER = (ignoredA, ignoredB) -> {}; + private static final TriConsumer, Set> NOOP_RELEASE_PARTITIONS_CONSUMER = (ignoredA, ignoredB, ignoredC) -> {}; private String address = "foobar:1234"; private String hostname = "foobar"; @@ -65,7 +66,7 @@ public class TestingTaskExecutorGatewayBuilder { private Consumer disconnectResourceManagerConsumer = NOOP_DISCONNECT_RESOURCE_MANAGER_CONSUMER; private Function> cancelTaskFunction = NOOP_CANCEL_TASK_FUNCTION; private Supplier> canBeReleasedSupplier = () -> CompletableFuture.completedFuture(true); - private BiConsumer> releasePartitionsConsumer = NOOP_RELEASE_PARTITIONS_CONSUMER; + private TriConsumer, Set> releaseOrPromotePartitionsConsumer = NOOP_RELEASE_PARTITIONS_CONSUMER; public TestingTaskExecutorGatewayBuilder setAddress(String address) { this.address = address; @@ -122,8 +123,8 @@ public TestingTaskExecutorGatewayBuilder setCanBeReleasedSupplier(Supplier> releasePartitionsConsumer) { - this.releasePartitionsConsumer = releasePartitionsConsumer; + public TestingTaskExecutorGatewayBuilder setReleaseOrPromotePartitionsConsumer(TriConsumer, Set> releasePartitionsConsumer) { + this.releaseOrPromotePartitionsConsumer = releasePartitionsConsumer; return this; } @@ -140,6 +141,6 @@ public TestingTaskExecutorGateway createTestingTaskExecutorGateway() { disconnectResourceManagerConsumer, cancelTaskFunction, canBeReleasedSupplier, - releasePartitionsConsumer); + releaseOrPromotePartitionsConsumer); } } From b6361a959b0ad3b025b81c66b1ebb3dfaff4ec3e Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Wed, 23 Oct 2019 15:40:15 -0700 Subject: [PATCH 301/746] [FLINK-14417][table] Develop CoreModule to provide Flink built-in functions Develop CoreModule to provide Flink built-in functions. This closes #9979. --- .../flink/table/catalog/FunctionCatalog.java | 40 ++++---------- .../table/catalog/FunctionCatalogUtil.java | 31 +++++++++++ .../flink/table/module/ModuleManager.java | 4 +- .../table/catalog/FunctionCatalogTest.java | 7 +-- .../apache/flink/table/module/CoreModule.java | 53 +++++++++++++++++++ .../flink/table/module/ModuleConfig.java | 27 ++++++++++ 6 files changed, 123 insertions(+), 39 deletions(-) create mode 100644 flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalogUtil.java create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/CoreModule.java create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/ModuleConfig.java 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 61f115440e9e..09313bffed5b 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 @@ -27,7 +27,6 @@ import org.apache.flink.table.delegation.PlannerTypeInferenceUtil; import org.apache.flink.table.functions.AggregateFunction; import org.apache.flink.table.functions.AggregateFunctionDefinition; -import org.apache.flink.table.functions.BuiltInFunctionDefinitions; import org.apache.flink.table.functions.FunctionDefinition; import org.apache.flink.table.functions.FunctionDefinitionUtil; import org.apache.flink.table.functions.ScalarFunction; @@ -46,7 +45,6 @@ import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.function.Function; import java.util.stream.Collectors; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -203,12 +201,7 @@ public String[] getFunctions() { Set result = getUserDefinedFunctionNames(); // Get built-in functions - result.addAll( - BuiltInFunctionDefinitions.getDefinitions() - .stream() - .map(f -> normalizeName(f.getName())) - .collect(Collectors.toSet()) - ); + result.addAll(moduleManager.listFunctions()); return result.toArray(new String[0]); } @@ -235,7 +228,7 @@ private Set getUserDefinedFunctionNames() { @Override public Optional lookupFunction(String name) { - String functionName = normalizeName(name); + String normalizedName = FunctionCatalogUtil.normalizeName(name); FunctionDefinition userCandidate; @@ -243,12 +236,12 @@ public Optional lookupFunction(String name) { try { CatalogFunction catalogFunction = catalog.getFunction( - new ObjectPath(catalogManager.getCurrentDatabase(), functionName)); + new ObjectPath(catalogManager.getCurrentDatabase(), normalizedName)); if (catalog.getFunctionDefinitionFactory().isPresent()) { - userCandidate = catalog.getFunctionDefinitionFactory().get().createFunctionDefinition(functionName, catalogFunction); + userCandidate = catalog.getFunctionDefinitionFactory().get().createFunctionDefinition(normalizedName, catalogFunction); } else { - userCandidate = FunctionDefinitionUtil.createFunctionDefinition(functionName, catalogFunction); + userCandidate = FunctionDefinitionUtil.createFunctionDefinition(normalizedName, catalogFunction); } return Optional.of( @@ -261,23 +254,13 @@ public Optional lookupFunction(String name) { } // If no corresponding function is found in catalog, check in-memory functions - userCandidate = tempSystemFunctions.get(functionName); + userCandidate = tempSystemFunctions.get(normalizedName); final Optional foundDefinition; if (userCandidate != null) { foundDefinition = Optional.of(userCandidate); } else { - - // TODO once we connect this class with the Catalog APIs we need to make sure that - // built-in functions are present in "root" built-in catalog. This allows to - // overwrite built-in functions but also fallback to the "root" catalog. It should be - // possible to disable the "root" catalog if that is desired. - - foundDefinition = BuiltInFunctionDefinitions.getDefinitions() - .stream() - .filter(f -> functionName.equals(normalizeName(f.getName()))) - .findFirst() - .map(Function.identity()); + foundDefinition = moduleManager.getFunctionDefinition(normalizedName); } return foundDefinition.map(definition -> new FunctionLookup.Result( @@ -298,23 +281,18 @@ public PlannerTypeInferenceUtil getPlannerTypeInferenceUtil() { } private void registerTempSystemFunction(String name, FunctionDefinition functionDefinition) { - tempSystemFunctions.put(normalizeName(name), functionDefinition); + tempSystemFunctions.put(FunctionCatalogUtil.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()); + FunctionCatalogUtil.normalizeName(oi.getObjectName())); } } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalogUtil.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalogUtil.java new file mode 100644 index 000000000000..a22c7d40bf91 --- /dev/null +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalogUtil.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.catalog; + +/** + * Utils for FunctionCatalog. + */ +public class FunctionCatalogUtil { + /** + * Normalize a function name. + */ + public static String normalizeName(String name) { + return name.toLowerCase(); + } +} diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/module/ModuleManager.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/module/ModuleManager.java index 9216237cd31a..fdc26b7f653b 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/module/ModuleManager.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/module/ModuleManager.java @@ -42,7 +42,7 @@ public class ModuleManager { public ModuleManager() { this.modules = new LinkedHashMap<>(); - // TODO: Add Core module to modules + modules.put(ModuleConfig.CORE_MODULE_NAME, CoreModule.INSTANCE); } /** @@ -109,7 +109,7 @@ public Set listFunctions() { */ public Optional getFunctionDefinition(String name) { Optional module = modules.values().stream() - .filter(p -> p.listFunctions().stream().anyMatch(e -> e.equals(name))) + .filter(p -> p.listFunctions().stream().anyMatch(e -> e.equalsIgnoreCase(name))) .findFirst(); return module.isPresent() ? module.get().getFunctionDefinition(name) : Optional.empty(); diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java index 5f79dd5e0a35..852d691b43e8 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java @@ -18,7 +18,6 @@ package org.apache.flink.table.catalog; -import org.apache.flink.table.functions.BuiltInFunctionDefinitions; import org.apache.flink.table.module.ModuleManager; import org.junit.Test; @@ -26,7 +25,6 @@ import java.util.Collections; import java.util.HashSet; import java.util.Set; -import java.util.stream.Collectors; import static org.junit.Assert.assertTrue; @@ -44,10 +42,7 @@ public void testGetBuiltInFunctions() { Set actual = new HashSet<>(); Collections.addAll(actual, functionCatalog.getFunctions()); - Set expected = BuiltInFunctionDefinitions.getDefinitions() - .stream() - .map(f -> FunctionCatalog.normalizeName(f.getName())) - .collect(Collectors.toSet()); + Set expected = new ModuleManager().listFunctions(); assertTrue(actual.containsAll(expected)); } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/CoreModule.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/CoreModule.java new file mode 100644 index 000000000000..a4b371a9b2b2 --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/CoreModule.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.table.module; + +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.functions.FunctionDefinition; + +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Module of default core metadata in Flink. + */ +public class CoreModule implements Module { + public static final CoreModule INSTANCE = new CoreModule(); + + private CoreModule() { + } + + @Override + public Set listFunctions() { + return BuiltInFunctionDefinitions.getDefinitions() + .stream() + .map(f -> f.getName()) + .collect(Collectors.toSet()); + } + + @Override + public Optional getFunctionDefinition(String name) { + return Optional.ofNullable( + BuiltInFunctionDefinitions.getDefinitions().stream() + .filter(f -> f.getName().equalsIgnoreCase(name)) + .findFirst() + .get()); + } +} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/ModuleConfig.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/ModuleConfig.java new file mode 100644 index 000000000000..f4dbb6b4cf83 --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/ModuleConfig.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.module; + +/** + * Configs for modules. + */ +public class ModuleConfig { + // Name of the core module + public static final String CORE_MODULE_NAME = "core"; +} From 23a34b818d8267d16f6c22c77b79c67387da5e88 Mon Sep 17 00:00:00 2001 From: xuyang1706 Date: Fri, 20 Sep 2019 20:22:56 +0800 Subject: [PATCH 302/746] [FLINK-14153][ml] Add to BLAS a method that performs DenseMatrix and SparseVector multiplication. This closes #9732. --- .../apache/flink/ml/common/linalg/BLAS.java | 121 +++++++++-- .../flink/ml/common/linalg/BLASTest.java | 188 ++++++++++++++++++ 2 files changed, 293 insertions(+), 16 deletions(-) create mode 100644 flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/common/linalg/BLASTest.java diff --git a/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/linalg/BLAS.java b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/linalg/BLAS.java index f5d9e8497e8e..80b5d7fe3c5c 100644 --- a/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/linalg/BLAS.java +++ b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/linalg/BLAS.java @@ -19,18 +19,56 @@ package org.apache.flink.ml.common.linalg; +import org.apache.flink.util.Preconditions; + /** * A utility class that provides BLAS routines over matrices and vectors. */ public class BLAS { + + /** + * For level-1 routines, we use Java implementation. + */ private static final com.github.fommil.netlib.BLAS NATIVE_BLAS = com.github.fommil.netlib.BLAS.getInstance(); + + /** + * For level-2 and level-3 routines, we use the native BLAS. + * + * The NATIVE_BLAS instance tries to load BLAS implementations in the order: + * 1) optimized system libraries such as Intel MKL, + * 2) self-contained native builds using the reference Fortran from netlib.org, + * 3) F2J implementation. + * If to use optimized system libraries, it is important to turn of their multi-thread support. + * Otherwise, it will conflict with Flink's executor and leads to performance loss. + */ private static final com.github.fommil.netlib.BLAS F2J_BLAS = com.github.fommil.netlib.F2jBLAS.getInstance(); + /** + * \sum_i |x_i| . + */ + public static double asum(int n, double[] x, int offset) { + return F2J_BLAS.dasum(n, x, offset, 1); + } + + /** + * \sum_i |x_i| . + */ + public static double asum(DenseVector x) { + return asum(x.data.length, x.data, 0); + } + + /** + * \sum_i |x_i| . + */ + public static double asum(SparseVector x) { + return asum(x.values.length, x.values, 0); + } + /** * y += a * x . */ public static void axpy(double a, double[] x, double[] y) { - assert x.length == y.length : "Array dimension mismatched."; + Preconditions.checkArgument(x.length == y.length, "Array dimension mismatched."); F2J_BLAS.daxpy(x.length, a, x, 1, y, 1); } @@ -38,7 +76,7 @@ public static void axpy(double a, double[] x, double[] y) { * y += a * x . */ public static void axpy(double a, DenseVector x, DenseVector y) { - assert x.data.length == y.data.length : "Vector dimension mismatched."; + Preconditions.checkArgument(x.data.length == y.data.length, "Vector dimension mismatched."); F2J_BLAS.daxpy(x.data.length, a, x.data, 1, y.data, 1); } @@ -55,24 +93,34 @@ public static void axpy(double a, SparseVector x, DenseVector y) { * y += a * x . */ public static void axpy(double a, DenseMatrix x, DenseMatrix y) { - assert x.m == y.m && x.n == y.n : "Matrix dimension mismatched."; + Preconditions.checkArgument(x.m == y.m && x.n == y.n, "Matrix dimension mismatched."); F2J_BLAS.daxpy(x.data.length, a, x.data, 1, y.data, 1); } + /** + * y[yOffset:yOffset+n] += a * x[xOffset:xOffset+n] . + */ + public static void axpy(int n, double a, double[] x, int xOffset, double[] y, int yOffset) { + F2J_BLAS.daxpy(n, a, x, xOffset, 1, y, yOffset, 1); + } + /** * x \cdot y . */ public static double dot(double[] x, double[] y) { - assert x.length == y.length : "Array dimension mismatched."; - return F2J_BLAS.ddot(x.length, x, 1, y, 1); + Preconditions.checkArgument(x.length == y.length, "Array dimension mismatched."); + double s = 0.; + for (int i = 0; i < x.length; i++) { + s += x[i] * y[i]; + } + return s; } /** * x \cdot y . */ public static double dot(DenseVector x, DenseVector y) { - assert x.data.length == y.data.length : "Vector dimension mismatched."; - return F2J_BLAS.ddot(x.data.length, x.data, 1, y.data, 1); + return dot(x.getData(), y.getData()); } /** @@ -109,14 +157,18 @@ public static void scal(double a, DenseMatrix x) { public static void gemm(double alpha, DenseMatrix matA, boolean transA, DenseMatrix matB, boolean transB, double beta, DenseMatrix matC) { if (transA) { - assert matA.numCols() == matC.numRows() : "The columns of A does not match the rows of C"; + Preconditions.checkArgument(matA.numCols() == matC.numRows(), + "The columns of A does not match the rows of C"); } else { - assert matA.numRows() == matC.numRows() : "The rows of A does not match the rows of C"; + Preconditions.checkArgument(matA.numRows() == matC.numRows(), + "The rows of A does not match the rows of C"); } if (transB) { - assert matB.numRows() == matC.numCols() : "The rows of B does not match the columns of C"; + Preconditions.checkArgument(matB.numRows() == matC.numCols(), + "The rows of B does not match the columns of C"); } else { - assert matB.numCols() == matC.numCols() : "The columns of B does not match the columns of C"; + Preconditions.checkArgument(matB.numCols() == matC.numCols(), + "The columns of B does not match the columns of C"); } final int m = matC.numRows(); @@ -131,19 +183,56 @@ public static void gemm(double alpha, DenseMatrix matA, boolean transA, DenseMat } /** - * y := alpha * A * x + beta * y . + * Check the compatibility of matrix and vector sizes in gemv. */ - public static void gemv(double alpha, DenseMatrix matA, boolean transA, - DenseVector x, double beta, DenseVector y) { + private static void gemvDimensionCheck(DenseMatrix matA, boolean transA, Vector x, Vector y) { if (transA) { - assert (matA.numCols() == y.size() && matA.numRows() == x.size()) : "Matrix and vector size mismatched."; + Preconditions.checkArgument(matA.numCols() == y.size() && matA.numRows() == x.size(), + "Matrix and vector size mismatched."); } else { - assert (matA.numRows() == y.size() && matA.numCols() == x.size()) : "Matrix and vector size mismatched."; + Preconditions.checkArgument(matA.numRows() == y.size() && matA.numCols() == x.size(), + "Matrix and vector size mismatched."); } + } + + /** + * y := alpha * A * x + beta * y . + */ + public static void gemv(double alpha, DenseMatrix matA, boolean transA, + DenseVector x, double beta, DenseVector y) { + gemvDimensionCheck(matA, transA, x, y); final int m = matA.numRows(); final int n = matA.numCols(); final int lda = matA.numRows(); final String ta = transA ? "T" : "N"; NATIVE_BLAS.dgemv(ta, m, n, alpha, matA.getData(), lda, x.getData(), 1, beta, y.getData(), 1); } + + /** + * y := alpha * A * x + beta * y . + */ + public static void gemv(double alpha, DenseMatrix matA, boolean transA, + SparseVector x, double beta, DenseVector y) { + gemvDimensionCheck(matA, transA, x, y); + final int m = matA.numRows(); + final int n = matA.numCols(); + if (transA) { + int start = 0; + for (int i = 0; i < n; i++) { + double s = 0.; + for (int j = 0; j < x.indices.length; j++) { + s += x.values[j] * matA.data[start + x.indices[j]]; + } + y.data[i] = beta * y.data[i] + alpha * s; + start += m; + } + } else { + scal(beta, y); + for (int i = 0; i < x.indices.length; i++) { + int index = x.indices[i]; + double value = alpha * x.values[i]; + F2J_BLAS.daxpy(m, value, matA.data, index * m, 1, y.data, 0, 1); + } + } + } } diff --git a/flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/common/linalg/BLASTest.java b/flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/common/linalg/BLASTest.java new file mode 100644 index 000000000000..c30b0dfaee57 --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/common/linalg/BLASTest.java @@ -0,0 +1,188 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.linalg; + +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +/** + * The test cases for {@link BLAS}. + */ +public class BLASTest { + private static final double TOL = 1.0e-8; + private DenseMatrix mat = new DenseMatrix(2, 3, new double[]{1, 4, 2, 5, 3, 6}); + private DenseVector dv1 = new DenseVector(new double[]{1, 2}); + private DenseVector dv2 = new DenseVector(new double[]{1, 2, 3}); + private SparseVector spv1 = new SparseVector(2, new int[]{0, 1}, new double[]{1, 2}); + private SparseVector spv2 = new SparseVector(3, new int[]{0, 2}, new double[]{1, 3}); + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Test + public void testAsum() throws Exception { + Assert.assertEquals(BLAS.asum(dv1), 3.0, TOL); + Assert.assertEquals(BLAS.asum(spv1), 3.0, TOL); + } + + @Test + public void testScal() throws Exception { + DenseVector v1 = dv1.clone(); + BLAS.scal(0.5, v1); + Assert.assertArrayEquals(v1.getData(), new double[]{0.5, 1.0}, TOL); + + SparseVector v2 = spv1.clone(); + BLAS.scal(0.5, v2); + Assert.assertArrayEquals(v2.getIndices(), spv1.getIndices()); + Assert.assertArrayEquals(v2.getValues(), new double[]{0.5, 1.0}, TOL); + } + + @Test + public void testDot() throws Exception { + DenseVector v = DenseVector.ones(2); + Assert.assertEquals(BLAS.dot(dv1, v), 3.0, TOL); + } + + @Test + public void testAxpy() throws Exception { + DenseVector v = DenseVector.ones(2); + BLAS.axpy(1.0, dv1, v); + Assert.assertArrayEquals(v.getData(), new double[]{2, 3}, TOL); + BLAS.axpy(1.0, spv1, v); + Assert.assertArrayEquals(v.getData(), new double[]{3, 5}, TOL); + BLAS.axpy(1, 1.0, new double[]{1}, 0, v.getData(), 1); + Assert.assertArrayEquals(v.getData(), new double[]{3, 6}, TOL); + } + + private DenseMatrix simpleMM(DenseMatrix m1, DenseMatrix m2) { + DenseMatrix mm = new DenseMatrix(m1.numRows(), m2.numCols()); + for (int i = 0; i < m1.numRows(); i++) { + for (int j = 0; j < m2.numCols(); j++) { + double s = 0.; + for (int k = 0; k < m1.numCols(); k++) { + s += m1.get(i, k) * m2.get(k, j); + } + mm.set(i, j, s); + } + } + return mm; + } + + @Test + public void testGemm() throws Exception { + DenseMatrix m32 = DenseMatrix.rand(3, 2); + DenseMatrix m24 = DenseMatrix.rand(2, 4); + DenseMatrix m34 = DenseMatrix.rand(3, 4); + DenseMatrix m42 = DenseMatrix.rand(4, 2); + DenseMatrix m43 = DenseMatrix.rand(4, 3); + + DenseMatrix a34 = DenseMatrix.zeros(3, 4); + BLAS.gemm(1.0, m32, false, m24, false, 0., a34); + Assert.assertArrayEquals(a34.getData(), simpleMM(m32, m24).getData(), TOL); + + BLAS.gemm(1.0, m32, false, m42, true, 0., a34); + Assert.assertArrayEquals(a34.getData(), simpleMM(m32, m42.transpose()).getData(), TOL); + + DenseMatrix a24 = DenseMatrix.zeros(2, 4); + BLAS.gemm(1.0, m32, true, m34, false, 0., a24); + Assert.assertArrayEquals(a24.getData(), simpleMM(m32.transpose(), m34).getData(), TOL); + + BLAS.gemm(1.0, m32, true, m43, true, 0., a24); + Assert.assertArrayEquals(a24.getData(), simpleMM(m32.transpose(), m43.transpose()).getData(), TOL); + } + + @Test + public void testGemmSizeCheck() throws Exception { + thrown.expect(IllegalArgumentException.class); + DenseMatrix m32 = DenseMatrix.rand(3, 2); + DenseMatrix m42 = DenseMatrix.rand(4, 2); + DenseMatrix a34 = DenseMatrix.zeros(3, 4); + BLAS.gemm(1.0, m32, false, m42, false, 0., a34); + } + + @Test + public void testGemmTransposeSizeCheck() throws Exception { + thrown.expect(IllegalArgumentException.class); + DenseMatrix m32 = DenseMatrix.rand(3, 2); + DenseMatrix m42 = DenseMatrix.rand(4, 2); + DenseMatrix a34 = DenseMatrix.zeros(3, 4); + BLAS.gemm(1.0, m32, true, m42, true, 0., a34); + } + + @Test + public void testGemvDense() throws Exception { + DenseVector y1 = DenseVector.ones(2); + BLAS.gemv(2.0, mat, false, dv2, 0., y1); + Assert.assertArrayEquals(new double[]{28, 64}, y1.data, TOL); + + DenseVector y2 = DenseVector.ones(2); + BLAS.gemv(2.0, mat, false, dv2, 1., y2); + Assert.assertArrayEquals(new double[]{29, 65}, y2.data, TOL); + } + + @Test + public void testGemvDenseTranspose() throws Exception { + DenseVector y1 = DenseVector.ones(3); + BLAS.gemv(1.0, mat, true, dv1, 0., y1); + Assert.assertArrayEquals(new double[]{9, 12, 15}, y1.data, TOL); + + DenseVector y2 = DenseVector.ones(3); + BLAS.gemv(1.0, mat, true, dv1, 1., y2); + Assert.assertArrayEquals(new double[]{10, 13, 16}, y2.data, TOL); + } + + @Test + public void testGemvSparse() throws Exception { + DenseVector y1 = DenseVector.ones(2); + BLAS.gemv(2.0, mat, false, spv2, 0., y1); + Assert.assertArrayEquals(new double[]{20, 44}, y1.data, TOL); + + DenseVector y2 = DenseVector.ones(2); + BLAS.gemv(2.0, mat, false, spv2, 1., y2); + Assert.assertArrayEquals(new double[]{21, 45}, y2.data, TOL); + } + + @Test + public void testGemvSparseTranspose() throws Exception { + DenseVector y1 = DenseVector.ones(3); + BLAS.gemv(2.0, mat, true, spv1, 0., y1); + Assert.assertArrayEquals(new double[]{18, 24, 30}, y1.data, TOL); + + DenseVector y2 = DenseVector.ones(3); + BLAS.gemv(2.0, mat, true, spv1, 1., y2); + Assert.assertArrayEquals(new double[]{19, 25, 31}, y2.data, TOL); + } + + @Test + public void testGemvSizeCheck() throws Exception { + thrown.expect(IllegalArgumentException.class); + DenseVector y = DenseVector.ones(2); + BLAS.gemv(2.0, mat, false, dv1, 0., y); + } + + @Test + public void testGemvTransposeSizeCheck() throws Exception { + thrown.expect(IllegalArgumentException.class); + DenseVector y = DenseVector.ones(2); + BLAS.gemv(2.0, mat, true, dv1, 0., y); + } +} From 81e6493590925fb7d5ec6933bafbd94ad1ee8eb6 Mon Sep 17 00:00:00 2001 From: Wei Zhong Date: Fri, 25 Oct 2019 09:50:20 +0800 Subject: [PATCH 303/746] [FLINK-14478][python] Optimize current python test cases to reduce test time. This closes #9991. --- .../pyflink/table/tests/test_descriptor.py | 2 + .../table/tests/test_table_environment_api.py | 1 + flink-python/pyflink/table/tests/test_udf.py | 130 +++++++++--------- .../pyflink/testing/test_case_utils.py | 7 +- flink-python/tox.ini | 2 +- 5 files changed, 72 insertions(+), 70 deletions(-) diff --git a/flink-python/pyflink/table/tests/test_descriptor.py b/flink-python/pyflink/table/tests/test_descriptor.py index ddbcb011f80d..a3be6338247c 100644 --- a/flink-python/pyflink/table/tests/test_descriptor.py +++ b/flink-python/pyflink/table/tests/test_descriptor.py @@ -950,6 +950,7 @@ def test_with_schema(self): assert properties == expected def test_register_table_source_and_register_table_sink(self): + self.env.set_parallelism(1) source_path = os.path.join(self.tempdir + '/streaming.csv') field_names = ["a", "b", "c"] field_types = [DataTypes.INT(), DataTypes.STRING(), DataTypes.STRING()] @@ -996,6 +997,7 @@ def test_register_table_source_and_register_table_sink(self): assert lines == '2,Hi,Hello\n' + '3,Hello,Hello\n' def test_register_table_source_and_sink(self): + self.env.set_parallelism(1) source_path = os.path.join(self.tempdir + '/streaming.csv') field_names = ["a", "b", "c"] field_types = [DataTypes.INT(), DataTypes.STRING(), DataTypes.STRING()] diff --git a/flink-python/pyflink/table/tests/test_table_environment_api.py b/flink-python/pyflink/table/tests/test_table_environment_api.py index ff0ec0f06467..19f45f2bb83a 100644 --- a/flink-python/pyflink/table/tests/test_table_environment_api.py +++ b/flink-python/pyflink/table/tests/test_table_environment_api.py @@ -208,6 +208,7 @@ def test_create_table_environment_with_blink_planner(self): "org.apache.flink.table.planner.delegation.StreamPlanner") def test_table_environment_with_blink_planner(self): + self.env.set_parallelism(1) t_env = StreamTableEnvironment.create( self.env, environment_settings=EnvironmentSettings.new_instance().use_blink_planner().build()) diff --git a/flink-python/pyflink/table/tests/test_udf.py b/flink-python/pyflink/table/tests/test_udf.py index a433c904f4fa..73bfb922eedd 100644 --- a/flink-python/pyflink/table/tests/test_udf.py +++ b/flink-python/pyflink/table/tests/test_udf.py @@ -245,71 +245,6 @@ def test_open(self): 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())) - def test_udf_without_arguments(self): self.t_env.register_function("one", udf( lambda: 1, input_types=[], result_type=DataTypes.BIGINT(), deterministic=True)) @@ -468,7 +403,70 @@ class PyFlinkStreamUserDefinedFunctionTests(UserDefinedFunctionTests, class PyFlinkBlinkStreamUserDefinedFunctionTests(UserDefinedFunctionTests, PyFlinkBlinkStreamTableTestCase): - pass + 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())) class PyFlinkBlinkBatchUserDefinedFunctionTests(UserDefinedFunctionTests, diff --git a/flink-python/pyflink/testing/test_case_utils.py b/flink-python/pyflink/testing/test_case_utils.py index dc9da7735dce..edb65b0bfbd1 100644 --- a/flink-python/pyflink/testing/test_case_utils.py +++ b/flink-python/pyflink/testing/test_case_utils.py @@ -120,7 +120,7 @@ class PyFlinkStreamTableTestCase(PyFlinkTestCase): def setUp(self): super(PyFlinkStreamTableTestCase, self).setUp() self.env = StreamExecutionEnvironment.get_execution_environment() - self.env.set_parallelism(1) + self.env.set_parallelism(2) self.t_env = StreamTableEnvironment.create(self.env) @@ -132,7 +132,7 @@ class PyFlinkBatchTableTestCase(PyFlinkTestCase): def setUp(self): super(PyFlinkBatchTableTestCase, self).setUp() self.env = ExecutionEnvironment.get_execution_environment() - self.env.set_parallelism(1) + self.env.set_parallelism(2) self.t_env = BatchTableEnvironment.create(self.env) def collect(self, table): @@ -152,7 +152,7 @@ class PyFlinkBlinkStreamTableTestCase(PyFlinkTestCase): def setUp(self): super(PyFlinkBlinkStreamTableTestCase, self).setUp() self.env = StreamExecutionEnvironment.get_execution_environment() - + self.env.set_parallelism(2) self.t_env = StreamTableEnvironment.create( self.env, environment_settings=EnvironmentSettings.new_instance() .in_streaming_mode().use_blink_planner().build()) @@ -168,6 +168,7 @@ def setUp(self): self.t_env = BatchTableEnvironment.create( environment_settings=EnvironmentSettings.new_instance() .in_batch_mode().use_blink_planner().build()) + self.t_env._j_tenv.getPlanner().getExecEnv().setParallelism(2) class PythonAPICompletenessTestCase(object): diff --git a/flink-python/tox.ini b/flink-python/tox.ini index 381040cffa40..7b5c57185e51 100644 --- a/flink-python/tox.ini +++ b/flink-python/tox.ini @@ -30,7 +30,7 @@ deps = pytest commands = python --version - pytest + pytest --durations=0 bash ./dev/run_pip_test.sh [flake8] From 5da3d3e2cda8e6d984a729ec2d5bde0669f83a8b Mon Sep 17 00:00:00 2001 From: Rong Rong Date: Fri, 25 Oct 2019 19:11:24 -0700 Subject: [PATCH 304/746] [hotfix] fix java checkstyle exception --- .../src/main/java/org/apache/flink/ml/common/linalg/BLAS.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/linalg/BLAS.java b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/linalg/BLAS.java index 80b5d7fe3c5c..2c476f797aa8 100644 --- a/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/linalg/BLAS.java +++ b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/linalg/BLAS.java @@ -34,7 +34,7 @@ public class BLAS { /** * For level-2 and level-3 routines, we use the native BLAS. * - * The NATIVE_BLAS instance tries to load BLAS implementations in the order: + *

The NATIVE_BLAS instance tries to load BLAS implementations in the order: * 1) optimized system libraries such as Intel MKL, * 2) self-contained native builds using the reference Fortran from netlib.org, * 3) F2J implementation. From b1b4e00c158a9e3e7bd855e82075308c1279097b Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Wed, 23 Oct 2019 17:14:56 +0800 Subject: [PATCH 305/746] [FLINK-14040][travis] Enable NG scheduler testing in per-commit tests This closes #9783. --- .travis.yml | 8 ++++++++ tools/travis/stage.sh | 14 ++++++++++++++ 2 files changed, 22 insertions(+) diff --git a/.travis.yml b/.travis.yml index acf8b7e51aea..109533f34338 100644 --- a/.travis.yml +++ b/.travis.yml @@ -112,6 +112,14 @@ jobs: script: ./tools/travis_controller.sh tests env: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11" name: tests + - if: type in (pull_request, push) + script: ./tools/travis_controller.sh scheduler_ng_core + env: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11" + name: core - scheduler_ng + - if: type in (pull_request, push) + script: ./tools/travis_controller.sh scheduler_ng_tests + env: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11" + name: tests - scheduler_ng - if: type in (pull_request, push) script: ./tools/travis_controller.sh misc env: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11" diff --git a/tools/travis/stage.sh b/tools/travis/stage.sh index b0f4456d324c..62e43efb27e2 100644 --- a/tools/travis/stage.sh +++ b/tools/travis/stage.sh @@ -25,6 +25,8 @@ STAGE_BLINK_PLANNER="blink_planner" STAGE_CONNECTORS="connectors" STAGE_KAFKA_GELLY="kafka/gelly" STAGE_TESTS="tests" +STAGE_SCHEDULER_NG_CORE="scheduler_ng_core" +STAGE_SCHEDULER_NG_TESTS="scheduler_ng_tests" STAGE_MISC="misc" STAGE_CLEANUP="cleanup" @@ -158,6 +160,12 @@ function get_compile_modules_for_stage() { (${STAGE_TESTS}) echo "-pl $MODULES_TESTS -am" ;; + (${STAGE_SCHEDULER_NG_CORE}) + echo "-pl $MODULES_CORE -am" + ;; + (${STAGE_SCHEDULER_NG_TESTS}) + echo "-pl $MODULES_TESTS -am" + ;; (${STAGE_MISC}) # compile everything; using the -am switch does not work with negated module lists! # the negation takes precedence, thus not all required modules would be built @@ -201,6 +209,12 @@ function get_test_modules_for_stage() { (${STAGE_TESTS}) echo "-pl $modules_tests" ;; + (${STAGE_SCHEDULER_NG_CORE}) + echo "-Dscheduler-ng -pl $MODULES_CORE" + ;; + (${STAGE_SCHEDULER_NG_TESTS}) + echo "-Dscheduler-ng -pl $MODULES_TESTS" + ;; (${STAGE_MISC}) echo "-pl $modules_misc" ;; From f61d2231a89dd370c7197b566cf279bd6b640d65 Mon Sep 17 00:00:00 2001 From: Henvealf <1103693087@qq.com> Date: Sun, 29 Sep 2019 19:55:58 +0800 Subject: [PATCH 306/746] [FLINK-14227][docs-zh] Translate "Checkpointing" page to Chinese This closes #9805 --- docs/dev/stream/state/checkpointing.zh.md | 152 ++++++++++------------ 1 file changed, 70 insertions(+), 82 deletions(-) diff --git a/docs/dev/stream/state/checkpointing.zh.md b/docs/dev/stream/state/checkpointing.zh.md index c193fc37d71e..d4aa989523a6 100644 --- a/docs/dev/stream/state/checkpointing.zh.md +++ b/docs/dev/stream/state/checkpointing.zh.md @@ -25,85 +25,77 @@ under the License. * ToC {:toc} -Every function and operator in Flink can be **stateful** (see [working with state](state.html) for details). -Stateful functions store data across the processing of individual elements/events, making state a critical building block for -any type of more elaborate operation. +Flink 中的每个方法或算子都能够是**有状态的**(阅读 [working with state](state.html) 了解更多)。 +状态化的方法在处理单个 元素/事件 的时候存储数据,让状态成为使各个类型的算子更加精细的重要部分。 +为了让状态容错,Flink 需要为状态添加 **checkpoint(检查点)**。Checkpoint 使得 Flink 能够恢复状态和在流中的位置,从而向应用提供和无故障执行时一样的语义。 -In order to make state fault tolerant, Flink needs to **checkpoint** the state. Checkpoints allow Flink to recover state and positions -in the streams to give the application the same semantics as a failure-free execution. +[容错文档]({{ site.baseurl }}/zh/internals/stream_checkpointing.html) 中介绍了 Flink 流计算容错机制内部的技术原理。 -The [documentation on streaming fault tolerance]({{ site.baseurl }}/internals/stream_checkpointing.html) describes in detail the technique behind Flink's streaming fault tolerance mechanism. +## 前提条件 -## Prerequisites +Flink 的 checkpoint 机制会和持久化存储进行交互,读写流与状态。一般需要: -Flink's checkpointing mechanism interacts with durable storage for streams and state. In general, it requires: + - 一个能够回放一段时间内数据的持久化数据源,例如持久化消息队列(例如 Apache Kafka、RabbitMQ、 Amazon Kinesis、 Google PubSub 等)或文件系统(例如 HDFS、 S3、 GFS、 NFS、 Ceph 等)。 + - 存放状态的持久化存储,通常为分布式文件系统(比如 HDFS、 S3、 GFS、 NFS、 Ceph 等)。 - - A *persistent* (or *durable*) data source that can replay records for a certain amount of time. Examples for such sources are persistent messages queues (e.g., Apache Kafka, RabbitMQ, Amazon Kinesis, Google PubSub) or file systems (e.g., HDFS, S3, GFS, NFS, Ceph, ...). - - A persistent storage for state, typically a distributed filesystem (e.g., HDFS, S3, GFS, NFS, Ceph, ...) +## 开启与配置 Checkpoint +默认情况下 checkpoint 是禁用的。通过调用 `StreamExecutionEnvironment` 的 `enableCheckpointing(n)` 来启用 checkpoint,里面的 *n* 是进行 checkpoint 的间隔,单位毫秒。 -## Enabling and Configuring Checkpointing +Checkpoint 其他的属性包括: -By default, checkpointing is disabled. To enable checkpointing, call `enableCheckpointing(n)` on the `StreamExecutionEnvironment`, where *n* is the checkpoint interval in milliseconds. - -Other parameters for checkpointing include: - - - *exactly-once vs. at-least-once*: You can optionally pass a mode to the `enableCheckpointing(n)` method to choose between the two guarantee levels. - Exactly-once is preferable for most applications. At-least-once may be relevant for certain super-low-latency (consistently few milliseconds) applications. - - - *checkpoint timeout*: The time after which a checkpoint-in-progress is aborted, if it did not complete by then. - - - *minimum time between checkpoints*: To make sure that the streaming application makes a certain amount of progress between checkpoints, - one can define how much time needs to pass between checkpoints. If this value is set for example to *5000*, the next checkpoint will be - started no sooner than 5 seconds after the previous checkpoint completed, regardless of the checkpoint duration and the checkpoint interval. - Note that this implies that the checkpoint interval will never be smaller than this parameter. + - *精确一次(exactly-once)对比至少一次(at-least-once)*:你可以选择向 `enableCheckpointing(long interval, CheckpointingMode mode)` 方法中传入一个模式来选择使用两种保证等级中的哪一种。 + 对于大多数应用来说,精确一次是较好的选择。至少一次可能与某些延迟超低(始终只有几毫秒)的应用的关联较大。 + + - *checkpoint 超时*:如果 checkpoint 执行的时间超过了该配置的阈值,还在进行中的 checkpoint 操作就会被抛弃。 + + - *checkpoints 之间的最小时间*:该属性定义在 checkpoint 之间需要多久的时间,以确保流应用在 checkpoint 之间有足够的进展。如果值设置为了 *5000*, + 无论 checkpoint 持续时间与间隔是多久,在前一个 checkpoint 完成时的至少五秒后会才开始下一个 checkpoint。 - It is often easier to configure applications by defining the "time between checkpoints" than the checkpoint interval, because the "time between checkpoints" - is not susceptible to the fact that checkpoints may sometimes take longer than on average (for example if the target storage system is temporarily slow). - - Note that this value also implies that the number of concurrent checkpoints is *one*. - - - *number of concurrent checkpoints*: By default, the system will not trigger another checkpoint while one is still in progress. - This ensures that the topology does not spend too much time on checkpoints and not make progress with processing the streams. - It is possible to allow for multiple overlapping checkpoints, which is interesting for pipelines that have a certain processing delay - (for example because the functions call external services that need some time to respond) but that still want to do very frequent checkpoints - (100s of milliseconds) to re-process very little upon failures. - - This option cannot be used when a minimum time between checkpoints is defined. - - - *externalized checkpoints*: You can configure periodic checkpoints to be persisted externally. Externalized checkpoints write their meta data out to persistent storage and are *not* automatically cleaned up when the job fails. This way, you will have a checkpoint around to resume from if your job fails. There are more details in the [deployment notes on externalized checkpoints]({{ site.baseurl }}/ops/state/checkpoints.html#externalized-checkpoints). - - - *fail/continue task on checkpoint errors*: This determines if a task will be failed if an error occurs in the execution of the task's checkpoint procedure. This is the default behaviour. Alternatively, when this is disabled, the task will simply decline the checkpoint to the checkpoint coordinator and continue running. + 往往使用“checkpoints 之间的最小时间”来配置应用会比 checkpoint 间隔容易很多,因为“checkpoints 之间的最小时间”在 checkpoint 的执行时间超过平均值时不会受到影响(例如如果目标的存储系统忽然变得很慢)。 + + 注意这个值也意味着并发 checkpoint 的数目是*一*。 - - *prefer checkpoint for recovery*: This determines if a job will fallback to latest checkpoint even when there are more recent savepoints available to potentially reduce recovery time. + - *并发 checkpoint 的数目*: 默认情况下,在上一个 checkpoint 未完成(失败或者成功)的情况下,系统不会触发另一个 checkpoint。这确保了拓扑不会在 checkpoint 上花费太多时间,从而影响正常的处理流程。 + 不过允许多个 checkpoint 并行进行是可行的,对于有确定的处理延迟(例如某方法所调用比较耗时的外部服务),但是仍然想进行频繁的 checkpoint 去最小化故障后重跑的 pipelines 来说,是有意义的。 + + 该选项不能和 "checkpoints 间的最小时间"同时使用。 + + - *externalized checkpoints*: 你可以配置周期存储 checkpoint 到外部系统中。Externalized checkpoints 将他们的元数据写到持久化存储上并且在 job 失败的时候*不会*被自动删除。 + 这种方式下,如果你的 job 失败,你将会有一个现有的 checkpoint 去恢复。更多的细节请看 [Externalized checkpoints 的部署文档]({{ site.baseurl }}/zh/ops/state/checkpoints.html#externalized-checkpoints)。 + + - *在 checkpoint 出错时使 task 失败或者继续进行 task*:他决定了在 task checkpoint 的过程中发生错误时,是否使 task 也失败,使失败是默认的行为。 + 或者禁用它时,这个任务将会简单的把 checkpoint 错误信息报告给 checkpoint coordinator 并继续运行。 + + - *优先从 checkpoint 恢复(prefer checkpoint for recovery)*:该属性确定 job 是否在最新的 checkpoint 回退,即使有更近的 savepoint 可用,这可以潜在地减少恢复时间(checkpoint 恢复比 savepoint 恢复更快)。

{% highlight java %} StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); -// start a checkpoint every 1000 ms +// 每 1000ms 开始一次 checkpoint env.enableCheckpointing(1000); -// advanced options: +// 高级选项: -// set mode to exactly-once (this is the default) +// 设置模式为精确一次 (这是默认值) env.getCheckpointConfig().setCheckpointingMode(CheckpointingMode.EXACTLY_ONCE); -// make sure 500 ms of progress happen between checkpoints +// 确认 checkpoints 之间的时间会进行 500 ms env.getCheckpointConfig().setMinPauseBetweenCheckpoints(500); -// checkpoints have to complete within one minute, or are discarded +// Checkpoint 必须在一分钟内完成,否则就会被抛弃 env.getCheckpointConfig().setCheckpointTimeout(60000); -// allow only one checkpoint to be in progress at the same time +// 同一时间只允许一个 checkpoint 进行 env.getCheckpointConfig().setMaxConcurrentCheckpoints(1); -// enable externalized checkpoints which are retained after job cancellation +// 开启在 job 中止后仍然保留的 externalized checkpoints env.getCheckpointConfig().enableExternalizedCheckpoints(ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION); -// allow job recovery fallback to checkpoint when there is a more recent savepoint +// 允许在有更近 savepoint 时回退到 checkpoint env.getCheckpointConfig().setPreferCheckpointForRecovery(true); {% endhighlight %}
@@ -111,24 +103,24 @@ env.getCheckpointConfig().setPreferCheckpointForRecovery(true); {% highlight scala %} val env = StreamExecutionEnvironment.getExecutionEnvironment() -// start a checkpoint every 1000 ms +// 每 1000ms 开始一次 checkpoint env.enableCheckpointing(1000) -// advanced options: +// 高级选项: -// set mode to exactly-once (this is the default) +// 设置模式为精确一次 (这是默认值) env.getCheckpointConfig.setCheckpointingMode(CheckpointingMode.EXACTLY_ONCE) -// make sure 500 ms of progress happen between checkpoints +// 确认 checkpoints 之间的时间会进行 500 ms env.getCheckpointConfig.setMinPauseBetweenCheckpoints(500) -// checkpoints have to complete within one minute, or are discarded +// Checkpoint 必须在一分钟内完成,否则就会被抛弃 env.getCheckpointConfig.setCheckpointTimeout(60000) -// prevent the tasks from failing if an error happens in their checkpointing, the checkpoint will just be declined. +// 如果 task 的 checkpoint 发生错误,会阻止 task 失败,checkpoint 仅仅会被抛弃 env.getCheckpointConfig.setFailTasksOnCheckpointingErrors(false) -// allow only one checkpoint to be in progress at the same time +// 同一时间只允许一个 checkpoint 进行 env.getCheckpointConfig.setMaxConcurrentCheckpoints(1) {% endhighlight %}
@@ -136,67 +128,63 @@ env.getCheckpointConfig.setMaxConcurrentCheckpoints(1) {% highlight python %} env = StreamExecutionEnvironment.get_execution_environment() -# start a checkpoint every 1000 ms +# 每 1000ms 开始一次 checkpoint env.enable_checkpointing(1000) -# advanced options: +# 高级选项: -# set mode to exactly-once (this is the default) +# 设置模式为精确一次 (这是默认值) env.get_checkpoint_config().set_checkpointing_mode(CheckpointingMode.EXACTLY_ONCE) -# make sure 500 ms of progress happen between checkpoints +# 确认 checkpoints 之间的时间会进行 500 ms env.get_checkpoint_config().set_min_pause_between_checkpoints(500) -# checkpoints have to complete within one minute, or are discarded +# Checkpoint 必须在一分钟内完成,否则就会被抛弃 env.get_checkpoint_config().set_checkpoint_timeout(60000) -# allow only one checkpoint to be in progress at the same time +# 同一时间只允许一个 checkpoint 进行 env.get_checkpoint_config().set_max_concurrent_checkpoints(1) -# enable externalized checkpoints which are retained after job cancellation +# 开启在 job 中止后仍然保留的 externalized checkpoints env.get_checkpoint_config().enable_externalized_checkpoints(ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION) -# allow job recovery fallback to checkpoint when there is a more recent savepoint +# 允许在有更近 savepoint 时回退到 checkpoint env.get_checkpoint_config().set_prefer_checkpoint_for_recovery(True) {% endhighlight %} -### Related Config Options +### 相关的配置选项 -Some more parameters and/or defaults may be set via `conf/flink-conf.yaml` (see [configuration]({{ site.baseurl }}/ops/config.html) for a full guide): +更多的属性与默认值能在 `conf/flink-conf.yaml` 中设置(完整教程请阅读 [配置]({{ site.baseurl }}/zh/ops/config.html))。 {% include generated/checkpointing_configuration.html %} {% top %} -## Selecting a State Backend - -Flink's [checkpointing mechanism]({{ site.baseurl }}/internals/stream_checkpointing.html) stores consistent snapshots -of all the state in timers and stateful operators, including connectors, windows, and any [user-defined state](state.html). -Where the checkpoints are stored (e.g., JobManager memory, file system, database) depends on the configured -**State Backend**. +## 选择一个 State Backend -By default, state is kept in memory in the TaskManagers and checkpoints are stored in memory in the JobManager. For proper persistence of large state, -Flink supports various approaches for storing and checkpointing state in other state backends. The choice of state backend can be configured via `StreamExecutionEnvironment.setStateBackend(…)`. +Flink 的 [checkpointing 机制]({{ site.baseurl }}/zh/internals/stream_checkpointing.html) 会将 timer 以及 stateful 的 operator 进行快照,然后存储下来, +包括连接器(connectors),窗口(windows)以及任何用户[自定义的状态](state.html)。 +Checkpoint 存储在哪里取决于所配置的 **State Backend**(比如 JobManager memory、 file system、 database)。 -See [state backends]({{ site.baseurl }}/ops/state/state_backends.html) for more details on the available state backends and options for job-wide and cluster-wide configuration. +默认情况下,状态是保持在 TaskManagers 的内存中,checkpoint 保存在 JobManager 的内存中。为了合适地持久化大体量状态, +Flink 支持各种各样的途径去存储 checkpoint 状态到其他的 state backends 上。通过 `StreamExecutionEnvironment.setStateBackend(…)` 来配置所选的 state backends。 +阅读 [state backends]({{ site.baseurl }}/zh/ops/state/state_backends.html) 来查看在 job 范围和集群范围上可用的 state backends 与选项的更多细节。 -## State Checkpoints in Iterative Jobs +## 迭代作业中的状态和 checkpoint -Flink currently only provides processing guarantees for jobs without iterations. Enabling checkpointing on an iterative job causes an exception. In order to force checkpointing on an iterative program the user needs to set a special flag when enabling checkpointing: `env.enableCheckpointing(interval, CheckpointingMode.EXACTLY_ONCE, force = true)`. +Flink 现在为没有迭代(iterations)的作业提供一致性的处理保证。在迭代作业上开启 checkpoint 会导致异常。为了在迭代程序中强制进行 checkpoint,用户需要在开启 checkpoint 时设置一个特殊的标志: `env.enableCheckpointing(interval, CheckpointingMode.EXACTLY_ONCE, force = true)`。 -Please note that records in flight in the loop edges (and the state changes associated with them) will be lost during failure. +请注意在环形边上游走的记录(以及与之相关的状态变化)在故障时会丢失。 {% top %} +## 重启策略 -## Restart Strategies - -Flink supports different restart strategies which control how the jobs are restarted in case of a failure. For more -information, see [Restart Strategies]({{ site.baseurl }}/dev/restart_strategies.html). +Flink 支持不同的重启策略,来控制 job 万一故障时该如何重启。更多信息请阅读 [重启策略]({{ site.baseurl }}/zh/dev/restart_strategies.html)。 {% top %} From a3852546de7c03187e15298b591779a6d919a0dc Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Tue, 15 Oct 2019 18:26:59 +0800 Subject: [PATCH 307/746] [hotfix][runtime] Implement AvailabilityHelper for providing resetAvailable/Unavailble functions for InputGate In existing abstract InputGate it covers some logics of switching between available and unavailable states, which could be extracted as a separate helpful implementation to be reused by other components future. --- .../runtime/io/AvailabilityProvider.java | 44 +++++++++++++++++++ .../network/partition/consumer/InputGate.java | 11 +---- .../partition/consumer/SingleInputGate.java | 14 +++--- .../partition/consumer/UnionInputGate.java | 16 +++---- ...kpointBarrierAlignerMassiveRandomTest.java | 2 +- .../streaming/runtime/io/MockInputGate.java | 4 +- 6 files changed, 62 insertions(+), 29 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 42e09366de7e..09d8ed1ac2d1 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 @@ -53,4 +53,48 @@ public interface AvailabilityProvider { * the input is finished. */ CompletableFuture isAvailable(); + + /** + * A availability implementation for providing the helpful functions of resetting the + * available/unavailable states. + */ + final class AvailabilityHelper implements AvailabilityProvider { + + private CompletableFuture isAvailable = new CompletableFuture<>(); + + /** + * Judges to reset the current available state as unavailable. + */ + public void resetUnavailable() { + // try to avoid volatile access in isDone()} + if (isAvailable == AVAILABLE || isAvailable.isDone()) { + isAvailable = new CompletableFuture<>(); + } + } + + /** + * Resets the constant completed {@link #AVAILABLE} as the current state. + */ + public void resetAvailable() { + isAvailable = AVAILABLE; + } + + /** + * Returns the previously not completed future and resets the constant completed + * {@link #AVAILABLE} as the current state. + */ + public CompletableFuture getUnavailableToResetAvailable() { + CompletableFuture toNotify = isAvailable; + isAvailable = AVAILABLE; + return toNotify; + } + + /** + * @return a future that is completed if the respective provider is available. + */ + @Override + public CompletableFuture isAvailable() { + return isAvailable; + } + } } 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 6a9ac1744f80..070c848cf12f 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 @@ -71,7 +71,7 @@ */ public abstract class InputGate implements PullingAsyncDataInput, AutoCloseable { - protected CompletableFuture isAvailable = new CompletableFuture<>(); + protected final AvailabilityHelper availabilityHelper = new AvailabilityHelper(); public abstract int getNumberOfInputChannels(); @@ -100,14 +100,7 @@ public abstract class InputGate implements PullingAsyncDataInput, */ @Override public CompletableFuture isAvailable() { - return isAvailable; - } - - protected void resetIsAvailable() { - // try to avoid volatile access in isDone()} - if (isAvailable == AVAILABLE || isAvailable.isDone()) { - isAvailable = new CompletableFuture<>(); - } + return availabilityHelper.isAvailable(); } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java index e1366d0e9cd9..9bd0b96b5920 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java @@ -516,7 +516,7 @@ private Optional> waitAndGetN } if (inputChannelsWithData.isEmpty()) { - resetIsAvailable(); + availabilityHelper.resetUnavailable(); } if (result.isPresent()) { @@ -567,12 +567,11 @@ private BufferOrEvent transformToBufferOrEvent( } private void markAvailable() { - CompletableFuture toNotfiy; + CompletableFuture toNotify; synchronized (inputChannelsWithData) { - toNotfiy = isAvailable; - isAvailable = AVAILABLE; + toNotify = availabilityHelper.getUnavailableToResetAvailable(); } - toNotfiy.complete(null); + toNotify.complete(null); } @Override @@ -629,8 +628,7 @@ private void queueChannel(InputChannel channel) { if (availableChannels == 0) { inputChannelsWithData.notifyAll(); - toNotify = isAvailable; - isAvailable = AVAILABLE; + toNotify = availabilityHelper.getUnavailableToResetAvailable(); } } @@ -650,7 +648,7 @@ private Optional getChannel(boolean blocking) throws InterruptedEx inputChannelsWithData.wait(); } else { - resetIsAvailable(); + availabilityHelper.resetUnavailable(); return Optional.empty(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java index c61204463f95..9a023eaeb5a4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java @@ -118,7 +118,7 @@ public UnionInputGate(InputGate... inputGates) { } if (!inputGatesWithData.isEmpty()) { - isAvailable = AVAILABLE; + availabilityHelper.resetAvailable(); } } @@ -189,7 +189,7 @@ private Optional> waitAndGetNextData(boo } if (inputGatesWithData.isEmpty()) { - resetIsAvailable(); + availabilityHelper.resetUnavailable(); } if (bufferOrEvent.isPresent()) { @@ -232,12 +232,11 @@ private void handleEndOfPartitionEvent(BufferOrEvent bufferOrEvent, InputGate in } private void markAvailable() { - CompletableFuture toNotfiy; + CompletableFuture toNotify; synchronized (inputGatesWithData) { - toNotfiy = isAvailable; - isAvailable = AVAILABLE; + toNotify = availabilityHelper.getUnavailableToResetAvailable(); } - toNotfiy.complete(null); + toNotify.complete(null); } @Override @@ -271,8 +270,7 @@ private void queueInputGate(InputGate inputGate) { if (availableInputGates == 0) { inputGatesWithData.notifyAll(); - toNotify = isAvailable; - isAvailable = AVAILABLE; + toNotify = availabilityHelper.getUnavailableToResetAvailable(); } } @@ -287,7 +285,7 @@ private Optional getInputGate(boolean blocking) throws InterruptedExc if (blocking) { inputGatesWithData.wait(); } else { - resetIsAvailable(); + availabilityHelper.resetUnavailable(); return Optional.empty(); } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierAlignerMassiveRandomTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierAlignerMassiveRandomTest.java index 182afef82072..fbdabb9e5f17 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierAlignerMassiveRandomTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/CheckpointBarrierAlignerMassiveRandomTest.java @@ -138,7 +138,7 @@ public RandomGeneratingInputGate(BufferPool[] bufferPools, BarrierGenerator[] ba this.currentBarriers = new int[numberOfChannels]; this.bufferPools = bufferPools; this.barrierGens = barrierGens; - this.isAvailable = AVAILABLE; + availabilityHelper.resetAvailable(); } @Override diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/MockInputGate.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/MockInputGate.java index 7cc71945cfe9..ec8308601b87 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/MockInputGate.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/MockInputGate.java @@ -54,7 +54,7 @@ public MockInputGate( this.closed = new boolean[numberOfChannels]; this.finishAfterLastBuffer = finishAfterLastBuffer; - isAvailable = AVAILABLE; + availabilityHelper.resetAvailable(); } @Override @@ -75,7 +75,7 @@ public boolean isFinished() { public Optional getNext() { BufferOrEvent next = bufferOrEvents.poll(); if (!finishAfterLastBuffer && bufferOrEvents.isEmpty()) { - resetIsAvailable(); + availabilityHelper.resetUnavailable(); } if (next == null) { return Optional.empty(); From ad70e062c1d4a867b7eec7a03239c5a6ce52c23e Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Tue, 22 Oct 2019 12:07:20 +0800 Subject: [PATCH 308/746] [FLINK-14396][network] Implement rudimentary non-blocking network output MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Considering the mailbox model and unaligned checkpoints requirements in future, task network output should be non-blocking. In other words, as long as output is available, it should never block for a subsequent/future single record write. In the first version, we only implement the non-blocking output for the most regular case, and do not solve the following cases which still keep the previous behavior. 1. Big record which might span multiple buffers 2. Flatmap-like operators which might emit multiple records in every process 3. Broadcast watermark which might request multiple buffers at a time The solution is to provide the RecordWriter#isAvailable method and respective LocalBufferPool#isAvailable for judging the output beforehand. As long as there is at-least one available buffer in LocalBufferPool, the RecordWriter is available for network output in most cases. This PR doesn’t include runtime handling of this non-blocking and availability behavior in StreamInputProcessor. Note: It requires the minimum number of buffers in output LocalBufferPool adjusting to (numberOfSubpartitions + 1) and also adjusting the monitor of backpressure future. --- .../io/network/api/writer/RecordWriter.java | 9 +- .../api/writer/ResultPartitionWriter.java | 9 ++ .../io/network/buffer/BufferProvider.java | 4 +- .../io/network/buffer/LocalBufferPool.java | 83 ++++++++++++++----- .../io/network/partition/ResultPartition.java | 6 ++ ...tifyingResultPartitionWriterDecorator.java | 6 ++ ...stractCollectingResultPartitionWriter.java | 7 ++ .../network/api/writer/RecordWriterTest.java | 62 ++++++++++++++ .../io/network/buffer/NoOpBufferPool.java | 6 ++ .../util/TestPooledBufferProvider.java | 6 ++ 10 files changed, 176 insertions(+), 22 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 93fa82630546..9264f0fd24df 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 @@ -23,6 +23,7 @@ import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.SimpleCounter; import org.apache.flink.runtime.event.AbstractEvent; +import org.apache.flink.runtime.io.AvailabilityProvider; 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; @@ -38,6 +39,7 @@ import java.io.IOException; import java.util.Random; +import java.util.concurrent.CompletableFuture; import static org.apache.flink.runtime.io.network.api.serialization.RecordSerializer.SerializationResult; import static org.apache.flink.util.Preconditions.checkArgument; @@ -56,7 +58,7 @@ * * @param the type of the record that can be emitted with this record writer */ -public abstract class RecordWriter { +public abstract class RecordWriter implements AvailabilityProvider { /** Default name for the output flush thread, if no name with a task reference is given. */ @VisibleForTesting @@ -187,6 +189,11 @@ protected void finishBufferBuilder(BufferBuilder bufferBuilder) { numBuffersOut.inc(); } + @Override + public CompletableFuture isAvailable() { + return targetPartition.isAvailable(); + } + /** * This is used to send regular records. */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java index 1945bb0a7d80..cedf4de3487c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java @@ -25,6 +25,7 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.util.concurrent.CompletableFuture; /** * A buffer-oriented runtime result writer API for producing results. @@ -92,4 +93,12 @@ public interface ResultPartitionWriter extends AutoCloseable { *

Closing of partition is still needed afterwards. */ void finish() throws IOException; + + /** + * Check whether the writer is available for output or not. + * + * @return a future that is completed if it is available for output. + */ + + CompletableFuture isAvailable(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferProvider.java index 1217b08b01d5..5e17b3087fab 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferProvider.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferProvider.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.io.network.buffer; +import org.apache.flink.runtime.io.AvailabilityProvider; + import java.io.IOException; /** @@ -26,7 +28,7 @@ *

The data producing side (result partition writers) request buffers in a synchronous fashion, * whereas the input side requests asynchronously. */ -public interface BufferProvider { +public interface BufferProvider extends AvailabilityProvider { /** * Returns a {@link Buffer} instance from the buffer provider, if one is available. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java index afa76a5bcecc..56055dbb52fe 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java @@ -29,6 +29,7 @@ import java.io.IOException; import java.util.ArrayDeque; +import java.util.concurrent.CompletableFuture; import static org.apache.flink.util.Preconditions.checkArgument; @@ -90,6 +91,8 @@ class LocalBufferPool implements BufferPool { @Nullable private final BufferPoolOwner bufferPoolOwner; + private final AvailabilityHelper availabilityHelper = new AvailabilityHelper(); + /** * Local buffer pool based on the given networkBufferPool with a minimal number of * network buffers being available. @@ -153,6 +156,8 @@ class LocalBufferPool implements BufferPool { this.currentPoolSize = numberOfRequiredMemorySegments; this.maxNumberOfMemorySegments = maxNumberOfMemorySegments; this.bufferPoolOwner = bufferPoolOwner; + + availabilityHelper.resetAvailable(); } // ------------------------------------------------------------------------ @@ -224,44 +229,60 @@ private BufferBuilder toBufferBuilder(MemorySegment memorySegment) { return new BufferBuilder(memorySegment, this); } + @Nullable private MemorySegment requestMemorySegment(boolean isBlocking) throws InterruptedException, IOException { + MemorySegment segment = null; synchronized (availableMemorySegments) { returnExcessMemorySegments(); // fill availableMemorySegments with at least one element, wait if required while (availableMemorySegments.isEmpty()) { - if (isDestroyed) { - throw new IllegalStateException("Buffer pool is destroyed."); + segment = requestMemorySegmentFromGlobal(); + if (segment != null || !isBlocking) { + break; } - if (numberOfRequestedMemorySegments < currentPoolSize) { - final MemorySegment segment = networkBufferPool.requestMemorySegment(); + availableMemorySegments.wait(2000); + } - if (segment != null) { - numberOfRequestedMemorySegments++; - return segment; - } - } + if (segment == null) { + segment = availableMemorySegments.poll(); + } - if (bufferPoolOwner != null) { - bufferPoolOwner.releaseMemory(1); - } + if (isBlocking && isUnavailable()) { + availabilityHelper.resetUnavailable(); + } + } + return segment; + } - if (isBlocking) { - availableMemorySegments.wait(2000); - } - else { - return null; - } + @Nullable + private MemorySegment requestMemorySegmentFromGlobal() throws IOException { + assert Thread.holdsLock(availableMemorySegments); + + if (isDestroyed) { + throw new IllegalStateException("Buffer pool is destroyed."); + } + + if (numberOfRequestedMemorySegments < currentPoolSize) { + final MemorySegment segment = networkBufferPool.requestMemorySegment(); + if (segment != null) { + numberOfRequestedMemorySegments++; + return segment; } + } - return availableMemorySegments.poll(); + if (bufferPoolOwner != null) { + bufferPoolOwner.releaseMemory(1); } + + return null; } @Override public void recycle(MemorySegment segment) { BufferListener listener; + CompletableFuture toNotify = null; NotificationResult notificationResult = NotificationResult.BUFFER_NOT_USED; while (!notificationResult.isBufferUsed()) { synchronized (availableMemorySegments) { @@ -271,14 +292,22 @@ public void recycle(MemorySegment segment) { } else { listener = registeredListeners.poll(); if (listener == null) { + boolean wasUnavailable = isUnavailable(); availableMemorySegments.add(segment); availableMemorySegments.notify(); - return; + if (wasUnavailable) { + toNotify = availabilityHelper.getUnavailableToResetAvailable(); + } + break; } } } notificationResult = fireBufferAvailableNotification(listener, segment); } + + if (toNotify != null) { + toNotify.complete(null); + } } private NotificationResult fireBufferAvailableNotification(BufferListener listener, MemorySegment segment) { @@ -299,6 +328,15 @@ private NotificationResult fireBufferAvailableNotification(BufferListener listen return notificationResult; } + /** + * @return true if there is no available buffers in queue and the global quota is also exhausted. + */ + private boolean isUnavailable() { + assert Thread.holdsLock(availableMemorySegments); + + return availableMemorySegments.isEmpty() && numberOfRequestedMemorySegments >= currentPoolSize; + } + /** * Destroy is called after the produce or consume phase of a task finishes. */ @@ -366,6 +404,11 @@ public void setNumBuffers(int numBuffers) throws IOException { } } + @Override + public CompletableFuture isAvailable() { + return availabilityHelper.isAvailable(); + } + @Override public String toString() { synchronized (availableMemorySegments) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java index bfbdd5871d06..26238b93dc16 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java @@ -37,6 +37,7 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.flink.util.Preconditions.checkArgument; @@ -322,6 +323,11 @@ public boolean isReleased() { return isReleased.get(); } + @Override + public CompletableFuture isAvailable() { + return bufferPool.isAvailable(); + } + @Override public String toString() { return "ResultPartition " + partitionId.toString() + " [" + partitionType + ", " diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ConsumableNotifyingResultPartitionWriterDecorator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ConsumableNotifyingResultPartitionWriterDecorator.java index d9add0689112..d606a10d0ef4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ConsumableNotifyingResultPartitionWriterDecorator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ConsumableNotifyingResultPartitionWriterDecorator.java @@ -28,6 +28,7 @@ import java.io.IOException; import java.util.Collection; +import java.util.concurrent.CompletableFuture; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -119,6 +120,11 @@ public void fail(Throwable throwable) { partitionWriter.fail(throwable); } + @Override + public CompletableFuture isAvailable() { + return partitionWriter.isAvailable(); + } + @Override public void close() throws Exception { partitionWriter.close(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/AbstractCollectingResultPartitionWriter.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/AbstractCollectingResultPartitionWriter.java index 035fa565b1e9..f4e6f95bf731 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/AbstractCollectingResultPartitionWriter.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/AbstractCollectingResultPartitionWriter.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.io.network.api.writer; +import org.apache.flink.runtime.io.AvailabilityProvider; 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.BufferConsumer; @@ -29,6 +30,7 @@ import java.io.IOException; import java.util.ArrayDeque; +import java.util.concurrent.CompletableFuture; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkState; @@ -122,5 +124,10 @@ public void finish() { throw new UnsupportedOperationException(); } + @Override + public CompletableFuture isAvailable() { + return AvailabilityProvider.AVAILABLE; + } + protected abstract void deserializeBuffer(Buffer buffer) throws IOException; } 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 239c47d3a5cf..c7cd1bcb672e 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 @@ -18,6 +18,7 @@ package org.apache.flink.runtime.io.network.api.writer; +import org.apache.flink.api.common.JobID; import org.apache.flink.core.io.IOReadableWritable; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -33,15 +34,22 @@ 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.buffer.BufferPool; import org.apache.flink.runtime.io.network.buffer.BufferProvider; import org.apache.flink.runtime.io.network.buffer.BufferRecycler; +import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; +import org.apache.flink.runtime.io.network.partition.NoOpResultPartitionConsumableNotifier; +import org.apache.flink.runtime.io.network.partition.ResultPartitionBuilder; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.io.network.util.DeserializationUtils; import org.apache.flink.runtime.io.network.util.TestPooledBufferProvider; import org.apache.flink.runtime.operators.shipping.OutputEmitter; import org.apache.flink.runtime.operators.shipping.ShipStrategyType; +import org.apache.flink.runtime.taskmanager.ConsumableNotifyingResultPartitionWriterDecorator; +import org.apache.flink.runtime.taskmanager.NoOpTaskActions; import org.apache.flink.testutils.serialization.types.SerializationTestType; import org.apache.flink.testutils.serialization.types.SerializationTestTypeFactory; import org.apache.flink.testutils.serialization.types.Util; @@ -64,13 +72,17 @@ import java.util.Queue; import java.util.Random; import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import static org.apache.flink.runtime.io.AvailabilityProvider.AVAILABLE; import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.buildSingleBuffer; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; @@ -417,6 +429,46 @@ public void testBroadcastEmitRecord() throws Exception { } } + /** + * Tests that the RecordWriter is available iif the respective LocalBufferPool has at-least one available buffer. + */ + @Test + public void testIsAvailableOrNot() throws Exception { + // setup + final NetworkBufferPool globalPool = new NetworkBufferPool(10, 128, 2); + final BufferPool localPool = globalPool.createBufferPool(1, 1); + final ResultPartitionWriter resultPartition = new ResultPartitionBuilder() + .setBufferPoolFactory(p -> localPool) + .build(); + resultPartition.setup(); + final ResultPartitionWriter partitionWrapper = new ConsumableNotifyingResultPartitionWriterDecorator( + new NoOpTaskActions(), + new JobID(), + resultPartition, + new NoOpResultPartitionConsumableNotifier()); + final RecordWriter recordWriter = createRecordWriter(partitionWrapper); + + try { + // record writer is available because of initial available local pool + assertTrue(recordWriter.isAvailable().isDone()); + assertEquals(recordWriter.AVAILABLE, recordWriter.isAvailable()); + + // request one buffer from the local pool to make it unavailable afterwards + final BufferBuilder bufferBuilder = resultPartition.getBufferBuilder(); + assertNotNull(bufferBuilder); + assertFalse(recordWriter.isAvailable().isDone()); + + // recycle the buffer to make the local pool available again + final Buffer buffer = BufferBuilderTestUtils.buildSingleBuffer(bufferBuilder); + buffer.recycleBuffer(); + assertTrue(recordWriter.isAvailable().isDone()); + assertEquals(recordWriter.AVAILABLE, recordWriter.isAvailable()); + } finally { + localPool.lazyDestroy(); + globalPool.destroy(); + } + } + private void verifyBroadcastBufferOrEventIndependence(boolean broadcastEvent) throws Exception { @SuppressWarnings("unchecked") ArrayDeque[] queues = new ArrayDeque[]{new ArrayDeque(), new ArrayDeque()}; @@ -543,6 +595,11 @@ public void finish() { throw new UnsupportedOperationException(); } + @Override + public CompletableFuture isAvailable() { + return AVAILABLE; + } + @Override public void close() { } @@ -619,6 +676,11 @@ public void finish() { throw new UnsupportedOperationException(); } + @Override + public CompletableFuture isAvailable() { + return AVAILABLE; + } + @Override public void close() { } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NoOpBufferPool.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NoOpBufferPool.java index b9c5c5fa0b39..05d56e193975 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NoOpBufferPool.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NoOpBufferPool.java @@ -23,6 +23,7 @@ import org.apache.flink.core.memory.MemorySegment; import java.io.IOException; +import java.util.concurrent.CompletableFuture; /** * No-op implementation of {@link BufferPool}. @@ -87,4 +88,9 @@ public int bestEffortGetNumOfUsedBuffers() { public void recycle(MemorySegment memorySegment) { throw new UnsupportedOperationException(); } + + @Override + public CompletableFuture isAvailable() { + return AVAILABLE; + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPooledBufferProvider.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPooledBufferProvider.java index 45aea352dcc7..4bb26f96e86f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPooledBufferProvider.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPooledBufferProvider.java @@ -31,6 +31,7 @@ import java.io.IOException; import java.util.Queue; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.LinkedBlockingDeque; @@ -95,6 +96,11 @@ public boolean isDestroyed() { return false; } + @Override + public CompletableFuture isAvailable() { + return AVAILABLE; + } + public int getNumberOfAvailableBuffers() { return buffers.size(); } From 60ce77a6ca5d52afe7d98b5b30d585b98b4a2514 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Tue, 22 Oct 2019 14:18:30 +0800 Subject: [PATCH 309/746] [hotfix][tests] Fix the missing class reference in LocalInputChannelTest --- .../io/network/partition/consumer/LocalInputChannelTest.java | 1 + 1 file changed, 1 insertion(+) 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 d049f9b0a745..cc2da0de1b67 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 @@ -36,6 +36,7 @@ import org.apache.flink.runtime.io.network.util.TestBufferFactory; import org.apache.flink.runtime.io.network.util.TestPartitionProducer; import org.apache.flink.runtime.io.network.util.TestProducerSource; +import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.util.function.CheckedSupplier; import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; From 27307b4f44d42a4d79a4ee52cb27f4f1581745c1 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Mon, 28 Oct 2019 14:34:47 +0800 Subject: [PATCH 310/746] [FLINK-14415][table-common] ValueLiteralExpression#equals should take array value into account (#9915) This closes #9915 --- .../expressions/ValueLiteralExpression.java | 2 +- .../table/expressions/ExpressionTest.java | 22 +++++++++++++++++++ 2 files changed, 23 insertions(+), 1 deletion(-) diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/ValueLiteralExpression.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/ValueLiteralExpression.java index bef3829f3256..f8bbaf625884 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/ValueLiteralExpression.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/ValueLiteralExpression.java @@ -187,7 +187,7 @@ public boolean equals(Object o) { return false; } ValueLiteralExpression that = (ValueLiteralExpression) o; - return Objects.equals(value, that.value) && dataType.equals(that.dataType); + return Objects.deepEquals(value, that.value) && dataType.equals(that.dataType); } @Override 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 fcc8ef1772ee..3a0abc8a4e90 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 @@ -28,6 +28,7 @@ import org.junit.rules.ExpectedException; import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; import java.sql.Timestamp; import java.time.LocalDateTime; import java.time.Period; @@ -72,6 +73,27 @@ public void testExpressionEquality() { assertEquals(TREE_WITH_VALUE, TREE_WITH_SAME_VALUE); } + @Test + public void testArrayValueLiteralEquality() { + assertEquals( + new ValueLiteralExpression(new Integer[][]{null, null, {1, 2, 3}}), + new ValueLiteralExpression(new Integer[][]{null, null, {1, 2, 3}})); + + assertEquals( + new ValueLiteralExpression( + new String[][]{null, null, {"1", "2", "3", "Dog's"}}, + DataTypes.ARRAY(DataTypes.ARRAY(DataTypes.STRING()))), + new ValueLiteralExpression( + new String[][]{null, null, {"1", "2", "3", "Dog's"}}, + DataTypes.ARRAY(DataTypes.ARRAY(DataTypes.STRING()))) + ); + + assertEquals( + new ValueLiteralExpression("abc".getBytes(StandardCharsets.UTF_8)), + new ValueLiteralExpression("abc".getBytes(StandardCharsets.UTF_8)) + ); + } + @Test public void testExpressionInequality() { assertNotEquals(TREE_WITH_NULL, TREE_WITH_VALUE); From ecc802064d35d22162a3bd57a522b4c77c56e390 Mon Sep 17 00:00:00 2001 From: sunhaibotb Date: Mon, 28 Oct 2019 11:53:23 +0800 Subject: [PATCH 311/746] [FLINK-14230][task] Change the endInput call of the downstream operator to after the upstream operator closes This change fixes the error of propagating "endInput" on the chain immediately after the input of the head operator is finished. Correctly, "endInput" of the downstream operator should be invoked only after closing the upstream operator. --- .../streaming/api/operators/StreamSource.java | 4 +- .../runtime/io/StreamOneInputProcessor.java | 2 +- .../runtime/io/StreamTwoInputProcessor.java | 2 +- .../runtime/tasks/OperatorChain.java | 62 +++++++------------ .../streaming/runtime/tasks/StreamTask.java | 7 +++ .../runtime/tasks/OneInputStreamTaskTest.java | 2 + .../runtime/tasks/SourceStreamTaskTest.java | 30 ++++++++- .../TestBoundedOneInputStreamOperator.java | 6 ++ .../runtime/tasks/TwoInputStreamTaskTest.java | 6 +- .../util/TestBoundedTwoInputOperator.java | 8 ++- 10 files changed, 80 insertions(+), 49 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java index cd9c3684db27..08b88f4b209d 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java @@ -106,8 +106,10 @@ public void run(final Object lockingObject, if (!isCanceledOrStopped()) { advanceToEndOfEventTime(); + // in theory, the subclasses of StreamSource may implement the BoundedOneInput interface, + // so we still need the following call to end the input synchronized (lockingObject) { - operatorChain.endInput(1); + operatorChain.endHeadOperatorInput(1); } } } finally { 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 ddeaf7be04ab..396355e28671 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 @@ -70,7 +70,7 @@ public InputStatus processInput() throws Exception { if (status == InputStatus.END_OF_INPUT) { synchronized (lock) { - operatorChain.endInput(1); + operatorChain.endHeadOperatorInput(1); } } 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 a18defe8eda4..51344637de36 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 @@ -203,7 +203,7 @@ private int selectFirstReadingInputIndex() throws IOException { private void checkFinished(InputStatus status, int inputIndex) throws Exception { if (status == InputStatus.END_OF_INPUT) { synchronized (lock) { - operatorChain.endInput(getInputId(inputIndex)); + operatorChain.endHeadOperatorInput(getInputId(inputIndex)); inputSelectionHandler.nextSelection(); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java index 824b0fc68a76..fa7faf23678f 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java @@ -41,13 +41,11 @@ import org.apache.flink.streaming.api.operators.BoundedMultiInput; import org.apache.flink.streaming.api.operators.BoundedOneInput; import org.apache.flink.streaming.api.operators.InputSelectable; -import org.apache.flink.streaming.api.operators.InputSelection; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; 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.StreamOperatorFactoryUtil; -import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.io.RecordWriterOutput; import org.apache.flink.streaming.runtime.metrics.WatermarkGauge; @@ -73,6 +71,7 @@ import java.util.Random; import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; /** * The {@code OperatorChain} contains all operators that are executed as one chain within a single @@ -105,9 +104,6 @@ public class OperatorChain> implements Strea */ private StreamStatus streamStatus = StreamStatus.ACTIVE; - /** The flag that tracks finished inputs. */ - private InputSelection finishedInputs = new InputSelection.Builder().build(); - public OperatorChain( StreamTask containingTask, List>>> recordWriters) { @@ -244,46 +240,30 @@ public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { } /** - * Ends an input (specified by {@code inputId}) of the {@link StreamTask}. The {@code inputId} - * is numbered starting from 1, and `1` indicates the first input. + * Ends the head operator input specified by {@code inputId}). * - * @param inputId The ID of the input. - * @throws Exception if some exception happens in the endInput function of an operator. + * @param inputId the input ID starts from 1 which indicates the first input. */ - public void endInput(int inputId) throws Exception { - if (finishedInputs.areAllInputsSelected()) { - return; - } - - if (headOperator instanceof TwoInputStreamOperator) { - if (finishedInputs.isInputSelected(inputId)) { - return; - } + public void endHeadOperatorInput(int inputId) throws Exception { + endOperatorInput(headOperator, inputId); + } - if (headOperator instanceof BoundedMultiInput) { - ((BoundedMultiInput) headOperator).endInput(inputId); - } + /** + * Ends all inputs of the non-head operator specified by {@code streamOperator}) + * (now there is only one input for each non-head operator). + * + * @param streamOperator non-head operator for ending the only input. + */ + public void endNonHeadOperatorInput(StreamOperator streamOperator) throws Exception { + checkState(streamOperator != headOperator); + endOperatorInput(streamOperator, 1); + } - finishedInputs = InputSelection.Builder - .from(finishedInputs) - .select(finishedInputs.getInputMask() == 0 ? inputId : -1) - .build(); - } else { - // here, the head operator is a stream source or an one-input stream operator, - // so all inputs are finished - finishedInputs = new InputSelection.Builder() - .select(-1) - .build(); - } - - if (finishedInputs.areAllInputsSelected()) { - // executing #endInput() happens from head to tail operator in the chain - for (int i = allOperators.length - 1; i >= 0; i--) { - StreamOperator operator = allOperators[i]; - if (operator instanceof BoundedOneInput) { - ((BoundedOneInput) operator).endInput(); - } - } + private void endOperatorInput(StreamOperator streamOperator, int inputId) throws Exception { + if (streamOperator instanceof BoundedOneInput) { + ((BoundedOneInput) streamOperator).endInput(); + } else if (streamOperator instanceof BoundedMultiInput) { + ((BoundedMultiInput) streamOperator).endInput(inputId); } } 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 4469da26ee54..c276d488fb44 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 @@ -583,6 +583,13 @@ private void closeAllOperators() throws Exception { if (operator != null) { operator.close(); } + + // The operators on the chain, except for the head operator, must be one-input operators. + // So after the upstream operator on the chain is closed, the input of its downstream operator + // reaches the end. + if (i > 0) { + operatorChain.endNonHeadOperatorInput(allOperators[i - 1]); + } } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java index 9765abacbc26..b6d1ad4f6043 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java @@ -623,7 +623,9 @@ public void testHandlingEndOfInput() throws Exception { testHarness.waitForTaskCompletion(); expectedOutput.add(new StreamRecord<>("Hello")); + expectedOutput.add(new StreamRecord<>("[Operator0]: EndOfInput")); expectedOutput.add(new StreamRecord<>("[Operator0]: Bye")); + expectedOutput.add(new StreamRecord<>("[Operator1]: EndOfInput")); expectedOutput.add(new StreamRecord<>("[Operator1]: Bye")); TestHarnessUtil.assertOutputEquals("Output was not correct.", diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java index bd083e7b13e2..f5b73211bc86 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.configuration.Configuration; @@ -34,6 +35,7 @@ import org.apache.flink.streaming.api.functions.source.RichSourceFunction; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.graph.StreamConfig; +import org.apache.flink.streaming.api.operators.BoundedOneInput; import org.apache.flink.streaming.api.operators.StreamSource; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.TestHarnessUtil; @@ -172,8 +174,8 @@ public void testMarkingEndOfInput() throws Exception { testHarness .setupOperatorChain( new OperatorID(), - new StreamSource<>(new FromElementsFunction<>( - BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), "Hello"))) + new OutputRecordInCloseTestSource<>( + "Source0", new FromElementsFunction<>(StringSerializer.INSTANCE, "Hello"))) .chain( new OperatorID(), new TestBoundedOneInputStreamOperator("Operator1"), @@ -189,6 +191,9 @@ public void testMarkingEndOfInput() throws Exception { testHarness.waitForTaskCompletion(); expectedOutput.add(new StreamRecord<>("Hello")); + expectedOutput.add(new StreamRecord<>("[Source0]: EndOfInput")); + expectedOutput.add(new StreamRecord<>("[Source0]: Bye")); + expectedOutput.add(new StreamRecord<>("[Operator1]: EndOfInput")); expectedOutput.add(new StreamRecord<>("[Operator1]: Bye")); TestHarnessUtil.assertOutputEquals("Output was not correct.", @@ -479,5 +484,26 @@ public void cancel() { running = false; } } + + private static final class OutputRecordInCloseTestSource> + extends StreamSource implements BoundedOneInput { + + private final String name; + + public OutputRecordInCloseTestSource(String name, SRC sourceFunction) { + super(sourceFunction); + this.name = name; + } + + @Override + public void endInput() { + output.collect(new StreamRecord<>("[" + name + "]: EndOfInput")); + } + + @Override + public void close() { + output.collect(new StreamRecord<>("[" + name + "]: Bye")); + } + } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TestBoundedOneInputStreamOperator.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TestBoundedOneInputStreamOperator.java index c7d517ce1dec..a5b784923b8e 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TestBoundedOneInputStreamOperator.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TestBoundedOneInputStreamOperator.java @@ -44,6 +44,12 @@ public void processElement(StreamRecord element) { @Override public void endInput() { + output.collect(new StreamRecord<>("[" + name + "]: EndOfInput")); + } + + @Override + public void close() throws Exception { output.collect(new StreamRecord<>("[" + name + "]: Bye")); + super.close(); } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java index af15fbc8d3cc..ca25380c6f9f 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java @@ -606,9 +606,11 @@ public void testHandlingEndOfInput() throws Exception { testHarness.waitForTaskCompletion(); expectedOutput.add(new StreamRecord<>("[Operator0-1]: Hello-1")); - expectedOutput.add(new StreamRecord<>("[Operator0-1]: Bye")); + expectedOutput.add(new StreamRecord<>("[Operator0-1]: EndOfInput")); expectedOutput.add(new StreamRecord<>("[Operator0-2]: Hello-2")); - expectedOutput.add(new StreamRecord<>("[Operator0-2]: Bye")); + expectedOutput.add(new StreamRecord<>("[Operator0-2]: EndOfInput")); + expectedOutput.add(new StreamRecord<>("[Operator0]: Bye")); + expectedOutput.add(new StreamRecord<>("[Operator1]: EndOfInput")); expectedOutput.add(new StreamRecord<>("[Operator1]: Bye")); TestHarnessUtil.assertOutputEquals("Output was not correct.", diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestBoundedTwoInputOperator.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestBoundedTwoInputOperator.java index cf0ebd4d1403..64f25b3c5e1f 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestBoundedTwoInputOperator.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestBoundedTwoInputOperator.java @@ -49,6 +49,12 @@ public void processElement2(StreamRecord element) { @Override public void endInput(int inputId) { - output.collect(new StreamRecord<>("[" + name + "-" + inputId + "]: Bye")); + output.collect(new StreamRecord<>("[" + name + "-" + inputId + "]: EndOfInput")); + } + + @Override + public void close() throws Exception { + output.collect(new StreamRecord<>("[" + name + "]: Bye")); + super.close(); } } From d4be2574e594c8ffc8e41fb924401b107723bbc0 Mon Sep 17 00:00:00 2001 From: sunhaibotb Date: Mon, 28 Oct 2019 12:00:57 +0800 Subject: [PATCH 312/746] [FLINK-14230][datastream] Remove the BoundedOneInput implementation of AsyncWaitOperator and ContinuousFileReaderOperator After "endInput" of the downstream operator on the chain is invoked correctly, we revert the changes of PR#9298 and PR#9221. --- .../source/ContinuousFileReaderOperator.java | 8 +- .../operators/async/AsyncWaitOperator.java | 8 +- .../async/AsyncWaitOperatorTest.java | 62 ------------ .../OneInputStreamOperatorTestHarness.java | 9 -- .../ContinuousFileReaderOperatorITCase.java | 98 ------------------- 5 files changed, 2 insertions(+), 183 deletions(-) delete mode 100644 flink-tests/src/test/java/org/apache/flink/test/streaming/api/ContinuousFileReaderOperatorITCase.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileReaderOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileReaderOperator.java index 50f0d5c45fc8..f55141d0be0b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileReaderOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileReaderOperator.java @@ -30,7 +30,6 @@ import org.apache.flink.runtime.state.StateSnapshotContext; import org.apache.flink.streaming.api.TimeCharacteristic; 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.operators.OutputTypeConfigurable; import org.apache.flink.streaming.api.operators.StreamSourceContexts; @@ -62,7 +61,7 @@ */ @Internal public class ContinuousFileReaderOperator extends AbstractStreamOperator - implements OneInputStreamOperator, OutputTypeConfigurable, BoundedOneInput { + implements OneInputStreamOperator, OutputTypeConfigurable { private static final long serialVersionUID = 1L; @@ -202,11 +201,6 @@ public void close() throws Exception { output.close(); } - @Override - public void endInput() throws Exception { - waitSplitReaderFinished(); - } - private void waitSplitReaderFinished() throws InterruptedException { // make sure that we hold the checkpointing lock assert Thread.holdsLock(checkpointLock); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java index 5fbc19e0fe67..7d6d5ab00e30 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java @@ -30,7 +30,6 @@ import org.apache.flink.streaming.api.functions.async.ResultFuture; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator; -import org.apache.flink.streaming.api.operators.BoundedOneInput; import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.Output; @@ -76,7 +75,7 @@ @Internal public class AsyncWaitOperator extends AbstractUdfStreamOperator> - implements OneInputStreamOperator, BoundedOneInput { + implements OneInputStreamOperator { private static final long serialVersionUID = 1L; private static final String STATE_NAME = "_async_wait_operator_state_"; @@ -234,11 +233,6 @@ public void initializeState(StateInitializationContext context) throws Exception } - @Override - public void endInput() throws Exception { - waitInFlightInputsFinished(); - } - @Override public void close() throws Exception { try { diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java index cfa6ce66f69a..69e78b744fd3 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java @@ -204,30 +204,6 @@ public static void countDown() { } } - /** - * AsyncFunction supports a specific delay(ms) before async invocation. - */ - private static class DelayedAsyncFunction extends MyAsyncFunction { - - private final long delayed; - - public DelayedAsyncFunction(long delayed) { - this.delayed = delayed; - } - - @Override - public void asyncInvoke(final Integer input, final ResultFuture resultFuture) throws Exception { - executorService.submit(() -> { - try { - Thread.sleep(delayed); - } catch (InterruptedException e) { - resultFuture.completeExceptionally(e); - } - resultFuture.complete(Collections.singletonList(input * 2)); - }); - } - } - /** * A special {@link LazyAsyncFunction} for timeout handling. * Complete the result future with 3 times the input when the timeout occurred. @@ -1023,44 +999,6 @@ private SingleOutputStreamOperator addAsyncOperatorLegacyChained( return in.transform("async wait operator", outTypeInfo, factory); } - /** - * Delay a while before async invocation to check whether end input waits for all elements finished or not. - */ - @Test - public void testEndInput() throws Exception { - final OneInputStreamOperatorTestHarness testHarness = - createTestHarness(new DelayedAsyncFunction(10), -1, 2, AsyncDataStream.OutputMode.ORDERED); - - final long initialTime = 0L; - final ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); - expectedOutput.add(new StreamRecord<>(2, initialTime + 1)); - expectedOutput.add(new StreamRecord<>(4, initialTime + 2)); - expectedOutput.add(new Watermark(initialTime + 2)); - expectedOutput.add(new StreamRecord<>(6, initialTime + 3)); - - testHarness.open(); - - try { - synchronized (testHarness.getCheckpointLock()) { - testHarness.processElement(new StreamRecord<>(1, initialTime + 1)); - testHarness.processElement(new StreamRecord<>(2, initialTime + 2)); - testHarness.processWatermark(new Watermark(initialTime + 2)); - testHarness.processElement(new StreamRecord<>(3, initialTime + 3)); - } - - // wait until all async collectors in the buffer have been emitted out. - synchronized (testHarness.getCheckpointLock()) { - testHarness.endInput(); - } - - TestHarnessUtil.assertOutputEquals("Output with watermark was not correct.", expectedOutput, testHarness.getOutput()); - } finally { - synchronized (testHarness.getCheckpointLock()) { - testHarness.close(); - } - } - } - private static OneInputStreamOperatorTestHarness createTestHarness( AsyncFunction function, long timeout, diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java index 4440d9d2e57e..508f44b01bd2 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.testutils.MockEnvironment; -import org.apache.flink.streaming.api.operators.BoundedOneInput; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.streaming.api.watermark.Watermark; @@ -170,12 +169,4 @@ public void processWatermark(Watermark mark) throws Exception { public long getCurrentWatermark() { return currentWatermark; } - - public void endInput() throws Exception { - if (getOneInputOperator() instanceof BoundedOneInput) { - ((BoundedOneInput) getOneInputOperator()).endInput(); - } else { - throw new UnsupportedOperationException("The operator is not BoundedOneInput"); - } - } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/ContinuousFileReaderOperatorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/ContinuousFileReaderOperatorITCase.java deleted file mode 100644 index 8d9995f8653f..000000000000 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/ContinuousFileReaderOperatorITCase.java +++ /dev/null @@ -1,98 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.test.streaming.api; - -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.DiscardingSink; -import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.api.operators.ChainingStrategy; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.test.util.AbstractTestBase; - -import org.junit.Test; - -import java.io.File; -import java.io.PrintWriter; - -import static org.apache.flink.api.common.typeinfo.BasicTypeInfo.STRING_TYPE_INFO; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; - -/** - * Integration tests for {@link org.apache.flink.streaming.api.functions.source.ContinuousFileReaderOperator}. - */ -public class ContinuousFileReaderOperatorITCase extends AbstractTestBase { - - @Test - public void testEndInput() throws Exception { - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - final File sourceFile = TEMPORARY_FOLDER.newFile(); - final int elementCount = 10000; - try (PrintWriter printWriter = new PrintWriter(sourceFile)) { - for (int i = 0; i < elementCount; i++) { - printWriter.println(i); - } - } - - DataStreamSource source = env.readTextFile(sourceFile.getAbsolutePath()); - - // check the endInput is invoked at the right time - TestBoundedOneInputStreamOperator checkingOperator = new TestBoundedOneInputStreamOperator(elementCount); - DataStream endInputChecking = source.transform("EndInputChecking", STRING_TYPE_INFO, checkingOperator); - - endInputChecking.addSink(new DiscardingSink<>()); - - env.execute("ContinuousFileReaderOperatorITCase.testEndInput"); - } - - private static class TestBoundedOneInputStreamOperator extends AbstractStreamOperator - implements OneInputStreamOperator, BoundedOneInput { - - private final int expectedProcessedElementCount; - - private boolean hasEnded = false; - - private int processedElementCount = 0; - - TestBoundedOneInputStreamOperator(int expectedProcessedElementCount) { - // this operator must be chained with ContinuousFileReaderOperator - // that way, this end input would be triggered after ContinuousFileReaderOperator - chainingStrategy = ChainingStrategy.ALWAYS; - this.expectedProcessedElementCount = expectedProcessedElementCount; - } - - @Override - public void endInput() throws Exception { - assertEquals(expectedProcessedElementCount, processedElementCount); - hasEnded = true; - } - - @Override - public void processElement(StreamRecord element) throws Exception { - assertFalse(hasEnded); - output.collect(element); - processedElementCount++; - } - } -} From 85433349249df77b2c372874f0561fd6fb9809c7 Mon Sep 17 00:00:00 2001 From: sunhaibotb Date: Mon, 28 Oct 2019 12:09:37 +0800 Subject: [PATCH 313/746] [hotfix][test] Clean up the test code in SourceStreamTaskTest and OneInputStreamTaskTest These cleanups include removing unnecessary type parameter declarations and redundant suppression, etc. --- .../runtime/tasks/OneInputStreamTaskTest.java | 79 ++++++++++--------- .../runtime/tasks/SourceStreamTaskTest.java | 8 +- 2 files changed, 43 insertions(+), 44 deletions(-) diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java index b6d1ad4f6043..c2d82e9b4695 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java @@ -109,20 +109,20 @@ public void testOpenCloseAndTimestamps() throws Exception { testHarness.setupOutputForSingletonOperatorChain(); StreamConfig streamConfig = testHarness.getStreamConfig(); - StreamMap mapOperator = new StreamMap(new TestOpenCloseMapFunction()); + StreamMap mapOperator = new StreamMap<>(new TestOpenCloseMapFunction()); streamConfig.setStreamOperator(mapOperator); streamConfig.setOperatorID(new OperatorID()); long initialTime = 0L; - ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); testHarness.invoke(); testHarness.waitForTaskRunning(); - testHarness.processElement(new StreamRecord("Hello", initialTime + 1)); - testHarness.processElement(new StreamRecord("Ciao", initialTime + 2)); - expectedOutput.add(new StreamRecord("Hello", initialTime + 1)); - expectedOutput.add(new StreamRecord("Ciao", initialTime + 2)); + testHarness.processElement(new StreamRecord<>("Hello", initialTime + 1)); + testHarness.processElement(new StreamRecord<>("Ciao", initialTime + 2)); + expectedOutput.add(new StreamRecord<>("Hello", initialTime + 1)); + expectedOutput.add(new StreamRecord<>("Ciao", initialTime + 2)); testHarness.waitForInputProcessing(); @@ -143,7 +143,6 @@ public void testOpenCloseAndTimestamps() throws Exception { * forwarded watermark must be the minimum of the watermarks of all active inputs. */ @Test - @SuppressWarnings("unchecked") public void testWatermarkAndStreamStatusForwarding() throws Exception { final OneInputStreamTaskTestHarness testHarness = @@ -155,11 +154,11 @@ public void testWatermarkAndStreamStatusForwarding() throws Exception { testHarness.setupOutputForSingletonOperatorChain(); StreamConfig streamConfig = testHarness.getStreamConfig(); - StreamMap mapOperator = new StreamMap(new IdentityMap()); + StreamMap mapOperator = new StreamMap<>(new IdentityMap()); streamConfig.setStreamOperator(mapOperator); streamConfig.setOperatorID(new OperatorID()); - ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); long initialTime = 0L; testHarness.invoke(); @@ -183,10 +182,10 @@ public void testWatermarkAndStreamStatusForwarding() throws Exception { testHarness.getOutput()); // contrary to checkpoint barriers these elements are not blocked by watermarks - testHarness.processElement(new StreamRecord("Hello", initialTime)); - testHarness.processElement(new StreamRecord("Ciao", initialTime)); - expectedOutput.add(new StreamRecord("Hello", initialTime)); - expectedOutput.add(new StreamRecord("Ciao", initialTime)); + testHarness.processElement(new StreamRecord<>("Hello", initialTime)); + testHarness.processElement(new StreamRecord<>("Ciao", initialTime)); + expectedOutput.add(new StreamRecord<>("Hello", initialTime)); + expectedOutput.add(new StreamRecord<>("Ciao", initialTime)); testHarness.processElement(new Watermark(initialTime + 4), 0, 0); testHarness.processElement(new Watermark(initialTime + 3), 0, 1); @@ -277,7 +276,7 @@ public void testWatermarksNotForwardedWithinChainWhenIdle() throws Exception { // --------------------- begin test --------------------- - ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); testHarness.invoke(); testHarness.waitForTaskRunning(); @@ -367,11 +366,11 @@ public void testCheckpointBarriers() throws Exception { testHarness.setupOutputForSingletonOperatorChain(); StreamConfig streamConfig = testHarness.getStreamConfig(); - StreamMap mapOperator = new StreamMap(new IdentityMap()); + StreamMap mapOperator = new StreamMap<>(new IdentityMap()); streamConfig.setStreamOperator(mapOperator); streamConfig.setOperatorID(new OperatorID()); - ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); long initialTime = 0L; testHarness.invoke(); @@ -381,16 +380,16 @@ public void testCheckpointBarriers() throws Exception { // These elements should be buffered until we receive barriers from // all inputs - testHarness.processElement(new StreamRecord("Hello-0-0", initialTime), 0, 0); - testHarness.processElement(new StreamRecord("Ciao-0-0", initialTime), 0, 0); + testHarness.processElement(new StreamRecord<>("Hello-0-0", initialTime), 0, 0); + testHarness.processElement(new StreamRecord<>("Ciao-0-0", initialTime), 0, 0); // These elements should be forwarded, since we did not yet receive a checkpoint barrier // on that input, only add to same input, otherwise we would not know the ordering // of the output since the Task might read the inputs in any order - testHarness.processElement(new StreamRecord("Hello-1-1", initialTime), 1, 1); - testHarness.processElement(new StreamRecord("Ciao-1-1", initialTime), 1, 1); - expectedOutput.add(new StreamRecord("Hello-1-1", initialTime)); - expectedOutput.add(new StreamRecord("Ciao-1-1", initialTime)); + testHarness.processElement(new StreamRecord<>("Hello-1-1", initialTime), 1, 1); + testHarness.processElement(new StreamRecord<>("Ciao-1-1", initialTime), 1, 1); + expectedOutput.add(new StreamRecord<>("Hello-1-1", initialTime)); + expectedOutput.add(new StreamRecord<>("Ciao-1-1", initialTime)); testHarness.waitForInputProcessing(); // we should not yet see the barrier, only the two elements from non-blocked input @@ -404,8 +403,8 @@ public void testCheckpointBarriers() throws Exception { // now we should see the barrier and after that the buffered elements expectedOutput.add(new CheckpointBarrier(0, 0, CheckpointOptions.forCheckpointWithDefaultLocation())); - expectedOutput.add(new StreamRecord("Hello-0-0", initialTime)); - expectedOutput.add(new StreamRecord("Ciao-0-0", initialTime)); + expectedOutput.add(new StreamRecord<>("Hello-0-0", initialTime)); + expectedOutput.add(new StreamRecord<>("Ciao-0-0", initialTime)); testHarness.endInput(); @@ -430,11 +429,11 @@ public void testOvertakingCheckpointBarriers() throws Exception { testHarness.setupOutputForSingletonOperatorChain(); StreamConfig streamConfig = testHarness.getStreamConfig(); - StreamMap mapOperator = new StreamMap(new IdentityMap()); + StreamMap mapOperator = new StreamMap<>(new IdentityMap()); streamConfig.setStreamOperator(mapOperator); streamConfig.setOperatorID(new OperatorID()); - ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); long initialTime = 0L; testHarness.invoke(); @@ -444,16 +443,16 @@ public void testOvertakingCheckpointBarriers() throws Exception { // These elements should be buffered until we receive barriers from // all inputs - testHarness.processElement(new StreamRecord("Hello-0-0", initialTime), 0, 0); - testHarness.processElement(new StreamRecord("Ciao-0-0", initialTime), 0, 0); + testHarness.processElement(new StreamRecord<>("Hello-0-0", initialTime), 0, 0); + testHarness.processElement(new StreamRecord<>("Ciao-0-0", initialTime), 0, 0); // These elements should be forwarded, since we did not yet receive a checkpoint barrier // on that input, only add to same input, otherwise we would not know the ordering // of the output since the Task might read the inputs in any order - testHarness.processElement(new StreamRecord("Hello-1-1", initialTime), 1, 1); - testHarness.processElement(new StreamRecord("Ciao-1-1", initialTime), 1, 1); - expectedOutput.add(new StreamRecord("Hello-1-1", initialTime)); - expectedOutput.add(new StreamRecord("Ciao-1-1", initialTime)); + testHarness.processElement(new StreamRecord<>("Hello-1-1", initialTime), 1, 1); + testHarness.processElement(new StreamRecord<>("Ciao-1-1", initialTime), 1, 1); + expectedOutput.add(new StreamRecord<>("Hello-1-1", initialTime)); + expectedOutput.add(new StreamRecord<>("Ciao-1-1", initialTime)); testHarness.waitForInputProcessing(); // we should not yet see the barrier, only the two elements from non-blocked input @@ -467,8 +466,8 @@ public void testOvertakingCheckpointBarriers() throws Exception { testHarness.processEvent(new CheckpointBarrier(1, 1, CheckpointOptions.forCheckpointWithDefaultLocation()), 1, 1); expectedOutput.add(new CancelCheckpointMarker(0)); - expectedOutput.add(new StreamRecord("Hello-0-0", initialTime)); - expectedOutput.add(new StreamRecord("Ciao-0-0", initialTime)); + expectedOutput.add(new StreamRecord<>("Hello-0-0", initialTime)); + expectedOutput.add(new StreamRecord<>("Ciao-0-0", initialTime)); expectedOutput.add(new CheckpointBarrier(1, 1, CheckpointOptions.forCheckpointWithDefaultLocation())); testHarness.waitForInputProcessing(); @@ -708,6 +707,7 @@ public void processElement(StreamRecord element) { } @Test + @SuppressWarnings("unchecked") public void testWatermarkMetrics() throws Exception { final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness<>(OneInputStreamTask::new, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); @@ -797,7 +797,7 @@ public void processElement(StreamRecord element) throws Exception { } @Override - public void processWatermark(Watermark mark) throws Exception { + public void processWatermark(Watermark mark) { output.emitWatermark(new Watermark(mark.getTimestamp() * 2)); } } @@ -848,7 +848,7 @@ private void configureChainedTestingStreamOperator( null ), 0, - Collections.emptyList(), + Collections.emptyList(), null, null ); @@ -921,9 +921,10 @@ public void processElement(StreamRecord element) throws Exception { } } - - // This must only be used in one test, otherwise the static fields will be changed - // by several tests concurrently + /** + * This must only be used in one test, otherwise the static fields will be changed + * by several tests concurrently. + */ private static class TestOpenCloseMapFunction extends RichMapFunction { private static final long serialVersionUID = 1L; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java index f5b73211bc86..6a46a86f4850 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java @@ -74,7 +74,6 @@ public class SourceStreamTaskTest { * This test verifies that open() and close() are correctly called by the StreamTask. */ @Test - @SuppressWarnings("unchecked") public void testOpenClose() throws Exception { final StreamTaskTestHarness testHarness = new StreamTaskTestHarness<>( SourceStreamTask::new, BasicTypeInfo.STRING_TYPE_INFO); @@ -288,7 +287,6 @@ public MockSource(int maxElements, int checkpointDelay, int readDelay) { @Override public void run(SourceContext> ctx) { - final Object lockObject = ctx.getCheckpointLock(); while (isRunning && count < maxElements) { // simulate some work try { @@ -299,8 +297,8 @@ public void run(SourceContext> ctx) { Thread.currentThread().interrupt(); } - synchronized (lockObject) { - ctx.collect(new Tuple2(lastCheckpointId, count)); + synchronized (ctx.getCheckpointLock()) { + ctx.collect(new Tuple2<>(lastCheckpointId, count)); count++; } } @@ -330,7 +328,7 @@ public List snapshotState(long checkpointId, long timestamp) throw Assert.fail("Count is different at start end end of snapshot."); } semaphore.release(); - return Collections.singletonList(sum); + return Collections.singletonList(sum); } @Override From 126cff6ab4e433a69c774e126b2e85335e30ee7a Mon Sep 17 00:00:00 2001 From: ifndef-SleePy Date: Wed, 18 Sep 2019 17:30:04 +0800 Subject: [PATCH 314/746] [hotfix] Extract utils of CheckpointCoordinatorTest into a separate utils class and correct codestyle --- .../CheckpointCoordinatorFailureTest.java | 5 +- .../CheckpointCoordinatorMasterHooksTest.java | 4 +- .../checkpoint/CheckpointCoordinatorTest.java | 517 ++---------------- .../CheckpointCoordinatorTestingUtils.java | 472 ++++++++++++++++ .../CheckpointStateRestoreTest.java | 4 +- .../messages/CheckpointMessagesTest.java | 13 +- 6 files changed, 542 insertions(+), 473 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java index da181bada169..b6b79302a68f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java @@ -49,6 +49,9 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +/** + * Tests for failure of checkpoint coordinator. + */ @RunWith(PowerMockRunner.class) @PrepareForTest(PendingCheckpoint.class) public class CheckpointCoordinatorFailureTest extends TestLogger { @@ -62,7 +65,7 @@ public void testFailingCompletedCheckpointStoreAdd() throws Exception { JobID jid = new JobID(); final ExecutionAttemptID executionAttemptId = new ExecutionAttemptID(); - final ExecutionVertex vertex = CheckpointCoordinatorTest.mockExecutionVertex(executionAttemptId); + final ExecutionVertex vertex = CheckpointCoordinatorTestingUtils.mockExecutionVertex(executionAttemptId); final long triggerTimestamp = 1L; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java index 48b6583d4341..8453cba62438 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java @@ -48,7 +48,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; -import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTest.mockExecutionVertex; +import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.mockExecutionVertex; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -353,7 +353,7 @@ public void checkUnMatchedStateOnRestore() throws Exception { // ------------------------------------------------------------------------ /** - * This test makes sure that the checkpoint is already registered by the time + * This test makes sure that the checkpoint is already registered by the time. * that the hooks are called */ @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java index 98d80fd694b4..034fac4ce3dc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java @@ -19,9 +19,7 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.core.fs.FSDataInputStream; import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.execution.ExecutionState; @@ -39,7 +37,6 @@ import org.apache.flink.runtime.state.IncrementalRemoteKeyedStateHandle; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyGroupRangeAssignment; -import org.apache.flink.runtime.state.KeyGroupRangeOffsets; import org.apache.flink.runtime.state.KeyGroupsStateHandle; import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; @@ -55,8 +52,6 @@ import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.RecoverableCompletedCheckpointStore; import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.InstantiationUtil; -import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializableObject; import org.apache.flink.util.TestLogger; @@ -76,7 +71,6 @@ import org.mockito.verification.VerificationMode; import java.io.IOException; -import java.io.Serializable; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -91,11 +85,20 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import java.util.concurrent.Executor; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.compareKeyedState; +import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.comparePartitionableState; +import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.generateChainedPartitionableStateHandle; +import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.generateKeyGroupState; +import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.generatePartitionableStateHandle; +import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.mockExecution; +import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.mockExecutionJobVertex; +import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.mockExecutionVertex; +import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.mockSubtaskState; +import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.verifyStateRestore; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; @@ -439,7 +442,6 @@ public void testTriggerAndDeclineCheckpointSimple() { assertFalse(checkpoint.isDiscarded()); assertFalse(checkpoint.isFullyAcknowledged()); - // decline checkpoint from the other task, this should cancel the checkpoint // and trigger a new one coord.receiveDeclineMessage(new DeclineCheckpoint(jid, attemptID1, checkpointId), TASK_MANAGER_LOCATION_INFO); @@ -923,19 +925,19 @@ public void testSuccessfulCheckpointSubsumesUnsuccessful() { OperatorID opID2 = OperatorID.fromJobVertexID(ackVertex2.getJobvertexId()); OperatorID opID3 = OperatorID.fromJobVertexID(ackVertex3.getJobvertexId()); - TaskStateSnapshot taskOperatorSubtaskStates1_1 = spy(new TaskStateSnapshot()); - TaskStateSnapshot taskOperatorSubtaskStates1_2 = spy(new TaskStateSnapshot()); - TaskStateSnapshot taskOperatorSubtaskStates1_3 = spy(new TaskStateSnapshot()); + TaskStateSnapshot taskOperatorSubtaskStates11 = spy(new TaskStateSnapshot()); + TaskStateSnapshot taskOperatorSubtaskStates12 = spy(new TaskStateSnapshot()); + TaskStateSnapshot taskOperatorSubtaskStates13 = spy(new TaskStateSnapshot()); - OperatorSubtaskState subtaskState1_1 = mock(OperatorSubtaskState.class); - OperatorSubtaskState subtaskState1_2 = mock(OperatorSubtaskState.class); - OperatorSubtaskState subtaskState1_3 = mock(OperatorSubtaskState.class); - taskOperatorSubtaskStates1_1.putSubtaskStateByOperatorID(opID1, subtaskState1_1); - taskOperatorSubtaskStates1_2.putSubtaskStateByOperatorID(opID2, subtaskState1_2); - taskOperatorSubtaskStates1_3.putSubtaskStateByOperatorID(opID3, subtaskState1_3); + OperatorSubtaskState subtaskState11 = mock(OperatorSubtaskState.class); + OperatorSubtaskState subtaskState12 = mock(OperatorSubtaskState.class); + OperatorSubtaskState subtaskState13 = mock(OperatorSubtaskState.class); + taskOperatorSubtaskStates11.putSubtaskStateByOperatorID(opID1, subtaskState11); + taskOperatorSubtaskStates12.putSubtaskStateByOperatorID(opID2, subtaskState12); + taskOperatorSubtaskStates13.putSubtaskStateByOperatorID(opID3, subtaskState13); // acknowledge one of the three tasks - coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID2, checkpointId1, new CheckpointMetrics(), taskOperatorSubtaskStates1_2), TASK_MANAGER_LOCATION_INFO); + coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID2, checkpointId1, new CheckpointMetrics(), taskOperatorSubtaskStates12), TASK_MANAGER_LOCATION_INFO); // start the second checkpoint // trigger the first checkpoint. this should succeed @@ -953,17 +955,17 @@ public void testSuccessfulCheckpointSubsumesUnsuccessful() { } long checkpointId2 = pending2.getCheckpointId(); - TaskStateSnapshot taskOperatorSubtaskStates2_1 = spy(new TaskStateSnapshot()); - TaskStateSnapshot taskOperatorSubtaskStates2_2 = spy(new TaskStateSnapshot()); - TaskStateSnapshot taskOperatorSubtaskStates2_3 = spy(new TaskStateSnapshot()); + TaskStateSnapshot taskOperatorSubtaskStates21 = spy(new TaskStateSnapshot()); + TaskStateSnapshot taskOperatorSubtaskStates22 = spy(new TaskStateSnapshot()); + TaskStateSnapshot taskOperatorSubtaskStates23 = spy(new TaskStateSnapshot()); - OperatorSubtaskState subtaskState2_1 = mock(OperatorSubtaskState.class); - OperatorSubtaskState subtaskState2_2 = mock(OperatorSubtaskState.class); - OperatorSubtaskState subtaskState2_3 = mock(OperatorSubtaskState.class); + OperatorSubtaskState subtaskState21 = mock(OperatorSubtaskState.class); + OperatorSubtaskState subtaskState22 = mock(OperatorSubtaskState.class); + OperatorSubtaskState subtaskState23 = mock(OperatorSubtaskState.class); - taskOperatorSubtaskStates2_1.putSubtaskStateByOperatorID(opID1, subtaskState2_1); - taskOperatorSubtaskStates2_2.putSubtaskStateByOperatorID(opID2, subtaskState2_2); - taskOperatorSubtaskStates2_3.putSubtaskStateByOperatorID(opID3, subtaskState2_3); + taskOperatorSubtaskStates21.putSubtaskStateByOperatorID(opID1, subtaskState21); + taskOperatorSubtaskStates22.putSubtaskStateByOperatorID(opID2, subtaskState22); + taskOperatorSubtaskStates23.putSubtaskStateByOperatorID(opID3, subtaskState23); // trigger messages should have been sent verify(triggerVertex1.getCurrentExecutionAttempt(), times(1)).triggerCheckpoint(eq(checkpointId2), eq(timestamp2), any(CheckpointOptions.class)); @@ -972,13 +974,13 @@ public void testSuccessfulCheckpointSubsumesUnsuccessful() { // we acknowledge one more task from the first checkpoint and the second // checkpoint completely. The second checkpoint should then subsume the first checkpoint - coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID3, checkpointId2, new CheckpointMetrics(), taskOperatorSubtaskStates2_3), TASK_MANAGER_LOCATION_INFO); + coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID3, checkpointId2, new CheckpointMetrics(), taskOperatorSubtaskStates23), TASK_MANAGER_LOCATION_INFO); - coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID1, checkpointId2, new CheckpointMetrics(), taskOperatorSubtaskStates2_1), TASK_MANAGER_LOCATION_INFO); + coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID1, checkpointId2, new CheckpointMetrics(), taskOperatorSubtaskStates21), TASK_MANAGER_LOCATION_INFO); - coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID1, checkpointId1, new CheckpointMetrics(), taskOperatorSubtaskStates1_1), TASK_MANAGER_LOCATION_INFO); + coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID1, checkpointId1, new CheckpointMetrics(), taskOperatorSubtaskStates11), TASK_MANAGER_LOCATION_INFO); - coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID2, checkpointId2, new CheckpointMetrics(), taskOperatorSubtaskStates2_2), TASK_MANAGER_LOCATION_INFO); + coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID2, checkpointId2, new CheckpointMetrics(), taskOperatorSubtaskStates22), TASK_MANAGER_LOCATION_INFO); // now, the second checkpoint should be confirmed, and the first discarded // actually both pending checkpoints are discarded, and the second has been transformed @@ -990,13 +992,13 @@ public void testSuccessfulCheckpointSubsumesUnsuccessful() { assertEquals(1, coord.getNumberOfRetainedSuccessfulCheckpoints()); // validate that all received subtask states in the first checkpoint have been discarded - verify(subtaskState1_1, times(1)).discardState(); - verify(subtaskState1_2, times(1)).discardState(); + verify(subtaskState11, times(1)).discardState(); + verify(subtaskState12, times(1)).discardState(); // validate that all subtask states in the second checkpoint are not discarded - verify(subtaskState2_1, never()).discardState(); - verify(subtaskState2_2, never()).discardState(); - verify(subtaskState2_3, never()).discardState(); + verify(subtaskState21, never()).discardState(); + verify(subtaskState22, never()).discardState(); + verify(subtaskState23, never()).discardState(); // validate the committed checkpoints List scs = coord.getSuccessfulCheckpoints(); @@ -1010,15 +1012,15 @@ public void testSuccessfulCheckpointSubsumesUnsuccessful() { verify(commitVertex.getCurrentExecutionAttempt(), times(1)).notifyCheckpointComplete(eq(checkpointId2), eq(timestamp2)); // send the last remaining ack for the first checkpoint. This should not do anything - coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID3, checkpointId1, new CheckpointMetrics(), taskOperatorSubtaskStates1_3), TASK_MANAGER_LOCATION_INFO); - verify(subtaskState1_3, times(1)).discardState(); + coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID3, checkpointId1, new CheckpointMetrics(), taskOperatorSubtaskStates13), TASK_MANAGER_LOCATION_INFO); + verify(subtaskState13, times(1)).discardState(); coord.shutdown(JobStatus.FINISHED); // validate that the states in the second checkpoint have been discarded - verify(subtaskState2_1, times(1)).discardState(); - verify(subtaskState2_2, times(1)).discardState(); - verify(subtaskState2_3, times(1)).discardState(); + verify(subtaskState21, times(1)).discardState(); + verify(subtaskState22, times(1)).discardState(); + verify(subtaskState23, times(1)).discardState(); } catch (Exception e) { @@ -1377,7 +1379,6 @@ public Void answer(InvocationOnMock invocation) throws Throwable { coord.stopCheckpointScheduler(); - // for 400 ms, no further calls may come. // there may be the case that one trigger was fired and about to // acquire the lock, such that after cancelling it will still do @@ -1385,7 +1386,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { int numCallsSoFar = numCalls.get(); Thread.sleep(400); assertTrue(numCallsSoFar == numCalls.get() || - numCallsSoFar+1 == numCalls.get()); + numCallsSoFar + 1 == numCalls.get()); // start another sequence of periodic scheduling numCalls.set(0); @@ -2200,7 +2201,7 @@ public void testRestoreLatestCheckpointedState() throws Exception { * * @throws Exception */ - @Test(expected=IllegalStateException.class) + @Test(expected = IllegalStateException.class) public void testRestoreLatestCheckpointFailureWhenMaxParallelismChanges() throws Exception { final JobID jid = new JobID(); final long timestamp = System.currentTimeMillis(); @@ -2275,7 +2276,6 @@ public void testRestoreLatestCheckpointFailureWhenMaxParallelismChanges() throws coord.receiveAcknowledgeMessage(acknowledgeCheckpoint, TASK_MANAGER_LOCATION_INFO); } - for (int index = 0; index < jobVertex2.getParallelism(); index++) { KeyGroupsStateHandle keyGroupState = generateKeyGroupState(jobVertexID2, keyGroupPartitions2.get(index), false); OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(null, null, keyGroupState, null); @@ -2391,9 +2391,9 @@ private void testRestoreLatestCheckpointIsPreferSavepoint(boolean isPreferCheckp long checkpointId = checkpointIDCounter.getLast(); - KeyGroupRange keyGroupRange = KeyGroupRange.of(0,0); + KeyGroupRange keyGroupRange = KeyGroupRange.of(0, 0); List testStates = Collections.singletonList(new SerializableObject()); - KeyedStateHandle serializedKeyGroupStates = CheckpointCoordinatorTest.generateKeyGroupState(keyGroupRange, testStates); + KeyedStateHandle serializedKeyGroupStates = generateKeyGroupState(keyGroupRange, testStates); TaskStateSnapshot subtaskStatesForCheckpoint = new TaskStateSnapshot(); @@ -2416,10 +2416,9 @@ private void testRestoreLatestCheckpointIsPreferSavepoint(boolean isPreferCheckp timestamp = System.currentTimeMillis(); CompletableFuture savepointFuture = coord.triggerSavepoint(timestamp, savepointDir); - KeyGroupRange keyGroupRangeForSavepoint = KeyGroupRange.of(1, 1); List testStatesForSavepoint = Collections.singletonList(new SerializableObject()); - KeyedStateHandle serializedKeyGroupStatesForSavepoint = CheckpointCoordinatorTest.generateKeyGroupState(keyGroupRangeForSavepoint, testStatesForSavepoint); + KeyedStateHandle serializedKeyGroupStatesForSavepoint = generateKeyGroupState(keyGroupRangeForSavepoint, testStatesForSavepoint); TaskStateSnapshot subtaskStatesForSavepoint = new TaskStateSnapshot(); @@ -2677,16 +2676,18 @@ private static Tuple2 generateIDPair() { OperatorID operatorID = OperatorID.fromJobVertexID(jobVertexID); return new Tuple2<>(jobVertexID, operatorID); } - + /** - * old topology + *

+ * old topology. * [operator1,operator2] * parallelism1 -> [operator3,operator4] * parallelism2 + *

* - * + *

* new topology * * [operator5,operator1,operator3] * newParallelism1 -> [operator3, operator6] * newParallelism2 - * + *

* scaleType: * 0 increase parallelism * 1 decrease parallelism @@ -2956,7 +2957,6 @@ public void testStateRecoveryWithTopologyChange(int scaleType) throws Exception Collection keyedStateBackend = headOpState.getManagedKeyedState(); Collection keyGroupStateRaw = headOpState.getRawKeyedState(); - compareKeyedState(Collections.singletonList(originalKeyedStateBackend), keyedStateBackend); compareKeyedState(Collections.singletonList(originalKeyedStateRaw), keyGroupStateRaw); } @@ -3019,383 +3019,6 @@ public void testExternalizedCheckpoints() throws Exception { } } - // ------------------------------------------------------------------------ - // Utilities - // ------------------------------------------------------------------------ - - public static KeyGroupsStateHandle generateKeyGroupState( - JobVertexID jobVertexID, - KeyGroupRange keyGroupPartition, boolean rawState) throws IOException { - - List testStatesLists = new ArrayList<>(keyGroupPartition.getNumberOfKeyGroups()); - - // generate state for one keygroup - for (int keyGroupIndex : keyGroupPartition) { - int vertexHash = jobVertexID.hashCode(); - int seed = rawState ? (vertexHash * (31 + keyGroupIndex)) : (vertexHash + keyGroupIndex); - Random random = new Random(seed); - int simulatedStateValue = random.nextInt(); - testStatesLists.add(simulatedStateValue); - } - - return generateKeyGroupState(keyGroupPartition, testStatesLists); - } - - public static KeyGroupsStateHandle generateKeyGroupState( - KeyGroupRange keyGroupRange, - List states) throws IOException { - - Preconditions.checkArgument(keyGroupRange.getNumberOfKeyGroups() == states.size()); - - Tuple2> serializedDataWithOffsets = - serializeTogetherAndTrackOffsets(Collections.>singletonList(states)); - - KeyGroupRangeOffsets keyGroupRangeOffsets = new KeyGroupRangeOffsets(keyGroupRange, serializedDataWithOffsets.f1.get(0)); - - ByteStreamStateHandle allSerializedStatesHandle = new ByteStreamStateHandle( - String.valueOf(UUID.randomUUID()), - serializedDataWithOffsets.f0); - - return new KeyGroupsStateHandle(keyGroupRangeOffsets, allSerializedStatesHandle); - } - - public static Tuple2> serializeTogetherAndTrackOffsets( - List> serializables) throws IOException { - - List offsets = new ArrayList<>(serializables.size()); - List serializedGroupValues = new ArrayList<>(); - - int runningGroupsOffset = 0; - for(List list : serializables) { - - long[] currentOffsets = new long[list.size()]; - offsets.add(currentOffsets); - - for (int i = 0; i < list.size(); ++i) { - currentOffsets[i] = runningGroupsOffset; - byte[] serializedValue = InstantiationUtil.serializeObject(list.get(i)); - serializedGroupValues.add(serializedValue); - runningGroupsOffset += serializedValue.length; - } - } - - //write all generated values in a single byte array, which is index by groupOffsetsInFinalByteArray - byte[] allSerializedValuesConcatenated = new byte[runningGroupsOffset]; - runningGroupsOffset = 0; - for (byte[] serializedGroupValue : serializedGroupValues) { - System.arraycopy( - serializedGroupValue, - 0, - allSerializedValuesConcatenated, - runningGroupsOffset, - serializedGroupValue.length); - runningGroupsOffset += serializedGroupValue.length; - } - return new Tuple2<>(allSerializedValuesConcatenated, offsets); - } - - public static OperatorStateHandle generatePartitionableStateHandle( - JobVertexID jobVertexID, - int index, - int namedStates, - int partitionsPerState, - boolean rawState) throws IOException { - - Map> statesListsMap = new HashMap<>(namedStates); - - for (int i = 0; i < namedStates; ++i) { - List testStatesLists = new ArrayList<>(partitionsPerState); - // generate state - int seed = jobVertexID.hashCode() * index + i * namedStates; - if (rawState) { - seed = (seed + 1) * 31; - } - Random random = new Random(seed); - for (int j = 0; j < partitionsPerState; ++j) { - int simulatedStateValue = random.nextInt(); - testStatesLists.add(simulatedStateValue); - } - statesListsMap.put("state-" + i, testStatesLists); - } - - return generatePartitionableStateHandle(statesListsMap); - } - - public static ChainedStateHandle generateChainedPartitionableStateHandle( - JobVertexID jobVertexID, - int index, - int namedStates, - int partitionsPerState, - boolean rawState) throws IOException { - - Map> statesListsMap = new HashMap<>(namedStates); - - for (int i = 0; i < namedStates; ++i) { - List testStatesLists = new ArrayList<>(partitionsPerState); - // generate state - int seed = jobVertexID.hashCode() * index + i * namedStates; - if (rawState) { - seed = (seed + 1) * 31; - } - Random random = new Random(seed); - for (int j = 0; j < partitionsPerState; ++j) { - int simulatedStateValue = random.nextInt(); - testStatesLists.add(simulatedStateValue); - } - statesListsMap.put("state-" + i, testStatesLists); - } - - return ChainedStateHandle.wrapSingleHandle(generatePartitionableStateHandle(statesListsMap)); - } - - private static OperatorStateHandle generatePartitionableStateHandle( - Map> states) throws IOException { - - List> namedStateSerializables = new ArrayList<>(states.size()); - - for (Map.Entry> entry : states.entrySet()) { - namedStateSerializables.add(entry.getValue()); - } - - Tuple2> serializationWithOffsets = serializeTogetherAndTrackOffsets(namedStateSerializables); - - Map offsetsMap = new HashMap<>(states.size()); - - int idx = 0; - for (Map.Entry> entry : states.entrySet()) { - offsetsMap.put( - entry.getKey(), - new OperatorStateHandle.StateMetaInfo( - serializationWithOffsets.f1.get(idx), - OperatorStateHandle.Mode.SPLIT_DISTRIBUTE)); - ++idx; - } - - ByteStreamStateHandle streamStateHandle = new ByteStreamStateHandle( - String.valueOf(UUID.randomUUID()), - serializationWithOffsets.f0); - - return new OperatorStreamStateHandle(offsetsMap, streamStateHandle); - } - - static ExecutionJobVertex mockExecutionJobVertex( - JobVertexID jobVertexID, - int parallelism, - int maxParallelism) { - - return mockExecutionJobVertex( - jobVertexID, - Collections.singletonList(OperatorID.fromJobVertexID(jobVertexID)), - parallelism, - maxParallelism - ); - } - - static ExecutionJobVertex mockExecutionJobVertex( - JobVertexID jobVertexID, - List jobVertexIDs, - int parallelism, - int maxParallelism) { - final ExecutionJobVertex executionJobVertex = mock(ExecutionJobVertex.class); - - ExecutionVertex[] executionVertices = new ExecutionVertex[parallelism]; - - for (int i = 0; i < parallelism; i++) { - executionVertices[i] = mockExecutionVertex( - new ExecutionAttemptID(), - jobVertexID, - jobVertexIDs, - parallelism, - maxParallelism, - ExecutionState.RUNNING); - - when(executionVertices[i].getParallelSubtaskIndex()).thenReturn(i); - } - - when(executionJobVertex.getJobVertexId()).thenReturn(jobVertexID); - when(executionJobVertex.getTaskVertices()).thenReturn(executionVertices); - when(executionJobVertex.getParallelism()).thenReturn(parallelism); - when(executionJobVertex.getMaxParallelism()).thenReturn(maxParallelism); - when(executionJobVertex.isMaxParallelismConfigured()).thenReturn(true); - when(executionJobVertex.getOperatorIDs()).thenReturn(jobVertexIDs); - when(executionJobVertex.getUserDefinedOperatorIDs()).thenReturn(Arrays.asList(new OperatorID[jobVertexIDs.size()])); - - return executionJobVertex; - } - - static ExecutionVertex mockExecutionVertex(ExecutionAttemptID attemptID) { - JobVertexID jobVertexID = new JobVertexID(); - return mockExecutionVertex( - attemptID, - jobVertexID, - Collections.singletonList(OperatorID.fromJobVertexID(jobVertexID)), - 1, - 1, - ExecutionState.RUNNING); - } - - private static ExecutionVertex mockExecutionVertex( - ExecutionAttemptID attemptID, - JobVertexID jobVertexID, - List jobVertexIDs, - int parallelism, - int maxParallelism, - ExecutionState state, - ExecutionState ... successiveStates) { - - ExecutionVertex vertex = mock(ExecutionVertex.class); - - final Execution exec = spy(new Execution( - mock(Executor.class), - vertex, - 1, - 1L, - 1L, - Time.milliseconds(500L) - )); - when(exec.getAttemptId()).thenReturn(attemptID); - when(exec.getState()).thenReturn(state, successiveStates); - - when(vertex.getJobvertexId()).thenReturn(jobVertexID); - when(vertex.getCurrentExecutionAttempt()).thenReturn(exec); - when(vertex.getTotalNumberOfParallelSubtasks()).thenReturn(parallelism); - when(vertex.getMaxParallelism()).thenReturn(maxParallelism); - - ExecutionJobVertex jobVertex = mock(ExecutionJobVertex.class); - when(jobVertex.getOperatorIDs()).thenReturn(jobVertexIDs); - - when(vertex.getJobVertex()).thenReturn(jobVertex); - - return vertex; - } - - static TaskStateSnapshot mockSubtaskState( - JobVertexID jobVertexID, - int index, - KeyGroupRange keyGroupRange) throws IOException { - - OperatorStateHandle partitionableState = generatePartitionableStateHandle(jobVertexID, index, 2, 8, false); - KeyGroupsStateHandle partitionedKeyGroupState = generateKeyGroupState(jobVertexID, keyGroupRange, false); - - TaskStateSnapshot subtaskStates = spy(new TaskStateSnapshot()); - OperatorSubtaskState subtaskState = spy(new OperatorSubtaskState( - partitionableState, null, partitionedKeyGroupState, null) - ); - - subtaskStates.putSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID), subtaskState); - - return subtaskStates; - } - - public static void verifyStateRestore( - JobVertexID jobVertexID, ExecutionJobVertex executionJobVertex, - List keyGroupPartitions) throws Exception { - - for (int i = 0; i < executionJobVertex.getParallelism(); i++) { - - JobManagerTaskRestore taskRestore = executionJobVertex.getTaskVertices()[i].getCurrentExecutionAttempt().getTaskRestore(); - Assert.assertEquals(1L, taskRestore.getRestoreCheckpointId()); - TaskStateSnapshot stateSnapshot = taskRestore.getTaskStateSnapshot(); - - OperatorSubtaskState operatorState = stateSnapshot.getSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID)); - - ChainedStateHandle expectedOpStateBackend = - generateChainedPartitionableStateHandle(jobVertexID, i, 2, 8, false); - - assertTrue(CommonTestUtils.isStreamContentEqual( - expectedOpStateBackend.get(0).openInputStream(), - operatorState.getManagedOperatorState().iterator().next().openInputStream())); - - KeyGroupsStateHandle expectPartitionedKeyGroupState = generateKeyGroupState( - jobVertexID, keyGroupPartitions.get(i), false); - compareKeyedState(Collections.singletonList(expectPartitionedKeyGroupState), operatorState.getManagedKeyedState()); - } - } - - public static void compareKeyedState( - Collection expectPartitionedKeyGroupState, - Collection actualPartitionedKeyGroupState) throws Exception { - - KeyGroupsStateHandle expectedHeadOpKeyGroupStateHandle = expectPartitionedKeyGroupState.iterator().next(); - int expectedTotalKeyGroups = expectedHeadOpKeyGroupStateHandle.getKeyGroupRange().getNumberOfKeyGroups(); - int actualTotalKeyGroups = 0; - for(KeyedStateHandle keyedStateHandle: actualPartitionedKeyGroupState) { - assertTrue(keyedStateHandle instanceof KeyGroupsStateHandle); - - actualTotalKeyGroups += keyedStateHandle.getKeyGroupRange().getNumberOfKeyGroups(); - } - - assertEquals(expectedTotalKeyGroups, actualTotalKeyGroups); - - try (FSDataInputStream inputStream = expectedHeadOpKeyGroupStateHandle.openInputStream()) { - for (int groupId : expectedHeadOpKeyGroupStateHandle.getKeyGroupRange()) { - long offset = expectedHeadOpKeyGroupStateHandle.getOffsetForKeyGroup(groupId); - inputStream.seek(offset); - int expectedKeyGroupState = - InstantiationUtil.deserializeObject(inputStream, Thread.currentThread().getContextClassLoader()); - for (KeyedStateHandle oneActualKeyedStateHandle : actualPartitionedKeyGroupState) { - - assertTrue(oneActualKeyedStateHandle instanceof KeyGroupsStateHandle); - - KeyGroupsStateHandle oneActualKeyGroupStateHandle = (KeyGroupsStateHandle) oneActualKeyedStateHandle; - if (oneActualKeyGroupStateHandle.getKeyGroupRange().contains(groupId)) { - long actualOffset = oneActualKeyGroupStateHandle.getOffsetForKeyGroup(groupId); - try (FSDataInputStream actualInputStream = oneActualKeyGroupStateHandle.openInputStream()) { - actualInputStream.seek(actualOffset); - int actualGroupState = InstantiationUtil. - deserializeObject(actualInputStream, Thread.currentThread().getContextClassLoader()); - assertEquals(expectedKeyGroupState, actualGroupState); - } - } - } - } - } - } - - public static void comparePartitionableState( - List> expected, - List>> actual) throws Exception { - - List expectedResult = new ArrayList<>(); - for (ChainedStateHandle chainedStateHandle : expected) { - for (int i = 0; i < chainedStateHandle.getLength(); ++i) { - OperatorStateHandle operatorStateHandle = chainedStateHandle.get(i); - collectResult(i, operatorStateHandle, expectedResult); - } - } - Collections.sort(expectedResult); - - List actualResult = new ArrayList<>(); - for (List> collectionList : actual) { - if (collectionList != null) { - for (int i = 0; i < collectionList.size(); ++i) { - Collection stateHandles = collectionList.get(i); - Assert.assertNotNull(stateHandles); - for (OperatorStateHandle operatorStateHandle : stateHandles) { - collectResult(i, operatorStateHandle, actualResult); - } - } - } - } - - Collections.sort(actualResult); - Assert.assertEquals(expectedResult, actualResult); - } - - private static void collectResult(int opIdx, OperatorStateHandle operatorStateHandle, List resultCollector) throws Exception { - try (FSDataInputStream in = operatorStateHandle.openInputStream()) { - for (Map.Entry entry : operatorStateHandle.getStateNameToPartitionOffsets().entrySet()) { - for (long offset : entry.getValue().getOffsets()) { - in.seek(offset); - Integer state = InstantiationUtil. - deserializeObject(in, Thread.currentThread().getContextClassLoader()); - resultCollector.add(opIdx + " : " + entry.getKey() + " : " + state); - } - } - } - } - - @Test public void testCreateKeyGroupPartitions() { testCreateKeyGroupPartitions(1, 1); @@ -4071,36 +3694,4 @@ private void performIncrementalCheckpoint( coord.receiveAcknowledgeMessage(acknowledgeCheckpoint, TASK_MANAGER_LOCATION_INFO); } } - - private Execution mockExecution() { - Execution mock = mock(Execution.class); - when(mock.getAttemptId()).thenReturn(new ExecutionAttemptID()); - when(mock.getState()).thenReturn(ExecutionState.RUNNING); - return mock; - } - - private ExecutionVertex mockExecutionVertex(Execution execution, JobVertexID vertexId, int subtask, int parallelism) { - ExecutionVertex mock = mock(ExecutionVertex.class); - when(mock.getJobvertexId()).thenReturn(vertexId); - when(mock.getParallelSubtaskIndex()).thenReturn(subtask); - when(mock.getCurrentExecutionAttempt()).thenReturn(execution); - when(mock.getTotalNumberOfParallelSubtasks()).thenReturn(parallelism); - when(mock.getMaxParallelism()).thenReturn(parallelism); - return mock; - } - - private ExecutionJobVertex mockExecutionJobVertex(JobVertexID id, ExecutionVertex[] vertices) { - ExecutionJobVertex vertex = mock(ExecutionJobVertex.class); - when(vertex.getParallelism()).thenReturn(vertices.length); - when(vertex.getMaxParallelism()).thenReturn(vertices.length); - when(vertex.getJobVertexId()).thenReturn(id); - when(vertex.getTaskVertices()).thenReturn(vertices); - when(vertex.getOperatorIDs()).thenReturn(Collections.singletonList(OperatorID.fromJobVertexID(id))); - when(vertex.getUserDefinedOperatorIDs()).thenReturn(Collections.singletonList(null)); - - for (ExecutionVertex v : vertices) { - when(v.getJobVertex()).thenReturn(vertex); - } - return vertex; - } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java new file mode 100644 index 000000000000..9578ac74a66d --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java @@ -0,0 +1,472 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.checkpoint; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.Execution; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.state.ChainedStateHandle; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyGroupRangeOffsets; +import org.apache.flink.runtime.state.KeyGroupsStateHandle; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.OperatorStreamStateHandle; +import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; +import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.util.InstantiationUtil; +import org.apache.flink.util.Preconditions; + +import org.junit.Assert; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.UUID; +import java.util.concurrent.Executor; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +/** + * Testing utils for checkpoint coordinator. + */ +public class CheckpointCoordinatorTestingUtils { + + public static OperatorStateHandle generatePartitionableStateHandle( + JobVertexID jobVertexID, + int index, + int namedStates, + int partitionsPerState, + boolean rawState) throws IOException { + + Map> statesListsMap = new HashMap<>(namedStates); + + for (int i = 0; i < namedStates; ++i) { + List testStatesLists = new ArrayList<>(partitionsPerState); + // generate state + int seed = jobVertexID.hashCode() * index + i * namedStates; + if (rawState) { + seed = (seed + 1) * 31; + } + Random random = new Random(seed); + for (int j = 0; j < partitionsPerState; ++j) { + int simulatedStateValue = random.nextInt(); + testStatesLists.add(simulatedStateValue); + } + statesListsMap.put("state-" + i, testStatesLists); + } + + return generatePartitionableStateHandle(statesListsMap); + } + + static ChainedStateHandle generateChainedPartitionableStateHandle( + JobVertexID jobVertexID, + int index, + int namedStates, + int partitionsPerState, + boolean rawState) throws IOException { + + Map> statesListsMap = new HashMap<>(namedStates); + + for (int i = 0; i < namedStates; ++i) { + List testStatesLists = new ArrayList<>(partitionsPerState); + // generate state + int seed = jobVertexID.hashCode() * index + i * namedStates; + if (rawState) { + seed = (seed + 1) * 31; + } + Random random = new Random(seed); + for (int j = 0; j < partitionsPerState; ++j) { + int simulatedStateValue = random.nextInt(); + testStatesLists.add(simulatedStateValue); + } + statesListsMap.put("state-" + i, testStatesLists); + } + + return ChainedStateHandle.wrapSingleHandle(generatePartitionableStateHandle(statesListsMap)); + } + + static OperatorStateHandle generatePartitionableStateHandle( + Map> states) throws IOException { + + List> namedStateSerializables = new ArrayList<>(states.size()); + + for (Map.Entry> entry : states.entrySet()) { + namedStateSerializables.add(entry.getValue()); + } + + Tuple2> serializationWithOffsets = serializeTogetherAndTrackOffsets(namedStateSerializables); + + Map offsetsMap = new HashMap<>(states.size()); + + int idx = 0; + for (Map.Entry> entry : states.entrySet()) { + offsetsMap.put( + entry.getKey(), + new OperatorStateHandle.StateMetaInfo( + serializationWithOffsets.f1.get(idx), + OperatorStateHandle.Mode.SPLIT_DISTRIBUTE)); + ++idx; + } + + ByteStreamStateHandle streamStateHandle = new ByteStreamStateHandle( + String.valueOf(UUID.randomUUID()), + serializationWithOffsets.f0); + + return new OperatorStreamStateHandle(offsetsMap, streamStateHandle); + } + + static Tuple2> serializeTogetherAndTrackOffsets( + List> serializables) throws IOException { + + List offsets = new ArrayList<>(serializables.size()); + List serializedGroupValues = new ArrayList<>(); + + int runningGroupsOffset = 0; + for (List list : serializables) { + + long[] currentOffsets = new long[list.size()]; + offsets.add(currentOffsets); + + for (int i = 0; i < list.size(); ++i) { + currentOffsets[i] = runningGroupsOffset; + byte[] serializedValue = InstantiationUtil.serializeObject(list.get(i)); + serializedGroupValues.add(serializedValue); + runningGroupsOffset += serializedValue.length; + } + } + + //write all generated values in a single byte array, which is index by groupOffsetsInFinalByteArray + byte[] allSerializedValuesConcatenated = new byte[runningGroupsOffset]; + runningGroupsOffset = 0; + for (byte[] serializedGroupValue : serializedGroupValues) { + System.arraycopy( + serializedGroupValue, + 0, + allSerializedValuesConcatenated, + runningGroupsOffset, + serializedGroupValue.length); + runningGroupsOffset += serializedGroupValue.length; + } + return new Tuple2<>(allSerializedValuesConcatenated, offsets); + } + + public static void verifyStateRestore( + JobVertexID jobVertexID, ExecutionJobVertex executionJobVertex, + List keyGroupPartitions) throws Exception { + + for (int i = 0; i < executionJobVertex.getParallelism(); i++) { + + JobManagerTaskRestore taskRestore = executionJobVertex.getTaskVertices()[i].getCurrentExecutionAttempt().getTaskRestore(); + Assert.assertEquals(1L, taskRestore.getRestoreCheckpointId()); + TaskStateSnapshot stateSnapshot = taskRestore.getTaskStateSnapshot(); + + OperatorSubtaskState operatorState = stateSnapshot.getSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID)); + + ChainedStateHandle expectedOpStateBackend = + generateChainedPartitionableStateHandle(jobVertexID, i, 2, 8, false); + + assertTrue(CommonTestUtils.isStreamContentEqual( + expectedOpStateBackend.get(0).openInputStream(), + operatorState.getManagedOperatorState().iterator().next().openInputStream())); + + KeyGroupsStateHandle expectPartitionedKeyGroupState = generateKeyGroupState( + jobVertexID, keyGroupPartitions.get(i), false); + compareKeyedState(Collections.singletonList(expectPartitionedKeyGroupState), operatorState.getManagedKeyedState()); + } + } + + static void compareKeyedState( + Collection expectPartitionedKeyGroupState, + Collection actualPartitionedKeyGroupState) throws Exception { + + KeyGroupsStateHandle expectedHeadOpKeyGroupStateHandle = expectPartitionedKeyGroupState.iterator().next(); + int expectedTotalKeyGroups = expectedHeadOpKeyGroupStateHandle.getKeyGroupRange().getNumberOfKeyGroups(); + int actualTotalKeyGroups = 0; + for (KeyedStateHandle keyedStateHandle: actualPartitionedKeyGroupState) { + assertTrue(keyedStateHandle instanceof KeyGroupsStateHandle); + + actualTotalKeyGroups += keyedStateHandle.getKeyGroupRange().getNumberOfKeyGroups(); + } + + assertEquals(expectedTotalKeyGroups, actualTotalKeyGroups); + + try (FSDataInputStream inputStream = expectedHeadOpKeyGroupStateHandle.openInputStream()) { + for (int groupId : expectedHeadOpKeyGroupStateHandle.getKeyGroupRange()) { + long offset = expectedHeadOpKeyGroupStateHandle.getOffsetForKeyGroup(groupId); + inputStream.seek(offset); + int expectedKeyGroupState = + InstantiationUtil.deserializeObject(inputStream, Thread.currentThread().getContextClassLoader()); + for (KeyedStateHandle oneActualKeyedStateHandle : actualPartitionedKeyGroupState) { + + assertTrue(oneActualKeyedStateHandle instanceof KeyGroupsStateHandle); + + KeyGroupsStateHandle oneActualKeyGroupStateHandle = (KeyGroupsStateHandle) oneActualKeyedStateHandle; + if (oneActualKeyGroupStateHandle.getKeyGroupRange().contains(groupId)) { + long actualOffset = oneActualKeyGroupStateHandle.getOffsetForKeyGroup(groupId); + try (FSDataInputStream actualInputStream = oneActualKeyGroupStateHandle.openInputStream()) { + actualInputStream.seek(actualOffset); + int actualGroupState = InstantiationUtil. + deserializeObject(actualInputStream, Thread.currentThread().getContextClassLoader()); + assertEquals(expectedKeyGroupState, actualGroupState); + } + } + } + } + } + } + + static void comparePartitionableState( + List> expected, + List>> actual) throws Exception { + + List expectedResult = new ArrayList<>(); + for (ChainedStateHandle chainedStateHandle : expected) { + for (int i = 0; i < chainedStateHandle.getLength(); ++i) { + OperatorStateHandle operatorStateHandle = chainedStateHandle.get(i); + collectResult(i, operatorStateHandle, expectedResult); + } + } + Collections.sort(expectedResult); + + List actualResult = new ArrayList<>(); + for (List> collectionList : actual) { + if (collectionList != null) { + for (int i = 0; i < collectionList.size(); ++i) { + Collection stateHandles = collectionList.get(i); + Assert.assertNotNull(stateHandles); + for (OperatorStateHandle operatorStateHandle : stateHandles) { + collectResult(i, operatorStateHandle, actualResult); + } + } + } + } + + Collections.sort(actualResult); + Assert.assertEquals(expectedResult, actualResult); + } + + static void collectResult(int opIdx, OperatorStateHandle operatorStateHandle, List resultCollector) throws Exception { + try (FSDataInputStream in = operatorStateHandle.openInputStream()) { + for (Map.Entry entry : operatorStateHandle.getStateNameToPartitionOffsets().entrySet()) { + for (long offset : entry.getValue().getOffsets()) { + in.seek(offset); + Integer state = InstantiationUtil. + deserializeObject(in, Thread.currentThread().getContextClassLoader()); + resultCollector.add(opIdx + " : " + entry.getKey() + " : " + state); + } + } + } + } + + static ExecutionJobVertex mockExecutionJobVertex( + JobVertexID jobVertexID, + int parallelism, + int maxParallelism) { + + return mockExecutionJobVertex( + jobVertexID, + Collections.singletonList(OperatorID.fromJobVertexID(jobVertexID)), + parallelism, + maxParallelism + ); + } + + static ExecutionJobVertex mockExecutionJobVertex( + JobVertexID jobVertexID, + List jobVertexIDs, + int parallelism, + int maxParallelism) { + final ExecutionJobVertex executionJobVertex = mock(ExecutionJobVertex.class); + + ExecutionVertex[] executionVertices = new ExecutionVertex[parallelism]; + + for (int i = 0; i < parallelism; i++) { + executionVertices[i] = mockExecutionVertex( + new ExecutionAttemptID(), + jobVertexID, + jobVertexIDs, + parallelism, + maxParallelism, + ExecutionState.RUNNING); + + when(executionVertices[i].getParallelSubtaskIndex()).thenReturn(i); + } + + when(executionJobVertex.getJobVertexId()).thenReturn(jobVertexID); + when(executionJobVertex.getTaskVertices()).thenReturn(executionVertices); + when(executionJobVertex.getParallelism()).thenReturn(parallelism); + when(executionJobVertex.getMaxParallelism()).thenReturn(maxParallelism); + when(executionJobVertex.isMaxParallelismConfigured()).thenReturn(true); + when(executionJobVertex.getOperatorIDs()).thenReturn(jobVertexIDs); + when(executionJobVertex.getUserDefinedOperatorIDs()).thenReturn(Arrays.asList(new OperatorID[jobVertexIDs.size()])); + + return executionJobVertex; + } + + static ExecutionVertex mockExecutionVertex(ExecutionAttemptID attemptID) { + JobVertexID jobVertexID = new JobVertexID(); + return mockExecutionVertex( + attemptID, + jobVertexID, + Collections.singletonList(OperatorID.fromJobVertexID(jobVertexID)), + 1, + 1, + ExecutionState.RUNNING); + } + + static ExecutionVertex mockExecutionVertex( + ExecutionAttemptID attemptID, + JobVertexID jobVertexID, + List jobVertexIDs, + int parallelism, + int maxParallelism, + ExecutionState state, + ExecutionState ... successiveStates) { + + ExecutionVertex vertex = mock(ExecutionVertex.class); + + final Execution exec = spy(new Execution( + mock(Executor.class), + vertex, + 1, + 1L, + 1L, + Time.milliseconds(500L) + )); + when(exec.getAttemptId()).thenReturn(attemptID); + when(exec.getState()).thenReturn(state, successiveStates); + + when(vertex.getJobvertexId()).thenReturn(jobVertexID); + when(vertex.getCurrentExecutionAttempt()).thenReturn(exec); + when(vertex.getTotalNumberOfParallelSubtasks()).thenReturn(parallelism); + when(vertex.getMaxParallelism()).thenReturn(maxParallelism); + + ExecutionJobVertex jobVertex = mock(ExecutionJobVertex.class); + when(jobVertex.getOperatorIDs()).thenReturn(jobVertexIDs); + + when(vertex.getJobVertex()).thenReturn(jobVertex); + + return vertex; + } + + static TaskStateSnapshot mockSubtaskState( + JobVertexID jobVertexID, + int index, + KeyGroupRange keyGroupRange) throws IOException { + + OperatorStateHandle partitionableState = generatePartitionableStateHandle(jobVertexID, index, 2, 8, false); + KeyGroupsStateHandle partitionedKeyGroupState = generateKeyGroupState(jobVertexID, keyGroupRange, false); + + TaskStateSnapshot subtaskStates = spy(new TaskStateSnapshot()); + OperatorSubtaskState subtaskState = spy(new OperatorSubtaskState( + partitionableState, null, partitionedKeyGroupState, null) + ); + + subtaskStates.putSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID), subtaskState); + + return subtaskStates; + } + + public static KeyGroupsStateHandle generateKeyGroupState( + JobVertexID jobVertexID, + KeyGroupRange keyGroupPartition, boolean rawState) throws IOException { + + List testStatesLists = new ArrayList<>(keyGroupPartition.getNumberOfKeyGroups()); + + // generate state for one keygroup + for (int keyGroupIndex : keyGroupPartition) { + int vertexHash = jobVertexID.hashCode(); + int seed = rawState ? (vertexHash * (31 + keyGroupIndex)) : (vertexHash + keyGroupIndex); + Random random = new Random(seed); + int simulatedStateValue = random.nextInt(); + testStatesLists.add(simulatedStateValue); + } + + return generateKeyGroupState(keyGroupPartition, testStatesLists); + } + + public static KeyGroupsStateHandle generateKeyGroupState( + KeyGroupRange keyGroupRange, + List states) throws IOException { + + Preconditions.checkArgument(keyGroupRange.getNumberOfKeyGroups() == states.size()); + + Tuple2> serializedDataWithOffsets = + serializeTogetherAndTrackOffsets(Collections.>singletonList(states)); + + KeyGroupRangeOffsets keyGroupRangeOffsets = new KeyGroupRangeOffsets(keyGroupRange, serializedDataWithOffsets.f1.get(0)); + + ByteStreamStateHandle allSerializedStatesHandle = new ByteStreamStateHandle( + String.valueOf(UUID.randomUUID()), + serializedDataWithOffsets.f0); + + return new KeyGroupsStateHandle(keyGroupRangeOffsets, allSerializedStatesHandle); + } + + static Execution mockExecution() { + Execution mock = mock(Execution.class); + when(mock.getAttemptId()).thenReturn(new ExecutionAttemptID()); + when(mock.getState()).thenReturn(ExecutionState.RUNNING); + return mock; + } + + static ExecutionVertex mockExecutionVertex(Execution execution, JobVertexID vertexId, int subtask, int parallelism) { + ExecutionVertex mock = mock(ExecutionVertex.class); + when(mock.getJobvertexId()).thenReturn(vertexId); + when(mock.getParallelSubtaskIndex()).thenReturn(subtask); + when(mock.getCurrentExecutionAttempt()).thenReturn(execution); + when(mock.getTotalNumberOfParallelSubtasks()).thenReturn(parallelism); + when(mock.getMaxParallelism()).thenReturn(parallelism); + return mock; + } + + static ExecutionJobVertex mockExecutionJobVertex(JobVertexID id, ExecutionVertex[] vertices) { + ExecutionJobVertex vertex = mock(ExecutionJobVertex.class); + when(vertex.getParallelism()).thenReturn(vertices.length); + when(vertex.getMaxParallelism()).thenReturn(vertices.length); + when(vertex.getJobVertexId()).thenReturn(id); + when(vertex.getTaskVertices()).thenReturn(vertices); + when(vertex.getOperatorIDs()).thenReturn(Collections.singletonList(OperatorID.fromJobVertexID(id))); + when(vertex.getUserDefinedOperatorIDs()).thenReturn(Collections.singletonList(null)); + + for (ExecutionVertex v : vertices) { + when(v.getJobVertex()).thenReturn(vertex); + } + return vertex; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java index af2e5a37523e..42849b5e6e48 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java @@ -79,9 +79,9 @@ public void setUp() throws Exception { public void testSetState() { try { - KeyGroupRange keyGroupRange = KeyGroupRange.of(0,0); + KeyGroupRange keyGroupRange = KeyGroupRange.of(0, 0); List testStates = Collections.singletonList(new SerializableObject()); - final KeyedStateHandle serializedKeyGroupStates = CheckpointCoordinatorTest.generateKeyGroupState(keyGroupRange, testStates); + final KeyedStateHandle serializedKeyGroupStates = CheckpointCoordinatorTestingUtils.generateKeyGroupState(keyGroupRange, testStates); final JobID jid = new JobID(); final JobVertexID statefulId = new JobVertexID(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/messages/CheckpointMessagesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/messages/CheckpointMessagesTest.java index f18b4c81b5ad..0fb46e0c230c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/messages/CheckpointMessagesTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/messages/CheckpointMessagesTest.java @@ -21,7 +21,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.core.fs.FSDataInputStream; import org.apache.flink.core.testutils.CommonTestUtils; -import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTest; +import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; @@ -45,6 +45,9 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.fail; +/** + * Tests for checkpoint messages. + */ public class CheckpointMessagesTest { @Test @@ -69,15 +72,15 @@ public void testConfirmTaskCheckpointed() { AcknowledgeCheckpoint noState = new AcknowledgeCheckpoint( new JobID(), new ExecutionAttemptID(), 569345L); - KeyGroupRange keyGroupRange = KeyGroupRange.of(42,42); + KeyGroupRange keyGroupRange = KeyGroupRange.of(42, 42); TaskStateSnapshot checkpointStateHandles = new TaskStateSnapshot(); checkpointStateHandles.putSubtaskStateByOperatorID( new OperatorID(), new OperatorSubtaskState( - CheckpointCoordinatorTest.generatePartitionableStateHandle(new JobVertexID(), 0, 2, 8, false), + CheckpointCoordinatorTestingUtils.generatePartitionableStateHandle(new JobVertexID(), 0, 2, 8, false), null, - CheckpointCoordinatorTest.generateKeyGroupState(keyGroupRange, Collections.singletonList(new MyHandle())), + CheckpointCoordinatorTestingUtils.generateKeyGroupState(keyGroupRange, Collections.singletonList(new MyHandle())), null ) ); @@ -105,7 +108,7 @@ private static void testSerializabilityEqualsHashCode(Serializable o) throws IOE assertNotNull(copy.toString()); } - public static class MyHandle implements StreamStateHandle { + private static class MyHandle implements StreamStateHandle { private static final long serialVersionUID = 8128146204128728332L; From 22c32483f800da95fb2281ac58f287e866d0e96b Mon Sep 17 00:00:00 2001 From: ifndef-SleePy Date: Wed, 18 Sep 2019 19:25:12 +0800 Subject: [PATCH 315/746] [hotfix] Split too large file CheckpointCoordinatorTest.java into several small files --- .../CheckpointCoordinatorRestoringTest.java | 996 +++++++++++++ .../checkpoint/CheckpointCoordinatorTest.java | 1256 +---------------- .../CheckpointCoordinatorTriggeringTest.java | 308 ++++ 3 files changed, 1353 insertions(+), 1207 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java new file mode 100644 index 000000000000..1259144fba29 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java @@ -0,0 +1,996 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.checkpoint; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.concurrent.Executors; +import org.apache.flink.runtime.executiongraph.Execution; +import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; +import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; +import org.apache.flink.runtime.state.ChainedStateHandle; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyGroupsStateHandle; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.SharedStateRegistry; +import org.apache.flink.runtime.state.memory.MemoryStateBackend; +import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; +import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.runtime.testutils.RecoverableCompletedCheckpointStore; +import org.apache.flink.util.SerializableObject; +import org.apache.flink.util.TestLogger; + +import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables; + +import org.hamcrest.BaseMatcher; +import org.hamcrest.Description; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.mockito.Mockito; +import org.mockito.hamcrest.MockitoHamcrest; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.CompletableFuture; + +import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.compareKeyedState; +import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.comparePartitionableState; +import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.generateChainedPartitionableStateHandle; +import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.generateKeyGroupState; +import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.generatePartitionableStateHandle; +import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.mockExecution; +import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.mockExecutionJobVertex; +import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.mockExecutionVertex; +import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.mockSubtaskState; +import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.verifyStateRestore; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * Tests for restoring checkpoint. + */ +public class CheckpointCoordinatorRestoringTest extends TestLogger { + private static final String TASK_MANAGER_LOCATION_INFO = "Unknown location"; + + private CheckpointFailureManager failureManager; + + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + + @Before + public void setUp() throws Exception { + failureManager = new CheckpointFailureManager( + 0, + NoOpFailJobCall.INSTANCE); + } + + /** + * Tests that the checkpointed partitioned and non-partitioned state is assigned properly to + * the {@link Execution} upon recovery. + * + * @throws Exception + */ + @Test + public void testRestoreLatestCheckpointedState() throws Exception { + final JobID jid = new JobID(); + final long timestamp = System.currentTimeMillis(); + + final JobVertexID jobVertexID1 = new JobVertexID(); + final JobVertexID jobVertexID2 = new JobVertexID(); + int parallelism1 = 3; + int parallelism2 = 2; + int maxParallelism1 = 42; + int maxParallelism2 = 13; + + final ExecutionJobVertex jobVertex1 = mockExecutionJobVertex( + jobVertexID1, + parallelism1, + maxParallelism1); + final ExecutionJobVertex jobVertex2 = mockExecutionJobVertex( + jobVertexID2, + parallelism2, + maxParallelism2); + + List allExecutionVertices = new ArrayList<>(parallelism1 + parallelism2); + + allExecutionVertices.addAll(Arrays.asList(jobVertex1.getTaskVertices())); + allExecutionVertices.addAll(Arrays.asList(jobVertex2.getTaskVertices())); + + ExecutionVertex[] arrayExecutionVertices = + allExecutionVertices.toArray(new ExecutionVertex[allExecutionVertices.size()]); + + CompletedCheckpointStore store = new RecoverableCompletedCheckpointStore(); + + // set up the coordinator and validate the initial state + CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( + 600000, + 600000, + 0, + Integer.MAX_VALUE, + CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION, + true, + false, + 0); + CheckpointCoordinator coord = new CheckpointCoordinator( + jid, + chkConfig, + arrayExecutionVertices, + arrayExecutionVertices, + arrayExecutionVertices, + new StandaloneCheckpointIDCounter(), + store, + new MemoryStateBackend(), + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY, + failureManager); + + // trigger the checkpoint + coord.triggerCheckpoint(timestamp, false); + + assertTrue(coord.getPendingCheckpoints().keySet().size() == 1); + long checkpointId = Iterables.getOnlyElement(coord.getPendingCheckpoints().keySet()); + + List keyGroupPartitions1 = StateAssignmentOperation.createKeyGroupPartitions(maxParallelism1, parallelism1); + List keyGroupPartitions2 = StateAssignmentOperation.createKeyGroupPartitions(maxParallelism2, parallelism2); + + for (int index = 0; index < jobVertex1.getParallelism(); index++) { + TaskStateSnapshot subtaskState = mockSubtaskState(jobVertexID1, index, keyGroupPartitions1.get(index)); + + AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint( + jid, + jobVertex1.getTaskVertices()[index].getCurrentExecutionAttempt().getAttemptId(), + checkpointId, + new CheckpointMetrics(), + subtaskState); + + coord.receiveAcknowledgeMessage(acknowledgeCheckpoint, TASK_MANAGER_LOCATION_INFO); + } + + for (int index = 0; index < jobVertex2.getParallelism(); index++) { + TaskStateSnapshot subtaskState = mockSubtaskState(jobVertexID2, index, keyGroupPartitions2.get(index)); + + AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint( + jid, + jobVertex2.getTaskVertices()[index].getCurrentExecutionAttempt().getAttemptId(), + checkpointId, + new CheckpointMetrics(), + subtaskState); + + coord.receiveAcknowledgeMessage(acknowledgeCheckpoint, TASK_MANAGER_LOCATION_INFO); + } + + List completedCheckpoints = coord.getSuccessfulCheckpoints(); + + assertEquals(1, completedCheckpoints.size()); + + // shutdown the store + store.shutdown(JobStatus.SUSPENDED); + + // restore the store + Map tasks = new HashMap<>(); + + tasks.put(jobVertexID1, jobVertex1); + tasks.put(jobVertexID2, jobVertex2); + + coord.restoreLatestCheckpointedState(tasks, true, false); + + // validate that all shared states are registered again after the recovery. + for (CompletedCheckpoint completedCheckpoint : completedCheckpoints) { + for (OperatorState taskState : completedCheckpoint.getOperatorStates().values()) { + for (OperatorSubtaskState subtaskState : taskState.getStates()) { + verify(subtaskState, times(2)).registerSharedStates(any(SharedStateRegistry.class)); + } + } + } + + // verify the restored state + verifyStateRestore(jobVertexID1, jobVertex1, keyGroupPartitions1); + verifyStateRestore(jobVertexID2, jobVertex2, keyGroupPartitions2); + } + + @Test + public void testRestoreLatestCheckpointedStateScaleIn() throws Exception { + testRestoreLatestCheckpointedStateWithChangingParallelism(false); + } + + @Test + public void testRestoreLatestCheckpointedStateScaleOut() throws Exception { + testRestoreLatestCheckpointedStateWithChangingParallelism(true); + } + + @Test + public void testRestoreLatestCheckpointWhenPreferCheckpoint() throws Exception { + testRestoreLatestCheckpointIsPreferSavepoint(true); + } + + @Test + public void testRestoreLatestCheckpointWhenPreferSavepoint() throws Exception { + testRestoreLatestCheckpointIsPreferSavepoint(false); + } + + private void testRestoreLatestCheckpointIsPreferSavepoint(boolean isPreferCheckpoint) { + try { + final JobID jid = new JobID(); + long timestamp = System.currentTimeMillis(); + StandaloneCheckpointIDCounter checkpointIDCounter = new StandaloneCheckpointIDCounter(); + + final JobVertexID statefulId = new JobVertexID(); + final JobVertexID statelessId = new JobVertexID(); + + Execution statefulExec1 = mockExecution(); + Execution statelessExec1 = mockExecution(); + + ExecutionVertex stateful1 = mockExecutionVertex(statefulExec1, statefulId, 0, 1); + ExecutionVertex stateless1 = mockExecutionVertex(statelessExec1, statelessId, 0, 1); + + ExecutionJobVertex stateful = mockExecutionJobVertex(statefulId, + new ExecutionVertex[] { stateful1 }); + ExecutionJobVertex stateless = mockExecutionJobVertex(statelessId, + new ExecutionVertex[] { stateless1 }); + + Map map = new HashMap(); + map.put(statefulId, stateful); + map.put(statelessId, stateless); + + CompletedCheckpointStore store = new RecoverableCompletedCheckpointStore(2); + + CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( + 600000, + 600000, + 0, + Integer.MAX_VALUE, + CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION, + true, + isPreferCheckpoint, + 0); + CheckpointCoordinator coord = new CheckpointCoordinator( + jid, + chkConfig, + new ExecutionVertex[] { stateful1, stateless1 }, + new ExecutionVertex[] { stateful1, stateless1 }, + new ExecutionVertex[] { stateful1, stateless1 }, + checkpointIDCounter, + store, + new MemoryStateBackend(), + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY, + failureManager); + + //trigger a checkpoint and wait to become a completed checkpoint + assertTrue(coord.triggerCheckpoint(timestamp, false)); + + long checkpointId = checkpointIDCounter.getLast(); + + KeyGroupRange keyGroupRange = KeyGroupRange.of(0, 0); + List testStates = Collections.singletonList(new SerializableObject()); + KeyedStateHandle serializedKeyGroupStates = generateKeyGroupState(keyGroupRange, testStates); + + TaskStateSnapshot subtaskStatesForCheckpoint = new TaskStateSnapshot(); + + subtaskStatesForCheckpoint.putSubtaskStateByOperatorID( + OperatorID.fromJobVertexID(statefulId), + new OperatorSubtaskState( + StateObjectCollection.empty(), + StateObjectCollection.empty(), + StateObjectCollection.singleton(serializedKeyGroupStates), + StateObjectCollection.empty())); + + coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statefulExec1.getAttemptId(), checkpointId, new CheckpointMetrics(), subtaskStatesForCheckpoint), TASK_MANAGER_LOCATION_INFO); + coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statelessExec1.getAttemptId(), checkpointId), TASK_MANAGER_LOCATION_INFO); + + CompletedCheckpoint success = coord.getSuccessfulCheckpoints().get(0); + assertEquals(jid, success.getJobId()); + + // trigger a savepoint and wait it to be finished + String savepointDir = tmpFolder.newFolder().getAbsolutePath(); + timestamp = System.currentTimeMillis(); + CompletableFuture savepointFuture = coord.triggerSavepoint(timestamp, savepointDir); + + KeyGroupRange keyGroupRangeForSavepoint = KeyGroupRange.of(1, 1); + List testStatesForSavepoint = Collections.singletonList(new SerializableObject()); + KeyedStateHandle serializedKeyGroupStatesForSavepoint = generateKeyGroupState(keyGroupRangeForSavepoint, testStatesForSavepoint); + + TaskStateSnapshot subtaskStatesForSavepoint = new TaskStateSnapshot(); + + subtaskStatesForSavepoint.putSubtaskStateByOperatorID( + OperatorID.fromJobVertexID(statefulId), + new OperatorSubtaskState( + StateObjectCollection.empty(), + StateObjectCollection.empty(), + StateObjectCollection.singleton(serializedKeyGroupStatesForSavepoint), + StateObjectCollection.empty())); + + checkpointId = checkpointIDCounter.getLast(); + coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statefulExec1.getAttemptId(), checkpointId, new CheckpointMetrics(), subtaskStatesForSavepoint), TASK_MANAGER_LOCATION_INFO); + coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statelessExec1.getAttemptId(), checkpointId), TASK_MANAGER_LOCATION_INFO); + + assertTrue(savepointFuture.isDone()); + + //restore and jump the latest savepoint + coord.restoreLatestCheckpointedState(map, true, false); + + //compare and see if it used the checkpoint's subtaskStates + BaseMatcher matcher = new BaseMatcher() { + @Override + public boolean matches(Object o) { + if (o instanceof JobManagerTaskRestore) { + JobManagerTaskRestore taskRestore = (JobManagerTaskRestore) o; + if (isPreferCheckpoint) { + return Objects.equals(taskRestore.getTaskStateSnapshot(), subtaskStatesForCheckpoint); + } else { + return Objects.equals(taskRestore.getTaskStateSnapshot(), subtaskStatesForSavepoint); + } + } + return false; + } + + @Override + public void describeTo(Description description) { + if (isPreferCheckpoint) { + description.appendValue(subtaskStatesForCheckpoint); + } else { + description.appendValue(subtaskStatesForSavepoint); + } + } + }; + + verify(statefulExec1, times(1)).setInitialState(MockitoHamcrest.argThat(matcher)); + verify(statelessExec1, times(0)).setInitialState(Mockito.any()); + + coord.shutdown(JobStatus.FINISHED); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + /** + * Tests the checkpoint restoration with changing parallelism of job vertex with partitioned + * state. + * + * @throws Exception + */ + private void testRestoreLatestCheckpointedStateWithChangingParallelism(boolean scaleOut) throws Exception { + final JobID jid = new JobID(); + final long timestamp = System.currentTimeMillis(); + + final JobVertexID jobVertexID1 = new JobVertexID(); + final JobVertexID jobVertexID2 = new JobVertexID(); + int parallelism1 = 3; + int parallelism2 = scaleOut ? 2 : 13; + + int maxParallelism1 = 42; + int maxParallelism2 = 13; + + int newParallelism2 = scaleOut ? 13 : 2; + + final ExecutionJobVertex jobVertex1 = mockExecutionJobVertex( + jobVertexID1, + parallelism1, + maxParallelism1); + final ExecutionJobVertex jobVertex2 = mockExecutionJobVertex( + jobVertexID2, + parallelism2, + maxParallelism2); + + List allExecutionVertices = new ArrayList<>(parallelism1 + parallelism2); + + allExecutionVertices.addAll(Arrays.asList(jobVertex1.getTaskVertices())); + allExecutionVertices.addAll(Arrays.asList(jobVertex2.getTaskVertices())); + + ExecutionVertex[] arrayExecutionVertices = + allExecutionVertices.toArray(new ExecutionVertex[allExecutionVertices.size()]); + + // set up the coordinator and validate the initial state + CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( + 600000, + 600000, + 0, + Integer.MAX_VALUE, + CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION, + true, + false, + 0); + CheckpointCoordinator coord = new CheckpointCoordinator( + jid, + chkConfig, + arrayExecutionVertices, + arrayExecutionVertices, + arrayExecutionVertices, + new StandaloneCheckpointIDCounter(), + new StandaloneCompletedCheckpointStore(1), + new MemoryStateBackend(), + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY, + failureManager); + + // trigger the checkpoint + coord.triggerCheckpoint(timestamp, false); + + assertTrue(coord.getPendingCheckpoints().keySet().size() == 1); + long checkpointId = Iterables.getOnlyElement(coord.getPendingCheckpoints().keySet()); + + List keyGroupPartitions1 = + StateAssignmentOperation.createKeyGroupPartitions(maxParallelism1, parallelism1); + List keyGroupPartitions2 = + StateAssignmentOperation.createKeyGroupPartitions(maxParallelism2, parallelism2); + + //vertex 1 + for (int index = 0; index < jobVertex1.getParallelism(); index++) { + OperatorStateHandle opStateBackend = generatePartitionableStateHandle(jobVertexID1, index, 2, 8, false); + KeyGroupsStateHandle keyedStateBackend = generateKeyGroupState(jobVertexID1, keyGroupPartitions1.get(index), false); + KeyGroupsStateHandle keyedStateRaw = generateKeyGroupState(jobVertexID1, keyGroupPartitions1.get(index), true); + OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(opStateBackend, null, keyedStateBackend, keyedStateRaw); + TaskStateSnapshot taskOperatorSubtaskStates = new TaskStateSnapshot(); + taskOperatorSubtaskStates.putSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID1), operatorSubtaskState); + + AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint( + jid, + jobVertex1.getTaskVertices()[index].getCurrentExecutionAttempt().getAttemptId(), + checkpointId, + new CheckpointMetrics(), + taskOperatorSubtaskStates); + + coord.receiveAcknowledgeMessage(acknowledgeCheckpoint, TASK_MANAGER_LOCATION_INFO); + } + + //vertex 2 + final List> expectedOpStatesBackend = new ArrayList<>(jobVertex2.getParallelism()); + final List> expectedOpStatesRaw = new ArrayList<>(jobVertex2.getParallelism()); + for (int index = 0; index < jobVertex2.getParallelism(); index++) { + KeyGroupsStateHandle keyedStateBackend = generateKeyGroupState(jobVertexID2, keyGroupPartitions2.get(index), false); + KeyGroupsStateHandle keyedStateRaw = generateKeyGroupState(jobVertexID2, keyGroupPartitions2.get(index), true); + OperatorStateHandle opStateBackend = generatePartitionableStateHandle(jobVertexID2, index, 2, 8, false); + OperatorStateHandle opStateRaw = generatePartitionableStateHandle(jobVertexID2, index, 2, 8, true); + expectedOpStatesBackend.add(new ChainedStateHandle<>(Collections.singletonList(opStateBackend))); + expectedOpStatesRaw.add(new ChainedStateHandle<>(Collections.singletonList(opStateRaw))); + + OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(opStateBackend, opStateRaw, keyedStateBackend, keyedStateRaw); + TaskStateSnapshot taskOperatorSubtaskStates = new TaskStateSnapshot(); + taskOperatorSubtaskStates.putSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID2), operatorSubtaskState); + + AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint( + jid, + jobVertex2.getTaskVertices()[index].getCurrentExecutionAttempt().getAttemptId(), + checkpointId, + new CheckpointMetrics(), + taskOperatorSubtaskStates); + + coord.receiveAcknowledgeMessage(acknowledgeCheckpoint, TASK_MANAGER_LOCATION_INFO); + } + + List completedCheckpoints = coord.getSuccessfulCheckpoints(); + + assertEquals(1, completedCheckpoints.size()); + + Map tasks = new HashMap<>(); + + List newKeyGroupPartitions2 = + StateAssignmentOperation.createKeyGroupPartitions(maxParallelism2, newParallelism2); + + final ExecutionJobVertex newJobVertex1 = mockExecutionJobVertex( + jobVertexID1, + parallelism1, + maxParallelism1); + + // rescale vertex 2 + final ExecutionJobVertex newJobVertex2 = mockExecutionJobVertex( + jobVertexID2, + newParallelism2, + maxParallelism2); + + tasks.put(jobVertexID1, newJobVertex1); + tasks.put(jobVertexID2, newJobVertex2); + coord.restoreLatestCheckpointedState(tasks, true, false); + + // verify the restored state + verifyStateRestore(jobVertexID1, newJobVertex1, keyGroupPartitions1); + List>> actualOpStatesBackend = new ArrayList<>(newJobVertex2.getParallelism()); + List>> actualOpStatesRaw = new ArrayList<>(newJobVertex2.getParallelism()); + for (int i = 0; i < newJobVertex2.getParallelism(); i++) { + + List operatorIDs = newJobVertex2.getOperatorIDs(); + + KeyGroupsStateHandle originalKeyedStateBackend = generateKeyGroupState(jobVertexID2, newKeyGroupPartitions2.get(i), false); + KeyGroupsStateHandle originalKeyedStateRaw = generateKeyGroupState(jobVertexID2, newKeyGroupPartitions2.get(i), true); + + JobManagerTaskRestore taskRestore = newJobVertex2.getTaskVertices()[i].getCurrentExecutionAttempt().getTaskRestore(); + Assert.assertEquals(1L, taskRestore.getRestoreCheckpointId()); + TaskStateSnapshot taskStateHandles = taskRestore.getTaskStateSnapshot(); + + final int headOpIndex = operatorIDs.size() - 1; + List> allParallelManagedOpStates = new ArrayList<>(operatorIDs.size()); + List> allParallelRawOpStates = new ArrayList<>(operatorIDs.size()); + + for (int idx = 0; idx < operatorIDs.size(); ++idx) { + OperatorID operatorID = operatorIDs.get(idx); + OperatorSubtaskState opState = taskStateHandles.getSubtaskStateByOperatorID(operatorID); + Collection opStateBackend = opState.getManagedOperatorState(); + Collection opStateRaw = opState.getRawOperatorState(); + allParallelManagedOpStates.add(opStateBackend); + allParallelRawOpStates.add(opStateRaw); + if (idx == headOpIndex) { + Collection keyedStateBackend = opState.getManagedKeyedState(); + Collection keyGroupStateRaw = opState.getRawKeyedState(); + compareKeyedState(Collections.singletonList(originalKeyedStateBackend), keyedStateBackend); + compareKeyedState(Collections.singletonList(originalKeyedStateRaw), keyGroupStateRaw); + } + } + actualOpStatesBackend.add(allParallelManagedOpStates); + actualOpStatesRaw.add(allParallelRawOpStates); + } + + comparePartitionableState(expectedOpStatesBackend, actualOpStatesBackend); + comparePartitionableState(expectedOpStatesRaw, actualOpStatesRaw); + } + + /** + * Tests that the checkpoint restoration fails if the max parallelism of the job vertices has + * changed. + * + * @throws Exception + */ + @Test(expected = IllegalStateException.class) + public void testRestoreLatestCheckpointFailureWhenMaxParallelismChanges() throws Exception { + final JobID jid = new JobID(); + final long timestamp = System.currentTimeMillis(); + + final JobVertexID jobVertexID1 = new JobVertexID(); + final JobVertexID jobVertexID2 = new JobVertexID(); + int parallelism1 = 3; + int parallelism2 = 2; + int maxParallelism1 = 42; + int maxParallelism2 = 13; + + final ExecutionJobVertex jobVertex1 = mockExecutionJobVertex( + jobVertexID1, + parallelism1, + maxParallelism1); + final ExecutionJobVertex jobVertex2 = mockExecutionJobVertex( + jobVertexID2, + parallelism2, + maxParallelism2); + + List allExecutionVertices = new ArrayList<>(parallelism1 + parallelism2); + + allExecutionVertices.addAll(Arrays.asList(jobVertex1.getTaskVertices())); + allExecutionVertices.addAll(Arrays.asList(jobVertex2.getTaskVertices())); + + ExecutionVertex[] arrayExecutionVertices = allExecutionVertices.toArray(new ExecutionVertex[allExecutionVertices.size()]); + + // set up the coordinator and validate the initial state + CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( + 600000, + 600000, + 0, + Integer.MAX_VALUE, + CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION, + true, + false, + 0); + CheckpointCoordinator coord = new CheckpointCoordinator( + jid, + chkConfig, + arrayExecutionVertices, + arrayExecutionVertices, + arrayExecutionVertices, + new StandaloneCheckpointIDCounter(), + new StandaloneCompletedCheckpointStore(1), + new MemoryStateBackend(), + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY, + failureManager); + + // trigger the checkpoint + coord.triggerCheckpoint(timestamp, false); + + assertTrue(coord.getPendingCheckpoints().keySet().size() == 1); + long checkpointId = Iterables.getOnlyElement(coord.getPendingCheckpoints().keySet()); + + List keyGroupPartitions1 = StateAssignmentOperation.createKeyGroupPartitions(maxParallelism1, parallelism1); + List keyGroupPartitions2 = StateAssignmentOperation.createKeyGroupPartitions(maxParallelism2, parallelism2); + + for (int index = 0; index < jobVertex1.getParallelism(); index++) { + KeyGroupsStateHandle keyGroupState = generateKeyGroupState(jobVertexID1, keyGroupPartitions1.get(index), false); + OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(null, null, keyGroupState, null); + TaskStateSnapshot taskOperatorSubtaskStates = new TaskStateSnapshot(); + taskOperatorSubtaskStates.putSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID1), operatorSubtaskState); + AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint( + jid, + jobVertex1.getTaskVertices()[index].getCurrentExecutionAttempt().getAttemptId(), + checkpointId, + new CheckpointMetrics(), + taskOperatorSubtaskStates); + + coord.receiveAcknowledgeMessage(acknowledgeCheckpoint, TASK_MANAGER_LOCATION_INFO); + } + + for (int index = 0; index < jobVertex2.getParallelism(); index++) { + KeyGroupsStateHandle keyGroupState = generateKeyGroupState(jobVertexID2, keyGroupPartitions2.get(index), false); + OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(null, null, keyGroupState, null); + TaskStateSnapshot taskOperatorSubtaskStates = new TaskStateSnapshot(); + taskOperatorSubtaskStates.putSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID2), operatorSubtaskState); + AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint( + jid, + jobVertex2.getTaskVertices()[index].getCurrentExecutionAttempt().getAttemptId(), + checkpointId, + new CheckpointMetrics(), + taskOperatorSubtaskStates); + + coord.receiveAcknowledgeMessage(acknowledgeCheckpoint, TASK_MANAGER_LOCATION_INFO); + } + + List completedCheckpoints = coord.getSuccessfulCheckpoints(); + + assertEquals(1, completedCheckpoints.size()); + + Map tasks = new HashMap<>(); + + int newMaxParallelism1 = 20; + int newMaxParallelism2 = 42; + + final ExecutionJobVertex newJobVertex1 = mockExecutionJobVertex( + jobVertexID1, + parallelism1, + newMaxParallelism1); + + final ExecutionJobVertex newJobVertex2 = mockExecutionJobVertex( + jobVertexID2, + parallelism2, + newMaxParallelism2); + + tasks.put(jobVertexID1, newJobVertex1); + tasks.put(jobVertexID2, newJobVertex2); + + coord.restoreLatestCheckpointedState(tasks, true, false); + + fail("The restoration should have failed because the max parallelism changed."); + } + + @Test + public void testStateRecoveryWhenTopologyChangeOut() throws Exception { + testStateRecoveryWithTopologyChange(0); + } + + @Test + public void testStateRecoveryWhenTopologyChangeIn() throws Exception { + testStateRecoveryWithTopologyChange(1); + } + + @Test + public void testStateRecoveryWhenTopologyChange() throws Exception { + testStateRecoveryWithTopologyChange(2); + } + + private static Tuple2 generateIDPair() { + JobVertexID jobVertexID = new JobVertexID(); + OperatorID operatorID = OperatorID.fromJobVertexID(jobVertexID); + return new Tuple2<>(jobVertexID, operatorID); + } + + /** + *

+ * old topology. + * [operator1,operator2] * parallelism1 -> [operator3,operator4] * parallelism2 + *

+ * + *

+ * new topology + * + * [operator5,operator1,operator3] * newParallelism1 -> [operator3, operator6] * newParallelism2 + *

+ * scaleType: + * 0 increase parallelism + * 1 decrease parallelism + * 2 same parallelism + */ + public void testStateRecoveryWithTopologyChange(int scaleType) throws Exception { + + /* + * Old topology + * CHAIN(op1 -> op2) * parallelism1 -> CHAIN(op3 -> op4) * parallelism2 + */ + Tuple2 id1 = generateIDPair(); + Tuple2 id2 = generateIDPair(); + int parallelism1 = 10; + int maxParallelism1 = 64; + + Tuple2 id3 = generateIDPair(); + Tuple2 id4 = generateIDPair(); + int parallelism2 = 10; + int maxParallelism2 = 64; + + List keyGroupPartitions2 = + StateAssignmentOperation.createKeyGroupPartitions(maxParallelism2, parallelism2); + + Map operatorStates = new HashMap<>(); + + //prepare vertex1 state + for (Tuple2 id : Arrays.asList(id1, id2)) { + OperatorState taskState = new OperatorState(id.f1, parallelism1, maxParallelism1); + operatorStates.put(id.f1, taskState); + for (int index = 0; index < taskState.getParallelism(); index++) { + OperatorStateHandle subManagedOperatorState = + generatePartitionableStateHandle(id.f0, index, 2, 8, false); + OperatorStateHandle subRawOperatorState = + generatePartitionableStateHandle(id.f0, index, 2, 8, true); + OperatorSubtaskState subtaskState = new OperatorSubtaskState( + subManagedOperatorState, + subRawOperatorState, + null, + null); + taskState.putState(index, subtaskState); + } + } + + List>> expectedManagedOperatorStates = new ArrayList<>(); + List>> expectedRawOperatorStates = new ArrayList<>(); + //prepare vertex2 state + for (Tuple2 id : Arrays.asList(id3, id4)) { + OperatorState operatorState = new OperatorState(id.f1, parallelism2, maxParallelism2); + operatorStates.put(id.f1, operatorState); + List> expectedManagedOperatorState = new ArrayList<>(); + List> expectedRawOperatorState = new ArrayList<>(); + expectedManagedOperatorStates.add(expectedManagedOperatorState); + expectedRawOperatorStates.add(expectedRawOperatorState); + + for (int index = 0; index < operatorState.getParallelism(); index++) { + OperatorStateHandle subManagedOperatorState = + generateChainedPartitionableStateHandle(id.f0, index, 2, 8, false) + .get(0); + OperatorStateHandle subRawOperatorState = + generateChainedPartitionableStateHandle(id.f0, index, 2, 8, true) + .get(0); + KeyGroupsStateHandle subManagedKeyedState = id.f0.equals(id3.f0) + ? generateKeyGroupState(id.f0, keyGroupPartitions2.get(index), false) + : null; + KeyGroupsStateHandle subRawKeyedState = id.f0.equals(id3.f0) + ? generateKeyGroupState(id.f0, keyGroupPartitions2.get(index), true) + : null; + + expectedManagedOperatorState.add(ChainedStateHandle.wrapSingleHandle(subManagedOperatorState)); + expectedRawOperatorState.add(ChainedStateHandle.wrapSingleHandle(subRawOperatorState)); + + OperatorSubtaskState subtaskState = new OperatorSubtaskState( + subManagedOperatorState, + subRawOperatorState, + subManagedKeyedState, + subRawKeyedState); + operatorState.putState(index, subtaskState); + } + } + + /* + * New topology + * CHAIN(op5 -> op1 -> op2) * newParallelism1 -> CHAIN(op3 -> op6) * newParallelism2 + */ + Tuple2 id5 = generateIDPair(); + int newParallelism1 = 10; + + Tuple2 id6 = generateIDPair(); + int newParallelism2 = parallelism2; + + if (scaleType == 0) { + newParallelism2 = 20; + } else if (scaleType == 1) { + newParallelism2 = 8; + } + + List newKeyGroupPartitions2 = + StateAssignmentOperation.createKeyGroupPartitions(maxParallelism2, newParallelism2); + + final ExecutionJobVertex newJobVertex1 = mockExecutionJobVertex( + id5.f0, + Arrays.asList(id2.f1, id1.f1, id5.f1), + newParallelism1, + maxParallelism1); + + final ExecutionJobVertex newJobVertex2 = mockExecutionJobVertex( + id3.f0, + Arrays.asList(id6.f1, id3.f1), + newParallelism2, + maxParallelism2); + + Map tasks = new HashMap<>(); + + tasks.put(id5.f0, newJobVertex1); + tasks.put(id3.f0, newJobVertex2); + + JobID jobID = new JobID(); + StandaloneCompletedCheckpointStore standaloneCompletedCheckpointStore = + spy(new StandaloneCompletedCheckpointStore(1)); + + CompletedCheckpoint completedCheckpoint = new CompletedCheckpoint( + jobID, + 2, + System.currentTimeMillis(), + System.currentTimeMillis() + 3000, + operatorStates, + Collections.emptyList(), + CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION), + new TestCompletedCheckpointStorageLocation()); + + when(standaloneCompletedCheckpointStore.getLatestCheckpoint(false)).thenReturn(completedCheckpoint); + + // set up the coordinator and validate the initial state + CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( + 600000, + 600000, + 0, + Integer.MAX_VALUE, + CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION, + true, + false, + 0); + CheckpointCoordinator coord = new CheckpointCoordinator( + new JobID(), + chkConfig, + newJobVertex1.getTaskVertices(), + newJobVertex1.getTaskVertices(), + newJobVertex1.getTaskVertices(), + new StandaloneCheckpointIDCounter(), + standaloneCompletedCheckpointStore, + new MemoryStateBackend(), + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY, + failureManager); + + coord.restoreLatestCheckpointedState(tasks, false, true); + + for (int i = 0; i < newJobVertex1.getParallelism(); i++) { + + final List operatorIds = newJobVertex1.getOperatorIDs(); + + JobManagerTaskRestore taskRestore = newJobVertex1.getTaskVertices()[i].getCurrentExecutionAttempt().getTaskRestore(); + Assert.assertEquals(2L, taskRestore.getRestoreCheckpointId()); + TaskStateSnapshot stateSnapshot = taskRestore.getTaskStateSnapshot(); + + OperatorSubtaskState headOpState = stateSnapshot.getSubtaskStateByOperatorID(operatorIds.get(operatorIds.size() - 1)); + assertTrue(headOpState.getManagedKeyedState().isEmpty()); + assertTrue(headOpState.getRawKeyedState().isEmpty()); + + // operator5 + { + int operatorIndexInChain = 2; + OperatorSubtaskState opState = + stateSnapshot.getSubtaskStateByOperatorID(operatorIds.get(operatorIndexInChain)); + + assertTrue(opState.getManagedOperatorState().isEmpty()); + assertTrue(opState.getRawOperatorState().isEmpty()); + } + // operator1 + { + int operatorIndexInChain = 1; + OperatorSubtaskState opState = + stateSnapshot.getSubtaskStateByOperatorID(operatorIds.get(operatorIndexInChain)); + + OperatorStateHandle expectedManagedOpState = generatePartitionableStateHandle( + id1.f0, i, 2, 8, false); + OperatorStateHandle expectedRawOpState = generatePartitionableStateHandle( + id1.f0, i, 2, 8, true); + + Collection managedOperatorState = opState.getManagedOperatorState(); + assertEquals(1, managedOperatorState.size()); + assertTrue(CommonTestUtils.isStreamContentEqual(expectedManagedOpState.openInputStream(), + managedOperatorState.iterator().next().openInputStream())); + + Collection rawOperatorState = opState.getRawOperatorState(); + assertEquals(1, rawOperatorState.size()); + assertTrue(CommonTestUtils.isStreamContentEqual(expectedRawOpState.openInputStream(), + rawOperatorState.iterator().next().openInputStream())); + } + // operator2 + { + int operatorIndexInChain = 0; + OperatorSubtaskState opState = + stateSnapshot.getSubtaskStateByOperatorID(operatorIds.get(operatorIndexInChain)); + + OperatorStateHandle expectedManagedOpState = generatePartitionableStateHandle( + id2.f0, i, 2, 8, false); + OperatorStateHandle expectedRawOpState = generatePartitionableStateHandle( + id2.f0, i, 2, 8, true); + + Collection managedOperatorState = opState.getManagedOperatorState(); + assertEquals(1, managedOperatorState.size()); + assertTrue(CommonTestUtils.isStreamContentEqual(expectedManagedOpState.openInputStream(), + managedOperatorState.iterator().next().openInputStream())); + + Collection rawOperatorState = opState.getRawOperatorState(); + assertEquals(1, rawOperatorState.size()); + assertTrue(CommonTestUtils.isStreamContentEqual(expectedRawOpState.openInputStream(), + rawOperatorState.iterator().next().openInputStream())); + } + } + + List>> actualManagedOperatorStates = new ArrayList<>(newJobVertex2.getParallelism()); + List>> actualRawOperatorStates = new ArrayList<>(newJobVertex2.getParallelism()); + + for (int i = 0; i < newJobVertex2.getParallelism(); i++) { + + final List operatorIds = newJobVertex2.getOperatorIDs(); + + JobManagerTaskRestore taskRestore = newJobVertex2.getTaskVertices()[i].getCurrentExecutionAttempt().getTaskRestore(); + Assert.assertEquals(2L, taskRestore.getRestoreCheckpointId()); + TaskStateSnapshot stateSnapshot = taskRestore.getTaskStateSnapshot(); + + // operator 3 + { + int operatorIndexInChain = 1; + OperatorSubtaskState opState = + stateSnapshot.getSubtaskStateByOperatorID(operatorIds.get(operatorIndexInChain)); + + List> actualSubManagedOperatorState = new ArrayList<>(1); + actualSubManagedOperatorState.add(opState.getManagedOperatorState()); + + List> actualSubRawOperatorState = new ArrayList<>(1); + actualSubRawOperatorState.add(opState.getRawOperatorState()); + + actualManagedOperatorStates.add(actualSubManagedOperatorState); + actualRawOperatorStates.add(actualSubRawOperatorState); + } + + // operator 6 + { + int operatorIndexInChain = 0; + OperatorSubtaskState opState = + stateSnapshot.getSubtaskStateByOperatorID(operatorIds.get(operatorIndexInChain)); + assertTrue(opState.getManagedOperatorState().isEmpty()); + assertTrue(opState.getRawOperatorState().isEmpty()); + + } + + KeyGroupsStateHandle originalKeyedStateBackend = generateKeyGroupState(id3.f0, newKeyGroupPartitions2.get(i), false); + KeyGroupsStateHandle originalKeyedStateRaw = generateKeyGroupState(id3.f0, newKeyGroupPartitions2.get(i), true); + + OperatorSubtaskState headOpState = + stateSnapshot.getSubtaskStateByOperatorID(operatorIds.get(operatorIds.size() - 1)); + + Collection keyedStateBackend = headOpState.getManagedKeyedState(); + Collection keyGroupStateRaw = headOpState.getRawKeyedState(); + + compareKeyedState(Collections.singletonList(originalKeyedStateBackend), keyedStateBackend); + compareKeyedState(Collections.singletonList(originalKeyedStateRaw), keyGroupStateRaw); + } + + comparePartitionableState(expectedManagedOperatorStates.get(0), actualManagedOperatorStates); + comparePartitionableState(expectedRawOperatorStates.get(0), actualRawOperatorStates); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java index 034fac4ce3dc..2d86a0656c32 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java @@ -33,11 +33,9 @@ import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint; -import org.apache.flink.runtime.state.ChainedStateHandle; import org.apache.flink.runtime.state.IncrementalRemoteKeyedStateHandle; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyGroupRangeAssignment; -import org.apache.flink.runtime.state.KeyGroupsStateHandle; import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; import org.apache.flink.runtime.state.OperatorStreamStateHandle; @@ -49,25 +47,17 @@ import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; import org.apache.flink.runtime.state.memory.MemoryStateBackend; import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; -import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.RecoverableCompletedCheckpointStore; import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.SerializableObject; import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables; -import org.hamcrest.BaseMatcher; -import org.hamcrest.Description; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import org.mockito.Mockito; -import org.mockito.hamcrest.MockitoHamcrest; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; import org.mockito.verification.VerificationMode; import java.io.IOException; @@ -79,26 +69,15 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Random; import java.util.UUID; -import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; -import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.compareKeyedState; -import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.comparePartitionableState; -import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.generateChainedPartitionableStateHandle; -import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.generateKeyGroupState; -import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.generatePartitionableStateHandle; -import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.mockExecution; import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.mockExecutionJobVertex; import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.mockExecutionVertex; -import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.mockSubtaskState; -import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.verifyStateRestore; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; @@ -1305,194 +1284,6 @@ public void testStateCleanupForLateOrUnknownMessages() throws Exception { verify(unknownSubtaskState2, times(1)).discardState(); } - @Test - public void testPeriodicTriggering() { - try { - final JobID jid = new JobID(); - final long start = System.currentTimeMillis(); - - // create some mock execution vertices and trigger some checkpoint - - final ExecutionAttemptID triggerAttemptID = new ExecutionAttemptID(); - final ExecutionAttemptID ackAttemptID = new ExecutionAttemptID(); - final ExecutionAttemptID commitAttemptID = new ExecutionAttemptID(); - - ExecutionVertex triggerVertex = mockExecutionVertex(triggerAttemptID); - ExecutionVertex ackVertex = mockExecutionVertex(ackAttemptID); - ExecutionVertex commitVertex = mockExecutionVertex(commitAttemptID); - - final AtomicInteger numCalls = new AtomicInteger(); - - final Execution execution = triggerVertex.getCurrentExecutionAttempt(); - - doAnswer(new Answer() { - - private long lastId = -1; - private long lastTs = -1; - - @Override - public Void answer(InvocationOnMock invocation) throws Throwable { - long id = (Long) invocation.getArguments()[0]; - long ts = (Long) invocation.getArguments()[1]; - - assertTrue(id > lastId); - assertTrue(ts >= lastTs); - assertTrue(ts >= start); - - lastId = id; - lastTs = ts; - numCalls.incrementAndGet(); - return null; - } - }).when(execution).triggerCheckpoint(anyLong(), anyLong(), any(CheckpointOptions.class)); - - CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( - 10, // periodic interval is 10 ms - 200000, // timeout is very long (200 s) - 0, - Integer.MAX_VALUE, - CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION, - true, - false, - 0); - CheckpointCoordinator coord = new CheckpointCoordinator( - jid, - chkConfig, - new ExecutionVertex[] { triggerVertex }, - new ExecutionVertex[] { ackVertex }, - new ExecutionVertex[] { commitVertex }, - new StandaloneCheckpointIDCounter(), - new StandaloneCompletedCheckpointStore(2), - new MemoryStateBackend(), - Executors.directExecutor(), - SharedStateRegistry.DEFAULT_FACTORY, - failureManager); - - coord.startCheckpointScheduler(); - - long timeout = System.currentTimeMillis() + 60000; - do { - Thread.sleep(20); - } - while (timeout > System.currentTimeMillis() && numCalls.get() < 5); - assertTrue(numCalls.get() >= 5); - - coord.stopCheckpointScheduler(); - - // for 400 ms, no further calls may come. - // there may be the case that one trigger was fired and about to - // acquire the lock, such that after cancelling it will still do - // the remainder of its work - int numCallsSoFar = numCalls.get(); - Thread.sleep(400); - assertTrue(numCallsSoFar == numCalls.get() || - numCallsSoFar + 1 == numCalls.get()); - - // start another sequence of periodic scheduling - numCalls.set(0); - coord.startCheckpointScheduler(); - - timeout = System.currentTimeMillis() + 60000; - do { - Thread.sleep(20); - } - while (timeout > System.currentTimeMillis() && numCalls.get() < 5); - assertTrue(numCalls.get() >= 5); - - coord.stopCheckpointScheduler(); - - // for 400 ms, no further calls may come - // there may be the case that one trigger was fired and about to - // acquire the lock, such that after cancelling it will still do - // the remainder of its work - numCallsSoFar = numCalls.get(); - Thread.sleep(400); - assertTrue(numCallsSoFar == numCalls.get() || - numCallsSoFar + 1 == numCalls.get()); - - coord.shutdown(JobStatus.FINISHED); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - /** - * This test verified that after a completed checkpoint a certain time has passed before - * another is triggered. - */ - @Test - public void testMinTimeBetweenCheckpointsInterval() throws Exception { - final JobID jid = new JobID(); - - // create some mock execution vertices and trigger some checkpoint - final ExecutionAttemptID attemptID = new ExecutionAttemptID(); - final ExecutionVertex vertex = mockExecutionVertex(attemptID); - final Execution executionAttempt = vertex.getCurrentExecutionAttempt(); - - final BlockingQueue triggerCalls = new LinkedBlockingQueue<>(); - - doAnswer(invocation -> { - triggerCalls.add((Long) invocation.getArguments()[0]); - return null; - }).when(executionAttempt).triggerCheckpoint(anyLong(), anyLong(), any(CheckpointOptions.class)); - - final long delay = 50; - - CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( - 12, // periodic interval is 12 ms - 200_000, // timeout is very long (200 s) - delay, // 50 ms delay between checkpoints - 1, - CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION, - true, - false, - 0); - final CheckpointCoordinator coord = new CheckpointCoordinator( - jid, - chkConfig, - new ExecutionVertex[] { vertex }, - new ExecutionVertex[] { vertex }, - new ExecutionVertex[] { vertex }, - new StandaloneCheckpointIDCounter(), - new StandaloneCompletedCheckpointStore(2), - new MemoryStateBackend(), - Executors.directExecutor(), - SharedStateRegistry.DEFAULT_FACTORY, - failureManager); - - try { - coord.startCheckpointScheduler(); - - // wait until the first checkpoint was triggered - Long firstCallId = triggerCalls.take(); - assertEquals(1L, firstCallId.longValue()); - - AcknowledgeCheckpoint ackMsg = new AcknowledgeCheckpoint(jid, attemptID, 1L); - - // tell the coordinator that the checkpoint is done - final long ackTime = System.nanoTime(); - coord.receiveAcknowledgeMessage(ackMsg, TASK_MANAGER_LOCATION_INFO); - - // wait until the next checkpoint is triggered - Long nextCallId = triggerCalls.take(); - final long nextCheckpointTime = System.nanoTime(); - assertEquals(2L, nextCallId.longValue()); - - final long delayMillis = (nextCheckpointTime - ackTime) / 1_000_000; - - // we need to add one ms here to account for rounding errors - if (delayMillis + 1 < delay) { - fail("checkpoint came too early: delay was " + delayMillis + " but should have been at least " + delay); - } - } - finally { - coord.stopCheckpointScheduler(); - coord.shutdown(JobStatus.FINISHED); - } - } - @Test public void testMaxConcurrentAttempts1() { testMaxConcurrentAttempts(1); @@ -2072,961 +1863,67 @@ public void testMinDelayBetweenSavepoints() throws Exception { } /** - * Tests that the checkpointed partitioned and non-partitioned state is assigned properly to - * the {@link Execution} upon recovery. - * - * @throws Exception + * Tests that the externalized checkpoint configuration is respected. */ @Test - public void testRestoreLatestCheckpointedState() throws Exception { - final JobID jid = new JobID(); - final long timestamp = System.currentTimeMillis(); - - final JobVertexID jobVertexID1 = new JobVertexID(); - final JobVertexID jobVertexID2 = new JobVertexID(); - int parallelism1 = 3; - int parallelism2 = 2; - int maxParallelism1 = 42; - int maxParallelism2 = 13; - - final ExecutionJobVertex jobVertex1 = mockExecutionJobVertex( - jobVertexID1, - parallelism1, - maxParallelism1); - final ExecutionJobVertex jobVertex2 = mockExecutionJobVertex( - jobVertexID2, - parallelism2, - maxParallelism2); - - List allExecutionVertices = new ArrayList<>(parallelism1 + parallelism2); - - allExecutionVertices.addAll(Arrays.asList(jobVertex1.getTaskVertices())); - allExecutionVertices.addAll(Arrays.asList(jobVertex2.getTaskVertices())); - - ExecutionVertex[] arrayExecutionVertices = - allExecutionVertices.toArray(new ExecutionVertex[allExecutionVertices.size()]); - - CompletedCheckpointStore store = new RecoverableCompletedCheckpointStore(); - - // set up the coordinator and validate the initial state - CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( - 600000, - 600000, - 0, - Integer.MAX_VALUE, - CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION, - true, - false, - 0); - CheckpointCoordinator coord = new CheckpointCoordinator( - jid, - chkConfig, - arrayExecutionVertices, - arrayExecutionVertices, - arrayExecutionVertices, - new StandaloneCheckpointIDCounter(), - store, - new MemoryStateBackend(), - Executors.directExecutor(), - SharedStateRegistry.DEFAULT_FACTORY, - failureManager); - - // trigger the checkpoint - coord.triggerCheckpoint(timestamp, false); - - assertTrue(coord.getPendingCheckpoints().keySet().size() == 1); - long checkpointId = Iterables.getOnlyElement(coord.getPendingCheckpoints().keySet()); - - List keyGroupPartitions1 = StateAssignmentOperation.createKeyGroupPartitions(maxParallelism1, parallelism1); - List keyGroupPartitions2 = StateAssignmentOperation.createKeyGroupPartitions(maxParallelism2, parallelism2); + public void testExternalizedCheckpoints() throws Exception { + try { + final JobID jid = new JobID(); + final long timestamp = System.currentTimeMillis(); - for (int index = 0; index < jobVertex1.getParallelism(); index++) { - TaskStateSnapshot subtaskState = mockSubtaskState(jobVertexID1, index, keyGroupPartitions1.get(index)); + // create some mock Execution vertices that receive the checkpoint trigger messages + final ExecutionAttemptID attemptID1 = new ExecutionAttemptID(); + ExecutionVertex vertex1 = mockExecutionVertex(attemptID1); - AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint( - jid, - jobVertex1.getTaskVertices()[index].getCurrentExecutionAttempt().getAttemptId(), - checkpointId, - new CheckpointMetrics(), - subtaskState); + // set up the coordinator and validate the initial state + CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( + 600000, + 600000, + 0, + Integer.MAX_VALUE, + CheckpointRetentionPolicy.RETAIN_ON_FAILURE, + true, + false, + 0); + CheckpointCoordinator coord = new CheckpointCoordinator( + jid, + chkConfig, + new ExecutionVertex[] { vertex1 }, + new ExecutionVertex[] { vertex1 }, + new ExecutionVertex[] { vertex1 }, + new StandaloneCheckpointIDCounter(), + new StandaloneCompletedCheckpointStore(1), + new MemoryStateBackend(), + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY, + failureManager); - coord.receiveAcknowledgeMessage(acknowledgeCheckpoint, TASK_MANAGER_LOCATION_INFO); - } + assertTrue(coord.triggerCheckpoint(timestamp, false)); - for (int index = 0; index < jobVertex2.getParallelism(); index++) { - TaskStateSnapshot subtaskState = mockSubtaskState(jobVertexID2, index, keyGroupPartitions2.get(index)); + for (PendingCheckpoint checkpoint : coord.getPendingCheckpoints().values()) { + CheckpointProperties props = checkpoint.getProps(); + CheckpointProperties expected = CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.RETAIN_ON_FAILURE); - AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint( - jid, - jobVertex2.getTaskVertices()[index].getCurrentExecutionAttempt().getAttemptId(), - checkpointId, - new CheckpointMetrics(), - subtaskState); + assertEquals(expected, props); + } - coord.receiveAcknowledgeMessage(acknowledgeCheckpoint, TASK_MANAGER_LOCATION_INFO); + // the now we should have a completed checkpoint + coord.shutdown(JobStatus.FINISHED); } - - List completedCheckpoints = coord.getSuccessfulCheckpoints(); - - assertEquals(1, completedCheckpoints.size()); - - // shutdown the store - store.shutdown(JobStatus.SUSPENDED); - - // restore the store - Map tasks = new HashMap<>(); - - tasks.put(jobVertexID1, jobVertex1); - tasks.put(jobVertexID2, jobVertex2); - - coord.restoreLatestCheckpointedState(tasks, true, false); - - // validate that all shared states are registered again after the recovery. - for (CompletedCheckpoint completedCheckpoint : completedCheckpoints) { - for (OperatorState taskState : completedCheckpoint.getOperatorStates().values()) { - for (OperatorSubtaskState subtaskState : taskState.getStates()) { - verify(subtaskState, times(2)).registerSharedStates(any(SharedStateRegistry.class)); - } - } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); } - - // verify the restored state - verifyStateRestore(jobVertexID1, jobVertex1, keyGroupPartitions1); - verifyStateRestore(jobVertexID2, jobVertex2, keyGroupPartitions2); } - /** - * Tests that the checkpoint restoration fails if the max parallelism of the job vertices has - * changed. - * - * @throws Exception - */ - @Test(expected = IllegalStateException.class) - public void testRestoreLatestCheckpointFailureWhenMaxParallelismChanges() throws Exception { - final JobID jid = new JobID(); - final long timestamp = System.currentTimeMillis(); - - final JobVertexID jobVertexID1 = new JobVertexID(); - final JobVertexID jobVertexID2 = new JobVertexID(); - int parallelism1 = 3; - int parallelism2 = 2; - int maxParallelism1 = 42; - int maxParallelism2 = 13; - - final ExecutionJobVertex jobVertex1 = mockExecutionJobVertex( - jobVertexID1, - parallelism1, - maxParallelism1); - final ExecutionJobVertex jobVertex2 = mockExecutionJobVertex( - jobVertexID2, - parallelism2, - maxParallelism2); - - List allExecutionVertices = new ArrayList<>(parallelism1 + parallelism2); - - allExecutionVertices.addAll(Arrays.asList(jobVertex1.getTaskVertices())); - allExecutionVertices.addAll(Arrays.asList(jobVertex2.getTaskVertices())); - - ExecutionVertex[] arrayExecutionVertices = allExecutionVertices.toArray(new ExecutionVertex[allExecutionVertices.size()]); - - // set up the coordinator and validate the initial state - CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( - 600000, - 600000, - 0, - Integer.MAX_VALUE, - CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION, - true, - false, - 0); - CheckpointCoordinator coord = new CheckpointCoordinator( - jid, - chkConfig, - arrayExecutionVertices, - arrayExecutionVertices, - arrayExecutionVertices, - new StandaloneCheckpointIDCounter(), - new StandaloneCompletedCheckpointStore(1), - new MemoryStateBackend(), - Executors.directExecutor(), - SharedStateRegistry.DEFAULT_FACTORY, - failureManager); - - // trigger the checkpoint - coord.triggerCheckpoint(timestamp, false); - - assertTrue(coord.getPendingCheckpoints().keySet().size() == 1); - long checkpointId = Iterables.getOnlyElement(coord.getPendingCheckpoints().keySet()); - - List keyGroupPartitions1 = StateAssignmentOperation.createKeyGroupPartitions(maxParallelism1, parallelism1); - List keyGroupPartitions2 = StateAssignmentOperation.createKeyGroupPartitions(maxParallelism2, parallelism2); - - for (int index = 0; index < jobVertex1.getParallelism(); index++) { - KeyGroupsStateHandle keyGroupState = generateKeyGroupState(jobVertexID1, keyGroupPartitions1.get(index), false); - OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(null, null, keyGroupState, null); - TaskStateSnapshot taskOperatorSubtaskStates = new TaskStateSnapshot(); - taskOperatorSubtaskStates.putSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID1), operatorSubtaskState); - AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint( - jid, - jobVertex1.getTaskVertices()[index].getCurrentExecutionAttempt().getAttemptId(), - checkpointId, - new CheckpointMetrics(), - taskOperatorSubtaskStates); - - coord.receiveAcknowledgeMessage(acknowledgeCheckpoint, TASK_MANAGER_LOCATION_INFO); - } - - for (int index = 0; index < jobVertex2.getParallelism(); index++) { - KeyGroupsStateHandle keyGroupState = generateKeyGroupState(jobVertexID2, keyGroupPartitions2.get(index), false); - OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(null, null, keyGroupState, null); - TaskStateSnapshot taskOperatorSubtaskStates = new TaskStateSnapshot(); - taskOperatorSubtaskStates.putSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID2), operatorSubtaskState); - AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint( - jid, - jobVertex2.getTaskVertices()[index].getCurrentExecutionAttempt().getAttemptId(), - checkpointId, - new CheckpointMetrics(), - taskOperatorSubtaskStates); - - coord.receiveAcknowledgeMessage(acknowledgeCheckpoint, TASK_MANAGER_LOCATION_INFO); - } - - List completedCheckpoints = coord.getSuccessfulCheckpoints(); - - assertEquals(1, completedCheckpoints.size()); - - Map tasks = new HashMap<>(); - - int newMaxParallelism1 = 20; - int newMaxParallelism2 = 42; - - final ExecutionJobVertex newJobVertex1 = mockExecutionJobVertex( - jobVertexID1, - parallelism1, - newMaxParallelism1); - - final ExecutionJobVertex newJobVertex2 = mockExecutionJobVertex( - jobVertexID2, - parallelism2, - newMaxParallelism2); - - tasks.put(jobVertexID1, newJobVertex1); - tasks.put(jobVertexID2, newJobVertex2); - - coord.restoreLatestCheckpointedState(tasks, true, false); - - fail("The restoration should have failed because the max parallelism changed."); - } - - @Test - public void testRestoreLatestCheckpointedStateScaleIn() throws Exception { - testRestoreLatestCheckpointedStateWithChangingParallelism(false); - } - - @Test - public void testRestoreLatestCheckpointedStateScaleOut() throws Exception { - testRestoreLatestCheckpointedStateWithChangingParallelism(true); - } - - @Test - public void testRestoreLatestCheckpointWhenPreferCheckpoint() throws Exception { - testRestoreLatestCheckpointIsPreferSavepoint(true); - } - - @Test - public void testRestoreLatestCheckpointWhenPreferSavepoint() throws Exception { - testRestoreLatestCheckpointIsPreferSavepoint(false); - } - - private void testRestoreLatestCheckpointIsPreferSavepoint(boolean isPreferCheckpoint) { - try { - final JobID jid = new JobID(); - long timestamp = System.currentTimeMillis(); - StandaloneCheckpointIDCounter checkpointIDCounter = new StandaloneCheckpointIDCounter(); - - final JobVertexID statefulId = new JobVertexID(); - final JobVertexID statelessId = new JobVertexID(); - - Execution statefulExec1 = mockExecution(); - Execution statelessExec1 = mockExecution(); - - ExecutionVertex stateful1 = mockExecutionVertex(statefulExec1, statefulId, 0, 1); - ExecutionVertex stateless1 = mockExecutionVertex(statelessExec1, statelessId, 0, 1); - - ExecutionJobVertex stateful = mockExecutionJobVertex(statefulId, - new ExecutionVertex[] { stateful1 }); - ExecutionJobVertex stateless = mockExecutionJobVertex(statelessId, - new ExecutionVertex[] { stateless1 }); - - Map map = new HashMap(); - map.put(statefulId, stateful); - map.put(statelessId, stateless); - - CompletedCheckpointStore store = new RecoverableCompletedCheckpointStore(2); - - CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( - 600000, - 600000, - 0, - Integer.MAX_VALUE, - CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION, - true, - isPreferCheckpoint, - 0); - CheckpointCoordinator coord = new CheckpointCoordinator( - jid, - chkConfig, - new ExecutionVertex[] { stateful1, stateless1 }, - new ExecutionVertex[] { stateful1, stateless1 }, - new ExecutionVertex[] { stateful1, stateless1 }, - checkpointIDCounter, - store, - new MemoryStateBackend(), - Executors.directExecutor(), - SharedStateRegistry.DEFAULT_FACTORY, - failureManager); - - //trigger a checkpoint and wait to become a completed checkpoint - assertTrue(coord.triggerCheckpoint(timestamp, false)); - - long checkpointId = checkpointIDCounter.getLast(); - - KeyGroupRange keyGroupRange = KeyGroupRange.of(0, 0); - List testStates = Collections.singletonList(new SerializableObject()); - KeyedStateHandle serializedKeyGroupStates = generateKeyGroupState(keyGroupRange, testStates); - - TaskStateSnapshot subtaskStatesForCheckpoint = new TaskStateSnapshot(); - - subtaskStatesForCheckpoint.putSubtaskStateByOperatorID( - OperatorID.fromJobVertexID(statefulId), - new OperatorSubtaskState( - StateObjectCollection.empty(), - StateObjectCollection.empty(), - StateObjectCollection.singleton(serializedKeyGroupStates), - StateObjectCollection.empty())); - - coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statefulExec1.getAttemptId(), checkpointId, new CheckpointMetrics(), subtaskStatesForCheckpoint), TASK_MANAGER_LOCATION_INFO); - coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statelessExec1.getAttemptId(), checkpointId), TASK_MANAGER_LOCATION_INFO); - - CompletedCheckpoint success = coord.getSuccessfulCheckpoints().get(0); - assertEquals(jid, success.getJobId()); - - // trigger a savepoint and wait it to be finished - String savepointDir = tmpFolder.newFolder().getAbsolutePath(); - timestamp = System.currentTimeMillis(); - CompletableFuture savepointFuture = coord.triggerSavepoint(timestamp, savepointDir); - - KeyGroupRange keyGroupRangeForSavepoint = KeyGroupRange.of(1, 1); - List testStatesForSavepoint = Collections.singletonList(new SerializableObject()); - KeyedStateHandle serializedKeyGroupStatesForSavepoint = generateKeyGroupState(keyGroupRangeForSavepoint, testStatesForSavepoint); - - TaskStateSnapshot subtaskStatesForSavepoint = new TaskStateSnapshot(); - - subtaskStatesForSavepoint.putSubtaskStateByOperatorID( - OperatorID.fromJobVertexID(statefulId), - new OperatorSubtaskState( - StateObjectCollection.empty(), - StateObjectCollection.empty(), - StateObjectCollection.singleton(serializedKeyGroupStatesForSavepoint), - StateObjectCollection.empty())); - - checkpointId = checkpointIDCounter.getLast(); - coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statefulExec1.getAttemptId(), checkpointId, new CheckpointMetrics(), subtaskStatesForSavepoint), TASK_MANAGER_LOCATION_INFO); - coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statelessExec1.getAttemptId(), checkpointId), TASK_MANAGER_LOCATION_INFO); - - assertTrue(savepointFuture.isDone()); - - //restore and jump the latest savepoint - coord.restoreLatestCheckpointedState(map, true, false); - - //compare and see if it used the checkpoint's subtaskStates - BaseMatcher matcher = new BaseMatcher() { - @Override - public boolean matches(Object o) { - if (o instanceof JobManagerTaskRestore) { - JobManagerTaskRestore taskRestore = (JobManagerTaskRestore) o; - if (isPreferCheckpoint) { - return Objects.equals(taskRestore.getTaskStateSnapshot(), subtaskStatesForCheckpoint); - } else { - return Objects.equals(taskRestore.getTaskStateSnapshot(), subtaskStatesForSavepoint); - } - } - return false; - } - - @Override - public void describeTo(Description description) { - if (isPreferCheckpoint) { - description.appendValue(subtaskStatesForCheckpoint); - } else { - description.appendValue(subtaskStatesForSavepoint); - } - } - }; - - verify(statefulExec1, times(1)).setInitialState(MockitoHamcrest.argThat(matcher)); - verify(statelessExec1, times(0)).setInitialState(Mockito.any()); - - coord.shutdown(JobStatus.FINISHED); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testStateRecoveryWhenTopologyChangeOut() throws Exception { - testStateRecoveryWithTopologyChange(0); - } - - @Test - public void testStateRecoveryWhenTopologyChangeIn() throws Exception { - testStateRecoveryWithTopologyChange(1); - } - - @Test - public void testStateRecoveryWhenTopologyChange() throws Exception { - testStateRecoveryWithTopologyChange(2); - } - - - /** - * Tests the checkpoint restoration with changing parallelism of job vertex with partitioned - * state. - * - * @throws Exception - */ - private void testRestoreLatestCheckpointedStateWithChangingParallelism(boolean scaleOut) throws Exception { - final JobID jid = new JobID(); - final long timestamp = System.currentTimeMillis(); - - final JobVertexID jobVertexID1 = new JobVertexID(); - final JobVertexID jobVertexID2 = new JobVertexID(); - int parallelism1 = 3; - int parallelism2 = scaleOut ? 2 : 13; - - int maxParallelism1 = 42; - int maxParallelism2 = 13; - - int newParallelism2 = scaleOut ? 13 : 2; - - final ExecutionJobVertex jobVertex1 = mockExecutionJobVertex( - jobVertexID1, - parallelism1, - maxParallelism1); - final ExecutionJobVertex jobVertex2 = mockExecutionJobVertex( - jobVertexID2, - parallelism2, - maxParallelism2); - - List allExecutionVertices = new ArrayList<>(parallelism1 + parallelism2); - - allExecutionVertices.addAll(Arrays.asList(jobVertex1.getTaskVertices())); - allExecutionVertices.addAll(Arrays.asList(jobVertex2.getTaskVertices())); - - ExecutionVertex[] arrayExecutionVertices = - allExecutionVertices.toArray(new ExecutionVertex[allExecutionVertices.size()]); - - // set up the coordinator and validate the initial state - CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( - 600000, - 600000, - 0, - Integer.MAX_VALUE, - CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION, - true, - false, - 0); - CheckpointCoordinator coord = new CheckpointCoordinator( - jid, - chkConfig, - arrayExecutionVertices, - arrayExecutionVertices, - arrayExecutionVertices, - new StandaloneCheckpointIDCounter(), - new StandaloneCompletedCheckpointStore(1), - new MemoryStateBackend(), - Executors.directExecutor(), - SharedStateRegistry.DEFAULT_FACTORY, - failureManager); - - // trigger the checkpoint - coord.triggerCheckpoint(timestamp, false); - - assertTrue(coord.getPendingCheckpoints().keySet().size() == 1); - long checkpointId = Iterables.getOnlyElement(coord.getPendingCheckpoints().keySet()); - - List keyGroupPartitions1 = - StateAssignmentOperation.createKeyGroupPartitions(maxParallelism1, parallelism1); - List keyGroupPartitions2 = - StateAssignmentOperation.createKeyGroupPartitions(maxParallelism2, parallelism2); - - //vertex 1 - for (int index = 0; index < jobVertex1.getParallelism(); index++) { - OperatorStateHandle opStateBackend = generatePartitionableStateHandle(jobVertexID1, index, 2, 8, false); - KeyGroupsStateHandle keyedStateBackend = generateKeyGroupState(jobVertexID1, keyGroupPartitions1.get(index), false); - KeyGroupsStateHandle keyedStateRaw = generateKeyGroupState(jobVertexID1, keyGroupPartitions1.get(index), true); - OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(opStateBackend, null, keyedStateBackend, keyedStateRaw); - TaskStateSnapshot taskOperatorSubtaskStates = new TaskStateSnapshot(); - taskOperatorSubtaskStates.putSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID1), operatorSubtaskState); - - AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint( - jid, - jobVertex1.getTaskVertices()[index].getCurrentExecutionAttempt().getAttemptId(), - checkpointId, - new CheckpointMetrics(), - taskOperatorSubtaskStates); - - coord.receiveAcknowledgeMessage(acknowledgeCheckpoint, TASK_MANAGER_LOCATION_INFO); - } - - //vertex 2 - final List> expectedOpStatesBackend = new ArrayList<>(jobVertex2.getParallelism()); - final List> expectedOpStatesRaw = new ArrayList<>(jobVertex2.getParallelism()); - for (int index = 0; index < jobVertex2.getParallelism(); index++) { - KeyGroupsStateHandle keyedStateBackend = generateKeyGroupState(jobVertexID2, keyGroupPartitions2.get(index), false); - KeyGroupsStateHandle keyedStateRaw = generateKeyGroupState(jobVertexID2, keyGroupPartitions2.get(index), true); - OperatorStateHandle opStateBackend = generatePartitionableStateHandle(jobVertexID2, index, 2, 8, false); - OperatorStateHandle opStateRaw = generatePartitionableStateHandle(jobVertexID2, index, 2, 8, true); - expectedOpStatesBackend.add(new ChainedStateHandle<>(Collections.singletonList(opStateBackend))); - expectedOpStatesRaw.add(new ChainedStateHandle<>(Collections.singletonList(opStateRaw))); - - OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(opStateBackend, opStateRaw, keyedStateBackend, keyedStateRaw); - TaskStateSnapshot taskOperatorSubtaskStates = new TaskStateSnapshot(); - taskOperatorSubtaskStates.putSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID2), operatorSubtaskState); - - AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint( - jid, - jobVertex2.getTaskVertices()[index].getCurrentExecutionAttempt().getAttemptId(), - checkpointId, - new CheckpointMetrics(), - taskOperatorSubtaskStates); - - coord.receiveAcknowledgeMessage(acknowledgeCheckpoint, TASK_MANAGER_LOCATION_INFO); - } - - List completedCheckpoints = coord.getSuccessfulCheckpoints(); - - assertEquals(1, completedCheckpoints.size()); - - Map tasks = new HashMap<>(); - - List newKeyGroupPartitions2 = - StateAssignmentOperation.createKeyGroupPartitions(maxParallelism2, newParallelism2); - - final ExecutionJobVertex newJobVertex1 = mockExecutionJobVertex( - jobVertexID1, - parallelism1, - maxParallelism1); - - // rescale vertex 2 - final ExecutionJobVertex newJobVertex2 = mockExecutionJobVertex( - jobVertexID2, - newParallelism2, - maxParallelism2); - - tasks.put(jobVertexID1, newJobVertex1); - tasks.put(jobVertexID2, newJobVertex2); - coord.restoreLatestCheckpointedState(tasks, true, false); - - // verify the restored state - verifyStateRestore(jobVertexID1, newJobVertex1, keyGroupPartitions1); - List>> actualOpStatesBackend = new ArrayList<>(newJobVertex2.getParallelism()); - List>> actualOpStatesRaw = new ArrayList<>(newJobVertex2.getParallelism()); - for (int i = 0; i < newJobVertex2.getParallelism(); i++) { - - List operatorIDs = newJobVertex2.getOperatorIDs(); - - KeyGroupsStateHandle originalKeyedStateBackend = generateKeyGroupState(jobVertexID2, newKeyGroupPartitions2.get(i), false); - KeyGroupsStateHandle originalKeyedStateRaw = generateKeyGroupState(jobVertexID2, newKeyGroupPartitions2.get(i), true); - - JobManagerTaskRestore taskRestore = newJobVertex2.getTaskVertices()[i].getCurrentExecutionAttempt().getTaskRestore(); - Assert.assertEquals(1L, taskRestore.getRestoreCheckpointId()); - TaskStateSnapshot taskStateHandles = taskRestore.getTaskStateSnapshot(); - - final int headOpIndex = operatorIDs.size() - 1; - List> allParallelManagedOpStates = new ArrayList<>(operatorIDs.size()); - List> allParallelRawOpStates = new ArrayList<>(operatorIDs.size()); - - for (int idx = 0; idx < operatorIDs.size(); ++idx) { - OperatorID operatorID = operatorIDs.get(idx); - OperatorSubtaskState opState = taskStateHandles.getSubtaskStateByOperatorID(operatorID); - Collection opStateBackend = opState.getManagedOperatorState(); - Collection opStateRaw = opState.getRawOperatorState(); - allParallelManagedOpStates.add(opStateBackend); - allParallelRawOpStates.add(opStateRaw); - if (idx == headOpIndex) { - Collection keyedStateBackend = opState.getManagedKeyedState(); - Collection keyGroupStateRaw = opState.getRawKeyedState(); - compareKeyedState(Collections.singletonList(originalKeyedStateBackend), keyedStateBackend); - compareKeyedState(Collections.singletonList(originalKeyedStateRaw), keyGroupStateRaw); - } - } - actualOpStatesBackend.add(allParallelManagedOpStates); - actualOpStatesRaw.add(allParallelRawOpStates); - } - - comparePartitionableState(expectedOpStatesBackend, actualOpStatesBackend); - comparePartitionableState(expectedOpStatesRaw, actualOpStatesRaw); - } - - private static Tuple2 generateIDPair() { - JobVertexID jobVertexID = new JobVertexID(); - OperatorID operatorID = OperatorID.fromJobVertexID(jobVertexID); - return new Tuple2<>(jobVertexID, operatorID); - } - - /** - *

- * old topology. - * [operator1,operator2] * parallelism1 -> [operator3,operator4] * parallelism2 - *

- * - *

- * new topology - * - * [operator5,operator1,operator3] * newParallelism1 -> [operator3, operator6] * newParallelism2 - *

- * scaleType: - * 0 increase parallelism - * 1 decrease parallelism - * 2 same parallelism - */ - public void testStateRecoveryWithTopologyChange(int scaleType) throws Exception { - - /* - * Old topology - * CHAIN(op1 -> op2) * parallelism1 -> CHAIN(op3 -> op4) * parallelism2 - */ - Tuple2 id1 = generateIDPair(); - Tuple2 id2 = generateIDPair(); - int parallelism1 = 10; - int maxParallelism1 = 64; - - Tuple2 id3 = generateIDPair(); - Tuple2 id4 = generateIDPair(); - int parallelism2 = 10; - int maxParallelism2 = 64; - - List keyGroupPartitions2 = - StateAssignmentOperation.createKeyGroupPartitions(maxParallelism2, parallelism2); - - Map operatorStates = new HashMap<>(); - - //prepare vertex1 state - for (Tuple2 id : Arrays.asList(id1, id2)) { - OperatorState taskState = new OperatorState(id.f1, parallelism1, maxParallelism1); - operatorStates.put(id.f1, taskState); - for (int index = 0; index < taskState.getParallelism(); index++) { - OperatorStateHandle subManagedOperatorState = - generatePartitionableStateHandle(id.f0, index, 2, 8, false); - OperatorStateHandle subRawOperatorState = - generatePartitionableStateHandle(id.f0, index, 2, 8, true); - OperatorSubtaskState subtaskState = new OperatorSubtaskState( - subManagedOperatorState, - subRawOperatorState, - null, - null); - taskState.putState(index, subtaskState); - } - } - - List>> expectedManagedOperatorStates = new ArrayList<>(); - List>> expectedRawOperatorStates = new ArrayList<>(); - //prepare vertex2 state - for (Tuple2 id : Arrays.asList(id3, id4)) { - OperatorState operatorState = new OperatorState(id.f1, parallelism2, maxParallelism2); - operatorStates.put(id.f1, operatorState); - List> expectedManagedOperatorState = new ArrayList<>(); - List> expectedRawOperatorState = new ArrayList<>(); - expectedManagedOperatorStates.add(expectedManagedOperatorState); - expectedRawOperatorStates.add(expectedRawOperatorState); - - for (int index = 0; index < operatorState.getParallelism(); index++) { - OperatorStateHandle subManagedOperatorState = - generateChainedPartitionableStateHandle(id.f0, index, 2, 8, false) - .get(0); - OperatorStateHandle subRawOperatorState = - generateChainedPartitionableStateHandle(id.f0, index, 2, 8, true) - .get(0); - KeyGroupsStateHandle subManagedKeyedState = id.f0.equals(id3.f0) - ? generateKeyGroupState(id.f0, keyGroupPartitions2.get(index), false) - : null; - KeyGroupsStateHandle subRawKeyedState = id.f0.equals(id3.f0) - ? generateKeyGroupState(id.f0, keyGroupPartitions2.get(index), true) - : null; - - expectedManagedOperatorState.add(ChainedStateHandle.wrapSingleHandle(subManagedOperatorState)); - expectedRawOperatorState.add(ChainedStateHandle.wrapSingleHandle(subRawOperatorState)); - - OperatorSubtaskState subtaskState = new OperatorSubtaskState( - subManagedOperatorState, - subRawOperatorState, - subManagedKeyedState, - subRawKeyedState); - operatorState.putState(index, subtaskState); - } - } - - /* - * New topology - * CHAIN(op5 -> op1 -> op2) * newParallelism1 -> CHAIN(op3 -> op6) * newParallelism2 - */ - Tuple2 id5 = generateIDPair(); - int newParallelism1 = 10; - - Tuple2 id6 = generateIDPair(); - int newParallelism2 = parallelism2; - - if (scaleType == 0) { - newParallelism2 = 20; - } else if (scaleType == 1) { - newParallelism2 = 8; - } - - List newKeyGroupPartitions2 = - StateAssignmentOperation.createKeyGroupPartitions(maxParallelism2, newParallelism2); - - final ExecutionJobVertex newJobVertex1 = mockExecutionJobVertex( - id5.f0, - Arrays.asList(id2.f1, id1.f1, id5.f1), - newParallelism1, - maxParallelism1); - - final ExecutionJobVertex newJobVertex2 = mockExecutionJobVertex( - id3.f0, - Arrays.asList(id6.f1, id3.f1), - newParallelism2, - maxParallelism2); - - Map tasks = new HashMap<>(); - - tasks.put(id5.f0, newJobVertex1); - tasks.put(id3.f0, newJobVertex2); - - JobID jobID = new JobID(); - StandaloneCompletedCheckpointStore standaloneCompletedCheckpointStore = - spy(new StandaloneCompletedCheckpointStore(1)); - - CompletedCheckpoint completedCheckpoint = new CompletedCheckpoint( - jobID, - 2, - System.currentTimeMillis(), - System.currentTimeMillis() + 3000, - operatorStates, - Collections.emptyList(), - CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION), - new TestCompletedCheckpointStorageLocation()); - - when(standaloneCompletedCheckpointStore.getLatestCheckpoint(false)).thenReturn(completedCheckpoint); - - // set up the coordinator and validate the initial state - CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( - 600000, - 600000, - 0, - Integer.MAX_VALUE, - CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION, - true, - false, - 0); - CheckpointCoordinator coord = new CheckpointCoordinator( - new JobID(), - chkConfig, - newJobVertex1.getTaskVertices(), - newJobVertex1.getTaskVertices(), - newJobVertex1.getTaskVertices(), - new StandaloneCheckpointIDCounter(), - standaloneCompletedCheckpointStore, - new MemoryStateBackend(), - Executors.directExecutor(), - SharedStateRegistry.DEFAULT_FACTORY, - failureManager); - - coord.restoreLatestCheckpointedState(tasks, false, true); - - for (int i = 0; i < newJobVertex1.getParallelism(); i++) { - - final List operatorIds = newJobVertex1.getOperatorIDs(); - - JobManagerTaskRestore taskRestore = newJobVertex1.getTaskVertices()[i].getCurrentExecutionAttempt().getTaskRestore(); - Assert.assertEquals(2L, taskRestore.getRestoreCheckpointId()); - TaskStateSnapshot stateSnapshot = taskRestore.getTaskStateSnapshot(); - - OperatorSubtaskState headOpState = stateSnapshot.getSubtaskStateByOperatorID(operatorIds.get(operatorIds.size() - 1)); - assertTrue(headOpState.getManagedKeyedState().isEmpty()); - assertTrue(headOpState.getRawKeyedState().isEmpty()); - - // operator5 - { - int operatorIndexInChain = 2; - OperatorSubtaskState opState = - stateSnapshot.getSubtaskStateByOperatorID(operatorIds.get(operatorIndexInChain)); - - assertTrue(opState.getManagedOperatorState().isEmpty()); - assertTrue(opState.getRawOperatorState().isEmpty()); - } - // operator1 - { - int operatorIndexInChain = 1; - OperatorSubtaskState opState = - stateSnapshot.getSubtaskStateByOperatorID(operatorIds.get(operatorIndexInChain)); - - OperatorStateHandle expectedManagedOpState = generatePartitionableStateHandle( - id1.f0, i, 2, 8, false); - OperatorStateHandle expectedRawOpState = generatePartitionableStateHandle( - id1.f0, i, 2, 8, true); - - Collection managedOperatorState = opState.getManagedOperatorState(); - assertEquals(1, managedOperatorState.size()); - assertTrue(CommonTestUtils.isStreamContentEqual(expectedManagedOpState.openInputStream(), - managedOperatorState.iterator().next().openInputStream())); - - Collection rawOperatorState = opState.getRawOperatorState(); - assertEquals(1, rawOperatorState.size()); - assertTrue(CommonTestUtils.isStreamContentEqual(expectedRawOpState.openInputStream(), - rawOperatorState.iterator().next().openInputStream())); - } - // operator2 - { - int operatorIndexInChain = 0; - OperatorSubtaskState opState = - stateSnapshot.getSubtaskStateByOperatorID(operatorIds.get(operatorIndexInChain)); - - OperatorStateHandle expectedManagedOpState = generatePartitionableStateHandle( - id2.f0, i, 2, 8, false); - OperatorStateHandle expectedRawOpState = generatePartitionableStateHandle( - id2.f0, i, 2, 8, true); - - Collection managedOperatorState = opState.getManagedOperatorState(); - assertEquals(1, managedOperatorState.size()); - assertTrue(CommonTestUtils.isStreamContentEqual(expectedManagedOpState.openInputStream(), - managedOperatorState.iterator().next().openInputStream())); - - Collection rawOperatorState = opState.getRawOperatorState(); - assertEquals(1, rawOperatorState.size()); - assertTrue(CommonTestUtils.isStreamContentEqual(expectedRawOpState.openInputStream(), - rawOperatorState.iterator().next().openInputStream())); - } - } - - List>> actualManagedOperatorStates = new ArrayList<>(newJobVertex2.getParallelism()); - List>> actualRawOperatorStates = new ArrayList<>(newJobVertex2.getParallelism()); - - for (int i = 0; i < newJobVertex2.getParallelism(); i++) { - - final List operatorIds = newJobVertex2.getOperatorIDs(); - - JobManagerTaskRestore taskRestore = newJobVertex2.getTaskVertices()[i].getCurrentExecutionAttempt().getTaskRestore(); - Assert.assertEquals(2L, taskRestore.getRestoreCheckpointId()); - TaskStateSnapshot stateSnapshot = taskRestore.getTaskStateSnapshot(); - - // operator 3 - { - int operatorIndexInChain = 1; - OperatorSubtaskState opState = - stateSnapshot.getSubtaskStateByOperatorID(operatorIds.get(operatorIndexInChain)); - - List> actualSubManagedOperatorState = new ArrayList<>(1); - actualSubManagedOperatorState.add(opState.getManagedOperatorState()); - - List> actualSubRawOperatorState = new ArrayList<>(1); - actualSubRawOperatorState.add(opState.getRawOperatorState()); - - actualManagedOperatorStates.add(actualSubManagedOperatorState); - actualRawOperatorStates.add(actualSubRawOperatorState); - } - - // operator 6 - { - int operatorIndexInChain = 0; - OperatorSubtaskState opState = - stateSnapshot.getSubtaskStateByOperatorID(operatorIds.get(operatorIndexInChain)); - assertTrue(opState.getManagedOperatorState().isEmpty()); - assertTrue(opState.getRawOperatorState().isEmpty()); - - } - - KeyGroupsStateHandle originalKeyedStateBackend = generateKeyGroupState(id3.f0, newKeyGroupPartitions2.get(i), false); - KeyGroupsStateHandle originalKeyedStateRaw = generateKeyGroupState(id3.f0, newKeyGroupPartitions2.get(i), true); - - OperatorSubtaskState headOpState = - stateSnapshot.getSubtaskStateByOperatorID(operatorIds.get(operatorIds.size() - 1)); - - Collection keyedStateBackend = headOpState.getManagedKeyedState(); - Collection keyGroupStateRaw = headOpState.getRawKeyedState(); - - compareKeyedState(Collections.singletonList(originalKeyedStateBackend), keyedStateBackend); - compareKeyedState(Collections.singletonList(originalKeyedStateRaw), keyGroupStateRaw); - } - - comparePartitionableState(expectedManagedOperatorStates.get(0), actualManagedOperatorStates); - comparePartitionableState(expectedRawOperatorStates.get(0), actualRawOperatorStates); - } - - /** - * Tests that the externalized checkpoint configuration is respected. - */ - @Test - public void testExternalizedCheckpoints() throws Exception { - try { - final JobID jid = new JobID(); - final long timestamp = System.currentTimeMillis(); - - // create some mock Execution vertices that receive the checkpoint trigger messages - final ExecutionAttemptID attemptID1 = new ExecutionAttemptID(); - ExecutionVertex vertex1 = mockExecutionVertex(attemptID1); - - // set up the coordinator and validate the initial state - CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( - 600000, - 600000, - 0, - Integer.MAX_VALUE, - CheckpointRetentionPolicy.RETAIN_ON_FAILURE, - true, - false, - 0); - CheckpointCoordinator coord = new CheckpointCoordinator( - jid, - chkConfig, - new ExecutionVertex[] { vertex1 }, - new ExecutionVertex[] { vertex1 }, - new ExecutionVertex[] { vertex1 }, - new StandaloneCheckpointIDCounter(), - new StandaloneCompletedCheckpointStore(1), - new MemoryStateBackend(), - Executors.directExecutor(), - SharedStateRegistry.DEFAULT_FACTORY, - failureManager); - - assertTrue(coord.triggerCheckpoint(timestamp, false)); - - for (PendingCheckpoint checkpoint : coord.getPendingCheckpoints().values()) { - CheckpointProperties props = checkpoint.getProps(); - CheckpointProperties expected = CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.RETAIN_ON_FAILURE); - - assertEquals(expected, props); - } - - // the now we should have a completed checkpoint - coord.shutdown(JobStatus.FINISHED); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @Test - public void testCreateKeyGroupPartitions() { - testCreateKeyGroupPartitions(1, 1); - testCreateKeyGroupPartitions(13, 1); - testCreateKeyGroupPartitions(13, 2); - testCreateKeyGroupPartitions(Short.MAX_VALUE, 1); - testCreateKeyGroupPartitions(Short.MAX_VALUE, 13); - testCreateKeyGroupPartitions(Short.MAX_VALUE, Short.MAX_VALUE); + @Test + public void testCreateKeyGroupPartitions() { + testCreateKeyGroupPartitions(1, 1); + testCreateKeyGroupPartitions(13, 1); + testCreateKeyGroupPartitions(13, 2); + testCreateKeyGroupPartitions(Short.MAX_VALUE, 1); + testCreateKeyGroupPartitions(Short.MAX_VALUE, 13); + testCreateKeyGroupPartitions(Short.MAX_VALUE, Short.MAX_VALUE); Random r = new Random(1234); for (int k = 0; k < 1000; ++k) { @@ -3036,61 +1933,6 @@ public void testCreateKeyGroupPartitions() { } } - @Test - public void testStopPeriodicScheduler() throws Exception { - // create some mock Execution vertices that receive the checkpoint trigger messages - final ExecutionAttemptID attemptID1 = new ExecutionAttemptID(); - ExecutionVertex vertex1 = mockExecutionVertex(attemptID1); - - // set up the coordinator and validate the initial state - CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( - 600000, - 600000, - 0, - Integer.MAX_VALUE, - CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION, - true, - false, - 0); - CheckpointCoordinator coord = new CheckpointCoordinator( - new JobID(), - chkConfig, - new ExecutionVertex[] { vertex1 }, - new ExecutionVertex[] { vertex1 }, - new ExecutionVertex[] { vertex1 }, - new StandaloneCheckpointIDCounter(), - new StandaloneCompletedCheckpointStore(1), - new MemoryStateBackend(), - Executors.directExecutor(), - SharedStateRegistry.DEFAULT_FACTORY, - failureManager); - - // Periodic - try { - coord.triggerCheckpoint( - System.currentTimeMillis(), - CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION), - null, - true, - false); - fail("The triggerCheckpoint call expected an exception"); - } catch (CheckpointException e) { - assertEquals(CheckpointFailureReason.PERIODIC_SCHEDULER_SHUTDOWN, e.getCheckpointFailureReason()); - } - - // Not periodic - try { - coord.triggerCheckpoint( - System.currentTimeMillis(), - CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION), - null, - false, - false); - } catch (CheckpointException e) { - fail("Unexpected exception : " + e.getCheckpointFailureReason().message()); - } - } - private void testCreateKeyGroupPartitions(int maxParallelism, int parallelism) { List ranges = StateAssignmentOperation.createKeyGroupPartitions(maxParallelism, parallelism); for (int i = 0; i < maxParallelism; ++i) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java new file mode 100644 index 000000000000..b224d9ebd98a --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java @@ -0,0 +1,308 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.checkpoint; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.concurrent.Executors; +import org.apache.flink.runtime.executiongraph.Execution; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; +import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; +import org.apache.flink.runtime.state.SharedStateRegistry; +import org.apache.flink.runtime.state.memory.MemoryStateBackend; +import org.apache.flink.util.TestLogger; + +import org.junit.Before; +import org.junit.Test; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.mockExecutionVertex; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.doAnswer; + +/** + * Tests for checkpoint coordinator triggering. + */ +public class CheckpointCoordinatorTriggeringTest extends TestLogger { + private static final String TASK_MANAGER_LOCATION_INFO = "Unknown location"; + + private CheckpointFailureManager failureManager; + + @Before + public void setUp() throws Exception { + failureManager = new CheckpointFailureManager( + 0, + NoOpFailJobCall.INSTANCE); + } + + @Test + public void testPeriodicTriggering() { + try { + final JobID jid = new JobID(); + final long start = System.currentTimeMillis(); + + // create some mock execution vertices and trigger some checkpoint + + final ExecutionAttemptID triggerAttemptID = new ExecutionAttemptID(); + final ExecutionAttemptID ackAttemptID = new ExecutionAttemptID(); + final ExecutionAttemptID commitAttemptID = new ExecutionAttemptID(); + + ExecutionVertex triggerVertex = mockExecutionVertex(triggerAttemptID); + ExecutionVertex ackVertex = mockExecutionVertex(ackAttemptID); + ExecutionVertex commitVertex = mockExecutionVertex(commitAttemptID); + + final AtomicInteger numCalls = new AtomicInteger(); + + final Execution execution = triggerVertex.getCurrentExecutionAttempt(); + + doAnswer(new Answer() { + + private long lastId = -1; + private long lastTs = -1; + + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + long id = (Long) invocation.getArguments()[0]; + long ts = (Long) invocation.getArguments()[1]; + + assertTrue(id > lastId); + assertTrue(ts >= lastTs); + assertTrue(ts >= start); + + lastId = id; + lastTs = ts; + numCalls.incrementAndGet(); + return null; + } + }).when(execution).triggerCheckpoint(anyLong(), anyLong(), any(CheckpointOptions.class)); + + CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( + 10, // periodic interval is 10 ms + 200000, // timeout is very long (200 s) + 0, + Integer.MAX_VALUE, + CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION, + true, + false, + 0); + CheckpointCoordinator coord = new CheckpointCoordinator( + jid, + chkConfig, + new ExecutionVertex[] { triggerVertex }, + new ExecutionVertex[] { ackVertex }, + new ExecutionVertex[] { commitVertex }, + new StandaloneCheckpointIDCounter(), + new StandaloneCompletedCheckpointStore(2), + new MemoryStateBackend(), + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY, + failureManager); + + coord.startCheckpointScheduler(); + + long timeout = System.currentTimeMillis() + 60000; + do { + Thread.sleep(20); + } + while (timeout > System.currentTimeMillis() && numCalls.get() < 5); + assertTrue(numCalls.get() >= 5); + + coord.stopCheckpointScheduler(); + + // for 400 ms, no further calls may come. + // there may be the case that one trigger was fired and about to + // acquire the lock, such that after cancelling it will still do + // the remainder of its work + int numCallsSoFar = numCalls.get(); + Thread.sleep(400); + assertTrue(numCallsSoFar == numCalls.get() || + numCallsSoFar + 1 == numCalls.get()); + + // start another sequence of periodic scheduling + numCalls.set(0); + coord.startCheckpointScheduler(); + + timeout = System.currentTimeMillis() + 60000; + do { + Thread.sleep(20); + } + while (timeout > System.currentTimeMillis() && numCalls.get() < 5); + assertTrue(numCalls.get() >= 5); + + coord.stopCheckpointScheduler(); + + // for 400 ms, no further calls may come + // there may be the case that one trigger was fired and about to + // acquire the lock, such that after cancelling it will still do + // the remainder of its work + numCallsSoFar = numCalls.get(); + Thread.sleep(400); + assertTrue(numCallsSoFar == numCalls.get() || + numCallsSoFar + 1 == numCalls.get()); + + coord.shutdown(JobStatus.FINISHED); + } + catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } + } + + /** + * This test verified that after a completed checkpoint a certain time has passed before + * another is triggered. + */ + @Test + public void testMinTimeBetweenCheckpointsInterval() throws Exception { + final JobID jid = new JobID(); + + // create some mock execution vertices and trigger some checkpoint + final ExecutionAttemptID attemptID = new ExecutionAttemptID(); + final ExecutionVertex vertex = mockExecutionVertex(attemptID); + final Execution executionAttempt = vertex.getCurrentExecutionAttempt(); + + final BlockingQueue triggerCalls = new LinkedBlockingQueue<>(); + + doAnswer(invocation -> { + triggerCalls.add((Long) invocation.getArguments()[0]); + return null; + }).when(executionAttempt).triggerCheckpoint(anyLong(), anyLong(), any(CheckpointOptions.class)); + + final long delay = 50; + + CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( + 12, // periodic interval is 12 ms + 200_000, // timeout is very long (200 s) + delay, // 50 ms delay between checkpoints + 1, + CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION, + true, + false, + 0); + final CheckpointCoordinator coord = new CheckpointCoordinator( + jid, + chkConfig, + new ExecutionVertex[] { vertex }, + new ExecutionVertex[] { vertex }, + new ExecutionVertex[] { vertex }, + new StandaloneCheckpointIDCounter(), + new StandaloneCompletedCheckpointStore(2), + new MemoryStateBackend(), + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY, + failureManager); + + try { + coord.startCheckpointScheduler(); + + // wait until the first checkpoint was triggered + Long firstCallId = triggerCalls.take(); + assertEquals(1L, firstCallId.longValue()); + + AcknowledgeCheckpoint ackMsg = new AcknowledgeCheckpoint(jid, attemptID, 1L); + + // tell the coordinator that the checkpoint is done + final long ackTime = System.nanoTime(); + coord.receiveAcknowledgeMessage(ackMsg, TASK_MANAGER_LOCATION_INFO); + + // wait until the next checkpoint is triggered + Long nextCallId = triggerCalls.take(); + final long nextCheckpointTime = System.nanoTime(); + assertEquals(2L, nextCallId.longValue()); + + final long delayMillis = (nextCheckpointTime - ackTime) / 1_000_000; + + // we need to add one ms here to account for rounding errors + if (delayMillis + 1 < delay) { + fail("checkpoint came too early: delay was " + delayMillis + " but should have been at least " + delay); + } + } + finally { + coord.stopCheckpointScheduler(); + coord.shutdown(JobStatus.FINISHED); + } + } + + @Test + public void testStopPeriodicScheduler() throws Exception { + // create some mock Execution vertices that receive the checkpoint trigger messages + final ExecutionAttemptID attemptID1 = new ExecutionAttemptID(); + ExecutionVertex vertex1 = mockExecutionVertex(attemptID1); + + // set up the coordinator and validate the initial state + CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( + 600000, + 600000, + 0, + Integer.MAX_VALUE, + CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION, + true, + false, + 0); + CheckpointCoordinator coord = new CheckpointCoordinator( + new JobID(), + chkConfig, + new ExecutionVertex[] { vertex1 }, + new ExecutionVertex[] { vertex1 }, + new ExecutionVertex[] { vertex1 }, + new StandaloneCheckpointIDCounter(), + new StandaloneCompletedCheckpointStore(1), + new MemoryStateBackend(), + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY, + failureManager); + + // Periodic + try { + coord.triggerCheckpoint( + System.currentTimeMillis(), + CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION), + null, + true, + false); + fail("The triggerCheckpoint call expected an exception"); + } catch (CheckpointException e) { + assertEquals(CheckpointFailureReason.PERIODIC_SCHEDULER_SHUTDOWN, e.getCheckpointFailureReason()); + } + + // Not periodic + try { + coord.triggerCheckpoint( + System.currentTimeMillis(), + CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION), + null, + false, + false); + } catch (CheckpointException e) { + fail("Unexpected exception : " + e.getCheckpointFailureReason().message()); + } + } + +} From 3c84c058c67a02671cf325b27576b6faaba1d836 Mon Sep 17 00:00:00 2001 From: ifndef-SleePy Date: Thu, 19 Sep 2019 21:23:35 +0800 Subject: [PATCH 316/746] [hotfix] Correct code style of CheckpointCoordinator --- .../flink/runtime/checkpoint/CheckpointCoordinator.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index f30b135e099a..e7c7e172cdac 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -1249,7 +1249,7 @@ public void abortPendingCheckpoints(CheckpointException exception) { } /** - * If too many checkpoints are currently in progress, we need to mark that a request is queued + * If too many checkpoints are currently in progress, we need to mark that a request is queued. * * @throws CheckpointException If too many checkpoints are currently in progress. */ @@ -1265,7 +1265,7 @@ private void checkConcurrentCheckpoints() throws CheckpointException { } /** - * Make sure the minimum interval between checkpoints has passed + * Make sure the minimum interval between checkpoints has passed. * * @throws CheckpointException If the minimum interval between checkpoints has not passed. */ From 5ab6261df2efb3cb34403cd76e77ca3672c57066 Mon Sep 17 00:00:00 2001 From: ifndef-SleePy Date: Wed, 18 Sep 2019 21:04:09 +0800 Subject: [PATCH 317/746] [FLINK-13904][checkpointing] Make trigger thread of CheckpointCoordinator single-threaded --- .../checkpoint/CheckpointCoordinator.java | 25 +--- .../executiongraph/ExecutionGraph.java | 19 +++ .../CheckpointCoordinatorFailureTest.java | 7 + .../CheckpointCoordinatorMasterHooksTest.java | 9 +- .../CheckpointCoordinatorRestoringTest.java | 10 ++ .../checkpoint/CheckpointCoordinatorTest.java | 73 ++++++++--- .../CheckpointCoordinatorTestingUtils.java | 121 ++++++++++++++++++ .../CheckpointCoordinatorTriggeringTest.java | 9 ++ .../CheckpointStateRestoreTest.java | 9 ++ ...overStrategyCheckpointCoordinatorTest.java | 57 +-------- .../util/TestingScheduledExecutor.java | 62 +++++++++ 11 files changed, 309 insertions(+), 92 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingScheduledExecutor.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index e7c7e172cdac..df9278ec2785 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.checkpoint.hooks.MasterHooks; 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.Execution; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; @@ -41,7 +42,6 @@ import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.SharedStateRegistryFactory; import org.apache.flink.runtime.state.StateBackend; -import org.apache.flink.runtime.taskmanager.DispatcherThreadFactory; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.Preconditions; import org.apache.flink.util.StringUtils; @@ -62,7 +62,6 @@ import java.util.concurrent.CompletionException; import java.util.concurrent.Executor; import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -145,8 +144,9 @@ public class CheckpointCoordinator { /** The maximum number of checkpoints that may be in progress at the same time. */ private final int maxConcurrentCheckpointAttempts; - /** The timer that handles the checkpoint timeouts and triggers periodic checkpoints. */ - private final ScheduledThreadPoolExecutor timer; + /** The timer that handles the checkpoint timeouts and triggers periodic checkpoints. + * It must be single-threaded. Eventually it will be replaced by main thread executor. */ + private final ScheduledExecutor timer; /** The master checkpoint hooks executed by this checkpoint coordinator. */ private final HashMap> masterHooks; @@ -200,6 +200,7 @@ public CheckpointCoordinator( CompletedCheckpointStore completedCheckpointStore, StateBackend checkpointStateBackend, Executor executor, + ScheduledExecutor timer, SharedStateRegistryFactory sharedStateRegistryFactory, CheckpointFailureManager failureManager) { @@ -239,13 +240,7 @@ public CheckpointCoordinator( this.recentPendingCheckpoints = new ArrayDeque<>(NUM_GHOST_CHECKPOINT_IDS); this.masterHooks = new HashMap<>(); - this.timer = new ScheduledThreadPoolExecutor(1, - new DispatcherThreadFactory(Thread.currentThread().getThreadGroup(), "Checkpoint Timer")); - - // make sure the timer internally cleans up and does not hold onto stale scheduled tasks - this.timer.setRemoveOnCancelPolicy(true); - this.timer.setContinueExistingPeriodicTasksAfterShutdownPolicy(false); - this.timer.setExecuteExistingDelayedTasksAfterShutdownPolicy(false); + this.timer = timer; this.checkpointProperties = CheckpointProperties.forCheckpoint(chkConfig.getCheckpointRetentionPolicy()); @@ -336,9 +331,6 @@ public void shutdown(JobStatus jobStatus) throws Exception { MasterHooks.close(masterHooks.values(), LOG); masterHooks.clear(); - // shut down the thread that handles the timeouts and pending triggers - timer.shutdownNow(); - // clear and discard all pending checkpoints for (PendingCheckpoint pending : pendingCheckpoints.values()) { failPendingCheckpoint(pending, CheckpointFailureReason.CHECKPOINT_COORDINATOR_SHUTDOWN); @@ -997,11 +989,6 @@ private void triggerQueuedRequests() { } } - @VisibleForTesting - int getNumScheduledTasks() { - return timer.getQueue().size(); - } - // -------------------------------------------------------------------------------------------- // Checkpoint State Restoring // -------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index dab62df761fd..a3b637485ab9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -45,6 +45,7 @@ import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.concurrent.FutureUtils.ConjunctFuture; +import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.execution.SuppressRestartsException; import org.apache.flink.runtime.executiongraph.failover.FailoverStrategy; @@ -79,6 +80,7 @@ import org.apache.flink.runtime.shuffle.ShuffleMaster; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.StateBackend; +import org.apache.flink.runtime.taskmanager.DispatcherThreadFactory; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.types.Either; import org.apache.flink.util.ExceptionUtils; @@ -113,6 +115,7 @@ import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; +import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLongFieldUpdater; @@ -319,8 +322,13 @@ public class ExecutionGraph implements AccessExecutionGraph { // ------ Fields that are relevant to the execution and need to be cleared before archiving ------- /** The coordinator for checkpoints, if snapshot checkpoints are enabled. */ + @Nullable private CheckpointCoordinator checkpointCoordinator; + /** TODO, replace it with main thread executor. */ + @Nullable + private ScheduledExecutorService checkpointCoordinatorTimer; + /** Checkpoint stats tracker separate from the coordinator in order to be * available after archiving. */ private CheckpointStatsTracker checkpointStatsTracker; @@ -604,6 +612,12 @@ public void failJobDueToTaskFailure(Throwable cause, ExecutionAttemptID failingT } ); + checkState(checkpointCoordinatorTimer == null); + + checkpointCoordinatorTimer = Executors.newSingleThreadScheduledExecutor( + new DispatcherThreadFactory( + Thread.currentThread().getThreadGroup(), "Checkpoint Timer")); + // create the coordinator that triggers and commits checkpoints and holds the state checkpointCoordinator = new CheckpointCoordinator( jobInformation.getJobId(), @@ -615,6 +629,7 @@ public void failJobDueToTaskFailure(Throwable cause, ExecutionAttemptID failingT checkpointStore, checkpointStateBackend, ioExecutor, + new ScheduledExecutorServiceAdapter(checkpointCoordinatorTimer), SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -1559,6 +1574,10 @@ private void onTerminalState(JobStatus status) { if (coord != null) { coord.shutdown(status); } + if (checkpointCoordinatorTimer != null) { + checkpointCoordinatorTimer.shutdownNow(); + checkpointCoordinatorTimer = null; + } } catch (Exception e) { LOG.error("Error while cleaning up after execution", e); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java index b6b79302a68f..39a8c2e6bffe 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java @@ -31,8 +31,10 @@ import org.apache.flink.runtime.state.OperatorStreamStateHandle; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.memory.MemoryStateBackend; +import org.apache.flink.runtime.util.TestingScheduledExecutor; import org.apache.flink.util.TestLogger; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.powermock.core.classloader.annotations.PrepareForTest; @@ -56,6 +58,10 @@ @PrepareForTest(PendingCheckpoint.class) public class CheckpointCoordinatorFailureTest extends TestLogger { + @Rule + public final TestingScheduledExecutor testingScheduledExecutor = + new TestingScheduledExecutor(); + /** * Tests that a failure while storing a completed checkpoint in the completed checkpoint store * will properly fail the originating pending checkpoint and clean upt the completed checkpoint. @@ -93,6 +99,7 @@ public void testFailingCompletedCheckpointStoreAdd() throws Exception { new FailingCompletedCheckpointStore(), new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java index 8453cba62438..d85d01431c6a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java @@ -32,7 +32,9 @@ import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.memory.MemoryStateBackend; import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; +import org.apache.flink.runtime.util.TestingScheduledExecutor; +import org.junit.Rule; import org.junit.Test; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -69,6 +71,10 @@ */ public class CheckpointCoordinatorMasterHooksTest { + @Rule + public final TestingScheduledExecutor testingScheduledExecutor = + new TestingScheduledExecutor(); + // ------------------------------------------------------------------------ // hook registration // ------------------------------------------------------------------------ @@ -421,7 +427,7 @@ public void testTypeIncompatibleWithHookOnRestore() { // utilities // ------------------------------------------------------------------------ - private static CheckpointCoordinator instantiateCheckpointCoordinator(JobID jid, ExecutionVertex... ackVertices) { + private CheckpointCoordinator instantiateCheckpointCoordinator(JobID jid, ExecutionVertex... ackVertices) { CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( 10000000L, 600000L, @@ -441,6 +447,7 @@ private static CheckpointCoordinator instantiateCheckpointCoordinator(JobID jid, new StandaloneCompletedCheckpointStore(10), new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, new CheckpointFailureManager( 0, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java index 1259144fba29..1725ef225bdc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java @@ -39,6 +39,7 @@ import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.RecoverableCompletedCheckpointStore; +import org.apache.flink.runtime.util.TestingScheduledExecutor; import org.apache.flink.util.SerializableObject; import org.apache.flink.util.TestLogger; @@ -89,6 +90,10 @@ public class CheckpointCoordinatorRestoringTest extends TestLogger { private static final String TASK_MANAGER_LOCATION_INFO = "Unknown location"; + @Rule + public final TestingScheduledExecutor testingScheduledExecutor = + new TestingScheduledExecutor(); + private CheckpointFailureManager failureManager; @Rule @@ -158,6 +163,7 @@ public void testRestoreLatestCheckpointedState() throws Exception { store, new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -290,6 +296,7 @@ private void testRestoreLatestCheckpointIsPreferSavepoint(boolean isPreferCheckp store, new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -439,6 +446,7 @@ private void testRestoreLatestCheckpointedStateWithChangingParallelism(boolean s new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -616,6 +624,7 @@ public void testRestoreLatestCheckpointFailureWhenMaxParallelismChanges() throws new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -870,6 +879,7 @@ public void testStateRecoveryWithTopologyChange(int scaleType) throws Exception standaloneCompletedCheckpointStore, new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java index 2d86a0656c32..ddd2f8e30b12 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java @@ -21,7 +21,9 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.TestingScheduledServiceWithRecordingScheduledTasks; import org.apache.flink.runtime.concurrent.Executors; +import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.Execution; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; @@ -48,6 +50,7 @@ import org.apache.flink.runtime.state.memory.MemoryStateBackend; import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; import org.apache.flink.runtime.testutils.RecoverableCompletedCheckpointStore; +import org.apache.flink.runtime.util.TestingScheduledExecutor; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; @@ -103,6 +106,10 @@ public class CheckpointCoordinatorTest extends TestLogger { private static final String TASK_MANAGER_LOCATION_INFO = "Unknown location"; + @Rule + public final TestingScheduledExecutor testingScheduledExecutor = + new TestingScheduledExecutor(); + private CheckpointFailureManager failureManager; @Rule @@ -151,6 +158,7 @@ public void testCheckpointAbortsIfTriggerTasksAreNotExecuted() { new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -218,6 +226,7 @@ public void testCheckpointAbortsIfTriggerTasksAreFinished() { new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -276,6 +285,7 @@ public void testCheckpointAbortsIfAckTasksAreNotExecuted() { new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -325,8 +335,11 @@ public void failJobDueToTaskFailure(Throwable cause, ExecutionAttemptID failingT } }); + final TestingScheduledServiceWithRecordingScheduledTasks scheduledExecutorService = + new TestingScheduledServiceWithRecordingScheduledTasks(testingScheduledExecutor.getScheduledExecutor()); + // set up the coordinator - CheckpointCoordinator coord = getCheckpointCoordinator(jid, vertex1, vertex2, checkpointFailureManager); + CheckpointCoordinator coord = getCheckpointCoordinator(jid, vertex1, vertex2, checkpointFailureManager, scheduledExecutorService); try { // trigger the checkpoint. this should succeed @@ -376,8 +389,10 @@ public void testTriggerAndDeclineCheckpointSimple() { ExecutionVertex vertex1 = mockExecutionVertex(attemptID1); ExecutionVertex vertex2 = mockExecutionVertex(attemptID2); + final TestingScheduledServiceWithRecordingScheduledTasks scheduledExecutorService = + new TestingScheduledServiceWithRecordingScheduledTasks(testingScheduledExecutor.getScheduledExecutor()); // set up the coordinator and validate the initial state - CheckpointCoordinator coord = getCheckpointCoordinator(jid, vertex1, vertex2, failureManager); + CheckpointCoordinator coord = getCheckpointCoordinator(jid, vertex1, vertex2, failureManager, scheduledExecutorService); assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -390,7 +405,7 @@ public void testTriggerAndDeclineCheckpointSimple() { assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); // we have one task scheduled that will cancel after timeout - assertEquals(1, coord.getNumScheduledTasks()); + assertEquals(1, scheduledExecutorService.getNumScheduledOnceTasks()); long checkpointId = coord.getPendingCheckpoints().entrySet().iterator().next().getKey(); PendingCheckpoint checkpoint = coord.getPendingCheckpoints().get(checkpointId); @@ -427,7 +442,7 @@ public void testTriggerAndDeclineCheckpointSimple() { assertTrue(checkpoint.isDiscarded()); // the canceler is also removed - assertEquals(0, coord.getNumScheduledTasks()); + assertEquals(0, scheduledExecutorService.getNumScheduledOnceTasks()); // validate that we have no new pending checkpoint assertEquals(0, coord.getNumberOfPendingCheckpoints()); @@ -464,12 +479,14 @@ public void testTriggerAndDeclineCheckpointComplex() { ExecutionVertex vertex1 = mockExecutionVertex(attemptID1); ExecutionVertex vertex2 = mockExecutionVertex(attemptID2); + final TestingScheduledServiceWithRecordingScheduledTasks scheduledExecutorService = + new TestingScheduledServiceWithRecordingScheduledTasks(testingScheduledExecutor.getScheduledExecutor()); // set up the coordinator and validate the initial state - CheckpointCoordinator coord = getCheckpointCoordinator(jid, vertex1, vertex2, failureManager); + CheckpointCoordinator coord = getCheckpointCoordinator(jid, vertex1, vertex2, failureManager, scheduledExecutorService); assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); - assertEquals(0, coord.getNumScheduledTasks()); + assertEquals(0, scheduledExecutorService.getNumScheduledOnceTasks()); // trigger the first checkpoint. this should succeed assertTrue(coord.triggerCheckpoint(timestamp, false)); @@ -480,7 +497,7 @@ public void testTriggerAndDeclineCheckpointComplex() { // validate that we have a pending checkpoint assertEquals(2, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); - assertEquals(2, coord.getNumScheduledTasks()); + assertEquals(2, scheduledExecutorService.getNumScheduledOnceTasks()); Iterator> it = coord.getPendingCheckpoints().entrySet().iterator(); long checkpoint1Id = it.next().getKey(); @@ -527,7 +544,7 @@ public void testTriggerAndDeclineCheckpointComplex() { // validate that we have only one pending checkpoint left assertEquals(1, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); - assertEquals(1, coord.getNumScheduledTasks()); + assertEquals(1, scheduledExecutorService.getNumScheduledOnceTasks()); // validate that it is the same second checkpoint from earlier long checkpointIdNew = coord.getPendingCheckpoints().entrySet().iterator().next().getKey(); @@ -570,12 +587,14 @@ public void testTriggerAndConfirmSimpleCheckpoint() { ExecutionVertex vertex1 = mockExecutionVertex(attemptID1); ExecutionVertex vertex2 = mockExecutionVertex(attemptID2); + final TestingScheduledServiceWithRecordingScheduledTasks scheduledExecutorService = + new TestingScheduledServiceWithRecordingScheduledTasks(testingScheduledExecutor.getScheduledExecutor()); // set up the coordinator and validate the initial state - CheckpointCoordinator coord = getCheckpointCoordinator(jid, vertex1, vertex2, failureManager); + CheckpointCoordinator coord = getCheckpointCoordinator(jid, vertex1, vertex2, failureManager, scheduledExecutorService); assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); - assertEquals(0, coord.getNumScheduledTasks()); + assertEquals(0, scheduledExecutorService.getNumScheduledOnceTasks()); // trigger the first checkpoint. this should succeed assertTrue(coord.triggerCheckpoint(timestamp, false)); @@ -583,7 +602,7 @@ public void testTriggerAndConfirmSimpleCheckpoint() { // validate that we have a pending checkpoint assertEquals(1, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); - assertEquals(1, coord.getNumScheduledTasks()); + assertEquals(1, scheduledExecutorService.getNumScheduledOnceTasks()); long checkpointId = coord.getPendingCheckpoints().entrySet().iterator().next().getKey(); PendingCheckpoint checkpoint = coord.getPendingCheckpoints().get(checkpointId); @@ -640,7 +659,7 @@ public void testTriggerAndConfirmSimpleCheckpoint() { assertEquals(0, coord.getNumberOfPendingCheckpoints()); // the canceler should be removed now - assertEquals(0, coord.getNumScheduledTasks()); + assertEquals(0, scheduledExecutorService.getNumScheduledOnceTasks()); // validate that the subtasks states have registered their shared states. { @@ -672,7 +691,7 @@ public void testTriggerAndConfirmSimpleCheckpoint() { assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(1, coord.getNumberOfRetainedSuccessfulCheckpoints()); - assertEquals(0, coord.getNumScheduledTasks()); + assertEquals(0, scheduledExecutorService.getNumScheduledOnceTasks()); CompletedCheckpoint successNew = coord.getSuccessfulCheckpoints().get(0); assertEquals(jid, successNew.getJobId()); @@ -744,6 +763,7 @@ public void testMultipleConcurrentCheckpoints() { new StandaloneCompletedCheckpointStore(2), new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -881,6 +901,7 @@ public void testSuccessfulCheckpointSubsumesUnsuccessful() { new StandaloneCompletedCheckpointStore(10), new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -1051,6 +1072,7 @@ public void testCheckpointTimeoutIsolated() { new StandaloneCompletedCheckpointStore(2), new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -1135,6 +1157,7 @@ public void testHandleMessagesForNonExistingCheckpoints() { new StandaloneCompletedCheckpointStore(2), new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -1205,6 +1228,7 @@ public void testStateCleanupForLateOrUnknownMessages() throws Exception { new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -1310,8 +1334,10 @@ public void testTriggerAndConfirmSimpleSavepoint() throws Exception { ExecutionVertex vertex1 = mockExecutionVertex(attemptID1); ExecutionVertex vertex2 = mockExecutionVertex(attemptID2); + final TestingScheduledServiceWithRecordingScheduledTasks scheduledExecutorService = + new TestingScheduledServiceWithRecordingScheduledTasks(testingScheduledExecutor.getScheduledExecutor()); // set up the coordinator and validate the initial state - CheckpointCoordinator coord = getCheckpointCoordinator(jid, vertex1, vertex2, failureManager); + CheckpointCoordinator coord = getCheckpointCoordinator(jid, vertex1, vertex2, failureManager, scheduledExecutorService); assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -1468,6 +1494,7 @@ public void testSavepointsAreNotSubsumed() throws Exception { new StandaloneCompletedCheckpointStore(10), new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -1568,6 +1595,7 @@ private void testMaxConcurrentAttempts(int maxConcurrentAttempts) { new StandaloneCompletedCheckpointStore(2), new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -1648,6 +1676,7 @@ public void testMaxConcurrentAttempsWithSubsumption() { new StandaloneCompletedCheckpointStore(2), new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -1731,6 +1760,7 @@ public void testPeriodicSchedulingWithInactiveTasks() { new StandaloneCompletedCheckpointStore(2), new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -1790,6 +1820,7 @@ public void testConcurrentSavepoints() throws Exception { new StandaloneCompletedCheckpointStore(2), new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -1850,6 +1881,7 @@ public void testMinDelayBetweenSavepoints() throws Exception { new StandaloneCompletedCheckpointStore(2), new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -1895,6 +1927,7 @@ public void testExternalizedCheckpoints() throws Exception { new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -2131,6 +2164,7 @@ public void testCheckpointStatsTrackerPendingCheckpointCallback() { new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -2176,6 +2210,7 @@ public void testCheckpointStatsTrackerRestoreCallback() throws Exception { store, new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -2245,6 +2280,7 @@ public void testSharedStateRegistrationOnRestore() throws Exception { store, new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), deleteExecutor -> { SharedStateRegistry instance = new SharedStateRegistry(deleteExecutor); createdSharedStateRegistries.add(instance); @@ -2385,6 +2421,8 @@ public void jobFailsIfInFlightSynchronousSavepointIsDiscarded() throws Exception final ExecutionVertex vertex1 = mockExecutionVertex(attemptID1); final ExecutionVertex vertex2 = mockExecutionVertex(attemptID2); + final TestingScheduledServiceWithRecordingScheduledTasks scheduledExecutorService = + new TestingScheduledServiceWithRecordingScheduledTasks(testingScheduledExecutor.getScheduledExecutor()); // set up the coordinator and validate the initial state final CheckpointCoordinator coordinator = getCheckpointCoordinator(jobId, vertex1, vertex2, new CheckpointFailureManager( @@ -2400,7 +2438,8 @@ public void failJob(Throwable cause) { public void failJobDueToTaskFailure(Throwable cause, ExecutionAttemptID failingTask) { throw new AssertionError("This method should not be called for the test."); } - })); + }), + scheduledExecutorService); final CompletableFuture savepointFuture = coordinator .triggerSynchronousSavepoint(10L, false, "test-dir"); @@ -2430,7 +2469,8 @@ private CheckpointCoordinator getCheckpointCoordinator( final JobID jobId, final ExecutionVertex vertex1, final ExecutionVertex vertex2, - final CheckpointFailureManager failureManager) { + final CheckpointFailureManager failureManager, + final ScheduledExecutor timer) { final CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( 600000, @@ -2452,6 +2492,7 @@ private CheckpointCoordinator getCheckpointCoordinator( new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), + timer, SharedStateRegistry.DEFAULT_FACTORY, failureManager); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java index 9578ac74a66d..76f042b6cabf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.Execution; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; @@ -49,12 +50,21 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Random; +import java.util.Set; import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.Delayed; +import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import static org.apache.flink.util.Preconditions.checkNotNull; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; @@ -469,4 +479,115 @@ static ExecutionJobVertex mockExecutionJobVertex(JobVertexID id, ExecutionVertex } return vertex; } + + static class TestingScheduledServiceWithRecordingScheduledTasks implements ScheduledExecutor { + + private final ScheduledExecutor scheduledExecutor; + + private final Set tasksScheduledOnce; + + public TestingScheduledServiceWithRecordingScheduledTasks(ScheduledExecutor scheduledExecutor) { + this.scheduledExecutor = checkNotNull(scheduledExecutor); + tasksScheduledOnce = new HashSet<>(); + } + + public int getNumScheduledOnceTasks() { + synchronized (tasksScheduledOnce) { + return tasksScheduledOnce.size(); + } + } + + @Override + public ScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) { + final UUID id = UUID.randomUUID(); + synchronized (tasksScheduledOnce) { + tasksScheduledOnce.add(id); + } + return new TestingScheduledFuture<>(id, scheduledExecutor.schedule(() -> { + synchronized (tasksScheduledOnce) { + tasksScheduledOnce.remove(id); + } + command.run(); + }, delay, unit)); + } + + @Override + public ScheduledFuture schedule(Callable callable, long delay, TimeUnit unit) { + final UUID id = UUID.randomUUID(); + synchronized (tasksScheduledOnce) { + tasksScheduledOnce.add(id); + } + return new TestingScheduledFuture<>(id, scheduledExecutor.schedule(() -> { + synchronized (tasksScheduledOnce) { + tasksScheduledOnce.remove(id); + } + return callable.call(); + }, delay, unit)); + } + + @Override + public ScheduledFuture scheduleAtFixedRate(Runnable command, long initialDelay, long period, TimeUnit unit) { + return scheduledExecutor.scheduleAtFixedRate(command, initialDelay, period, unit); + } + + @Override + public ScheduledFuture scheduleWithFixedDelay(Runnable command, long initialDelay, long delay, TimeUnit unit) { + return scheduledExecutor.scheduleWithFixedDelay(command, initialDelay, delay, unit); + } + + @Override + public void execute(Runnable command) { + scheduledExecutor.execute(command); + } + + private class TestingScheduledFuture implements ScheduledFuture { + + private final ScheduledFuture scheduledFuture; + + private final UUID id; + + public TestingScheduledFuture(UUID id, ScheduledFuture scheduledFuture) { + this.id = checkNotNull(id); + this.scheduledFuture = checkNotNull(scheduledFuture); + } + + @Override + public long getDelay(TimeUnit unit) { + return scheduledFuture.getDelay(unit); + } + + @Override + public int compareTo(Delayed o) { + return scheduledFuture.compareTo(o); + } + + @Override + public boolean cancel(boolean mayInterruptIfRunning) { + synchronized (tasksScheduledOnce) { + tasksScheduledOnce.remove(id); + } + return scheduledFuture.cancel(mayInterruptIfRunning); + } + + @Override + public boolean isCancelled() { + return scheduledFuture.isCancelled(); + } + + @Override + public boolean isDone() { + return scheduledFuture.isDone(); + } + + @Override + public V get() throws InterruptedException, ExecutionException { + return scheduledFuture.get(); + } + + @Override + public V get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { + return scheduledFuture.get(timeout, unit); + } + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java index b224d9ebd98a..a157b6f00744 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java @@ -28,9 +28,11 @@ import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.memory.MemoryStateBackend; +import org.apache.flink.runtime.util.TestingScheduledExecutor; import org.apache.flink.util.TestLogger; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -53,6 +55,10 @@ public class CheckpointCoordinatorTriggeringTest extends TestLogger { private static final String TASK_MANAGER_LOCATION_INFO = "Unknown location"; + @Rule + public final TestingScheduledExecutor testingScheduledExecutor = + new TestingScheduledExecutor(); + private CheckpointFailureManager failureManager; @Before @@ -122,6 +128,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { new StandaloneCompletedCheckpointStore(2), new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -216,6 +223,7 @@ public void testMinTimeBetweenCheckpointsInterval() throws Exception { new StandaloneCompletedCheckpointStore(2), new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -276,6 +284,7 @@ public void testStopPeriodicScheduler() throws Exception { new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java index 42849b5e6e48..080e1c786ba8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java @@ -34,11 +34,13 @@ import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.memory.MemoryStateBackend; import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; +import org.apache.flink.runtime.util.TestingScheduledExecutor; import org.apache.flink.util.SerializableObject; import org.hamcrest.BaseMatcher; import org.hamcrest.Description; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; import org.mockito.Mockito; import org.mockito.hamcrest.MockitoHamcrest; @@ -63,6 +65,10 @@ public class CheckpointStateRestoreTest { private static final String TASK_MANAGER_LOCATION_INFO = "Unknown location"; + @Rule + public final TestingScheduledExecutor testingScheduledExecutor = + new TestingScheduledExecutor(); + private CheckpointFailureManager failureManager; @Before @@ -127,6 +133,7 @@ public void testSetState() { new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -211,6 +218,7 @@ public void testNoCheckpointAvailable() { new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -276,6 +284,7 @@ public void testNonRestoredState() throws Exception { new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), + testingScheduledExecutor.getScheduledExecutor(), SharedStateRegistry.DEFAULT_FACTORY, failureManager); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java index 17a5bccead2a..df3e97eee823 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java @@ -19,10 +19,8 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobID; -import org.apache.flink.mock.Whitebox; import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor; -import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.Execution; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; @@ -33,17 +31,12 @@ import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.memory.MemoryStateBackend; import org.apache.flink.util.TestLogger; -import org.apache.flink.util.concurrent.NeverCompleteFuture; import org.junit.Before; import org.junit.Test; import org.mockito.Mockito; -import java.util.concurrent.Callable; -import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.TimeUnit; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -90,19 +83,13 @@ public void testAbortPendingCheckpointsWithTriggerValidation() { new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), + manualThreadExecutor, SharedStateRegistry.DEFAULT_FACTORY, mock(CheckpointFailureManager.class)); // switch current execution's state to running to allow checkpoint could be triggered. mockExecutionRunning(executionVertex); - // use manual checkpoint timer to trigger period checkpoints as we expect. - ManualCheckpointTimer manualCheckpointTimer = new ManualCheckpointTimer(manualThreadExecutor); - // set the init delay as 0 to ensure first checkpoint could be triggered once we trigger the manual executor - // this is used to avoid the randomness of when to trigger the first checkpoint (introduced via FLINK-9352) - manualCheckpointTimer.setManualDelay(0L); - Whitebox.setInternalState(checkpointCoordinator, "timer", manualCheckpointTimer); - checkpointCoordinator.startCheckpointScheduler(); assertTrue(checkpointCoordinator.isCurrentPeriodicTriggerAvailable()); manualThreadExecutor.triggerAll(); @@ -140,46 +127,4 @@ private ExecutionVertex mockExecutionVertex() { private void mockExecutionRunning(ExecutionVertex executionVertex) { when(executionVertex.getCurrentExecutionAttempt().getState()).thenReturn(ExecutionState.RUNNING); } - - public static class ManualCheckpointTimer extends ScheduledThreadPoolExecutor { - private final ScheduledExecutor scheduledExecutor; - private long manualDelay = 0; - - ManualCheckpointTimer(final ScheduledExecutor scheduledExecutor) { - super(0); - this.scheduledExecutor = scheduledExecutor; - } - - void setManualDelay(long manualDelay) { - this.manualDelay = manualDelay; - } - - @Override - public ScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) { - // used as checkpoint canceller, as we don't want pending checkpoint cancelled, this should never be scheduled. - return new NeverCompleteFuture(delay); - } - - @Override - public ScheduledFuture schedule(Callable callable, long delay, TimeUnit unit) { - throw new UnsupportedOperationException(); - } - - @Override - public ScheduledFuture scheduleAtFixedRate(Runnable command, long initialDelay, long period, TimeUnit unit) { - // used to schedule periodic checkpoints. - // this would use configured 'manualDelay' to let the task schedule with the wanted delay. - return scheduledExecutor.scheduleWithFixedDelay(command, manualDelay, period, unit); - } - - @Override - public ScheduledFuture scheduleWithFixedDelay(Runnable command, long initialDelay, long delay, TimeUnit unit) { - throw new UnsupportedOperationException(); - } - - @Override - public void execute(Runnable command) { - scheduledExecutor.execute(command); - } - } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingScheduledExecutor.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingScheduledExecutor.java new file mode 100644 index 000000000000..930af9fd118b --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingScheduledExecutor.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.util; + +import org.apache.flink.runtime.concurrent.ScheduledExecutor; +import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter; +import org.apache.flink.util.ExecutorUtils; + +import org.junit.rules.ExternalResource; + +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +/** + * Provide an automatically shut down scheduled executor for testing. + */ +public class TestingScheduledExecutor extends ExternalResource { + + private long shutdownTimeoutMillis; + private ScheduledExecutor scheduledExecutor; + private ScheduledExecutorService innerExecutorService; + + public TestingScheduledExecutor() { + this(500L); + } + + public TestingScheduledExecutor(long shutdownTimeoutMillis) { + this.shutdownTimeoutMillis = shutdownTimeoutMillis; + } + + @Override + protected void before() { + this.innerExecutorService = Executors.newSingleThreadScheduledExecutor(); + this.scheduledExecutor = new ScheduledExecutorServiceAdapter(innerExecutorService); + } + + @Override + protected void after() { + ExecutorUtils.gracefulShutdown(shutdownTimeoutMillis, TimeUnit.MILLISECONDS, innerExecutorService); + } + + public ScheduledExecutor getScheduledExecutor() { + return scheduledExecutor; + } +} From fc196737b818261d039d6ecb2c1555c340f0e2c0 Mon Sep 17 00:00:00 2001 From: ifndef-SleePy Date: Sun, 29 Sep 2019 14:50:09 +0800 Subject: [PATCH 318/746] [FLINK-13904][tests] Support checkpoint consumer of SimpleAckingTaskManagerGateway --- .../utils/SimpleAckingTaskManagerGateway.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleAckingTaskManagerGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleAckingTaskManagerGateway.java index 0d07f3d31199..e09d8beda8aa 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleAckingTaskManagerGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleAckingTaskManagerGateway.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.java.tuple.Tuple6; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.concurrent.FutureUtils; @@ -55,6 +56,8 @@ public class SimpleAckingTaskManagerGateway implements TaskManagerGateway { private BiConsumer> releasePartitionsConsumer = (ignore1, ignore2) -> { }; + private Consumer> checkpointConsumer = ignore -> { }; + public void setSubmitConsumer(Consumer submitConsumer) { this.submitConsumer = submitConsumer; } @@ -71,6 +74,10 @@ public void setReleasePartitionsConsumer(BiConsumer> checkpointConsumer) { + this.checkpointConsumer = checkpointConsumer; + } + @Override public String getAddress() { return address; @@ -123,7 +130,9 @@ public void triggerCheckpoint( long checkpointId, long timestamp, CheckpointOptions checkpointOptions, - boolean advanceToEndOfEventTime) {} + boolean advanceToEndOfEventTime) { + checkpointConsumer.accept(Tuple6.of(executionAttemptID, jobId, checkpointId, timestamp, checkpointOptions, advanceToEndOfEventTime)); + } @Override public CompletableFuture freeSlot(AllocationID allocationId, Throwable cause, Time timeout) { From 41cda38e17c1884c72b9bb092e6447ff40f2b5bd Mon Sep 17 00:00:00 2001 From: ifndef-SleePy Date: Wed, 25 Sep 2019 21:00:52 +0800 Subject: [PATCH 319/746] [FLINK-13904][checkpointing] Avoid competition between savepoint and periodic checkpoint triggering --- .../checkpoint/CheckpointCoordinator.java | 46 +-- .../CheckpointCoordinatorFailureTest.java | 14 +- .../CheckpointCoordinatorMasterHooksTest.java | 43 ++- .../CheckpointCoordinatorRestoringTest.java | 36 ++- .../checkpoint/CheckpointCoordinatorTest.java | 285 ++++++++++-------- .../CheckpointCoordinatorTestingUtils.java | 224 ++++++-------- .../CheckpointCoordinatorTriggeringTest.java | 62 ++-- .../CheckpointStateRestoreTest.java | 15 +- ...overStrategyCheckpointCoordinatorTest.java | 5 +- .../ManuallyTriggeredScheduledExecutor.java | 123 ++++++-- .../utils/SimpleAckingTaskManagerGateway.java | 34 ++- .../util/TestingScheduledExecutor.java | 8 +- 12 files changed, 509 insertions(+), 386 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index df9278ec2785..b4843e482733 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -408,46 +408,57 @@ private CompletableFuture triggerSavepointInternal( checkNotNull(checkpointProperties); - try { - PendingCheckpoint pendingCheckpoint = triggerCheckpoint( + // TODO, call triggerCheckpoint directly after removing timer thread + // for now, execute the trigger in timer thread to avoid competition + final CompletableFuture resultFuture = new CompletableFuture<>(); + timer.execute(() -> { + try { + triggerCheckpoint( timestamp, checkpointProperties, targetLocation, false, - advanceToEndOfEventTime); - - return pendingCheckpoint.getCompletionFuture(); - } catch (CheckpointException e) { - Throwable cause = new CheckpointException("Failed to trigger savepoint.", e.getCheckpointFailureReason()); - return FutureUtils.completedExceptionally(cause); - } + advanceToEndOfEventTime). + whenComplete((completedCheckpoint, throwable) -> { + if (throwable == null) { + resultFuture.complete(completedCheckpoint); + } else { + resultFuture.completeExceptionally(throwable); + } + }); + } catch (CheckpointException e) { + Throwable cause = new CheckpointException("Failed to trigger savepoint.", e.getCheckpointFailureReason()); + resultFuture.completeExceptionally(cause); + } + }); + return resultFuture; } /** * Triggers a new standard checkpoint and uses the given timestamp as the checkpoint - * timestamp. + * timestamp. The return value is a future. It completes when the checkpoint triggered finishes + * or an error occurred. * * @param timestamp The timestamp for the checkpoint. * @param isPeriodic Flag indicating whether this triggered checkpoint is * periodic. If this flag is true, but the periodic scheduler is disabled, * the checkpoint will be declined. - * @return true if triggering the checkpoint succeeded. + * @return a future to the completed checkpoint. */ - public boolean triggerCheckpoint(long timestamp, boolean isPeriodic) { + public CompletableFuture triggerCheckpoint(long timestamp, boolean isPeriodic) { try { - triggerCheckpoint(timestamp, checkpointProperties, null, isPeriodic, false); - return true; + return triggerCheckpoint(timestamp, checkpointProperties, null, isPeriodic, false); } catch (CheckpointException e) { long latestGeneratedCheckpointId = getCheckpointIdCounter().get(); // here we can not get the failed pending checkpoint's id, // so we pass the negative latest generated checkpoint id as a special flag failureManager.handleJobLevelCheckpointException(e, -1 * latestGeneratedCheckpointId); - return false; + return FutureUtils.completedExceptionally(e); } } @VisibleForTesting - public PendingCheckpoint triggerCheckpoint( + public CompletableFuture triggerCheckpoint( long timestamp, CheckpointProperties props, @Nullable String externalSavepointLocation, @@ -643,7 +654,7 @@ else if (!props.forceCheckpoint()) { } numUnsuccessfulCheckpointsTriggers.set(0); - return checkpoint; + return checkpoint.getCompletionFuture(); } catch (Throwable t) { // guard the map against concurrent modifications @@ -668,7 +679,6 @@ else if (!props.forceCheckpoint()) { throw new CheckpointException(CheckpointFailureReason.EXCEPTION, t); } - } // end trigger lock } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java index 39a8c2e6bffe..92f42bccbc23 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.concurrent.Executors; +import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionVertex; import org.apache.flink.runtime.jobgraph.JobStatus; @@ -31,10 +32,8 @@ import org.apache.flink.runtime.state.OperatorStreamStateHandle; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.memory.MemoryStateBackend; -import org.apache.flink.runtime.util.TestingScheduledExecutor; import org.apache.flink.util.TestLogger; -import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.powermock.core.classloader.annotations.PrepareForTest; @@ -58,10 +57,6 @@ @PrepareForTest(PendingCheckpoint.class) public class CheckpointCoordinatorFailureTest extends TestLogger { - @Rule - public final TestingScheduledExecutor testingScheduledExecutor = - new TestingScheduledExecutor(); - /** * Tests that a failure while storing a completed checkpoint in the completed checkpoint store * will properly fail the originating pending checkpoint and clean upt the completed checkpoint. @@ -70,6 +65,9 @@ public class CheckpointCoordinatorFailureTest extends TestLogger { public void testFailingCompletedCheckpointStoreAdd() throws Exception { JobID jid = new JobID(); + final ManuallyTriggeredScheduledExecutor manuallyTriggeredScheduledExecutor = + new ManuallyTriggeredScheduledExecutor(); + final ExecutionAttemptID executionAttemptId = new ExecutionAttemptID(); final ExecutionVertex vertex = CheckpointCoordinatorTestingUtils.mockExecutionVertex(executionAttemptId); @@ -99,12 +97,14 @@ public void testFailingCompletedCheckpointStoreAdd() throws Exception { new FailingCompletedCheckpointStore(), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); coord.triggerCheckpoint(triggerTimestamp, false); + manuallyTriggeredScheduledExecutor.triggerAll(); + assertEquals(1, coord.getNumberOfPendingCheckpoints()); PendingCheckpoint pendingCheckpoint = coord.getPendingCheckpoints().values().iterator().next(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java index d85d01431c6a..7c9a3578aedc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java @@ -21,6 +21,8 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.runtime.concurrent.Executors; +import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor; +import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.executiongraph.ExecutionVertex; @@ -32,9 +34,7 @@ import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.memory.MemoryStateBackend; import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; -import org.apache.flink.runtime.util.TestingScheduledExecutor; -import org.junit.Rule; import org.junit.Test; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -71,10 +71,6 @@ */ public class CheckpointCoordinatorMasterHooksTest { - @Rule - public final TestingScheduledExecutor testingScheduledExecutor = - new TestingScheduledExecutor(); - // ------------------------------------------------------------------------ // hook registration // ------------------------------------------------------------------------ @@ -194,14 +190,20 @@ public void testHooksAreCalledOnTrigger() throws Exception { final JobID jid = new JobID(); final ExecutionAttemptID execId = new ExecutionAttemptID(); final ExecutionVertex ackVertex = mockExecutionVertex(execId); - final CheckpointCoordinator cc = instantiateCheckpointCoordinator(jid, ackVertex); + final ManuallyTriggeredScheduledExecutor manuallyTriggeredScheduledExecutor = + new ManuallyTriggeredScheduledExecutor(); + final CheckpointCoordinator cc = instantiateCheckpointCoordinator( + jid, manuallyTriggeredScheduledExecutor, ackVertex); cc.addMasterHook(statefulHook1); cc.addMasterHook(statelessHook); cc.addMasterHook(statefulHook2); // trigger a checkpoint - assertTrue(cc.triggerCheckpoint(System.currentTimeMillis(), false)); + final CompletableFuture checkpointFuture = + cc.triggerCheckpoint(System.currentTimeMillis(), false); + manuallyTriggeredScheduledExecutor.triggerAll(); + assertFalse(checkpointFuture.isCompletedExceptionally()); assertEquals(1, cc.getNumberOfPendingCheckpoints()); verify(statefulHook1, times(1)).triggerCheckpoint(anyLong(), anyLong(), any(Executor.class)); @@ -370,7 +372,10 @@ public void ensureRegisteredAtHookTime() throws Exception { final JobID jid = new JobID(); final ExecutionAttemptID execId = new ExecutionAttemptID(); final ExecutionVertex ackVertex = mockExecutionVertex(execId); - final CheckpointCoordinator cc = instantiateCheckpointCoordinator(jid, ackVertex); + final ManuallyTriggeredScheduledExecutor manuallyTriggeredScheduledExecutor = + new ManuallyTriggeredScheduledExecutor(); + final CheckpointCoordinator cc = instantiateCheckpointCoordinator( + jid, manuallyTriggeredScheduledExecutor, ackVertex); final MasterTriggerRestoreHook hook = mockGeneric(MasterTriggerRestoreHook.class); when(hook.getIdentifier()).thenReturn(id); @@ -391,7 +396,10 @@ public CompletableFuture answer(InvocationOnMock invocation) throws Throwa cc.addMasterHook(hook); // trigger a checkpoint - assertTrue(cc.triggerCheckpoint(System.currentTimeMillis(), false)); + final CompletableFuture checkpointFuture = + cc.triggerCheckpoint(System.currentTimeMillis(), false); + manuallyTriggeredScheduledExecutor.triggerAll(); + assertFalse(checkpointFuture.isCompletedExceptionally()); } @@ -427,7 +435,18 @@ public void testTypeIncompatibleWithHookOnRestore() { // utilities // ------------------------------------------------------------------------ - private CheckpointCoordinator instantiateCheckpointCoordinator(JobID jid, ExecutionVertex... ackVertices) { + private CheckpointCoordinator instantiateCheckpointCoordinator( + JobID jid, + ExecutionVertex... ackVertices) { + + return instantiateCheckpointCoordinator(jid, new ManuallyTriggeredScheduledExecutor(), ackVertices); + } + + private CheckpointCoordinator instantiateCheckpointCoordinator( + JobID jid, + ScheduledExecutor testingScheduledExecutor, + ExecutionVertex... ackVertices) { + CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( 10000000L, 600000L, @@ -447,7 +466,7 @@ private CheckpointCoordinator instantiateCheckpointCoordinator(JobID jid, Execut new StandaloneCompletedCheckpointStore(10), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + testingScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, new CheckpointFailureManager( 0, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java index 1725ef225bdc..e0be7eecc481 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.concurrent.Executors; +import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor; import org.apache.flink.runtime.executiongraph.Execution; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.executiongraph.ExecutionVertex; @@ -39,7 +40,6 @@ import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.RecoverableCompletedCheckpointStore; -import org.apache.flink.runtime.util.TestingScheduledExecutor; import org.apache.flink.util.SerializableObject; import org.apache.flink.util.TestLogger; @@ -76,6 +76,8 @@ import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.mockSubtaskState; import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.verifyStateRestore; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.ArgumentMatchers.any; @@ -90,9 +92,7 @@ public class CheckpointCoordinatorRestoringTest extends TestLogger { private static final String TASK_MANAGER_LOCATION_INFO = "Unknown location"; - @Rule - public final TestingScheduledExecutor testingScheduledExecutor = - new TestingScheduledExecutor(); + private ManuallyTriggeredScheduledExecutor manuallyTriggeredScheduledExecutor; private CheckpointFailureManager failureManager; @@ -104,6 +104,7 @@ public void setUp() throws Exception { failureManager = new CheckpointFailureManager( 0, NoOpFailJobCall.INSTANCE); + manuallyTriggeredScheduledExecutor = new ManuallyTriggeredScheduledExecutor(); } /** @@ -163,14 +164,15 @@ public void testRestoreLatestCheckpointedState() throws Exception { store, new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); // trigger the checkpoint coord.triggerCheckpoint(timestamp, false); + manuallyTriggeredScheduledExecutor.triggerAll(); - assertTrue(coord.getPendingCheckpoints().keySet().size() == 1); + assertEquals(1, coord.getPendingCheckpoints().size()); long checkpointId = Iterables.getOnlyElement(coord.getPendingCheckpoints().keySet()); List keyGroupPartitions1 = StateAssignmentOperation.createKeyGroupPartitions(maxParallelism1, parallelism1); @@ -296,12 +298,15 @@ private void testRestoreLatestCheckpointIsPreferSavepoint(boolean isPreferCheckp store, new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); //trigger a checkpoint and wait to become a completed checkpoint - assertTrue(coord.triggerCheckpoint(timestamp, false)); + final CompletableFuture checkpointFuture = + coord.triggerCheckpoint(timestamp, false); + manuallyTriggeredScheduledExecutor.triggerAll(); + assertFalse(checkpointFuture.isCompletedExceptionally()); long checkpointId = checkpointIDCounter.getLast(); @@ -344,11 +349,12 @@ private void testRestoreLatestCheckpointIsPreferSavepoint(boolean isPreferCheckp StateObjectCollection.singleton(serializedKeyGroupStatesForSavepoint), StateObjectCollection.empty())); + manuallyTriggeredScheduledExecutor.triggerAll(); checkpointId = checkpointIDCounter.getLast(); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statefulExec1.getAttemptId(), checkpointId, new CheckpointMetrics(), subtaskStatesForSavepoint), TASK_MANAGER_LOCATION_INFO); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statelessExec1.getAttemptId(), checkpointId), TASK_MANAGER_LOCATION_INFO); - assertTrue(savepointFuture.isDone()); + assertNotNull(savepointFuture.get()); //restore and jump the latest savepoint coord.restoreLatestCheckpointedState(map, true, false); @@ -446,14 +452,15 @@ private void testRestoreLatestCheckpointedStateWithChangingParallelism(boolean s new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); // trigger the checkpoint coord.triggerCheckpoint(timestamp, false); + manuallyTriggeredScheduledExecutor.triggerAll(); - assertTrue(coord.getPendingCheckpoints().keySet().size() == 1); + assertEquals(1, coord.getPendingCheckpoints().size()); long checkpointId = Iterables.getOnlyElement(coord.getPendingCheckpoints().keySet()); List keyGroupPartitions1 = @@ -624,14 +631,15 @@ public void testRestoreLatestCheckpointFailureWhenMaxParallelismChanges() throws new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); // trigger the checkpoint coord.triggerCheckpoint(timestamp, false); + manuallyTriggeredScheduledExecutor.triggerAll(); - assertTrue(coord.getPendingCheckpoints().keySet().size() == 1); + assertEquals(1, coord.getPendingCheckpoints().size()); long checkpointId = Iterables.getOnlyElement(coord.getPendingCheckpoints().keySet()); List keyGroupPartitions1 = StateAssignmentOperation.createKeyGroupPartitions(maxParallelism1, parallelism1); @@ -879,7 +887,7 @@ public void testStateRecoveryWithTopologyChange(int scaleType) throws Exception standaloneCompletedCheckpointStore, new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java index ddd2f8e30b12..22d59bcc00ab 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java @@ -21,8 +21,8 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.fs.Path; -import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.TestingScheduledServiceWithRecordingScheduledTasks; import org.apache.flink.runtime.concurrent.Executors; +import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor; import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.Execution; @@ -50,7 +50,6 @@ import org.apache.flink.runtime.state.memory.MemoryStateBackend; import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; import org.apache.flink.runtime.testutils.RecoverableCompletedCheckpointStore; -import org.apache.flink.runtime.util.TestingScheduledExecutor; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; @@ -76,6 +75,7 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ScheduledFuture; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; @@ -106,12 +106,10 @@ public class CheckpointCoordinatorTest extends TestLogger { private static final String TASK_MANAGER_LOCATION_INFO = "Unknown location"; - @Rule - public final TestingScheduledExecutor testingScheduledExecutor = - new TestingScheduledExecutor(); - private CheckpointFailureManager failureManager; + private ManuallyTriggeredScheduledExecutor manuallyTriggeredScheduledExecutor; + @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); @@ -120,6 +118,7 @@ public void setUp() throws Exception { failureManager = new CheckpointFailureManager( 0, NoOpFailJobCall.INSTANCE); + manuallyTriggeredScheduledExecutor = new ManuallyTriggeredScheduledExecutor(); } @Test @@ -158,7 +157,7 @@ public void testCheckpointAbortsIfTriggerTasksAreNotExecuted() { new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -167,7 +166,10 @@ public void testCheckpointAbortsIfTriggerTasksAreNotExecuted() { assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); // trigger the first checkpoint. this should not succeed - assertFalse(coord.triggerCheckpoint(timestamp, false)); + final CompletableFuture checkpointFuture = + coord.triggerCheckpoint(timestamp, false); + manuallyTriggeredScheduledExecutor.triggerAll(); + assertTrue(checkpointFuture.isCompletedExceptionally()); // still, nothing should be happening assertEquals(0, coord.getNumberOfPendingCheckpoints()); @@ -226,7 +228,7 @@ public void testCheckpointAbortsIfTriggerTasksAreFinished() { new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -235,7 +237,10 @@ public void testCheckpointAbortsIfTriggerTasksAreFinished() { assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); // trigger the first checkpoint. this should not succeed - assertFalse(coord.triggerCheckpoint(timestamp, false)); + final CompletableFuture checkpointFuture = + coord.triggerCheckpoint(timestamp, false); + manuallyTriggeredScheduledExecutor.triggerAll(); + assertTrue(checkpointFuture.isCompletedExceptionally()); // still, nothing should be happening assertEquals(0, coord.getNumberOfPendingCheckpoints()); @@ -285,7 +290,7 @@ public void testCheckpointAbortsIfAckTasksAreNotExecuted() { new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -294,7 +299,10 @@ public void testCheckpointAbortsIfAckTasksAreNotExecuted() { assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); // trigger the first checkpoint. this should not succeed - assertFalse(coord.triggerCheckpoint(timestamp, false)); + final CompletableFuture checkpointFuture = + coord.triggerCheckpoint(timestamp, false); + manuallyTriggeredScheduledExecutor.triggerAll(); + assertTrue(checkpointFuture.isCompletedExceptionally()); // still, nothing should be happening assertEquals(0, coord.getNumberOfPendingCheckpoints()); @@ -309,7 +317,7 @@ public void testCheckpointAbortsIfAckTasksAreNotExecuted() { } @Test - public void testTriggerAndDeclineCheckpointThenFailureManagerThrowsException() { + public void testTriggerAndDeclineCheckpointThenFailureManagerThrowsException() throws Exception { final JobID jid = new JobID(); final long timestamp = System.currentTimeMillis(); @@ -335,15 +343,15 @@ public void failJobDueToTaskFailure(Throwable cause, ExecutionAttemptID failingT } }); - final TestingScheduledServiceWithRecordingScheduledTasks scheduledExecutorService = - new TestingScheduledServiceWithRecordingScheduledTasks(testingScheduledExecutor.getScheduledExecutor()); - // set up the coordinator - CheckpointCoordinator coord = getCheckpointCoordinator(jid, vertex1, vertex2, checkpointFailureManager, scheduledExecutorService); + CheckpointCoordinator coord = getCheckpointCoordinator(jid, vertex1, vertex2, checkpointFailureManager, manuallyTriggeredScheduledExecutor); try { // trigger the checkpoint. this should succeed - assertTrue(coord.triggerCheckpoint(timestamp, false)); + final CompletableFuture checkPointFuture = + coord.triggerCheckpoint(timestamp, false); + manuallyTriggeredScheduledExecutor.triggerAll(); + assertFalse(checkPointFuture.isCompletedExceptionally()); long checkpointId = coord.getPendingCheckpoints().entrySet().iterator().next().getKey(); PendingCheckpoint checkpoint = coord.getPendingCheckpoints().get(checkpointId); @@ -389,23 +397,24 @@ public void testTriggerAndDeclineCheckpointSimple() { ExecutionVertex vertex1 = mockExecutionVertex(attemptID1); ExecutionVertex vertex2 = mockExecutionVertex(attemptID2); - final TestingScheduledServiceWithRecordingScheduledTasks scheduledExecutorService = - new TestingScheduledServiceWithRecordingScheduledTasks(testingScheduledExecutor.getScheduledExecutor()); // set up the coordinator and validate the initial state - CheckpointCoordinator coord = getCheckpointCoordinator(jid, vertex1, vertex2, failureManager, scheduledExecutorService); + CheckpointCoordinator coord = getCheckpointCoordinator(jid, vertex1, vertex2, failureManager, manuallyTriggeredScheduledExecutor); assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); // trigger the first checkpoint. this should succeed - assertTrue(coord.triggerCheckpoint(timestamp, false)); + final CompletableFuture checkpointFuture = + coord.triggerCheckpoint(timestamp, false); + manuallyTriggeredScheduledExecutor.triggerAll(); + assertFalse(checkpointFuture.isCompletedExceptionally()); // validate that we have a pending checkpoint assertEquals(1, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); // we have one task scheduled that will cancel after timeout - assertEquals(1, scheduledExecutorService.getNumScheduledOnceTasks()); + assertEquals(1, manuallyTriggeredScheduledExecutor.getScheduledTasks().size()); long checkpointId = coord.getPendingCheckpoints().entrySet().iterator().next().getKey(); PendingCheckpoint checkpoint = coord.getPendingCheckpoints().get(checkpointId); @@ -442,7 +451,7 @@ public void testTriggerAndDeclineCheckpointSimple() { assertTrue(checkpoint.isDiscarded()); // the canceler is also removed - assertEquals(0, scheduledExecutorService.getNumScheduledOnceTasks()); + assertEquals(0, manuallyTriggeredScheduledExecutor.getScheduledTasks().size()); // validate that we have no new pending checkpoint assertEquals(0, coord.getNumberOfPendingCheckpoints()); @@ -478,26 +487,29 @@ public void testTriggerAndDeclineCheckpointComplex() { final ExecutionAttemptID attemptID2 = new ExecutionAttemptID(); ExecutionVertex vertex1 = mockExecutionVertex(attemptID1); ExecutionVertex vertex2 = mockExecutionVertex(attemptID2); - - final TestingScheduledServiceWithRecordingScheduledTasks scheduledExecutorService = - new TestingScheduledServiceWithRecordingScheduledTasks(testingScheduledExecutor.getScheduledExecutor()); // set up the coordinator and validate the initial state - CheckpointCoordinator coord = getCheckpointCoordinator(jid, vertex1, vertex2, failureManager, scheduledExecutorService); + CheckpointCoordinator coord = getCheckpointCoordinator(jid, vertex1, vertex2, failureManager, manuallyTriggeredScheduledExecutor); assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); - assertEquals(0, scheduledExecutorService.getNumScheduledOnceTasks()); + assertEquals(0, manuallyTriggeredScheduledExecutor.getScheduledTasks().size()); // trigger the first checkpoint. this should succeed - assertTrue(coord.triggerCheckpoint(timestamp, false)); + final CompletableFuture checkpointFuture1 = + coord.triggerCheckpoint(timestamp, false); + manuallyTriggeredScheduledExecutor.triggerAll(); + assertFalse(checkpointFuture1.isCompletedExceptionally()); // trigger second checkpoint, should also succeed - assertTrue(coord.triggerCheckpoint(timestamp + 2, false)); + final CompletableFuture checkpointFuture2 = + coord.triggerCheckpoint(timestamp + 2, false); + manuallyTriggeredScheduledExecutor.triggerAll(); + assertFalse(checkpointFuture2.isCompletedExceptionally()); // validate that we have a pending checkpoint assertEquals(2, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); - assertEquals(2, scheduledExecutorService.getNumScheduledOnceTasks()); + assertEquals(2, manuallyTriggeredScheduledExecutor.getScheduledTasks().size()); Iterator> it = coord.getPendingCheckpoints().entrySet().iterator(); long checkpoint1Id = it.next().getKey(); @@ -544,7 +556,7 @@ public void testTriggerAndDeclineCheckpointComplex() { // validate that we have only one pending checkpoint left assertEquals(1, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); - assertEquals(1, scheduledExecutorService.getNumScheduledOnceTasks()); + assertEquals(1, manuallyTriggeredScheduledExecutor.getScheduledTasks().size()); // validate that it is the same second checkpoint from earlier long checkpointIdNew = coord.getPendingCheckpoints().entrySet().iterator().next().getKey(); @@ -587,22 +599,23 @@ public void testTriggerAndConfirmSimpleCheckpoint() { ExecutionVertex vertex1 = mockExecutionVertex(attemptID1); ExecutionVertex vertex2 = mockExecutionVertex(attemptID2); - final TestingScheduledServiceWithRecordingScheduledTasks scheduledExecutorService = - new TestingScheduledServiceWithRecordingScheduledTasks(testingScheduledExecutor.getScheduledExecutor()); // set up the coordinator and validate the initial state - CheckpointCoordinator coord = getCheckpointCoordinator(jid, vertex1, vertex2, failureManager, scheduledExecutorService); + CheckpointCoordinator coord = getCheckpointCoordinator(jid, vertex1, vertex2, failureManager, manuallyTriggeredScheduledExecutor); assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); - assertEquals(0, scheduledExecutorService.getNumScheduledOnceTasks()); + assertEquals(0, manuallyTriggeredScheduledExecutor.getScheduledTasks().size()); // trigger the first checkpoint. this should succeed - assertTrue(coord.triggerCheckpoint(timestamp, false)); + final CompletableFuture checkpointFuture = + coord.triggerCheckpoint(timestamp, false); + manuallyTriggeredScheduledExecutor.triggerAll(); + assertFalse(checkpointFuture.isCompletedExceptionally()); // validate that we have a pending checkpoint assertEquals(1, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); - assertEquals(1, scheduledExecutorService.getNumScheduledOnceTasks()); + assertEquals(1, manuallyTriggeredScheduledExecutor.getScheduledTasks().size()); long checkpointId = coord.getPendingCheckpoints().entrySet().iterator().next().getKey(); PendingCheckpoint checkpoint = coord.getPendingCheckpoints().get(checkpointId); @@ -659,7 +672,7 @@ public void testTriggerAndConfirmSimpleCheckpoint() { assertEquals(0, coord.getNumberOfPendingCheckpoints()); // the canceler should be removed now - assertEquals(0, scheduledExecutorService.getNumScheduledOnceTasks()); + assertEquals(0, manuallyTriggeredScheduledExecutor.getScheduledTasks().size()); // validate that the subtasks states have registered their shared states. { @@ -684,6 +697,7 @@ public void testTriggerAndConfirmSimpleCheckpoint() { // --------------- final long timestampNew = timestamp + 7; coord.triggerCheckpoint(timestampNew, false); + manuallyTriggeredScheduledExecutor.triggerAll(); long checkpointIdNew = coord.getPendingCheckpoints().entrySet().iterator().next().getKey(); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, checkpointIdNew), TASK_MANAGER_LOCATION_INFO); @@ -691,7 +705,7 @@ public void testTriggerAndConfirmSimpleCheckpoint() { assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(1, coord.getNumberOfRetainedSuccessfulCheckpoints()); - assertEquals(0, scheduledExecutorService.getNumScheduledOnceTasks()); + assertEquals(0, manuallyTriggeredScheduledExecutor.getScheduledTasks().size()); CompletedCheckpoint successNew = coord.getSuccessfulCheckpoints().get(0); assertEquals(jid, successNew.getJobId()); @@ -763,7 +777,7 @@ public void testMultipleConcurrentCheckpoints() { new StandaloneCompletedCheckpointStore(2), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -771,7 +785,10 @@ public void testMultipleConcurrentCheckpoints() { assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); // trigger the first checkpoint. this should succeed - assertTrue(coord.triggerCheckpoint(timestamp1, false)); + final CompletableFuture checkpointFuture1 = + coord.triggerCheckpoint(timestamp1, false); + manuallyTriggeredScheduledExecutor.triggerAll(); + assertFalse(checkpointFuture1.isCompletedExceptionally()); assertEquals(1, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -788,7 +805,10 @@ public void testMultipleConcurrentCheckpoints() { // start the second checkpoint // trigger the first checkpoint. this should succeed - assertTrue(coord.triggerCheckpoint(timestamp2, false)); + final CompletableFuture checkpointFuture2 = + coord.triggerCheckpoint(timestamp2, false); + manuallyTriggeredScheduledExecutor.triggerAll(); + assertFalse(checkpointFuture2.isCompletedExceptionally()); assertEquals(2, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -901,7 +921,7 @@ public void testSuccessfulCheckpointSubsumesUnsuccessful() { new StandaloneCompletedCheckpointStore(10), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -909,7 +929,10 @@ public void testSuccessfulCheckpointSubsumesUnsuccessful() { assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); // trigger the first checkpoint. this should succeed - assertTrue(coord.triggerCheckpoint(timestamp1, false)); + final CompletableFuture checkpointFuture1 = + coord.triggerCheckpoint(timestamp1, false); + manuallyTriggeredScheduledExecutor.triggerAll(); + assertFalse(checkpointFuture1.isCompletedExceptionally()); assertEquals(1, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -941,7 +964,10 @@ public void testSuccessfulCheckpointSubsumesUnsuccessful() { // start the second checkpoint // trigger the first checkpoint. this should succeed - assertTrue(coord.triggerCheckpoint(timestamp2, false)); + final CompletableFuture checkpointFuture2 = + coord.triggerCheckpoint(timestamp2, false); + manuallyTriggeredScheduledExecutor.triggerAll(); + assertFalse(checkpointFuture2.isCompletedExceptionally()); assertEquals(2, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -1072,12 +1098,15 @@ public void testCheckpointTimeoutIsolated() { new StandaloneCompletedCheckpointStore(2), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); // trigger a checkpoint, partially acknowledged - assertTrue(coord.triggerCheckpoint(timestamp, false)); + final CompletableFuture checkpointFuture = + coord.triggerCheckpoint(timestamp, false); + manuallyTriggeredScheduledExecutor.triggerAll(); + assertFalse(checkpointFuture.isCompletedExceptionally()); assertEquals(1, coord.getNumberOfPendingCheckpoints()); PendingCheckpoint checkpoint = coord.getPendingCheckpoints().values().iterator().next(); @@ -1091,17 +1120,8 @@ public void testCheckpointTimeoutIsolated() { coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID1, checkpoint.getCheckpointId(), new CheckpointMetrics(), taskOperatorSubtaskStates1), TASK_MANAGER_LOCATION_INFO); - // wait until the checkpoint must have expired. - // we check every 250 msecs conservatively for 5 seconds - // to give even slow build servers a very good chance of completing this - long deadline = System.currentTimeMillis() + 5000; - do { - Thread.sleep(250); - } - while (!checkpoint.isDiscarded() && - coord.getNumberOfPendingCheckpoints() > 0 && - System.currentTimeMillis() < deadline); - + // triggers cancelling + manuallyTriggeredScheduledExecutor.triggerScheduledTasks(); assertTrue("Checkpoint was not canceled by the timeout", checkpoint.isDiscarded()); assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -1157,11 +1177,14 @@ public void testHandleMessagesForNonExistingCheckpoints() { new StandaloneCompletedCheckpointStore(2), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); - assertTrue(coord.triggerCheckpoint(timestamp, false)); + final CompletableFuture checkpointFuture = + coord.triggerCheckpoint(timestamp, false); + manuallyTriggeredScheduledExecutor.triggerAll(); + assertFalse(checkpointFuture.isCompletedExceptionally()); long checkpointId = coord.getPendingCheckpoints().keySet().iterator().next(); @@ -1228,11 +1251,14 @@ public void testStateCleanupForLateOrUnknownMessages() throws Exception { new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); - assertTrue(coord.triggerCheckpoint(timestamp, false)); + final CompletableFuture checkpointFuture = + coord.triggerCheckpoint(timestamp, false); + manuallyTriggeredScheduledExecutor.triggerAll(); + assertFalse(checkpointFuture.isCompletedExceptionally()); assertEquals(1, coord.getNumberOfPendingCheckpoints()); @@ -1334,10 +1360,8 @@ public void testTriggerAndConfirmSimpleSavepoint() throws Exception { ExecutionVertex vertex1 = mockExecutionVertex(attemptID1); ExecutionVertex vertex2 = mockExecutionVertex(attemptID2); - final TestingScheduledServiceWithRecordingScheduledTasks scheduledExecutorService = - new TestingScheduledServiceWithRecordingScheduledTasks(testingScheduledExecutor.getScheduledExecutor()); // set up the coordinator and validate the initial state - CheckpointCoordinator coord = getCheckpointCoordinator(jid, vertex1, vertex2, failureManager, scheduledExecutorService); + CheckpointCoordinator coord = getCheckpointCoordinator(jid, vertex1, vertex2, failureManager, manuallyTriggeredScheduledExecutor); assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -1345,6 +1369,7 @@ public void testTriggerAndConfirmSimpleSavepoint() throws Exception { // trigger the first checkpoint. this should succeed String savepointDir = tmpFolder.newFolder().getAbsolutePath(); CompletableFuture savepointFuture = coord.triggerSavepoint(timestamp, savepointDir); + manuallyTriggeredScheduledExecutor.triggerAll(); assertFalse(savepointFuture.isDone()); // validate that we have a pending savepoint @@ -1394,7 +1419,7 @@ public void testTriggerAndConfirmSimpleSavepoint() throws Exception { // the checkpoint is internally converted to a successful checkpoint and the // pending checkpoint object is disposed assertTrue(pending.isDiscarded()); - assertTrue(savepointFuture.isDone()); + assertNotNull(savepointFuture.get()); // the now we should have a completed checkpoint assertEquals(1, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -1423,6 +1448,7 @@ public void testTriggerAndConfirmSimpleSavepoint() throws Exception { // --------------- final long timestampNew = timestamp + 7; savepointFuture = coord.triggerSavepoint(timestampNew, savepointDir); + manuallyTriggeredScheduledExecutor.triggerAll(); assertFalse(savepointFuture.isDone()); long checkpointIdNew = coord.getPendingCheckpoints().entrySet().iterator().next().getKey(); @@ -1437,7 +1463,7 @@ public void testTriggerAndConfirmSimpleSavepoint() throws Exception { assertEquals(timestampNew, successNew.getTimestamp()); assertEquals(checkpointIdNew, successNew.getCheckpointID()); assertTrue(successNew.getOperatorStates().isEmpty()); - assertTrue(savepointFuture.isDone()); + assertNotNull(savepointFuture.get()); // validate that the first savepoint does not discard its private states. verify(subtaskState1, never()).discardState(); @@ -1494,7 +1520,7 @@ public void testSavepointsAreNotSubsumed() throws Exception { new StandaloneCompletedCheckpointStore(10), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -1502,13 +1528,21 @@ public void testSavepointsAreNotSubsumed() throws Exception { // Trigger savepoint and checkpoint CompletableFuture savepointFuture1 = coord.triggerSavepoint(timestamp, savepointDir); + + manuallyTriggeredScheduledExecutor.triggerAll(); long savepointId1 = counter.getLast(); assertEquals(1, coord.getNumberOfPendingCheckpoints()); - assertTrue(coord.triggerCheckpoint(timestamp + 1, false)); + CompletableFuture checkpointFuture1 = + coord.triggerCheckpoint(timestamp + 1, false); + manuallyTriggeredScheduledExecutor.triggerAll(); assertEquals(2, coord.getNumberOfPendingCheckpoints()); + assertFalse(checkpointFuture1.isCompletedExceptionally()); - assertTrue(coord.triggerCheckpoint(timestamp + 2, false)); + CompletableFuture checkpointFuture2 = + coord.triggerCheckpoint(timestamp + 2, false); + manuallyTriggeredScheduledExecutor.triggerAll(); + assertFalse(checkpointFuture2.isCompletedExceptionally()); long checkpointId2 = counter.getLast(); assertEquals(3, coord.getNumberOfPendingCheckpoints()); @@ -1522,11 +1556,16 @@ public void testSavepointsAreNotSubsumed() throws Exception { assertFalse(coord.getPendingCheckpoints().get(savepointId1).isDiscarded()); assertFalse(savepointFuture1.isDone()); - assertTrue(coord.triggerCheckpoint(timestamp + 3, false)); + CompletableFuture checkpointFuture3 = + coord.triggerCheckpoint(timestamp + 3, false); + manuallyTriggeredScheduledExecutor.triggerAll(); + assertFalse(checkpointFuture3.isCompletedExceptionally()); assertEquals(2, coord.getNumberOfPendingCheckpoints()); CompletableFuture savepointFuture2 = coord.triggerSavepoint(timestamp + 4, savepointDir); + manuallyTriggeredScheduledExecutor.triggerAll(); long savepointId2 = counter.getLast(); + assertFalse(savepointFuture2.isCompletedExceptionally()); assertEquals(3, coord.getNumberOfPendingCheckpoints()); // 2nd savepoint should subsume the last checkpoint, but not the 1st savepoint @@ -1538,7 +1577,7 @@ public void testSavepointsAreNotSubsumed() throws Exception { assertFalse(coord.getPendingCheckpoints().get(savepointId1).isDiscarded()); assertFalse(savepointFuture1.isDone()); - assertTrue(savepointFuture2.isDone()); + assertNotNull(savepointFuture2.get()); // Ack first savepoint coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, savepointId1), TASK_MANAGER_LOCATION_INFO); @@ -1546,7 +1585,7 @@ public void testSavepointsAreNotSubsumed() throws Exception { assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(3, coord.getNumberOfRetainedSuccessfulCheckpoints()); - assertTrue(savepointFuture1.isDone()); + assertNotNull(savepointFuture1.get()); } private void testMaxConcurrentAttempts(int maxConcurrentAttempts) { @@ -1595,22 +1634,15 @@ private void testMaxConcurrentAttempts(int maxConcurrentAttempts) { new StandaloneCompletedCheckpointStore(2), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); coord.startCheckpointScheduler(); - // after a while, there should be exactly as many checkpoints - // as concurrently permitted - long now = System.currentTimeMillis(); - long timeout = now + 60000; - long minDuration = now + 100; - do { - Thread.sleep(20); + for (int i = 0; i < maxConcurrentAttempts; i++) { + manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); } - while ((now = System.currentTimeMillis()) < minDuration || - (numCalls.get() < maxConcurrentAttempts && now < timeout)); assertEquals(maxConcurrentAttempts, numCalls.get()); @@ -1620,18 +1652,17 @@ private void testMaxConcurrentAttempts(int maxConcurrentAttempts) { // now, once we acknowledge one checkpoint, it should trigger the next one coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID, 1L), TASK_MANAGER_LOCATION_INFO); - // this should have immediately triggered a new checkpoint - now = System.currentTimeMillis(); - timeout = now + 60000; - do { - Thread.sleep(20); - } - while (numCalls.get() < maxConcurrentAttempts + 1 && now < timeout); + final Collection> periodicScheduledTasks = + manuallyTriggeredScheduledExecutor.getPeriodicScheduledTask(); + assertEquals(1, periodicScheduledTasks.size()); + final ScheduledFuture scheduledFuture = periodicScheduledTasks.iterator().next(); + + manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); assertEquals(maxConcurrentAttempts + 1, numCalls.get()); // no further checkpoints should happen - Thread.sleep(200); + manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); assertEquals(maxConcurrentAttempts + 1, numCalls.get()); coord.shutdown(JobStatus.FINISHED); @@ -1676,22 +1707,16 @@ public void testMaxConcurrentAttempsWithSubsumption() { new StandaloneCompletedCheckpointStore(2), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); coord.startCheckpointScheduler(); - // after a while, there should be exactly as many checkpoints - // as concurrently permitted - long now = System.currentTimeMillis(); - long timeout = now + 60000; - long minDuration = now + 100; do { - Thread.sleep(20); + manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); } - while ((now = System.currentTimeMillis()) < minDuration || - (coord.getNumberOfPendingCheckpoints() < maxConcurrentAttempts && now < timeout)); + while (coord.getNumberOfPendingCheckpoints() < maxConcurrentAttempts); // validate that the pending checkpoints are there assertEquals(maxConcurrentAttempts, coord.getNumberOfPendingCheckpoints()); @@ -1704,12 +1729,10 @@ public void testMaxConcurrentAttempsWithSubsumption() { coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID, 2L), TASK_MANAGER_LOCATION_INFO); // after a while, there should be the new checkpoints - final long newTimeout = System.currentTimeMillis() + 60000; do { - Thread.sleep(20); + manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); } - while (coord.getPendingCheckpoints().get(4L) == null && - System.currentTimeMillis() < newTimeout); + while (coord.getNumberOfPendingCheckpoints() < maxConcurrentAttempts); // do the final check assertEquals(maxConcurrentAttempts, coord.getNumberOfPendingCheckpoints()); @@ -1760,26 +1783,21 @@ public void testPeriodicSchedulingWithInactiveTasks() { new StandaloneCompletedCheckpointStore(2), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); coord.startCheckpointScheduler(); + manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); // no checkpoint should have started so far - Thread.sleep(200); assertEquals(0, coord.getNumberOfPendingCheckpoints()); // now move the state to RUNNING currentState.set(ExecutionState.RUNNING); // the coordinator should start checkpointing now - final long timeout = System.currentTimeMillis() + 10000; - do { - Thread.sleep(20); - } - while (System.currentTimeMillis() < timeout && - coord.getNumberOfPendingCheckpoints() == 0); + manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); assertTrue(coord.getNumberOfPendingCheckpoints() > 0); } @@ -1795,6 +1813,7 @@ public void testPeriodicSchedulingWithInactiveTasks() { @Test public void testConcurrentSavepoints() throws Exception { JobID jobId = new JobID(); + int numSavepoints = 5; final ExecutionAttemptID attemptID1 = new ExecutionAttemptID(); ExecutionVertex vertex1 = mockExecutionVertex(attemptID1); @@ -1820,14 +1839,12 @@ public void testConcurrentSavepoints() throws Exception { new StandaloneCompletedCheckpointStore(2), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); List> savepointFutures = new ArrayList<>(); - int numSavepoints = 5; - String savepointDir = tmpFolder.newFolder().getAbsolutePath(); // Trigger savepoints @@ -1840,6 +1857,8 @@ public void testConcurrentSavepoints() throws Exception { assertFalse(savepointFuture.isDone()); } + manuallyTriggeredScheduledExecutor.triggerAll(); + // ACK all savepoints long checkpointId = checkpointIDCounter.getLast(); for (int i = 0; i < numSavepoints; i++, checkpointId--) { @@ -1848,7 +1867,7 @@ public void testConcurrentSavepoints() throws Exception { // After ACKs, all should be completed for (CompletableFuture savepointFuture : savepointFutures) { - assertTrue(savepointFuture.isDone()); + assertNotNull(savepointFuture.get()); } } @@ -1881,7 +1900,7 @@ public void testMinDelayBetweenSavepoints() throws Exception { new StandaloneCompletedCheckpointStore(2), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -1927,11 +1946,14 @@ public void testExternalizedCheckpoints() throws Exception { new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); - assertTrue(coord.triggerCheckpoint(timestamp, false)); + CompletableFuture checkpointFuture = + coord.triggerCheckpoint(timestamp, false); + manuallyTriggeredScheduledExecutor.triggerAll(); + assertFalse(checkpointFuture.isCompletedExceptionally()); for (PendingCheckpoint checkpoint : coord.getPendingCheckpoints().values()) { CheckpointProperties props = checkpoint.getProps(); @@ -2140,7 +2162,7 @@ private void doTestPartitionableStateRepartitioning( * Tests that the pending checkpoint stats callbacks are created. */ @Test - public void testCheckpointStatsTrackerPendingCheckpointCallback() { + public void testCheckpointStatsTrackerPendingCheckpointCallback() throws Exception { final long timestamp = System.currentTimeMillis(); ExecutionVertex vertex1 = mockExecutionVertex(new ExecutionAttemptID()); @@ -2164,7 +2186,7 @@ public void testCheckpointStatsTrackerPendingCheckpointCallback() { new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -2175,7 +2197,10 @@ public void testCheckpointStatsTrackerPendingCheckpointCallback() { .thenReturn(mock(PendingCheckpointStats.class)); // Trigger a checkpoint and verify callback - assertTrue(coord.triggerCheckpoint(timestamp, false)); + CompletableFuture checkpointFuture = + coord.triggerCheckpoint(timestamp, false); + manuallyTriggeredScheduledExecutor.triggerAll(); + assertFalse(checkpointFuture.isCompletedExceptionally()); verify(tracker, times(1)) .reportPendingCheckpoint(eq(1L), eq(timestamp), eq(CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION))); @@ -2210,7 +2235,7 @@ public void testCheckpointStatsTrackerRestoreCallback() throws Exception { store, new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -2280,7 +2305,7 @@ public void testSharedStateRegistrationOnRestore() throws Exception { store, new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, deleteExecutor -> { SharedStateRegistry instance = new SharedStateRegistry(deleteExecutor); createdSharedStateRegistries.add(instance); @@ -2421,8 +2446,6 @@ public void jobFailsIfInFlightSynchronousSavepointIsDiscarded() throws Exception final ExecutionVertex vertex1 = mockExecutionVertex(attemptID1); final ExecutionVertex vertex2 = mockExecutionVertex(attemptID2); - final TestingScheduledServiceWithRecordingScheduledTasks scheduledExecutorService = - new TestingScheduledServiceWithRecordingScheduledTasks(testingScheduledExecutor.getScheduledExecutor()); // set up the coordinator and validate the initial state final CheckpointCoordinator coordinator = getCheckpointCoordinator(jobId, vertex1, vertex2, new CheckpointFailureManager( @@ -2439,11 +2462,12 @@ public void failJobDueToTaskFailure(Throwable cause, ExecutionAttemptID failingT throw new AssertionError("This method should not be called for the test."); } }), - scheduledExecutorService); + manuallyTriggeredScheduledExecutor); final CompletableFuture savepointFuture = coordinator .triggerSynchronousSavepoint(10L, false, "test-dir"); + manuallyTriggeredScheduledExecutor.triggerAll(); final PendingCheckpoint syncSavepoint = declineSynchronousSavepoint(jobId, coordinator, attemptID1, expectedRootCause); assertTrue(syncSavepoint.isDiscarded()); @@ -2519,8 +2543,9 @@ private void performIncrementalCheckpoint( // trigger the checkpoint coord.triggerCheckpoint(timestamp, false); + manuallyTriggeredScheduledExecutor.triggerAll(); - assertTrue(coord.getPendingCheckpoints().keySet().size() == 1); + assertEquals(1, coord.getPendingCheckpoints().size()); long checkpointId = Iterables.getOnlyElement(coord.getPendingCheckpoints().keySet()); for (int index = 0; index < jobVertex1.getParallelism(); index++) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java index 76f042b6cabf..89af1531cc70 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java @@ -18,17 +18,24 @@ package org.apache.flink.runtime.checkpoint; +import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.fs.FSDataInputStream; -import org.apache.flink.runtime.concurrent.ScheduledExecutor; +import org.apache.flink.mock.Whitebox; +import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.Execution; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; +import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway.CheckpointConsumer; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; +import org.apache.flink.runtime.jobmaster.LogicalSlot; +import org.apache.flink.runtime.jobmaster.TestingLogicalSlotBuilder; import org.apache.flink.runtime.state.ChainedStateHandle; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyGroupRangeOffsets; @@ -42,6 +49,9 @@ import org.apache.flink.util.Preconditions; import org.junit.Assert; +import org.mockito.invocation.InvocationOnMock; + +import javax.annotation.Nullable; import java.io.IOException; import java.io.Serializable; @@ -50,23 +60,20 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Random; -import java.util.Set; import java.util.UUID; -import java.util.concurrent.Callable; -import java.util.concurrent.Delayed; -import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; -import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -306,7 +313,7 @@ static void collectResult(int opIdx, OperatorStateHandle operatorStateHandle, Li static ExecutionJobVertex mockExecutionJobVertex( JobVertexID jobVertexID, int parallelism, - int maxParallelism) { + int maxParallelism) throws Exception { return mockExecutionJobVertex( jobVertexID, @@ -320,7 +327,7 @@ static ExecutionJobVertex mockExecutionJobVertex( JobVertexID jobVertexID, List jobVertexIDs, int parallelism, - int maxParallelism) { + int maxParallelism) throws Exception { final ExecutionJobVertex executionJobVertex = mock(ExecutionJobVertex.class); ExecutionVertex[] executionVertices = new ExecutionVertex[parallelism]; @@ -348,12 +355,39 @@ static ExecutionJobVertex mockExecutionJobVertex( return executionJobVertex; } - static ExecutionVertex mockExecutionVertex(ExecutionAttemptID attemptID) { + static ExecutionVertex mockExecutionVertex(ExecutionAttemptID attemptID) throws Exception { + return mockExecutionVertex(attemptID, (LogicalSlot) null); + } + + static ExecutionVertex mockExecutionVertex( + ExecutionAttemptID attemptID, + CheckpointConsumer checkpointConsumer) throws Exception { + + final SimpleAckingTaskManagerGateway taskManagerGateway = new SimpleAckingTaskManagerGateway(); + taskManagerGateway.setCheckpointConsumer(checkpointConsumer); + return mockExecutionVertex(attemptID, taskManagerGateway); + } + + static ExecutionVertex mockExecutionVertex( + ExecutionAttemptID attemptID, + TaskManagerGateway taskManagerGateway) throws Exception { + + TestingLogicalSlotBuilder slotBuilder = new TestingLogicalSlotBuilder(); + slotBuilder.setTaskManagerGateway(taskManagerGateway); + LogicalSlot slot = slotBuilder.createTestingLogicalSlot(); + return mockExecutionVertex(attemptID, slot); + } + + static ExecutionVertex mockExecutionVertex( + ExecutionAttemptID attemptID, + @Nullable LogicalSlot slot) throws Exception { + JobVertexID jobVertexID = new JobVertexID(); return mockExecutionVertex( attemptID, jobVertexID, Collections.singletonList(OperatorID.fromJobVertexID(jobVertexID)), + slot, 1, 1, ExecutionState.RUNNING); @@ -366,7 +400,28 @@ static ExecutionVertex mockExecutionVertex( int parallelism, int maxParallelism, ExecutionState state, - ExecutionState ... successiveStates) { + ExecutionState ... successiveStates) throws Exception { + + return mockExecutionVertex( + attemptID, + jobVertexID, + jobVertexIDs, + null, + parallelism, + maxParallelism, + state, + successiveStates); + } + + static ExecutionVertex mockExecutionVertex( + ExecutionAttemptID attemptID, + JobVertexID jobVertexID, + List jobVertexIDs, + @Nullable LogicalSlot slot, + int parallelism, + int maxParallelism, + ExecutionState state, + ExecutionState ... successiveStates) throws Exception { ExecutionVertex vertex = mock(ExecutionVertex.class); @@ -378,6 +433,15 @@ static ExecutionVertex mockExecutionVertex( 1L, Time.milliseconds(500L) )); + if (slot != null) { + // is there a better way to do this? + //noinspection unchecked + AtomicReferenceFieldUpdater slotUpdater = + (AtomicReferenceFieldUpdater) + Whitebox.getInternalState(exec, "ASSIGNED_SLOT_UPDATER"); + slotUpdater.compareAndSet(exec, null, slot); + } + when(exec.getAttemptId()).thenReturn(attemptID); when(exec.getState()).thenReturn(state, successiveStates); @@ -455,6 +519,29 @@ static Execution mockExecution() { return mock; } + static Execution mockExecution(CheckpointConsumer checkpointConsumer) { + ExecutionVertex executionVertex = mock(ExecutionVertex.class); + final JobID jobId = new JobID(); + when(executionVertex.getJobId()).thenReturn(jobId); + Execution mock = mock(Execution.class); + ExecutionAttemptID executionAttemptID = new ExecutionAttemptID(); + when(mock.getAttemptId()).thenReturn(executionAttemptID); + when(mock.getState()).thenReturn(ExecutionState.RUNNING); + when(mock.getVertex()).thenReturn(executionVertex); + doAnswer((InvocationOnMock invocation) -> { + final Object[] args = invocation.getArguments(); + checkpointConsumer.accept( + executionAttemptID, + jobId, + (long) args[0], + (long) args[1], + (CheckpointOptions) args[2], + false); + return null; + }).when(mock).triggerCheckpoint(anyLong(), anyLong(), any(CheckpointOptions.class)); + return mock; + } + static ExecutionVertex mockExecutionVertex(Execution execution, JobVertexID vertexId, int subtask, int parallelism) { ExecutionVertex mock = mock(ExecutionVertex.class); when(mock.getJobvertexId()).thenReturn(vertexId); @@ -479,115 +566,4 @@ static ExecutionJobVertex mockExecutionJobVertex(JobVertexID id, ExecutionVertex } return vertex; } - - static class TestingScheduledServiceWithRecordingScheduledTasks implements ScheduledExecutor { - - private final ScheduledExecutor scheduledExecutor; - - private final Set tasksScheduledOnce; - - public TestingScheduledServiceWithRecordingScheduledTasks(ScheduledExecutor scheduledExecutor) { - this.scheduledExecutor = checkNotNull(scheduledExecutor); - tasksScheduledOnce = new HashSet<>(); - } - - public int getNumScheduledOnceTasks() { - synchronized (tasksScheduledOnce) { - return tasksScheduledOnce.size(); - } - } - - @Override - public ScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) { - final UUID id = UUID.randomUUID(); - synchronized (tasksScheduledOnce) { - tasksScheduledOnce.add(id); - } - return new TestingScheduledFuture<>(id, scheduledExecutor.schedule(() -> { - synchronized (tasksScheduledOnce) { - tasksScheduledOnce.remove(id); - } - command.run(); - }, delay, unit)); - } - - @Override - public ScheduledFuture schedule(Callable callable, long delay, TimeUnit unit) { - final UUID id = UUID.randomUUID(); - synchronized (tasksScheduledOnce) { - tasksScheduledOnce.add(id); - } - return new TestingScheduledFuture<>(id, scheduledExecutor.schedule(() -> { - synchronized (tasksScheduledOnce) { - tasksScheduledOnce.remove(id); - } - return callable.call(); - }, delay, unit)); - } - - @Override - public ScheduledFuture scheduleAtFixedRate(Runnable command, long initialDelay, long period, TimeUnit unit) { - return scheduledExecutor.scheduleAtFixedRate(command, initialDelay, period, unit); - } - - @Override - public ScheduledFuture scheduleWithFixedDelay(Runnable command, long initialDelay, long delay, TimeUnit unit) { - return scheduledExecutor.scheduleWithFixedDelay(command, initialDelay, delay, unit); - } - - @Override - public void execute(Runnable command) { - scheduledExecutor.execute(command); - } - - private class TestingScheduledFuture implements ScheduledFuture { - - private final ScheduledFuture scheduledFuture; - - private final UUID id; - - public TestingScheduledFuture(UUID id, ScheduledFuture scheduledFuture) { - this.id = checkNotNull(id); - this.scheduledFuture = checkNotNull(scheduledFuture); - } - - @Override - public long getDelay(TimeUnit unit) { - return scheduledFuture.getDelay(unit); - } - - @Override - public int compareTo(Delayed o) { - return scheduledFuture.compareTo(o); - } - - @Override - public boolean cancel(boolean mayInterruptIfRunning) { - synchronized (tasksScheduledOnce) { - tasksScheduledOnce.remove(id); - } - return scheduledFuture.cancel(mayInterruptIfRunning); - } - - @Override - public boolean isCancelled() { - return scheduledFuture.isCancelled(); - } - - @Override - public boolean isDone() { - return scheduledFuture.isDone(); - } - - @Override - public V get() throws InterruptedException, ExecutionException { - return scheduledFuture.get(); - } - - @Override - public V get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { - return scheduledFuture.get(timeout, unit); - } - } - } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java index a157b6f00744..4a761d957876 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.concurrent.Executors; +import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor; import org.apache.flink.runtime.executiongraph.Execution; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionVertex; @@ -28,11 +29,9 @@ import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.memory.MemoryStateBackend; -import org.apache.flink.runtime.util.TestingScheduledExecutor; import org.apache.flink.util.TestLogger; import org.junit.Before; -import org.junit.Rule; import org.junit.Test; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -55,9 +54,7 @@ public class CheckpointCoordinatorTriggeringTest extends TestLogger { private static final String TASK_MANAGER_LOCATION_INFO = "Unknown location"; - @Rule - public final TestingScheduledExecutor testingScheduledExecutor = - new TestingScheduledExecutor(); + private ManuallyTriggeredScheduledExecutor manuallyTriggeredScheduledExecutor; private CheckpointFailureManager failureManager; @@ -66,6 +63,7 @@ public void setUp() throws Exception { failureManager = new CheckpointFailureManager( 0, NoOpFailJobCall.INSTANCE); + manuallyTriggeredScheduledExecutor = new ManuallyTriggeredScheduledExecutor(); } @Test @@ -128,51 +126,39 @@ public Void answer(InvocationOnMock invocation) throws Throwable { new StandaloneCompletedCheckpointStore(2), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); coord.startCheckpointScheduler(); - long timeout = System.currentTimeMillis() + 60000; do { - Thread.sleep(20); + manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); } - while (timeout > System.currentTimeMillis() && numCalls.get() < 5); - assertTrue(numCalls.get() >= 5); + while (numCalls.get() < 5); + assertEquals(5, numCalls.get()); coord.stopCheckpointScheduler(); - // for 400 ms, no further calls may come. - // there may be the case that one trigger was fired and about to - // acquire the lock, such that after cancelling it will still do - // the remainder of its work - int numCallsSoFar = numCalls.get(); - Thread.sleep(400); - assertTrue(numCallsSoFar == numCalls.get() || - numCallsSoFar + 1 == numCalls.get()); + // no further calls may come. + manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); + assertEquals(5, numCalls.get()); // start another sequence of periodic scheduling numCalls.set(0); coord.startCheckpointScheduler(); - timeout = System.currentTimeMillis() + 60000; do { - Thread.sleep(20); + manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); } - while (timeout > System.currentTimeMillis() && numCalls.get() < 5); - assertTrue(numCalls.get() >= 5); + while (numCalls.get() < 5); + assertEquals(5, numCalls.get()); coord.stopCheckpointScheduler(); - // for 400 ms, no further calls may come - // there may be the case that one trigger was fired and about to - // acquire the lock, such that after cancelling it will still do - // the remainder of its work - numCallsSoFar = numCalls.get(); - Thread.sleep(400); - assertTrue(numCallsSoFar == numCalls.get() || - numCallsSoFar + 1 == numCalls.get()); + // no further calls may come + manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); + assertEquals(5, numCalls.get()); coord.shutdown(JobStatus.FINISHED); } @@ -203,9 +189,10 @@ public void testMinTimeBetweenCheckpointsInterval() throws Exception { }).when(executionAttempt).triggerCheckpoint(anyLong(), anyLong(), any(CheckpointOptions.class)); final long delay = 50; + final long checkpointInterval = 12; CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( - 12, // periodic interval is 12 ms + checkpointInterval, // periodic interval is 12 ms 200_000, // timeout is very long (200 s) delay, // 50 ms delay between checkpoints 1, @@ -223,12 +210,13 @@ public void testMinTimeBetweenCheckpointsInterval() throws Exception { new StandaloneCompletedCheckpointStore(2), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); try { coord.startCheckpointScheduler(); + manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); // wait until the first checkpoint was triggered Long firstCallId = triggerCalls.take(); @@ -240,6 +228,12 @@ public void testMinTimeBetweenCheckpointsInterval() throws Exception { final long ackTime = System.nanoTime(); coord.receiveAcknowledgeMessage(ackMsg, TASK_MANAGER_LOCATION_INFO); + manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); + while (triggerCalls.isEmpty()) { + // sleeps for a while to simulate periodic scheduling + Thread.sleep(checkpointInterval); + manuallyTriggeredScheduledExecutor.triggerPeriodicScheduledTasks(); + } // wait until the next checkpoint is triggered Long nextCallId = triggerCalls.take(); final long nextCheckpointTime = System.nanoTime(); @@ -284,7 +278,7 @@ public void testStopPeriodicScheduler() throws Exception { new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -296,6 +290,7 @@ public void testStopPeriodicScheduler() throws Exception { null, true, false); + manuallyTriggeredScheduledExecutor.triggerAll(); fail("The triggerCheckpoint call expected an exception"); } catch (CheckpointException e) { assertEquals(CheckpointFailureReason.PERIODIC_SCHEDULER_SHUTDOWN, e.getCheckpointFailureReason()); @@ -309,6 +304,7 @@ public void testStopPeriodicScheduler() throws Exception { null, false, false); + manuallyTriggeredScheduledExecutor.triggerAll(); } catch (CheckpointException e) { fail("Unexpected exception : " + e.getCheckpointFailureReason().message()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java index 080e1c786ba8..ca5a42e28fc4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.concurrent.Executors; +import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.Execution; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; @@ -34,13 +35,11 @@ import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.memory.MemoryStateBackend; import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; -import org.apache.flink.runtime.util.TestingScheduledExecutor; import org.apache.flink.util.SerializableObject; import org.hamcrest.BaseMatcher; import org.hamcrest.Description; import org.junit.Before; -import org.junit.Rule; import org.junit.Test; import org.mockito.Mockito; import org.mockito.hamcrest.MockitoHamcrest; @@ -65,9 +64,7 @@ public class CheckpointStateRestoreTest { private static final String TASK_MANAGER_LOCATION_INFO = "Unknown location"; - @Rule - public final TestingScheduledExecutor testingScheduledExecutor = - new TestingScheduledExecutor(); + private ManuallyTriggeredScheduledExecutor manuallyTriggeredScheduledExecutor; private CheckpointFailureManager failureManager; @@ -76,6 +73,7 @@ public void setUp() throws Exception { failureManager = new CheckpointFailureManager( 0, NoOpFailJobCall.INSTANCE); + manuallyTriggeredScheduledExecutor = new ManuallyTriggeredScheduledExecutor(); } /** @@ -133,13 +131,14 @@ public void testSetState() { new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); // create ourselves a checkpoint with state final long timestamp = 34623786L; coord.triggerCheckpoint(timestamp, false); + manuallyTriggeredScheduledExecutor.triggerAll(); PendingCheckpoint pending = coord.getPendingCheckpoints().values().iterator().next(); final long checkpointId = pending.getCheckpointId(); @@ -218,7 +217,7 @@ public void testNoCheckpointAvailable() { new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); @@ -284,7 +283,7 @@ public void testNonRestoredState() throws Exception { new StandaloneCompletedCheckpointStore(1), new MemoryStateBackend(), Executors.directExecutor(), - testingScheduledExecutor.getScheduledExecutor(), + manuallyTriggeredScheduledExecutor, SharedStateRegistry.DEFAULT_FACTORY, failureManager); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java index df3e97eee823..ab1424d16b43 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java @@ -92,8 +92,9 @@ public void testAbortPendingCheckpointsWithTriggerValidation() { checkpointCoordinator.startCheckpointScheduler(); assertTrue(checkpointCoordinator.isCurrentPeriodicTriggerAvailable()); - manualThreadExecutor.triggerAll(); - manualThreadExecutor.triggerScheduledTasks(); + // only trigger the periodic scheduling + // we can't trigger all scheduled task, because there is also a cancellation scheduled + manualThreadExecutor.triggerPeriodicScheduledTasks(); assertEquals(1, checkpointCoordinator.getNumberOfPendingCheckpoints()); for (int i = 1; i < maxConcurrentCheckpoints; i++) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/ManuallyTriggeredScheduledExecutor.java b/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/ManuallyTriggeredScheduledExecutor.java index 870cda8d7f7d..de2ec720476c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/ManuallyTriggeredScheduledExecutor.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/ManuallyTriggeredScheduledExecutor.java @@ -35,6 +35,7 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; /** * Simple {@link ScheduledExecutor} implementation for testing purposes. @@ -42,8 +43,14 @@ public class ManuallyTriggeredScheduledExecutor implements ScheduledExecutor { private final Executor executorDelegate; + private final ArrayDeque queuedRunnables = new ArrayDeque<>(); - private final ConcurrentLinkedQueue> scheduledTasks = new ConcurrentLinkedQueue<>(); + + private final ConcurrentLinkedQueue> nonPeriodicScheduledTasks = + new ConcurrentLinkedQueue<>(); + + private final ConcurrentLinkedQueue> periodicScheduledTasks = + new ConcurrentLinkedQueue<>(); public ManuallyTriggeredScheduledExecutor() { this.executorDelegate = Runnable::run; @@ -74,11 +81,7 @@ public void trigger() { next = queuedRunnables.removeFirst(); } - if (next != null) { - CompletableFuture.runAsync(next, executorDelegate).join(); - } else { - throw new IllegalStateException("No runnable available"); - } + CompletableFuture.runAsync(next, executorDelegate).join(); } /** @@ -92,58 +95,107 @@ public int numQueuedRunnables() { @Override public ScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) { - return insertRunnable(command, false); + return insertNonPeriodicTask(command, delay, unit); } @Override public ScheduledFuture schedule(Callable callable, long delay, TimeUnit unit) { - final ScheduledTask scheduledTask = new ScheduledTask<>(callable, false); - - scheduledTasks.offer(scheduledTask); - - return scheduledTask; + return insertNonPeriodicTask(callable, delay, unit); } @Override public ScheduledFuture scheduleAtFixedRate(Runnable command, long initialDelay, long period, TimeUnit unit) { - return insertRunnable(command, true); + return insertPeriodicRunnable(command, initialDelay, period, unit); } @Override public ScheduledFuture scheduleWithFixedDelay(Runnable command, long initialDelay, long delay, TimeUnit unit) { - return insertRunnable(command, true); + return insertPeriodicRunnable(command, initialDelay, delay, unit); } public Collection> getScheduledTasks() { - return new ArrayList<>(scheduledTasks); + final ArrayList> scheduledTasks = + new ArrayList<>(nonPeriodicScheduledTasks.size() + periodicScheduledTasks.size()); + scheduledTasks.addAll(getNonPeriodicScheduledTask()); + scheduledTasks.addAll(getPeriodicScheduledTask()); + return scheduledTasks; + } + + public Collection> getPeriodicScheduledTask() { + return periodicScheduledTasks + .stream() + .filter(scheduledTask -> !scheduledTask.isCancelled()) + .collect(Collectors.toList()); + } + + public Collection> getNonPeriodicScheduledTask() { + return nonPeriodicScheduledTasks + .stream() + .filter(scheduledTask -> !scheduledTask.isCancelled()) + .collect(Collectors.toList()); } /** * Triggers all registered tasks. */ public void triggerScheduledTasks() { - final Iterator> iterator = scheduledTasks.iterator(); + triggerPeriodicScheduledTasks(); + triggerNonPeriodicScheduledTasks(); + } + + public void triggerNonPeriodicScheduledTasks() { + final Iterator> iterator = nonPeriodicScheduledTasks.iterator(); while (iterator.hasNext()) { final ScheduledTask scheduledTask = iterator.next(); - scheduledTask.execute(); + if (!scheduledTask.isCancelled()) { + scheduledTask.execute(); + } + iterator.remove(); + } + } - if (!scheduledTask.isPeriodic) { - iterator.remove(); + public void triggerPeriodicScheduledTasks() { + for (ScheduledTask scheduledTask : periodicScheduledTasks) { + if (!scheduledTask.isCancelled()) { + scheduledTask.execute(); } } } - private ScheduledFuture insertRunnable(Runnable command, boolean isPeriodic) { + private ScheduledFuture insertPeriodicRunnable( + Runnable command, + long delay, + long period, + TimeUnit unit) { + final ScheduledTask scheduledTask = new ScheduledTask<>( () -> { command.run(); return null; }, - isPeriodic); + unit.convert(delay, TimeUnit.MILLISECONDS), + unit.convert(period, TimeUnit.MILLISECONDS)); - scheduledTasks.offer(scheduledTask); + periodicScheduledTasks.offer(scheduledTask); + + return scheduledTask; + } + + private ScheduledFuture insertNonPeriodicTask(Runnable command, long delay, TimeUnit unit) { + return insertNonPeriodicTask(() -> { + command.run(); + return null; + }, delay, unit); + } + + private ScheduledFuture insertNonPeriodicTask( + Callable callable, long delay, TimeUnit unit) { + final ScheduledTask scheduledTask = + new ScheduledTask<>(callable, unit.convert(delay, TimeUnit.MILLISECONDS)); + + nonPeriodicScheduledTasks.offer(scheduledTask); return scheduledTask; } @@ -152,20 +204,30 @@ private static final class ScheduledTask implements ScheduledFuture { private final Callable callable; - private final boolean isPeriodic; + private final long delay; + + private final long period; private final CompletableFuture result; - private ScheduledTask(Callable callable, boolean isPeriodic) { - this.callable = Preconditions.checkNotNull(callable); - this.isPeriodic = isPeriodic; + private ScheduledTask(Callable callable, long delay) { + this(callable, delay, 0); + } + private ScheduledTask(Callable callable, long delay, long period) { + this.callable = Preconditions.checkNotNull(callable); this.result = new CompletableFuture<>(); + this.delay = delay; + this.period = period; + } + + private boolean isPeriodic() { + return period > 0; } public void execute() { if (!result.isDone()) { - if (!isPeriodic) { + if (!isPeriodic()) { try { result.complete(callable.call()); } catch (Exception e) { @@ -183,12 +245,12 @@ public void execute() { @Override public long getDelay(TimeUnit unit) { - return 0; + return unit.convert(delay, TimeUnit.MILLISECONDS); } @Override public int compareTo(Delayed o) { - return 0; + return Long.compare(getDelay(TimeUnit.MILLISECONDS), o.getDelay(TimeUnit.MILLISECONDS)); } @Override @@ -212,7 +274,8 @@ public T get() throws InterruptedException, ExecutionException { } @Override - public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { + public T get(long timeout, @Nonnull TimeUnit unit) + throws InterruptedException, ExecutionException, TimeoutException { return result.get(timeout, unit); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleAckingTaskManagerGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleAckingTaskManagerGateway.java index e09d8beda8aa..3408e1132db9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleAckingTaskManagerGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleAckingTaskManagerGateway.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; -import org.apache.flink.api.java.tuple.Tuple6; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.concurrent.FutureUtils; @@ -56,7 +55,13 @@ public class SimpleAckingTaskManagerGateway implements TaskManagerGateway { private BiConsumer> releasePartitionsConsumer = (ignore1, ignore2) -> { }; - private Consumer> checkpointConsumer = ignore -> { }; + private CheckpointConsumer checkpointConsumer = ( + executionAttemptID, + jobId, + checkpointId, + timestamp, + checkpointOptions, + advanceToEndOfEventTime) -> { }; public void setSubmitConsumer(Consumer submitConsumer) { this.submitConsumer = submitConsumer; @@ -74,7 +79,7 @@ public void setReleasePartitionsConsumer(BiConsumer> checkpointConsumer) { + public void setCheckpointConsumer(CheckpointConsumer checkpointConsumer) { this.checkpointConsumer = checkpointConsumer; } @@ -131,7 +136,14 @@ public void triggerCheckpoint( long timestamp, CheckpointOptions checkpointOptions, boolean advanceToEndOfEventTime) { - checkpointConsumer.accept(Tuple6.of(executionAttemptID, jobId, checkpointId, timestamp, checkpointOptions, advanceToEndOfEventTime)); + + checkpointConsumer.accept( + executionAttemptID, + jobId, + checkpointId, + timestamp, + checkpointOptions, + advanceToEndOfEventTime); } @Override @@ -144,4 +156,18 @@ public CompletableFuture freeSlot(AllocationID allocationId, Throwa return CompletableFuture.completedFuture(Acknowledge.get()); } } + + /** + * Consumer that accepts checkpoint trigger information. + */ + public interface CheckpointConsumer { + + void accept( + ExecutionAttemptID executionAttemptID, + JobID jobId, + long checkpointId, + long timestamp, + CheckpointOptions checkpointOptions, + boolean advanceToEndOfEventTime); + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingScheduledExecutor.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingScheduledExecutor.java index 930af9fd118b..d9cfb11108bb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingScheduledExecutor.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingScheduledExecutor.java @@ -38,15 +38,15 @@ public class TestingScheduledExecutor extends ExternalResource { private ScheduledExecutorService innerExecutorService; public TestingScheduledExecutor() { - this(500L); - } + this(500L); + } public TestingScheduledExecutor(long shutdownTimeoutMillis) { this.shutdownTimeoutMillis = shutdownTimeoutMillis; } @Override - protected void before() { + public void before() { this.innerExecutorService = Executors.newSingleThreadScheduledExecutor(); this.scheduledExecutor = new ScheduledExecutorServiceAdapter(innerExecutorService); } @@ -56,7 +56,7 @@ protected void after() { ExecutorUtils.gracefulShutdown(shutdownTimeoutMillis, TimeUnit.MILLISECONDS, innerExecutorService); } - public ScheduledExecutor getScheduledExecutor() { + protected ScheduledExecutor getScheduledExecutor() { return scheduledExecutor; } } From 00fe29b1be59c9ee6d7d6f3f71a4759f5132e12d Mon Sep 17 00:00:00 2001 From: ifndef-SleePy Date: Mon, 30 Sep 2019 11:17:39 +0800 Subject: [PATCH 320/746] [FLINK-13904][checkpointing] Remove trigger lock of CheckpointCoordinator Now the checkpoint and savepoint triggering are executed in the one thread without any competition. So the triggerLock is no longer needed. --- .../checkpoint/CheckpointCoordinator.java | 228 ++++++++---------- 1 file changed, 107 insertions(+), 121 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index b4843e482733..7517a38db38b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -87,13 +87,6 @@ public class CheckpointCoordinator { /** Coordinator-wide lock to safeguard the checkpoint updates. */ private final Object lock = new Object(); - /** Lock specially to make sure that trigger requests do not overtake each other. - * This is not done with the coordinator-wide lock, because as part of triggering, - * blocking operations may happen (distributed atomic counters). - * Using a dedicated lock, we avoid blocking the processing of 'acknowledge/decline' - * messages during that phase. */ - private final Object triggerLock = new Object(); - /** The job whose checkpoint this coordinator coordinates. */ private final JobID job; @@ -537,149 +530,142 @@ public CompletableFuture triggerCheckpoint( // we will actually trigger this checkpoint! - // we lock with a special lock to make sure that trigger requests do not overtake each other. - // this is not done with the coordinator-wide lock, because the 'checkpointIdCounter' - // may issue blocking operations. Using a different lock than the coordinator-wide lock, - // we avoid blocking the processing of 'acknowledge/decline' messages during that time. - synchronized (triggerLock) { - - final CheckpointStorageLocation checkpointStorageLocation; - final long checkpointID; + final CheckpointStorageLocation checkpointStorageLocation; + final long checkpointID; - try { - // this must happen outside the coordinator-wide lock, because it communicates - // with external services (in HA mode) and may block for a while. - checkpointID = checkpointIdCounter.getAndIncrement(); + try { + // this must happen outside the coordinator-wide lock, because it communicates + // with external services (in HA mode) and may block for a while. + checkpointID = checkpointIdCounter.getAndIncrement(); - checkpointStorageLocation = props.isSavepoint() ? - checkpointStorage.initializeLocationForSavepoint(checkpointID, externalSavepointLocation) : - checkpointStorage.initializeLocationForCheckpoint(checkpointID); - } - catch (Throwable t) { - int numUnsuccessful = numUnsuccessfulCheckpointsTriggers.incrementAndGet(); - LOG.warn("Failed to trigger checkpoint for job {} ({} consecutive failed attempts so far).", - job, - numUnsuccessful, - t); - throw new CheckpointException(CheckpointFailureReason.EXCEPTION, t); - } + checkpointStorageLocation = props.isSavepoint() ? + checkpointStorage.initializeLocationForSavepoint(checkpointID, externalSavepointLocation) : + checkpointStorage.initializeLocationForCheckpoint(checkpointID); + } + catch (Throwable t) { + int numUnsuccessful = numUnsuccessfulCheckpointsTriggers.incrementAndGet(); + LOG.warn("Failed to trigger checkpoint for job {} ({} consecutive failed attempts so far).", + job, + numUnsuccessful, + t); + throw new CheckpointException(CheckpointFailureReason.EXCEPTION, t); + } - final PendingCheckpoint checkpoint = new PendingCheckpoint( - job, + final PendingCheckpoint checkpoint = new PendingCheckpoint( + job, + checkpointID, + timestamp, + ackTasks, + props, + checkpointStorageLocation, + executor); + + if (statsTracker != null) { + PendingCheckpointStats callback = statsTracker.reportPendingCheckpoint( checkpointID, timestamp, - ackTasks, - props, - checkpointStorageLocation, - executor); - - if (statsTracker != null) { - PendingCheckpointStats callback = statsTracker.reportPendingCheckpoint( - checkpointID, - timestamp, - props); + props); - checkpoint.setStatsCallback(callback); - } + checkpoint.setStatsCallback(callback); + } - // schedule the timer that will clean up the expired checkpoints - final Runnable canceller = () -> { - synchronized (lock) { - // only do the work if the checkpoint is not discarded anyways - // note that checkpoint completion discards the pending checkpoint object - if (!checkpoint.isDiscarded()) { - LOG.info("Checkpoint {} of job {} expired before completing.", checkpointID, job); + // schedule the timer that will clean up the expired checkpoints + final Runnable canceller = () -> { + synchronized (lock) { + // only do the work if the checkpoint is not discarded anyways + // note that checkpoint completion discards the pending checkpoint object + if (!checkpoint.isDiscarded()) { + LOG.info("Checkpoint {} of job {} expired before completing.", checkpointID, job); - failPendingCheckpoint(checkpoint, CheckpointFailureReason.CHECKPOINT_EXPIRED); - pendingCheckpoints.remove(checkpointID); - rememberRecentCheckpointId(checkpointID); + failPendingCheckpoint(checkpoint, CheckpointFailureReason.CHECKPOINT_EXPIRED); + pendingCheckpoints.remove(checkpointID); + rememberRecentCheckpointId(checkpointID); - triggerQueuedRequests(); - } + triggerQueuedRequests(); } - }; + } + }; - try { - // re-acquire the coordinator-wide lock - synchronized (lock) { - // since we released the lock in the meantime, we need to re-check - // that the conditions still hold. - if (shutdown) { - throw new CheckpointException(CheckpointFailureReason.CHECKPOINT_COORDINATOR_SHUTDOWN); + try { + // re-acquire the coordinator-wide lock + synchronized (lock) { + // since we released the lock in the meantime, we need to re-check + // that the conditions still hold. + if (shutdown) { + throw new CheckpointException(CheckpointFailureReason.CHECKPOINT_COORDINATOR_SHUTDOWN); + } + else if (!props.forceCheckpoint()) { + if (triggerRequestQueued) { + LOG.warn("Trying to trigger another checkpoint for job {} while one was queued already.", job); + throw new CheckpointException(CheckpointFailureReason.ALREADY_QUEUED); } - else if (!props.forceCheckpoint()) { - if (triggerRequestQueued) { - LOG.warn("Trying to trigger another checkpoint for job {} while one was queued already.", job); - throw new CheckpointException(CheckpointFailureReason.ALREADY_QUEUED); - } - checkConcurrentCheckpoints(); + checkConcurrentCheckpoints(); - checkMinPauseBetweenCheckpoints(); - } + checkMinPauseBetweenCheckpoints(); + } - LOG.info("Triggering checkpoint {} @ {} for job {}.", checkpointID, timestamp, job); + LOG.info("Triggering checkpoint {} @ {} for job {}.", checkpointID, timestamp, job); - pendingCheckpoints.put(checkpointID, checkpoint); + pendingCheckpoints.put(checkpointID, checkpoint); - ScheduledFuture cancellerHandle = timer.schedule( - canceller, - checkpointTimeout, TimeUnit.MILLISECONDS); + ScheduledFuture cancellerHandle = timer.schedule( + canceller, + checkpointTimeout, TimeUnit.MILLISECONDS); - if (!checkpoint.setCancellerHandle(cancellerHandle)) { - // checkpoint is already disposed! - cancellerHandle.cancel(false); - } + if (!checkpoint.setCancellerHandle(cancellerHandle)) { + // checkpoint is already disposed! + cancellerHandle.cancel(false); + } - // trigger the master hooks for the checkpoint - final List masterStates = MasterHooks.triggerMasterHooks(masterHooks.values(), - checkpointID, timestamp, executor, Time.milliseconds(checkpointTimeout)); - for (MasterState s : masterStates) { - checkpoint.addMasterState(s); - } + // trigger the master hooks for the checkpoint + final List masterStates = MasterHooks.triggerMasterHooks(masterHooks.values(), + checkpointID, timestamp, executor, Time.milliseconds(checkpointTimeout)); + for (MasterState s : masterStates) { + checkpoint.addMasterState(s); } - // end of lock scope + } + // end of lock scope - final CheckpointOptions checkpointOptions = new CheckpointOptions( - props.getCheckpointType(), - checkpointStorageLocation.getLocationReference()); + final CheckpointOptions checkpointOptions = new CheckpointOptions( + props.getCheckpointType(), + checkpointStorageLocation.getLocationReference()); - // send the messages to the tasks that trigger their checkpoint - for (Execution execution: executions) { - if (props.isSynchronous()) { - execution.triggerSynchronousSavepoint(checkpointID, timestamp, checkpointOptions, advanceToEndOfTime); - } else { - execution.triggerCheckpoint(checkpointID, timestamp, checkpointOptions); - } + // send the messages to the tasks that trigger their checkpoint + for (Execution execution: executions) { + if (props.isSynchronous()) { + execution.triggerSynchronousSavepoint(checkpointID, timestamp, checkpointOptions, advanceToEndOfTime); + } else { + execution.triggerCheckpoint(checkpointID, timestamp, checkpointOptions); } - - numUnsuccessfulCheckpointsTriggers.set(0); - return checkpoint.getCompletionFuture(); } - catch (Throwable t) { - // guard the map against concurrent modifications - synchronized (lock) { - pendingCheckpoints.remove(checkpointID); - } - int numUnsuccessful = numUnsuccessfulCheckpointsTriggers.incrementAndGet(); - LOG.warn("Failed to trigger checkpoint {} for job {}. ({} consecutive failed attempts so far)", - checkpointID, job, numUnsuccessful, t); + numUnsuccessfulCheckpointsTriggers.set(0); + return checkpoint.getCompletionFuture(); + } + catch (Throwable t) { + // guard the map against concurrent modifications + synchronized (lock) { + pendingCheckpoints.remove(checkpointID); + } - if (!checkpoint.isDiscarded()) { - failPendingCheckpoint(checkpoint, CheckpointFailureReason.TRIGGER_CHECKPOINT_FAILURE, t); - } + int numUnsuccessful = numUnsuccessfulCheckpointsTriggers.incrementAndGet(); + LOG.warn("Failed to trigger checkpoint {} for job {}. ({} consecutive failed attempts so far)", + checkpointID, job, numUnsuccessful, t); - try { - checkpointStorageLocation.disposeOnFailure(); - } - catch (Throwable t2) { - LOG.warn("Cannot dispose failed checkpoint storage location {}", checkpointStorageLocation, t2); - } + if (!checkpoint.isDiscarded()) { + failPendingCheckpoint(checkpoint, CheckpointFailureReason.TRIGGER_CHECKPOINT_FAILURE, t); + } - throw new CheckpointException(CheckpointFailureReason.EXCEPTION, t); + try { + checkpointStorageLocation.disposeOnFailure(); } - } // end trigger lock + catch (Throwable t2) { + LOG.warn("Cannot dispose failed checkpoint storage location {}", checkpointStorageLocation, t2); + } + + throw new CheckpointException(CheckpointFailureReason.EXCEPTION, t); + } } // -------------------------------------------------------------------------------------------- From beb3fb06bdca64c4732318667ab59ce298da3b97 Mon Sep 17 00:00:00 2001 From: ifndef-SleePy Date: Fri, 11 Oct 2019 16:40:27 +0800 Subject: [PATCH 321/746] [FLINK-13904][checkpointing] Encapsule and optimize the time relevant operation of CheckpointCoordinator --- .../checkpoint/CheckpointCoordinator.java | 61 +++++++++++++++---- 1 file changed, 50 insertions(+), 11 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index 7517a38db38b..6d34d8c1ccb8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -42,6 +42,8 @@ import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.SharedStateRegistryFactory; import org.apache.flink.runtime.state.StateBackend; +import org.apache.flink.runtime.util.clock.Clock; +import org.apache.flink.runtime.util.clock.SystemClock; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.Preconditions; import org.apache.flink.util.StringUtils; @@ -130,9 +132,9 @@ public class CheckpointCoordinator { /** The max time (in ms) that a checkpoint may take. */ private final long checkpointTimeout; - /** The min time(in ns) to delay after a checkpoint could be triggered. Allows to + /** The min time(in ms) to delay after a checkpoint could be triggered. Allows to * enforce minimum processing time between checkpoint attempts */ - private final long minPauseBetweenCheckpointsNanos; + private final long minPauseBetweenCheckpoints; /** The maximum number of checkpoints that may be in progress at the same time. */ private final int maxConcurrentCheckpointAttempts; @@ -153,8 +155,9 @@ public class CheckpointCoordinator { /** A handle to the current periodic trigger, to cancel it when necessary. */ private ScheduledFuture currentPeriodicTrigger; - /** The timestamp (via {@link System#nanoTime()}) when the last checkpoint completed. */ - private long lastCheckpointCompletionNanos; + /** The timestamp (via {@link Clock#relativeTimeMillis()}) when the last checkpoint + * completed. */ + private long lastCheckpointCompletionRelativeTime; /** Flag whether a triggered checkpoint should immediately schedule the next checkpoint. * Non-volatile, because only accessed in synchronized scope */ @@ -181,8 +184,41 @@ public class CheckpointCoordinator { private final CheckpointFailureManager failureManager; + private final Clock clock; + // -------------------------------------------------------------------------------------------- + public CheckpointCoordinator( + JobID job, + CheckpointCoordinatorConfiguration chkConfig, + ExecutionVertex[] tasksToTrigger, + ExecutionVertex[] tasksToWaitFor, + ExecutionVertex[] tasksToCommitTo, + CheckpointIDCounter checkpointIDCounter, + CompletedCheckpointStore completedCheckpointStore, + StateBackend checkpointStateBackend, + Executor executor, + ScheduledExecutor timer, + SharedStateRegistryFactory sharedStateRegistryFactory, + CheckpointFailureManager failureManager) { + + this( + job, + chkConfig, + tasksToTrigger, + tasksToWaitFor, + tasksToCommitTo, + checkpointIDCounter, + completedCheckpointStore, + checkpointStateBackend, + executor, + timer, + sharedStateRegistryFactory, + failureManager, + SystemClock.getInstance()); + } + + @VisibleForTesting public CheckpointCoordinator( JobID job, CheckpointCoordinatorConfiguration chkConfig, @@ -195,7 +231,8 @@ public CheckpointCoordinator( Executor executor, ScheduledExecutor timer, SharedStateRegistryFactory sharedStateRegistryFactory, - CheckpointFailureManager failureManager) { + CheckpointFailureManager failureManager, + Clock clock) { // sanity checks checkNotNull(checkpointStateBackend); @@ -216,7 +253,7 @@ public CheckpointCoordinator( this.job = checkNotNull(job); this.baseInterval = baseInterval; this.checkpointTimeout = chkConfig.getCheckpointTimeout(); - this.minPauseBetweenCheckpointsNanos = minPauseBetweenCheckpoints * 1_000_000; + this.minPauseBetweenCheckpoints = minPauseBetweenCheckpoints; this.maxConcurrentCheckpointAttempts = chkConfig.getMaxConcurrentCheckpoints(); this.tasksToTrigger = checkNotNull(tasksToTrigger); this.tasksToWaitFor = checkNotNull(tasksToWaitFor); @@ -229,6 +266,7 @@ public CheckpointCoordinator( this.sharedStateRegistry = sharedStateRegistryFactory.create(executor); this.isPreferCheckpointForRecovery = chkConfig.isPreferCheckpointForRecovery(); this.failureManager = checkNotNull(failureManager); + this.clock = checkNotNull(clock); this.recentPendingCheckpoints = new ArrayDeque<>(NUM_GHOST_CHECKPOINT_IDS); this.masterHooks = new HashMap<>(); @@ -890,7 +928,7 @@ public void run() { // record the time when this was completed, to calculate // the 'min delay between checkpoints' - lastCheckpointCompletionNanos = System.nanoTime(); + lastCheckpointCompletionRelativeTime = clock.relativeTimeMillis(); LOG.info("Completed checkpoint {} for job {} ({} bytes in {} ms).", checkpointId, job, completedCheckpoint.getStateSize(), completedCheckpoint.getDuration()); @@ -1253,8 +1291,10 @@ private void checkConcurrentCheckpoints() throws CheckpointException { * @throws CheckpointException If the minimum interval between checkpoints has not passed. */ private void checkMinPauseBetweenCheckpoints() throws CheckpointException { - final long earliestNext = lastCheckpointCompletionNanos + minPauseBetweenCheckpointsNanos; - final long durationTillNextMillis = (earliestNext - System.nanoTime()) / 1_000_000; + final long nextCheckpointTriggerRelativeTime = + lastCheckpointCompletionRelativeTime + minPauseBetweenCheckpoints; + final long durationTillNextMillis = + nextCheckpointTriggerRelativeTime - clock.relativeTimeMillis(); if (durationTillNextMillis > 0) { if (currentPeriodicTrigger != null) { @@ -1269,8 +1309,7 @@ private void checkMinPauseBetweenCheckpoints() throws CheckpointException { } private long getRandomInitDelay() { - return ThreadLocalRandom.current().nextLong( - minPauseBetweenCheckpointsNanos / 1_000_000L, baseInterval + 1L); + return ThreadLocalRandom.current().nextLong(minPauseBetweenCheckpoints, baseInterval + 1L); } private ScheduledFuture scheduleTriggerWithDelay(long initDelay) { From 6e7bdc3c30cadf4573154a474042c90dc6f8fb3b Mon Sep 17 00:00:00 2001 From: "Jiangjie (Becket) Qin" Date: Sat, 26 Oct 2019 18:56:59 +0800 Subject: [PATCH 322/746] [FLINK-14370][kafka][test-stability] Fix the cascading test failure in KafkaProducerTestBase. --- .../streaming/connectors/kafka/KafkaProducerTestBase.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) 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 10e4f552e738..86dfd85e5cf4 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 @@ -280,13 +280,12 @@ public int partition(Integer record, byte[] key, byte[] value, String targetTopi try { env.execute("One-to-one at least once test"); fail("Job should fail!"); - } - catch (JobExecutionException ex) { + } catch (JobExecutionException ex) { // ignore error, it can be one of many errors so it would be hard to check the exception message/cause + } finally { + kafkaServer.unblockProxyTraffic(); } - kafkaServer.unblockProxyTraffic(); - // assert that before failure we successfully snapshot/flushed all expected elements assertAtLeastOnceForTopic( properties, From 96640cad3d770756cb6e70c73b25bd4269065775 Mon Sep 17 00:00:00 2001 From: "Jiangjie (Becket) Qin" Date: Sat, 26 Oct 2019 18:55:44 +0800 Subject: [PATCH 323/746] [hotfix][kafka][test-stability] Accelerate the KafkaProducerTest by reducing the timeout values. --- .../streaming/connectors/kafka/KafkaProducerTestBase.java | 4 ++++ .../flink/streaming/connectors/kafka/KafkaTestBase.java | 4 ++++ 2 files changed, 8 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 86dfd85e5cf4..c0c98446b47c 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 @@ -243,6 +243,10 @@ protected void testOneToOneAtLeastOnce(boolean regularSink) throws Exception { properties.putAll(secureProps); // decrease timeout and block time from 60s down to 10s - this is how long KafkaProducer will try send pending (not flushed) data on close() properties.setProperty("timeout.ms", "10000"); + // KafkaProducer prior to KIP-91 (release 2.1) uses request timeout to expire the unsent records. + properties.setProperty("request.timeout.ms", "3000"); + // KafkaProducer in 2.1.0 and above uses delivery timeout to expire the the records. + properties.setProperty("delivery.timeout.ms", "5000"); properties.setProperty("max.block.ms", "10000"); // increase batch.size and linger.ms - this tells KafkaProducer to batch produced events instead of flushing them immediately properties.setProperty("batch.size", "10240000"); 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 d46e21c2b018..94d913356e36 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 @@ -218,6 +218,10 @@ protected void assertAtLeastOnceForTopic( while (System.currentTimeMillis() < startMillis + timeoutMillis) { properties.put("key.deserializer", "org.apache.kafka.common.serialization.IntegerDeserializer"); properties.put("value.deserializer", "org.apache.kafka.common.serialization.IntegerDeserializer"); + // We need to set these two properties so that they are lower than request.timeout.ms. This is + // required for some old KafkaConsumer versions. + properties.put("session.timeout.ms", "2000"); + properties.put("heartbeat.interval.ms", "500"); // query kafka for new records ... Collection> records = kafkaServer.getAllRecordsFromTopic(properties, topic, partition, 100); From e18320b76047af4e15297e3e89b6c46ef3dae9bf Mon Sep 17 00:00:00 2001 From: Rui Li Date: Thu, 17 Oct 2019 21:15:55 +0800 Subject: [PATCH 324/746] [FLINK-14397][hive] Failed to run Hive UDTF with array arguments Fix the issue that calling Hive UDTF with array arguments causes cast exception. This closes #9927. --- .../hive/conversion/HiveInspectors.java | 60 ++++++++++++++++++- .../hive/TableEnvHiveConnectorTest.java | 38 ++++++++++++ .../table/catalog/hive/HiveTestUtils.java | 32 ++++++++++ 3 files changed, 128 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/conversion/HiveInspectors.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/conversion/HiveInspectors.java index 22ce60bd6cdd..e7e88b2fe0ed 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/conversion/HiveInspectors.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/conversion/HiveInspectors.java @@ -96,7 +96,9 @@ import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; +import java.lang.reflect.Array; import java.math.BigDecimal; +import java.sql.Date; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -312,9 +314,11 @@ public static Object toFlinkObject(ObjectInspector inspector, Object data) { ListObjectInspector listInspector = (ListObjectInspector) inspector; List list = listInspector.getList(data); - Object[] result = new Object[list.size()]; + // flink expects a specific array type (e.g. Integer[] instead of Object[]), so we have to get the element class + ObjectInspector elementInspector = listInspector.getListElementObjectInspector(); + Object[] result = (Object[]) Array.newInstance(getClassFromObjectInspector(elementInspector), list.size()); for (int i = 0; i < list.size(); i++) { - result[i] = toFlinkObject(listInspector.getListElementObjectInspector(), list.get(i)); + result[i] = toFlinkObject(elementInspector, list.get(i)); } return result; } @@ -450,4 +454,56 @@ private static ObjectInspector getObjectInspector(TypeInfo type) { public static DataType toFlinkType(ObjectInspector inspector) { return HiveTypeUtil.toFlinkType(TypeInfoUtils.getTypeInfoFromTypeString(inspector.getTypeName())); } + + // given a Hive ObjectInspector, get the class for corresponding Flink object + private static Class getClassFromObjectInspector(ObjectInspector inspector) { + switch (inspector.getCategory()) { + case PRIMITIVE: { + PrimitiveObjectInspector primitiveOI = (PrimitiveObjectInspector) inspector; + switch (primitiveOI.getPrimitiveCategory()) { + case STRING: + case CHAR: + case VARCHAR: + return String.class; + case INT: + return Integer.class; + case LONG: + return Long.class; + case BYTE: + return Byte.class; + case SHORT: + return Short.class; + case FLOAT: + return Float.class; + case DOUBLE: + return Double.class; + case DECIMAL: + return BigDecimal.class; + case BOOLEAN: + return Boolean.class; + case BINARY: + return byte[].class; + case DATE: + return Date.class; + case TIMESTAMP: + case INTERVAL_DAY_TIME: + case INTERVAL_YEAR_MONTH: + default: + throw new IllegalArgumentException( + "Unsupported primitive type " + primitiveOI.getPrimitiveCategory().name()); + + } + } + case LIST: + ListObjectInspector listInspector = (ListObjectInspector) inspector; + Class elementClz = getClassFromObjectInspector(listInspector.getListElementObjectInspector()); + return Array.newInstance(elementClz, 0).getClass(); + case MAP: + return Map.class; + case STRUCT: + return Row.class; + default: + throw new IllegalArgumentException("Unsupported type " + inspector.getCategory().name()); + } + } } diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java index 784bcc8abbbe..9484e550b9d1 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java @@ -35,14 +35,17 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.Table; +import org.junit.Assume; import org.junit.BeforeClass; import org.junit.Test; import org.junit.runner.RunWith; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import scala.collection.JavaConverters; @@ -260,6 +263,41 @@ public void testPartialDynamicPartition() throws Exception { } } + @Test + public void testUDTF() throws Exception { + // W/o https://issues.apache.org/jira/browse/HIVE-11878 Hive registers the App classloader as the classloader + // for the UDTF and closes the App classloader when we tear down the session. This causes problems for JUnit code + // and shutdown hooks that have to run after the test finishes, because App classloader can no longer load new + // classes. And will crash the forked JVM, thus failing the test phase. + // Therefore disable such tests for older Hive versions. + String hiveVersion = HiveShimLoader.getHiveVersion(); + Assume.assumeTrue(hiveVersion.compareTo("2.0.0") >= 0 || hiveVersion.compareTo("1.3.0") >= 0); + hiveShell.execute("create database db1"); + try { + hiveShell.execute("create table db1.simple (i int,a array)"); + hiveShell.execute("create table db1.nested (a array>)"); + hiveShell.execute("create function hiveudtf as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDTFExplode'"); + hiveShell.insertInto("db1", "simple").addRow(3, Arrays.asList(1, 2, 3)).commit(); + Map map1 = new HashMap<>(); + map1.put(1, "a"); + map1.put(2, "b"); + Map map2 = new HashMap<>(); + map2.put(3, "c"); + hiveShell.insertInto("db1", "nested").addRow(Arrays.asList(map1, map2)).commit(); + + TableEnvironment tableEnv = getTableEnvWithHiveCatalog(); + List results = HiveTestUtils.collectTable(tableEnv, + tableEnv.sqlQuery("select x from db1.simple, lateral table(hiveudtf(a)) as T(x)")); + assertEquals("[1, 2, 3]", results.toString()); + results = HiveTestUtils.collectTable(tableEnv, + tableEnv.sqlQuery("select x from db1.nested, lateral table(hiveudtf(a)) as T(x)")); + assertEquals("[{1=a, 2=b}, {3=c}]", results.toString()); + } finally { + hiveShell.execute("drop database db1 cascade"); + hiveShell.execute("drop function hiveudtf"); + } + } + private TableEnvironment getTableEnvWithHiveCatalog() { TableEnvironment tableEnv = HiveTestUtils.createTableEnv(); tableEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java index deb9105e0d37..e274633472a5 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java @@ -18,12 +18,23 @@ package org.apache.flink.table.catalog.hive; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.accumulators.SerializedListAccumulator; +import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.SqlDialect; +import org.apache.flink.table.api.Table; import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.catalog.CatalogTest; import org.apache.flink.table.catalog.exceptions.CatalogException; import org.apache.flink.table.catalog.hive.client.HiveShimLoader; +import org.apache.flink.table.planner.sinks.CollectRowTableSink; +import org.apache.flink.table.planner.sinks.CollectTableSink; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.flink.types.Row; +import org.apache.flink.util.AbstractID; import org.apache.flink.util.StringUtils; import org.apache.hadoop.hive.conf.HiveConf; @@ -33,6 +44,9 @@ import java.net.BindException; import java.net.InetSocketAddress; import java.net.ServerSocket; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; import static org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM; @@ -108,4 +122,22 @@ public static TableEnvironment createTableEnv() { tableEnv.getConfig().setSqlDialect(SqlDialect.HIVE); return tableEnv; } + + public static List collectTable(TableEnvironment tableEnv, Table table) throws Exception { + CollectTableSink sink = new CollectRowTableSink(); + TableSchema tableSchema = table.getSchema(); + sink = (CollectTableSink) sink.configure(tableSchema.getFieldNames(), tableSchema.getFieldTypes()); + final String id = new AbstractID().toString(); + TypeSerializer serializer = TypeConversions.fromDataTypeToLegacyInfo(sink.getConsumedDataType()) + .createSerializer(new ExecutionConfig()); + sink.init(serializer, id); + String sinkName = UUID.randomUUID().toString(); + tableEnv.registerTableSink(sinkName, sink); + final String builtInCatalogName = EnvironmentSettings.DEFAULT_BUILTIN_CATALOG; + final String builtInDBName = EnvironmentSettings.DEFAULT_BUILTIN_DATABASE; + tableEnv.insertInto(table, builtInCatalogName, builtInDBName, sinkName); + JobExecutionResult result = tableEnv.execute("collect-table"); + ArrayList data = result.getAccumulatorResult(id); + return SerializedListAccumulator.deserializeList(data, serializer); + } } From 7420bdbe63c5cf8f9f04dcda00fbbe01b2b067fe Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Thu, 10 Oct 2019 15:52:38 +0800 Subject: [PATCH 325/746] [FLINK-14134][table-common] Introduce LimitableTableSource for optimizing limit --- .../table/sources/LimitableTableSource.java | 46 +++++++++++++++++++ 1 file changed, 46 insertions(+) create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/LimitableTableSource.java diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/LimitableTableSource.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/LimitableTableSource.java new file mode 100644 index 000000000000..da9b83280bed --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/sources/LimitableTableSource.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.sources; + +import org.apache.flink.annotation.Experimental; + +/** + * Adds support for limiting push-down to a {@link TableSource}. + * A {@link TableSource} extending this interface is able to limit the number of records. + * + *

After pushing down, source only needs to try its best to limit the number of output records, + * but does not need to guarantee that the number must be less than or equal to the limit. + */ +@Experimental +public interface LimitableTableSource { + + /** + * Return the flag to indicate whether limit push down has been tried. Must return true on + * the returned instance of {@link #applyLimit(long)}. + */ + boolean isLimitPushedDown(); + + /** + * Check and push down the limit to the table source. + * + * @param limit the value which limit the number of records. + * @return A new cloned instance of {@link TableSource}. + */ + TableSource applyLimit(long limit); +} From 4bf46c578d486355667c789d67b8a2e266e7c938 Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Thu, 10 Oct 2019 15:56:02 +0800 Subject: [PATCH 326/746] [FLINK-14134][table-planner-blink] Introduce PushLimitIntoTableSourceScanRule to apply LimitableTableSource --- .../plan/rules/FlinkBatchRuleSets.scala | 7 +- .../PushLimitIntoTableSourceScanRule.scala | 121 ++++++++++++++++++ .../planner/plan/batch/sql/LimitTest.xml | 105 +++++++++++++++ .../planner/plan/batch/sql/LimitTest.scala | 38 +++++- .../runtime/batch/sql/LimitITCase.scala | 22 ++-- .../utils/TestLimitableTableSource.scala | 78 +++++++++++ 6 files changed, 360 insertions(+), 11 deletions(-) create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoTableSourceScanRule.scala create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TestLimitableTableSource.scala 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 27191457fb43..2fd0fc418722 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 @@ -90,6 +90,10 @@ object FlinkBatchRuleSets { RewriteCoalesceRule.CALC_INSTANCE ) + private val LIMIT_RULES: RuleSet = RuleSets.ofList( + //push down localLimit + PushLimitIntoTableSourceScanRule.INSTANCE) + /** * RuleSet to simplify predicate expressions in filters and joins */ @@ -338,7 +342,8 @@ object FlinkBatchRuleSets { * RuleSet to do logical optimize for batch */ val LOGICAL_OPT_RULES: RuleSet = RuleSets.ofList(( - FILTER_RULES.asScala ++ + LIMIT_RULES.asScala ++ + FILTER_RULES.asScala ++ PROJECT_RULES.asScala ++ PRUNE_EMPTY_RULES.asScala ++ LOGICAL_RULES.asScala ++ diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoTableSourceScanRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoTableSourceScanRule.scala new file mode 100644 index 000000000000..0dd33c155d50 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoTableSourceScanRule.scala @@ -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.table.planner.plan.rules.logical + +import org.apache.flink.table.api.TableException +import org.apache.flink.table.plan.stats.TableStats +import org.apache.flink.table.planner.plan.nodes.logical.{FlinkLogicalSort, FlinkLogicalTableSourceScan} +import org.apache.flink.table.planner.plan.schema.{FlinkRelOptTable, TableSourceTable} +import org.apache.flink.table.planner.plan.stats.FlinkStatistic +import org.apache.flink.table.sources.LimitableTableSource + +import org.apache.calcite.plan.RelOptRule.{none, operand} +import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall} +import org.apache.calcite.rel.core.{Sort, TableScan} +import org.apache.calcite.rex.RexLiteral +import org.apache.calcite.tools.RelBuilder + +import java.util.Collections + +/** + * Planner rule that tries to push limit into a [[LimitableTableSource]]. + * The original limit will still be retained. + * + * The reasons why the limit still be retained: + * 1.If the source is required to return the exact number of limit number, the implementation + * of the source is highly required. The source is required to accurately control the record + * number of split, and the parallelism setting also need to be adjusted accordingly. + * 2.When remove the limit, maybe filter will be pushed down to the source after limit pushed + * down. The source need know it should do limit first and do the filter later, it is hard to + * implement. + * 3.We can support limit with offset, we can push down offset + fetch to table source. + */ +class PushLimitIntoTableSourceScanRule extends RelOptRule( + operand(classOf[FlinkLogicalSort], + operand(classOf[FlinkLogicalTableSourceScan], none)), "PushLimitIntoTableSourceScanRule") { + + override def matches(call: RelOptRuleCall): Boolean = { + val sort = call.rel(0).asInstanceOf[Sort] + val onlyLimit = sort.getCollation.getFieldCollations.isEmpty && sort.fetch != null + if (onlyLimit) { + call.rel(1).asInstanceOf[TableScan] + .getTable.unwrap(classOf[TableSourceTable[_]]) match { + case table: TableSourceTable[_] => + table.tableSource match { + case source: LimitableTableSource[_] => + return !source.isLimitPushedDown + case _ => + } + case _ => + } + } + false + } + + override def onMatch(call: RelOptRuleCall): Unit = { + val sort = call.rel(0).asInstanceOf[Sort] + val scan = call.rel(1).asInstanceOf[FlinkLogicalTableSourceScan] + val relOptTable = scan.getTable.asInstanceOf[FlinkRelOptTable] + val offset = if (sort.offset == null) 0 else RexLiteral.intValue(sort.offset) + val limit = offset + RexLiteral.intValue(sort.fetch) + val relBuilder = call.builder() + val newRelOptTable = applyLimit(limit, relOptTable, relBuilder) + val newScan = scan.copy(scan.getTraitSet, newRelOptTable) + + val newTableSource = newRelOptTable.unwrap(classOf[TableSourceTable[_]]).tableSource + val oldTableSource = relOptTable.unwrap(classOf[TableSourceTable[_]]).tableSource + + if (newTableSource.asInstanceOf[LimitableTableSource[_]].isLimitPushedDown + && newTableSource.explainSource().equals(oldTableSource.explainSource)) { + throw new TableException("Failed to push limit into table source! " + + "table source with pushdown capability must override and change " + + "explainSource() API to explain the pushdown applied!") + } + + call.transformTo(sort.copy(sort.getTraitSet, Collections.singletonList(newScan))) + } + + private def applyLimit( + limit: Long, + relOptTable: FlinkRelOptTable, + relBuilder: RelBuilder): FlinkRelOptTable = { + val tableSourceTable = relOptTable.unwrap(classOf[TableSourceTable[Any]]) + val limitedSource = tableSourceTable.tableSource.asInstanceOf[LimitableTableSource[Any]] + val newTableSource = limitedSource.applyLimit(limit) + + val statistic = relOptTable.getFlinkStatistic + val newRowCount = if (statistic.getRowCount != null) { + Math.min(limit, statistic.getRowCount.toLong) + } else { + limit + } + // Update TableStats after limit push down + val newTableStats = new TableStats(newRowCount) + val newStatistic = FlinkStatistic.builder() + .statistic(statistic) + .tableStats(newTableStats) + .build() + val newTableSourceTable = tableSourceTable.replaceTableSource(newTableSource).copy(newStatistic) + relOptTable.copy(newTableSourceTable, relOptTable.getRowType) + } +} + +object PushLimitIntoTableSourceScanRule { + val INSTANCE: RelOptRule = new PushLimitIntoTableSourceScanRule +} diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/LimitTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/LimitTest.xml index d2f4740dd9cd..de3fd4b08caa 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/LimitTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/LimitTest.xml @@ -30,6 +30,27 @@ LogicalSort(fetch=[0]) + + + + + + + + + + + @@ -51,6 +72,27 @@ Calc(select=[a, c]) +- Exchange(distribution=[single]) +- Limit(offset=[0], fetch=[20], global=[false]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +]]> + + + + + + + + + + + @@ -144,6 +186,27 @@ LogicalSort(fetch=[0]) + + + + + + + + + + + @@ -185,6 +248,27 @@ Calc(select=[a, c]) +- Limit(offset=[10], fetch=[unlimited], global=[true]) +- Exchange(distribution=[single]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +]]> + + + + + + + + + + + @@ -205,6 +289,27 @@ Limit(offset=[0], fetch=[5], global=[true]) +- Exchange(distribution=[single]) +- Limit(offset=[0], fetch=[5], global=[false]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +]]> + + + + + + + + + + + diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LimitTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LimitTest.scala index 432e0b184560..8e1c52d232e9 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LimitTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LimitTest.scala @@ -18,10 +18,13 @@ package org.apache.flink.table.planner.plan.batch.sql +import org.apache.flink.api.common.typeinfo.BasicTypeInfo.{INT_TYPE_INFO, LONG_TYPE_INFO, STRING_TYPE_INFO} +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.api.scala._ import org.apache.flink.table.api.SqlParserException import org.apache.flink.table.api.scala._ -import org.apache.flink.table.planner.utils.TableTestBase +import org.apache.flink.table.planner.utils.{TableTestBase, TestLimitableTableSource} import org.junit.Test @@ -29,6 +32,9 @@ class LimitTest extends TableTestBase { private val util = batchTestUtil() util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) + util.addTableSource("LimitTable", new TestLimitableTableSource(null, new RowTypeInfo( + Array[TypeInformation[_]](INT_TYPE_INFO, LONG_TYPE_INFO, STRING_TYPE_INFO), + Array("a", "b", "c")))) @Test def testLimitWithoutOffset(): Unit = { @@ -90,4 +96,34 @@ class LimitTest extends TableTestBase { util.verifyPlan("SELECT a, c FROM MyTable OFFSET 10 ROWS") } + @Test + def testFetchWithLimitSource(): Unit = { + val sqlQuery = "SELECT a, c FROM LimitTable FETCH FIRST 10 ROWS ONLY" + util.verifyPlan(sqlQuery) + } + + @Test + def testOrderByWithLimitSource(): Unit = { + val sqlQuery = "SELECT a, c FROM LimitTable ORDER BY c LIMIT 10" + util.verifyPlan(sqlQuery) + } + + @Test + def testLimitWithLimitSource(): Unit = { + val sqlQuery = "SELECT a, c FROM LimitTable LIMIT 10" + util.verifyPlan(sqlQuery) + } + + @Test + def testLimitWithOffsetAndLimitSource(): Unit = { + val sqlQuery = "SELECT a, c FROM LimitTable LIMIT 10 OFFSET 1" + util.verifyPlan(sqlQuery) + } + + @Test + def testFetchWithOffsetAndLimitSource(): Unit = { + val sqlQuery = "SELECT a, c FROM LimitTable OFFSET 10 ROWS FETCH NEXT 10 ROWS ONLY" + util.verifyPlan(sqlQuery) + } + } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/LimitITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/LimitITCase.scala index cca192a3fd5d..dd29c2412912 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/LimitITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/LimitITCase.scala @@ -18,8 +18,10 @@ package org.apache.flink.table.planner.runtime.batch.sql +import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.table.planner.runtime.utils.BatchTestBase import org.apache.flink.table.planner.runtime.utils.TestData._ +import org.apache.flink.table.planner.utils.TestLimitableTableSource import org.junit._ @@ -30,9 +32,8 @@ class LimitITCase extends BatchTestBase { super.before() registerCollection("Table3", data3, type3, "a, b, c", nullablesOfData3) - // TODO support LimitableTableSource -// val rowType = new RowTypeInfo(type3.getFieldTypes, Array("a", "b", "c")) -// tEnv.registerTableSource("LimitTable", new TestLimitableTableSource(data3, rowType)) + val rowType = new RowTypeInfo(type3.getFieldTypes, Array("a", "b", "c")) + tEnv.registerTableSource("LimitTable", new TestLimitableTableSource(data3, rowType)) } @Test @@ -56,7 +57,6 @@ class LimitITCase extends BatchTestBase { 10) } - @Ignore @Test def testFetchWithLimitTable(): Unit = { checkSize( @@ -71,7 +71,6 @@ class LimitITCase extends BatchTestBase { 10) } - @Ignore @Test def testFetchFirstWithLimitTable(): Unit = { checkSize( @@ -86,7 +85,13 @@ class LimitITCase extends BatchTestBase { 5) } - @Ignore + @Test + def testLimit0WithLimitTable(): Unit = { + checkSize( + "SELECT * FROM LimitTable LIMIT 0", + 0) + } + @Test def testLimitWithLimitTable(): Unit = { checkSize( @@ -94,7 +99,7 @@ class LimitITCase extends BatchTestBase { 5) } - @Ignore + @Ignore // TODO support limit without sort in table api. @Test def testTableLimitWithLimitTable(): Unit = { Assert.assertEquals( @@ -109,8 +114,7 @@ class LimitITCase extends BatchTestBase { 19) } - @Ignore - @Test(expected = classOf[AssertionError]) + @Test def testLessThanOffsetWithLimitSource(): Unit = { checkSize( "SELECT * FROM LimitTable OFFSET 2 ROWS FETCH NEXT 50 ROWS ONLY", diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TestLimitableTableSource.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TestLimitableTableSource.scala new file mode 100644 index 000000000000..63ada494cb2e --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TestLimitableTableSource.scala @@ -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.table.planner.utils + +import org.apache.flink.api.common.ExecutionConfig +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.io.CollectionInputFormat +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.streaming.api.datastream.DataStream +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment +import org.apache.flink.table.api.TableSchema +import org.apache.flink.table.sources._ +import org.apache.flink.types.Row + +import scala.collection.JavaConverters._ + +/** + * The table source which support push-down the limit to the source. + */ +class TestLimitableTableSource( + data: Seq[Row], + rowType: RowTypeInfo, + var limit: Long = -1, + var limitablePushedDown: Boolean = false) + extends StreamTableSource[Row] + with LimitableTableSource[Row] { + + override def isBounded = true + + override def getDataStream(execEnv: StreamExecutionEnvironment): DataStream[Row] = { + if (limit == 0) { + throw new RuntimeException("limit 0 should be optimize to single values.") + } + val dataSet = if (limit > 0) { + data.take(limit.toInt).asJava + } else { + data.asJava + } + execEnv.createInput( + new CollectionInputFormat(dataSet, rowType.createSerializer(new ExecutionConfig)), + rowType) + } + + override def applyLimit(limit: Long): TableSource[Row] = { + new TestLimitableTableSource(data, rowType, limit, limitablePushedDown) + } + + override def isLimitPushedDown: Boolean = limitablePushedDown + + override def getReturnType: TypeInformation[Row] = rowType + + override def explainSource(): String = { + if (limit > 0) { + "limit: " + limit + } else { + "" + } + } + + override def getTableSchema: TableSchema = TableSchema.fromTypeInfo(rowType) +} + From 005bda9be361ffe6c371ebefed32e614cbacd876 Mon Sep 17 00:00:00 2001 From: xuyang1706 Date: Wed, 31 Jul 2019 21:12:58 +0800 Subject: [PATCH 327/746] [FLINK-13513][ml] Add the Mapper and related classes for later algorithm implementations. --- .../apache/flink/ml/common/mapper/Mapper.java | 83 +++++++++++++++++++ .../flink/ml/common/mapper/MapperAdapter.java | 45 ++++++++++ .../flink/ml/common/mapper/ModelMapper.java | 72 ++++++++++++++++ .../ml/common/mapper/ModelMapperAdapter.java | 62 ++++++++++++++ .../model/BroadcastVariableModelSource.java | 50 +++++++++++ .../flink/ml/common/model/ModelSource.java | 39 +++++++++ .../ml/common/model/RowsModelSource.java | 50 +++++++++++ 7 files changed, 401 insertions(+) create mode 100644 flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/mapper/Mapper.java create mode 100644 flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/mapper/MapperAdapter.java create mode 100644 flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/mapper/ModelMapper.java create mode 100644 flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/mapper/ModelMapperAdapter.java create mode 100644 flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/model/BroadcastVariableModelSource.java create mode 100644 flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/model/ModelSource.java create mode 100644 flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/model/RowsModelSource.java diff --git a/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/mapper/Mapper.java b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/mapper/Mapper.java new file mode 100644 index 000000000000..84aafed52a13 --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/mapper/Mapper.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.ml.common.mapper; + +import org.apache.flink.ml.api.misc.param.Params; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.types.DataType; +import org.apache.flink.types.Row; + +import java.io.Serializable; + +/** + * Abstract class for mappers. A mapper takes one row as input and transform it into another row. + */ +public abstract class Mapper implements Serializable { + + /** + * Schema of the input rows. + */ + private final String[] dataFieldNames; + private final DataType[] dataFieldTypes; + + /** + * Parameters for the Mapper. + * Users can set the params before the Mapper is executed. + */ + protected final Params params; + + /** + * Construct a Mapper. + * + * @param dataSchema The schema of input rows. + * @param params The parameters for this mapper. + */ + public Mapper(TableSchema dataSchema, Params params) { + this.dataFieldNames = dataSchema.getFieldNames(); + this.dataFieldTypes = dataSchema.getFieldDataTypes(); + this.params = (null == params) ? new Params() : params.clone(); + } + + /** + * Get the schema of input rows. + * + * @return The schema of input rows. + */ + protected TableSchema getDataSchema() { + return TableSchema.builder().fields(dataFieldNames, dataFieldTypes).build(); + } + + /** + * Map a row to a new row. + * + * @param row The input row. + * @return A new row. + * @throws Exception This method may throw exceptions. Throwing an exception will cause the operation to fail. + */ + public abstract Row map(Row row) throws Exception; + + /** + * Get the schema of the output rows of {@link #map(Row)} method. + * + * @return The table schema of the output rows of {@link #map(Row)} method. + */ + public abstract TableSchema getOutputSchema(); + +} diff --git a/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/mapper/MapperAdapter.java b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/mapper/MapperAdapter.java new file mode 100644 index 000000000000..57e5c02ecdb8 --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/mapper/MapperAdapter.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.common.mapper; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.types.Row; + +/** + * A class that helps adapt a {@link Mapper} to a {@link MapFunction} so that the mapper can run in Flink. + */ +public class MapperAdapter implements MapFunction { + + private final Mapper mapper; + + /** + * Construct a MapperAdapter with the given mapper. + * + * @param mapper The {@link Mapper} to adapt. + */ + public MapperAdapter(Mapper mapper) { + this.mapper = mapper; + } + + @Override + public Row map(Row row) throws Exception { + return this.mapper.map(row); + } +} diff --git a/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/mapper/ModelMapper.java b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/mapper/ModelMapper.java new file mode 100644 index 000000000000..e831d0a1672e --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/mapper/ModelMapper.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.common.mapper; + +import org.apache.flink.ml.api.misc.param.Params; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.types.DataType; +import org.apache.flink.types.Row; + +import java.util.List; + +/** + * An abstract class for {@link Mapper Mappers} with a model. + */ +public abstract class ModelMapper extends Mapper { + + /** + * Field names of the model rows. + */ + private final String[] modelFieldNames; + + /** + * Field types of the model rows. + */ + private final DataType[] modelFieldTypes; + + /** + * Constructs a ModelMapper. + * + * @param modelSchema The schema of the model rows passed to {@link #loadModel(List)}. + * @param dataSchema The schema of the input data rows. + * @param params The parameters of this ModelMapper. + */ + public ModelMapper(TableSchema modelSchema, TableSchema dataSchema, Params params) { + super(dataSchema, params); + this.modelFieldNames = modelSchema.getFieldNames(); + this.modelFieldTypes = modelSchema.getFieldDataTypes(); + } + + /** + * Get the schema of the model rows that are passed to {@link #loadModel(List)}. + * + * @return The schema of the model rows. + */ + protected TableSchema getModelSchema() { + return TableSchema.builder().fields(this.modelFieldNames, this.modelFieldTypes).build(); + } + + /** + * Load the model from the list of rows. + * + * @param modelRows The list of rows that containing the model. + */ + public abstract void loadModel(List modelRows); +} diff --git a/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/mapper/ModelMapperAdapter.java b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/mapper/ModelMapperAdapter.java new file mode 100644 index 000000000000..e1d9f8f5c1c6 --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/mapper/ModelMapperAdapter.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flink.ml.common.mapper; + +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.ml.common.model.ModelSource; +import org.apache.flink.types.Row; + +import java.util.List; + +/** + * A class that adapts a {@link ModelMapper} to a Flink {@link RichMapFunction} so the model can be + * loaded in a Flink job. + * + *

This adapter class hold the target {@link ModelMapper} and it's {@link ModelSource}. Upon open(), + * it will load model rows from {@link ModelSource} into {@link ModelMapper}. + */ +public class ModelMapperAdapter extends RichMapFunction { + + private final ModelMapper mapper; + private final ModelSource modelSource; + + /** + * Construct a ModelMapperAdapter with the given ModelMapper and ModelSource. + * + * @param mapper The {@link ModelMapper} to adapt. + * @param modelSource The {@link ModelSource} to load the model from. + */ + public ModelMapperAdapter(ModelMapper mapper, ModelSource modelSource) { + this.mapper = mapper; + this.modelSource = modelSource; + } + + @Override + public void open(Configuration parameters) throws Exception { + List modelRows = this.modelSource.getModelRows(getRuntimeContext()); + this.mapper.loadModel(modelRows); + } + + @Override + public Row map(Row row) throws Exception { + return this.mapper.map(row); + } +} diff --git a/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/model/BroadcastVariableModelSource.java b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/model/BroadcastVariableModelSource.java new file mode 100644 index 000000000000..a86403bd206c --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/model/BroadcastVariableModelSource.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.ml.common.model; + +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.types.Row; + +import java.util.List; + +/** + * A {@link ModelSource} implementation that reads the model from the broadcast variable. + */ +public class BroadcastVariableModelSource implements ModelSource { + + /** + * The name of the broadcast variable that hosts the model. + */ + private final String modelVariableName; + + /** + * Construct a BroadcastVariableModelSource. + * + * @param modelVariableName The name of the broadcast variable that hosts a BroadcastVariableModelSource. + */ + public BroadcastVariableModelSource(String modelVariableName) { + this.modelVariableName = modelVariableName; + } + + @Override + public List getModelRows(RuntimeContext runtimeContext) { + return runtimeContext.getBroadcastVariable(modelVariableName); + } +} diff --git a/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/model/ModelSource.java b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/model/ModelSource.java new file mode 100644 index 000000000000..443b647e87bf --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/model/ModelSource.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flink.ml.common.model; + +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.types.Row; + +import java.io.Serializable; +import java.util.List; + +/** + * An interface that load the model from different sources. E.g. broadcast variables, list of rows, etc. + */ +public interface ModelSource extends Serializable { + + /** + * Get the rows that containing the model. + * + * @return the rows that containing the model. + */ + List getModelRows(RuntimeContext runtimeContext); +} diff --git a/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/model/RowsModelSource.java b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/model/RowsModelSource.java new file mode 100644 index 000000000000..89e3b33bb2dd --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/model/RowsModelSource.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.ml.common.model; + +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.types.Row; + +import java.util.List; + +/** + * A {@link ModelSource} implementation that reads the model from the memory. + */ +public class RowsModelSource implements ModelSource { + + /** + * The rows that hosts the model. + */ + private final List modelRows; + + /** + * Construct a RowsModelSource with the given rows containing a model. + * + * @param modelRows The rows that contains a model. + */ + public RowsModelSource(List modelRows) { + this.modelRows = modelRows; + } + + @Override + public List getModelRows(RuntimeContext runtimeContext) { + return modelRows; + } +} From b7cd6a984750d7dd98e6408f3703b1978b14355a Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Tue, 29 Oct 2019 09:03:44 +0100 Subject: [PATCH 328/746] [FLINK-14493][core] Introduce data types to ConfigOptions. NOTE: Starting from this commit getters in Configuration throw an exception when parsing of a value failed, instead of returning the default value. --- .../flink/configuration/ConfigOption.java | 66 +-- .../flink/configuration/ConfigOptions.java | 205 +++++++- .../flink/configuration/Configuration.java | 437 +++++++++++------- .../DelegatingConfiguration.java | 21 +- .../flink/configuration/ReadableConfig.java | 53 +++ .../StructuredOptionsSplitter.java | 176 +++++++ .../flink/configuration/WritableConfig.java | 40 ++ .../ConfigurationConversionsTest.java | 378 +++++++++++++++ ...onfigurationParsingInvalidFormatsTest.java | 87 ++++ .../configuration/ConfigurationTest.java | 126 ----- .../ReadableWritableConfigurationTest.java | 265 +++++++++++ .../StructuredOptionsSplitterTest.java | 158 +++++++ .../UnmodifiableConfigurationTest.java | 3 +- 13 files changed, 1678 insertions(+), 337 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/configuration/ReadableConfig.java create mode 100644 flink-core/src/main/java/org/apache/flink/configuration/StructuredOptionsSplitter.java create mode 100644 flink-core/src/main/java/org/apache/flink/configuration/WritableConfig.java create mode 100644 flink-core/src/test/java/org/apache/flink/configuration/ConfigurationConversionsTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/configuration/ConfigurationParsingInvalidFormatsTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/configuration/ReadableWritableConfigurationTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/configuration/StructuredOptionsSplitterTest.java diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigOption.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigOption.java index 9ad49c097e9b..10cdc3f1fb9e 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigOption.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigOption.java @@ -43,6 +43,8 @@ public class ConfigOption { private static final FallbackKey[] EMPTY = new FallbackKey[0]; + static final Description EMPTY_DESCRIPTION = Description.builder().text("").build(); + // ------------------------------------------------------------------------ /** The current key for that config option. */ @@ -57,51 +59,51 @@ public class ConfigOption { /** The description for this option. */ private final Description description; - // ------------------------------------------------------------------------ - /** - * Creates a new config option with no fallback keys. - * - * @param key The current key for that config option - * @param defaultValue The default value for this option + * Type of the value that this ConfigOption describes. + *

    + *
  • typeClass == atomic class (e.g. {@code Integer.class}) -> {@code ConfigOption}
  • + *
  • typeClass == {@code Map.class} -> {@code ConfigOption>}
  • + *
  • typeClass == atomic class and isList == true for {@code ConfigOption>}
  • + *
*/ - ConfigOption(String key, T defaultValue) { - this.key = checkNotNull(key); - this.description = Description.builder().text("").build(); - this.defaultValue = defaultValue; - this.fallbackKeys = EMPTY; + private final Class clazz; + + private final boolean isList; + + // ------------------------------------------------------------------------ + + Class getClazz() { + return clazz; } - /** - * Creates a new config option with fallback keys. - * - * @param key The current key for that config option - * @param description Description for that option - * @param defaultValue The default value for this option - * @param fallbackKeys The list of fallback keys, in the order to be checked - * @deprecated use version with {@link Description} instead - */ - @Deprecated - ConfigOption(String key, String description, T defaultValue, FallbackKey... fallbackKeys) { - this.key = checkNotNull(key); - this.description = Description.builder().text(description).build(); - this.defaultValue = defaultValue; - this.fallbackKeys = fallbackKeys == null || fallbackKeys.length == 0 ? EMPTY : fallbackKeys; + boolean isList() { + return isList; } /** * Creates a new config option with fallback keys. * * @param key The current key for that config option + * @param clazz describes type of the ConfigOption, see description of the clazz field * @param description Description for that option * @param defaultValue The default value for this option + * @param isList tells if the ConfigOption describes a list option, see description of the clazz field * @param fallbackKeys The list of fallback keys, in the order to be checked */ - ConfigOption(String key, Description description, T defaultValue, FallbackKey... fallbackKeys) { + ConfigOption( + String key, + Class clazz, + Description description, + T defaultValue, + boolean isList, + FallbackKey... fallbackKeys) { this.key = checkNotNull(key); this.description = description; this.defaultValue = defaultValue; this.fallbackKeys = fallbackKeys == null || fallbackKeys.length == 0 ? EMPTY : fallbackKeys; + this.clazz = checkNotNull(clazz); + this.isList = isList; } // ------------------------------------------------------------------------ @@ -124,7 +126,7 @@ public ConfigOption withFallbackKeys(String... fallbackKeys) { // put fallback keys first so that they are prioritized final FallbackKey[] mergedAlternativeKeys = Stream.concat(newFallbackKeys, currentAlternativeKeys) .toArray(FallbackKey[]::new); - return new ConfigOption<>(key, description, defaultValue, mergedAlternativeKeys); + return new ConfigOption<>(key, clazz, description, defaultValue, isList, mergedAlternativeKeys); } /** @@ -145,7 +147,7 @@ public ConfigOption withDeprecatedKeys(String... deprecatedKeys) { // put deprecated keys last so that they are de-prioritized final FallbackKey[] mergedAlternativeKeys = Stream.concat(currentAlternativeKeys, newDeprecatedKeys) .toArray(FallbackKey[]::new); - return new ConfigOption<>(key, description, defaultValue, mergedAlternativeKeys); + return new ConfigOption<>(key, clazz, description, defaultValue, isList, mergedAlternativeKeys); } /** @@ -167,7 +169,7 @@ public ConfigOption withDescription(final String description) { * @return A new config option, with given description. */ public ConfigOption withDescription(final Description description) { - return new ConfigOption<>(key, description, defaultValue, fallbackKeys); + return new ConfigOption<>(key, clazz, description, defaultValue, isList, fallbackKeys); } // ------------------------------------------------------------------------ @@ -203,8 +205,7 @@ public T defaultValue() { */ @Deprecated public boolean hasDeprecatedKeys() { - return fallbackKeys == EMPTY ? false : - Arrays.stream(fallbackKeys).anyMatch(FallbackKey::isDeprecated); + return fallbackKeys != EMPTY && Arrays.stream(fallbackKeys).anyMatch(FallbackKey::isDeprecated); } /** @@ -276,4 +277,5 @@ public String toString() { return String.format("Key: '%s' , default: %s (fallback keys: %s)", key, defaultValue, Arrays.toString(fallbackKeys)); } + } diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigOptions.java index 1ec5b3ca85dc..559cbf464e2b 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigOptions.java @@ -19,6 +19,12 @@ package org.apache.flink.configuration; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.description.Description; + +import java.time.Duration; +import java.util.Arrays; +import java.util.List; +import java.util.Map; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -30,21 +36,32 @@ * // simple string-valued option with a default value * ConfigOption tempDirs = ConfigOptions * .key("tmp.dir") + * .stringType() * .defaultValue("/tmp"); * * // simple integer-valued option with a default value * ConfigOption parallelism = ConfigOptions * .key("application.parallelism") + * .intType() * .defaultValue(100); * + * // option of list of integers with a default value + * ConfigOption parallelism = ConfigOptions + * .key("application.ports") + * .intType() + * .asList() + * .defaultValue(8000, 8001, 8002); + * * // option with no default value * ConfigOption userName = ConfigOptions * .key("user.name") + * .stringType() * .noDefaultValue(); * * // option with deprecated keys to check * ConfigOption threshold = ConfigOptions * .key("cpu.utilization.threshold") + * .doubleType() * .defaultValue(0.9). * .withDeprecatedKeys("cpu.threshold"); * } @@ -82,6 +99,79 @@ public static final class OptionBuilder { this.key = key; } + /** + * Defines that the value of the option should be of {@link Boolean} type. + */ + public TypedConfigOptionBuilder booleanType() { + return new TypedConfigOptionBuilder<>(key, Boolean.class); + } + + /** + * Defines that the value of the option should be of {@link Integer} type. + */ + public TypedConfigOptionBuilder intType() { + return new TypedConfigOptionBuilder<>(key, Integer.class); + } + + /** + * Defines that the value of the option should be of {@link Long} type. + */ + public TypedConfigOptionBuilder longType() { + return new TypedConfigOptionBuilder<>(key, Long.class); + } + + /** + * Defines that the value of the option should be of {@link Float} type. + */ + public TypedConfigOptionBuilder floatType() { + return new TypedConfigOptionBuilder<>(key, Float.class); + } + + /** + * Defines that the value of the option should be of {@link Double} type. + */ + public TypedConfigOptionBuilder doubleType() { + return new TypedConfigOptionBuilder<>(key, Double.class); + } + + /** + * Defines that the value of the option should be of {@link String} type. + */ + public TypedConfigOptionBuilder stringType() { + return new TypedConfigOptionBuilder<>(key, String.class); + } + + /** + * Defines that the value of the option should be of {@link Duration} type. + */ + public TypedConfigOptionBuilder durationType() { + return new TypedConfigOptionBuilder<>(key, Duration.class); + } + + /** + * Defines that the value of the option should be of {@link MemorySize} type. + */ + public TypedConfigOptionBuilder memoryType() { + return new TypedConfigOptionBuilder<>(key, MemorySize.class); + } + + /** + * Defines that the value of the option should be of {@link Enum} type. + * + * @param enumClass Concrete type of the expected enum. + */ + public > TypedConfigOptionBuilder enumType(Class enumClass) { + return new TypedConfigOptionBuilder<>(key, enumClass); + } + + /** + * Defines that the value of the option should be a set of properties, which can be represented as + * {@code Map}. + */ + public TypedConfigOptionBuilder> mapType() { + return new TypedConfigOptionBuilder<>(key, Map.class); + } + /** * Creates a ConfigOption with the given default value. * @@ -91,10 +181,17 @@ public static final class OptionBuilder { * @param value The default value for the config option * @param The type of the default value. * @return The config option with the default value. + * @deprecated define the type explicitly first with one of the intType(), stringType(), etc. */ + @Deprecated public ConfigOption defaultValue(T value) { checkNotNull(value); - return new ConfigOption<>(key, value); + return new ConfigOption<>( + key, + value.getClass(), + ConfigOption.EMPTY_DESCRIPTION, + value, + false); } /** @@ -103,12 +200,116 @@ public ConfigOption defaultValue(T value) { * default value. * * @return The created ConfigOption. + * @deprecated define the type explicitly first with one of the intType(), stringType(), etc. */ + @Deprecated public ConfigOption noDefaultValue() { - return new ConfigOption<>(key, null); + return new ConfigOption<>( + key, + String.class, + ConfigOption.EMPTY_DESCRIPTION, + null, + false); + } + } + + /** + * Builder for {@link ConfigOption} with a defined atomic type. + * + * @param atomic type of the option + */ + public static class TypedConfigOptionBuilder { + private final String key; + private final Class clazz; + + TypedConfigOptionBuilder(String key, Class clazz) { + this.key = key; + this.clazz = clazz; + } + + /** + * Defines that the option's type should be a list of previously defined atomic type. + */ + public ListConfigOptionBuilder asList() { + return new ListConfigOptionBuilder<>(key, clazz); + } + + /** + * Creates a ConfigOption with the given default value. + * + * @param value The default value for the config option + * @return The config option with the default value. + */ + public ConfigOption defaultValue(T value) { + return new ConfigOption<>( + key, + clazz, + ConfigOption.EMPTY_DESCRIPTION, + value, + false); + } + + /** + * Creates a ConfigOption without a default value. + * + * @return The config option without a default value. + */ + public ConfigOption noDefaultValue() { + return new ConfigOption( + key, + clazz, + Description.builder().text("").build(), + null, + false); } } + /** + * Builder for {@link ConfigOption} of list of type T. + * + * @param atomic type of the option + */ + public static class ListConfigOptionBuilder { + private final String key; + private final Class clazz; + + ListConfigOptionBuilder(String key, Class clazz) { + this.key = key; + this.clazz = clazz; + } + + /** + * Creates a ConfigOption with the given default value. + * + * @param values The list of default values for the config option + * @return The config option with the default value. + */ + @SafeVarargs + public final ConfigOption> defaultValues(T... values) { + return new ConfigOption<>( + key, + clazz, + ConfigOption.EMPTY_DESCRIPTION, + Arrays.asList(values), + true); + } + + /** + * Creates a ConfigOption without a default value. + * + * @return The config option without a default value. + */ + public ConfigOption> noDefaultValue() { + return new ConfigOption<>( + key, + clazz, + ConfigOption.EMPTY_DESCRIPTION, + null, + true); + } + } + + // ------------------------------------------------------------------------ /** Not intended to be instantiated. */ diff --git a/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java b/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java index 186cf10722be..79cc7e4ccb19 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java @@ -25,18 +25,23 @@ import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.types.StringValue; +import org.apache.flink.util.TimeUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +import java.time.Duration; import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Optional; import java.util.Properties; import java.util.Set; +import java.util.stream.Collectors; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -45,7 +50,7 @@ */ @Public public class Configuration extends ExecutionConfig.GlobalJobParameters - implements IOReadableWritable, java.io.Serializable, Cloneable { + implements IOReadableWritable, java.io.Serializable, Cloneable, ReadableConfig, WritableConfig { private static final long serialVersionUID = 1L; @@ -96,18 +101,21 @@ public Configuration(Configuration other) { * @return The value associated with the given key, or the default value, if to entry for the key exists. */ @SuppressWarnings("unchecked") - public Class getClass(String key, Class defaultValue, ClassLoader classLoader) throws ClassNotFoundException { - Object o = getRawValue(key); - if (o == null) { + public Class getClass( + String key, + Class defaultValue, + ClassLoader classLoader) throws ClassNotFoundException { + Optional o = getRawValue(key); + if (!o.isPresent()) { return (Class) defaultValue; } - if (o.getClass() == String.class) { - return (Class) Class.forName((String) o, true, classLoader); + if (o.get().getClass() == String.class) { + return (Class) Class.forName((String) o.get(), true, classLoader); } - LOG.warn("Configuration cannot evaluate value " + o + " as a class name"); - return (Class) defaultValue; + throw new IllegalArgumentException( + "Configuration cannot evaluate object of class " + o.get().getClass() + " as a class name"); } /** @@ -130,14 +138,13 @@ public void setClass(String key, Class klazz) { * @param defaultValue * the default value which is returned in case there is no value associated with the given key * @return the (default) value associated with the given key + * @deprecated use {@link #getString(ConfigOption, String)} or {@link #getOptional(ConfigOption)} */ + @Deprecated public String getString(String key, String defaultValue) { - Object o = getRawValue(key); - if (o == null) { - return defaultValue; - } else { - return o.toString(); - } + return getRawValue(key) + .map(this::convertToString) + .orElse(defaultValue); } /** @@ -148,8 +155,8 @@ public String getString(String key, String defaultValue) { */ @PublicEvolving public String getString(ConfigOption configOption) { - Object o = getValueOrDefaultFromOption(configOption); - return o == null ? null : o.toString(); + return getOptional(configOption) + .orElseGet(configOption::defaultValue); } /** @@ -162,8 +169,8 @@ public String getString(ConfigOption configOption) { */ @PublicEvolving public String getString(ConfigOption configOption, String overrideDefault) { - Object o = getRawValueFromOption(configOption); - return o == null ? overrideDefault : o.toString(); + return getOptional(configOption) + .orElse(overrideDefault); } /** @@ -200,14 +207,13 @@ public void setString(ConfigOption key, String value) { * @param defaultValue * the default value which is returned in case there is no value associated with the given key * @return the (default) value associated with the given key + * @deprecated use {@link #getInteger(ConfigOption, int)} or {@link #getOptional(ConfigOption)} */ + @Deprecated public int getInteger(String key, int defaultValue) { - Object o = getRawValue(key); - if (o == null) { - return defaultValue; - } - - return convertToInt(o, defaultValue); + return getRawValue(key) + .map(this::convertToInt) + .orElse(defaultValue); } /** @@ -218,8 +224,8 @@ public int getInteger(String key, int defaultValue) { */ @PublicEvolving public int getInteger(ConfigOption configOption) { - Object o = getValueOrDefaultFromOption(configOption); - return convertToInt(o, configOption.defaultValue()); + return getOptional(configOption) + .orElseGet(configOption::defaultValue); } /** @@ -233,11 +239,8 @@ public int getInteger(ConfigOption configOption) { */ @PublicEvolving public int getInteger(ConfigOption configOption, int overrideDefault) { - Object o = getRawValueFromOption(configOption); - if (o == null) { - return overrideDefault; - } - return convertToInt(o, configOption.defaultValue()); + return getOptional(configOption) + .orElse(overrideDefault); } /** @@ -274,14 +277,13 @@ public void setInteger(ConfigOption key, int value) { * @param defaultValue * the default value which is returned in case there is no value associated with the given key * @return the (default) value associated with the given key + * @deprecated use {@link #getLong(ConfigOption, long)} or {@link #getOptional(ConfigOption)} */ + @Deprecated public long getLong(String key, long defaultValue) { - Object o = getRawValue(key); - if (o == null) { - return defaultValue; - } - - return convertToLong(o, defaultValue); + return getRawValue(key) + .map(this::convertToLong) + .orElse(defaultValue); } /** @@ -292,8 +294,8 @@ public long getLong(String key, long defaultValue) { */ @PublicEvolving public long getLong(ConfigOption configOption) { - Object o = getValueOrDefaultFromOption(configOption); - return convertToLong(o, configOption.defaultValue()); + return getOptional(configOption) + .orElseGet(configOption::defaultValue); } /** @@ -307,11 +309,8 @@ public long getLong(ConfigOption configOption) { */ @PublicEvolving public long getLong(ConfigOption configOption, long overrideDefault) { - Object o = getRawValueFromOption(configOption); - if (o == null) { - return overrideDefault; - } - return convertToLong(o, configOption.defaultValue()); + return getOptional(configOption) + .orElse(overrideDefault); } /** @@ -348,14 +347,13 @@ public void setLong(ConfigOption key, long value) { * @param defaultValue * the default value which is returned in case there is no value associated with the given key * @return the (default) value associated with the given key + * @deprecated use {@link #getBoolean(ConfigOption, boolean)} or {@link #getOptional(ConfigOption)} */ + @Deprecated public boolean getBoolean(String key, boolean defaultValue) { - Object o = getRawValue(key); - if (o == null) { - return defaultValue; - } - - return convertToBoolean(o); + return getRawValue(key) + .map(this::convertToBoolean) + .orElse(defaultValue); } /** @@ -366,8 +364,8 @@ public boolean getBoolean(String key, boolean defaultValue) { */ @PublicEvolving public boolean getBoolean(ConfigOption configOption) { - Object o = getValueOrDefaultFromOption(configOption); - return convertToBoolean(o); + return getOptional(configOption) + .orElseGet(configOption::defaultValue); } /** @@ -381,11 +379,8 @@ public boolean getBoolean(ConfigOption configOption) { */ @PublicEvolving public boolean getBoolean(ConfigOption configOption, boolean overrideDefault) { - Object o = getRawValueFromOption(configOption); - if (o == null) { - return overrideDefault; - } - return convertToBoolean(o); + return getOptional(configOption) + .orElse(overrideDefault); } /** @@ -422,14 +417,13 @@ public void setBoolean(ConfigOption key, boolean value) { * @param defaultValue * the default value which is returned in case there is no value associated with the given key * @return the (default) value associated with the given key + * @deprecated use {@link #getFloat(ConfigOption, float)} or {@link #getOptional(ConfigOption)} */ + @Deprecated public float getFloat(String key, float defaultValue) { - Object o = getRawValue(key); - if (o == null) { - return defaultValue; - } - - return convertToFloat(o, defaultValue); + return getRawValue(key) + .map(this::convertToFloat) + .orElse(defaultValue); } /** @@ -440,8 +434,8 @@ public float getFloat(String key, float defaultValue) { */ @PublicEvolving public float getFloat(ConfigOption configOption) { - Object o = getValueOrDefaultFromOption(configOption); - return convertToFloat(o, configOption.defaultValue()); + return getOptional(configOption) + .orElseGet(configOption::defaultValue); } /** @@ -455,11 +449,8 @@ public float getFloat(ConfigOption configOption) { */ @PublicEvolving public float getFloat(ConfigOption configOption, float overrideDefault) { - Object o = getRawValueFromOption(configOption); - if (o == null) { - return overrideDefault; - } - return convertToFloat(o, configOption.defaultValue()); + return getOptional(configOption) + .orElse(overrideDefault); } /** @@ -496,14 +487,13 @@ public void setFloat(ConfigOption key, float value) { * @param defaultValue * the default value which is returned in case there is no value associated with the given key * @return the (default) value associated with the given key + * @deprecated use {@link #getDouble(ConfigOption, double)} or {@link #getOptional(ConfigOption)} */ + @Deprecated public double getDouble(String key, double defaultValue) { - Object o = getRawValue(key); - if (o == null) { - return defaultValue; - } - - return convertToDouble(o, defaultValue); + return getRawValue(key) + .map(this::convertToDouble) + .orElse(defaultValue); } /** @@ -514,8 +504,8 @@ public double getDouble(String key, double defaultValue) { */ @PublicEvolving public double getDouble(ConfigOption configOption) { - Object o = getValueOrDefaultFromOption(configOption); - return convertToDouble(o, configOption.defaultValue()); + return getOptional(configOption) + .orElseGet(configOption::defaultValue); } /** @@ -529,11 +519,8 @@ public double getDouble(ConfigOption configOption) { */ @PublicEvolving public double getDouble(ConfigOption configOption, double overrideDefault) { - Object o = getRawValueFromOption(configOption); - if (o == null) { - return overrideDefault; - } - return convertToDouble(o, configOption.defaultValue()); + return getOptional(configOption) + .orElse(overrideDefault); } /** @@ -571,20 +558,17 @@ public void setDouble(ConfigOption key, double value) { * The default value which is returned in case there is no value associated with the given key. * @return the (default) value associated with the given key. */ - @SuppressWarnings("EqualsBetweenInconvertibleTypes") public byte[] getBytes(String key, byte[] defaultValue) { - - Object o = getRawValue(key); - if (o == null) { - return defaultValue; - } - else if (o.getClass().equals(byte[].class)) { - return (byte[]) o; - } - else { - LOG.warn("Configuration cannot evaluate value {} as a byte[] value", o); - return defaultValue; - } + return getRawValue(key).map(o -> { + if (o.getClass().equals(byte[].class)) { + return (byte[]) o; + } else { + throw new IllegalArgumentException(String.format( + "Configuration cannot evaluate value %s as a byte[] value", + o)); + } + } + ).orElse(defaultValue); } /** @@ -607,8 +591,9 @@ public void setBytes(String key, byte[] bytes) { */ @PublicEvolving public String getValue(ConfigOption configOption) { - Object o = getValueOrDefaultFromOption(configOption); - return o == null ? null : o.toString(); + return Optional.ofNullable(getRawValueFromOption(configOption).orElseGet(configOption::defaultValue)) + .map(String::valueOf) + .orElse(null); } /** @@ -626,15 +611,17 @@ public > T getEnum( checkNotNull(enumClass, "enumClass must not be null"); checkNotNull(configOption, "configOption must not be null"); - final String configValue = getString(configOption); + Object rawValue = getRawValueFromOption(configOption) + .orElseGet(configOption::defaultValue); try { - return Enum.valueOf(enumClass, configValue.toUpperCase(Locale.ROOT)); - } catch (final IllegalArgumentException | NullPointerException e) { - final String errorMessage = String.format("Value for config option %s must be one of %s (was %s)", + return convertToEnum(rawValue, enumClass); + } catch (IllegalArgumentException ex) { + final String errorMessage = String.format( + "Value for config option %s must be one of %s (was %s)", configOption.key(), Arrays.toString(enumClass.getEnumConstants()), - configValue); - throw new IllegalArgumentException(errorMessage, e); + rawValue); + throw new IllegalArgumentException(errorMessage); } } @@ -745,6 +732,36 @@ else if (configOption.hasFallbackKeys()) { } } + @Override + public T get(ConfigOption option) { + return getOptional(option).orElseGet(option::defaultValue); + } + + @Override + public Optional getOptional(ConfigOption option) { + Optional rawValue = getRawValueFromOption(option); + Class clazz = option.getClazz(); + + try { + if (option.isList()) { + return rawValue.map(v -> convertToList(v, clazz)); + } else { + return rawValue.map(v -> convertValue(v, clazz)); + } + } catch (Exception e) { + throw new IllegalArgumentException(String.format( + "Could not parse value '%s' for key '%s'.", + rawValue.map(Object::toString).orElse(""), + option.key()), e); + } + } + + @Override + public WritableConfig set(ConfigOption option, T value) { + setValueInternal(option.key(), value); + return this; + } + // -------------------------------------------------------------------------------------------- @Override @@ -799,41 +816,36 @@ void setValueInternal(String key, T value) { } } - private Object getRawValue(String key) { + private Optional getRawValue(String key) { if (key == null) { throw new NullPointerException("Key must not be null."); } synchronized (this.confData) { - return this.confData.get(key); + return Optional.ofNullable(this.confData.get(key)); } } - private Object getRawValueFromOption(ConfigOption configOption) { + private Optional getRawValueFromOption(ConfigOption configOption) { // first try the current key - Object o = getRawValue(configOption.key()); + Optional o = getRawValue(configOption.key()); - if (o != null) { + if (o.isPresent()) { // found a value for the current proper key return o; } else if (configOption.hasFallbackKeys()) { // try the deprecated keys for (FallbackKey fallbackKey : configOption.fallbackKeys()) { - Object oo = getRawValue(fallbackKey.getKey()); - if (oo != null) { + Optional oo = getRawValue(fallbackKey.getKey()); + if (oo.isPresent()) { loggingFallback(fallbackKey, configOption); return oo; } } } - return null; - } - - private Object getValueOrDefaultFromOption(ConfigOption configOption) { - Object o = getRawValueFromOption(configOption); - return o != null ? o : configOption.defaultValue(); + return Optional.empty(); } private void loggingFallback(FallbackKey fallbackKey, ConfigOption configOption) { @@ -850,99 +862,173 @@ private void loggingFallback(FallbackKey fallbackKey, ConfigOption configOpti // Type conversion // -------------------------------------------------------------------------------------------- - private int convertToInt(Object o, int defaultValue) { + @SuppressWarnings("unchecked") + private T convertValue(Object rawValue, Class clazz) { + if (Integer.class.equals(clazz)) { + return (T) convertToInt(rawValue); + } else if (Long.class.equals(clazz)) { + return (T) convertToLong(rawValue); + } else if (Boolean.class.equals(clazz)) { + return (T) convertToBoolean(rawValue); + } else if (Float.class.equals(clazz)) { + return (T) convertToFloat(rawValue); + } else if (Double.class.equals(clazz)) { + return (T) convertToDouble(rawValue); + } else if (String.class.equals(clazz)) { + return (T) convertToString(rawValue); + } else if (clazz.isEnum()) { + return (T) convertToEnum(rawValue, clazz); + } else if (clazz == Duration.class) { + return (T) convertToDuration(rawValue); + } else if (clazz == MemorySize.class) { + return (T) convertToMemorySize(rawValue); + } else if (clazz == Map.class) { + return (T) convertToProperties(rawValue); + } + + throw new IllegalArgumentException("Unsupported type: " + clazz); + } + + @SuppressWarnings("unchecked") + private T convertToList(Object rawValue, Class atomicClass) { + if (rawValue instanceof List) { + return (T) rawValue; + } else { + return (T) StructuredOptionsSplitter.splitEscaped(rawValue.toString(), ';').stream() + .map(s -> convertValue(s, atomicClass)) + .collect(Collectors.toList()); + } + } + + @SuppressWarnings("unchecked") + private Map convertToProperties(Object o) { + if (o instanceof Map) { + return (Map) o; + } else { + List listOfRawProperties = StructuredOptionsSplitter.splitEscaped(o.toString(), ','); + return listOfRawProperties.stream() + .map(s -> StructuredOptionsSplitter.splitEscaped(s, ':')) + .map(pair -> { + if (pair.size() != 2) { + throw new IllegalArgumentException("Could not parse pair in the map " + pair); + } + + return pair; + }) + .collect(Collectors.toMap( + a -> a.get(0), + a -> a.get(1) + )); + } + } + + @SuppressWarnings("unchecked") + private > E convertToEnum(Object o, Class clazz) { + if (o.getClass().equals(clazz)) { + return (E) o; + } + + return Enum.valueOf(clazz, o.toString().toUpperCase(Locale.ROOT)); + } + + private Duration convertToDuration(Object o) { + if (o.getClass() == Duration.class) { + return (Duration) o; + } + + return TimeUtils.parseDuration(o.toString()); + } + + private MemorySize convertToMemorySize(Object o) { + if (o.getClass() == MemorySize.class) { + return (MemorySize) o; + } + + return MemorySize.parse(o.toString()); + } + + private String convertToString(Object o) { + if (o.getClass() == String.class) { + return (String) o; + } else if (o.getClass() == Duration.class) { + Duration duration = (Duration) o; + return String.format("%d ns", duration.toNanos()); + } + + return o.toString(); + } + + private Integer convertToInt(Object o) { if (o.getClass() == Integer.class) { return (Integer) o; - } - else if (o.getClass() == Long.class) { + } else if (o.getClass() == Long.class) { long value = (Long) o; if (value <= Integer.MAX_VALUE && value >= Integer.MIN_VALUE) { return (int) value; } else { - LOG.warn("Configuration value {} overflows/underflows the integer type.", value); - return defaultValue; - } - } - else { - try { - return Integer.parseInt(o.toString()); - } - catch (NumberFormatException e) { - LOG.warn("Configuration cannot evaluate value {} as an integer number", o); - return defaultValue; + throw new IllegalArgumentException(String.format( + "Configuration value %s overflows/underflows the integer type.", + value)); } } + + return Integer.parseInt(o.toString()); } - private long convertToLong(Object o, long defaultValue) { + private Long convertToLong(Object o) { if (o.getClass() == Long.class) { return (Long) o; - } - else if (o.getClass() == Integer.class) { + } else if (o.getClass() == Integer.class) { return ((Integer) o).longValue(); } - else { - try { - return Long.parseLong(o.toString()); - } - catch (NumberFormatException e) { - LOG.warn("Configuration cannot evaluate value " + o + " as a long integer number"); - return defaultValue; - } - } + + return Long.parseLong(o.toString()); } - private boolean convertToBoolean(Object o) { + private Boolean convertToBoolean(Object o) { if (o.getClass() == Boolean.class) { return (Boolean) o; } - else { - return Boolean.parseBoolean(o.toString()); + + switch (o.toString().toUpperCase()) { + case "TRUE": + return true; + case "FALSE": + return false; + default: + throw new IllegalArgumentException(String.format( + "Unrecognized option for boolean: %s. Expected either true or false(case insensitive)", + o)); } } - private float convertToFloat(Object o, float defaultValue) { + private Float convertToFloat(Object o) { if (o.getClass() == Float.class) { return (Float) o; - } - else if (o.getClass() == Double.class) { + } else if (o.getClass() == Double.class) { double value = ((Double) o); if (value == 0.0 || (value >= Float.MIN_VALUE && value <= Float.MAX_VALUE) || (value >= -Float.MAX_VALUE && value <= -Float.MIN_VALUE)) { return (float) value; } else { - LOG.warn("Configuration value {} overflows/underflows the float type.", value); - return defaultValue; - } - } - else { - try { - return Float.parseFloat(o.toString()); - } - catch (NumberFormatException e) { - LOG.warn("Configuration cannot evaluate value {} as a float value", o); - return defaultValue; + throw new IllegalArgumentException(String.format( + "Configuration value %s overflows/underflows the float type.", + value)); } } + + return Float.parseFloat(o.toString()); } - private double convertToDouble(Object o, double defaultValue) { + private Double convertToDouble(Object o) { if (o.getClass() == Double.class) { return (Double) o; - } - else if (o.getClass() == Float.class) { + } else if (o.getClass() == Float.class) { return ((Float) o).doubleValue(); } - else { - try { - return Double.parseDouble(o.toString()); - } - catch (NumberFormatException e) { - LOG.warn("Configuration cannot evaluate value {} as a double value", o); - return defaultValue; - } - } + + return Double.parseDouble(o.toString()); } // -------------------------------------------------------------------------------------------- @@ -984,7 +1070,8 @@ public void read(DataInputView in) throws IOException { value = bytes; break; default: - throw new IOException("Unrecognized type: " + type); + throw new IOException(String.format("Unrecognized type: %s. This method is deprecated and" + + " might not work for all supported types.", type)); } this.confData.put(key, value); @@ -1035,7 +1122,8 @@ else if (clazz == Boolean.class) { out.writeBoolean((Boolean) val); } else { - throw new IllegalArgumentException("Unrecognized type"); + throw new IllegalArgumentException("Unrecognized type. This method is deprecated and might not work" + + " for all supported types."); } } } @@ -1052,7 +1140,6 @@ public int hashCode() { return hash; } - @SuppressWarnings("EqualsBetweenInconvertibleTypes") @Override public boolean equals(Object obj) { if (this == obj) { diff --git a/flink-core/src/main/java/org/apache/flink/configuration/DelegatingConfiguration.java b/flink-core/src/main/java/org/apache/flink/configuration/DelegatingConfiguration.java index b0249a050707..450325b9b943 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/DelegatingConfiguration.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/DelegatingConfiguration.java @@ -29,6 +29,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; import java.util.Set; @@ -332,6 +333,21 @@ public boolean contains(ConfigOption configOption) { return backingConfig.contains(prefixOption(configOption, prefix)); } + @Override + public T get(ConfigOption option) { + return backingConfig.get(prefixOption(option, prefix)); + } + + @Override + public Optional getOptional(ConfigOption option) { + return backingConfig.getOptional(prefixOption(option, prefix)); + } + + @Override + public WritableConfig set(ConfigOption option, T value) { + return backingConfig.set(prefixOption(option, prefix), value); + } + // -------------------------------------------------------------------------------------------- @Override @@ -379,9 +395,12 @@ private static ConfigOption prefixOption(ConfigOption option, String p } FallbackKey[] deprecated = deprecatedKeys.toArray(new FallbackKey[0]); - return new ConfigOption<>(key, + return new ConfigOption( + key, + option.getClazz(), option.description(), option.defaultValue(), + option.isList(), deprecated); } } diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ReadableConfig.java b/flink-core/src/main/java/org/apache/flink/configuration/ReadableConfig.java new file mode 100644 index 000000000000..9b4c3419fcf1 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/configuration/ReadableConfig.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.configuration; + +import org.apache.flink.annotation.PublicEvolving; + +import java.util.Optional; + +/** + * Read access to a configuration object. Allows reading values described with meta information + * included in {@link ConfigOption}. + */ +@PublicEvolving +public interface ReadableConfig { + + /** + * Reads a value using the metada included in {@link ConfigOption}. Returns the + * {@link ConfigOption#defaultValue()} if value key not present in the configuration. + * + * @param option metadata of the option to read + * @param type of the value to read + * @return read value or {@link ConfigOption#defaultValue()} if not found + * @see #getOptional(ConfigOption) + */ + T get(ConfigOption option); + + /** + * Reads a value using the metada included in {@link ConfigOption}. In contrast to + * {@link #get(ConfigOption)} returns {@link Optional#empty()} if value not present. + * + * @param option metadata of the option to read + * @param type of the value to read + * @return read value or {@link Optional#empty()} if not found + * @see #get(ConfigOption) + */ + Optional getOptional(ConfigOption option); +} diff --git a/flink-core/src/main/java/org/apache/flink/configuration/StructuredOptionsSplitter.java b/flink-core/src/main/java/org/apache/flink/configuration/StructuredOptionsSplitter.java new file mode 100644 index 000000000000..44ae82ad6109 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/configuration/StructuredOptionsSplitter.java @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.Internal; + +import java.util.ArrayList; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Helper class for splitting a string on a given delimiter with quoting logic. + */ +@Internal +class StructuredOptionsSplitter { + + /** + * Splits the given string on the given delimiter. It supports quoting parts of the string with + * either single (') or double quotes ("). Quotes can be escaped by doubling the quotes. + * + *

Examples: + *

    + *
  • 'A;B';C => [A;B], [C]
  • + *
  • "AB'D";B;C => [AB'D], [B], [C]
  • + *
  • "AB'""D;B";C => [AB'\"D;B], [C]
  • + *
+ * + *

For more examples check the tests. + * @param string a string to split + * @param delimiter delimiter to split on + * @return a list of splits + */ + static List splitEscaped(String string, char delimiter) { + List tokens = tokenize(checkNotNull(string), delimiter); + return processTokens(tokens); + } + + private static List processTokens(List tokens) { + final List splits = new ArrayList<>(); + for (int i = 0; i < tokens.size(); i++) { + Token token = tokens.get(i); + switch (token.getTokenType()) { + case DOUBLE_QUOTED: + case SINGLE_QUOTED: + if (i + 1 < tokens.size() && tokens.get(i + 1).getTokenType() != TokenType.DELIMITER) { + int illegalPosition = tokens.get(i + 1).getPosition() - 1; + throw new IllegalArgumentException( + "Could not split string. Illegal quoting at position: " + illegalPosition); + } + splits.add(token.getString()); + break; + case UNQUOTED: + splits.add(token.getString()); + break; + case DELIMITER: + if (i + 1 < tokens.size() && tokens.get(i + 1).getTokenType() == TokenType.DELIMITER) { + splits.add(""); + } + break; + } + } + + return splits; + } + + private static List tokenize(String string, char delimiter) { + final List tokens = new ArrayList<>(); + final StringBuilder builder = new StringBuilder(); + for (int cursor = 0; cursor < string.length(); ) { + final char c = string.charAt(cursor); + + int nextChar = cursor + 1; + if (c == '\'') { + nextChar = consumeInQuotes(string, '\'', cursor, builder); + tokens.add(new Token(TokenType.SINGLE_QUOTED, builder.toString(), cursor)); + } else if (c == '"') { + nextChar = consumeInQuotes(string, '"', cursor, builder); + tokens.add(new Token(TokenType.DOUBLE_QUOTED, builder.toString(), cursor)); + } else if (c == delimiter) { + tokens.add(new Token(TokenType.DELIMITER, String.valueOf(c), cursor)); + } else if (!Character.isWhitespace(c)) { + nextChar = consumeUnquoted(string, delimiter, cursor, builder); + tokens.add(new Token(TokenType.UNQUOTED, builder.toString().trim(), cursor)); + } + builder.setLength(0); + cursor = nextChar; + } + + return tokens; + } + + private static int consumeInQuotes(String string, char quote, int cursor, StringBuilder builder) { + for (int i = cursor + 1; i < string.length(); i++) { + char c = string.charAt(i); + if (c == quote) { + if (i + 1 < string.length() && string.charAt(i + 1) == quote) { + builder.append(c); + i += 1; + } else { + return i + 1; + } + } else { + builder.append(c); + } + } + + throw new IllegalArgumentException("Could not split string. Quoting was not closed properly."); + } + + private static int consumeUnquoted(String string, char delimiter, int cursor, StringBuilder builder) { + int i; + for (i = cursor; i < string.length(); i++) { + char c = string.charAt(i); + if (c == delimiter) { + return i; + } else if (c == '\'' || c == '"') { + throw new IllegalArgumentException("Could not split string. Illegal quoting at position: " + i); + } + + builder.append(c); + } + + return i; + } + + private enum TokenType { + DOUBLE_QUOTED, + SINGLE_QUOTED, + UNQUOTED, + DELIMITER + } + + private static class Token { + private final TokenType tokenType; + private final String string; + private final int position; + + private Token(TokenType tokenType, String string, int position) { + this.tokenType = tokenType; + this.string = string; + this.position = position; + } + + public TokenType getTokenType() { + return tokenType; + } + + public String getString() { + return string; + } + + public int getPosition() { + return position; + } + } + + private StructuredOptionsSplitter() { + } +} diff --git a/flink-core/src/main/java/org/apache/flink/configuration/WritableConfig.java b/flink-core/src/main/java/org/apache/flink/configuration/WritableConfig.java new file mode 100644 index 000000000000..594f5b77e9e9 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/configuration/WritableConfig.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.configuration; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * Write access to a configuration object. Allows storing values described with meta information + * included in {@link ConfigOption}. + */ +@PublicEvolving +public interface WritableConfig { + + /** + * Stores a given value using the metadata included in the {@link ConfigOption}. + * The value should be readable back through {@link ReadableConfig}. + * + * @param option metadata information + * @param value value to be stored + * @param type of the value to be stored + * @return instance of this configuration for fluent API + */ + WritableConfig set(ConfigOption option, T value); +} diff --git a/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationConversionsTest.java b/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationConversionsTest.java new file mode 100644 index 000000000000..4dc377f5c2c7 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationConversionsTest.java @@ -0,0 +1,378 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.hamcrest.Description; +import org.hamcrest.Matcher; +import org.hamcrest.TypeSafeMatcher; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import javax.annotation.Nullable; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Optional; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.startsWith; +import static org.hamcrest.Matchers.closeTo; +import static org.junit.Assert.assertThat; + +/** + * Tests for {@link Configuration} conversion between types. Extracted from {@link ConfigurationTest}. + */ +@RunWith(Parameterized.class) +public class ConfigurationConversionsTest { + + private static final byte[] EMPTY_BYTES = new byte[0]; + private static final long TOO_LONG = Integer.MAX_VALUE + 10L; + private static final double TOO_LONG_DOUBLE = Double.MAX_VALUE; + + private Configuration pc; + + @Before + public void init() { + pc = new Configuration(); + + pc.setInteger("int", 5); + pc.setLong("long", 15); + pc.setLong("too_long", TOO_LONG); + pc.setFloat("float", 2.1456775f); + pc.setDouble("double", Math.PI); + pc.setDouble("negative_double", -1.0); + pc.setDouble("zero", 0.0); + pc.setDouble("too_long_double", TOO_LONG_DOUBLE); + pc.setString("string", "42"); + pc.setString("non_convertible_string", "bcdefg&&"); + pc.setBoolean("boolean", true); + } + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Parameterized.Parameters + public static Collection getSpecs() { + return Arrays.asList( + // from integer + TestSpec.whenAccessed(conf -> conf.getInteger("int", 0)).expect(5), + TestSpec.whenAccessed(conf -> conf.getLong("int", 0)).expect(5L), + TestSpec.whenAccessed(conf -> conf.getFloat("int", 0)).expect(5f), + TestSpec.whenAccessed(conf -> conf.getDouble("int", 0)).expect(5.0), + TestSpec.whenAccessed(conf -> conf.getBoolean("int", true)) + .expectException("Unrecognized option for boolean: 5. Expected either true or false(case insensitive)"), + TestSpec.whenAccessed(conf -> conf.getString("int", "0")).expect("5"), + TestSpec.whenAccessed(conf -> conf.getBytes("int", EMPTY_BYTES)) + .expectException("Configuration cannot evaluate value 5 as a byte[] value"), + TestSpec.whenAccessed(conf -> conf.getClass( + "int", + ConfigurationConversionsTest.class, + ConfigurationConversionsTest.class.getClassLoader())) + .expectException("Configuration cannot evaluate object of class class java.lang.Integer as a class name"), + + // from long + TestSpec.whenAccessed(conf -> conf.getInteger("long", 0)).expect(15), + TestSpec.whenAccessed(conf -> conf.getLong("long", 0)).expect(15L), + TestSpec.whenAccessed(conf -> conf.getFloat("long", 0)).expect(15f), + TestSpec.whenAccessed(conf -> conf.getDouble("long", 0)).expect(15.0), + TestSpec.whenAccessed(conf -> conf.getBoolean("long", true)) + .expectException("Unrecognized option for boolean: 15. Expected either true or false(case insensitive)"), + TestSpec.whenAccessed(conf -> conf.getString("long", "0")).expect("15"), + TestSpec.whenAccessed(conf -> conf.getBytes("long", EMPTY_BYTES)) + .expectException("Configuration cannot evaluate value 15 as a byte[] value"), + TestSpec.whenAccessed(conf -> conf.getClass( + "long", + ConfigurationConversionsTest.class, + ConfigurationConversionsTest.class.getClassLoader())) + .expectException("Configuration cannot evaluate object of class class java.lang.Long as a class name"), + + // from too long + TestSpec.whenAccessed(conf -> conf.getInteger("too_long", 0)) + .expectException("Configuration value 2147483657 overflows/underflows the integer type"), + TestSpec.whenAccessed(conf -> conf.getLong("too_long", 0)).expect(TOO_LONG), + TestSpec.whenAccessed(conf -> conf.getFloat("too_long", 0)).expect((float) TOO_LONG), + TestSpec.whenAccessed(conf -> conf.getDouble("too_long", 0)).expect((double) TOO_LONG), + TestSpec.whenAccessed(conf -> conf.getBoolean("too_long", true)) + .expectException( + "Unrecognized option for boolean: 2147483657. Expected either true or false(case insensitive)"), + TestSpec.whenAccessed(conf -> conf.getString("too_long", "0")).expect(String.valueOf(TOO_LONG)), + TestSpec.whenAccessed(conf -> conf.getBytes("too_long", EMPTY_BYTES)) + .expectException("Configuration cannot evaluate value 2147483657 as a byte[] value"), + TestSpec.whenAccessed(conf -> conf.getClass( + "too_long", + ConfigurationConversionsTest.class, + ConfigurationConversionsTest.class.getClassLoader())) + .expectException("Configuration cannot evaluate object of class class java.lang.Long as a class name"), + + // from float + TestSpec.whenAccessed(conf -> conf.getInteger("float", 0)) + .expectException("For input string: \"2.1456776\"", NumberFormatException.class), + TestSpec.whenAccessed(conf -> conf.getLong("float", 0)) + .expectException("For input string: \"2.1456776\"", NumberFormatException.class), + TestSpec.whenAccessed(conf -> conf.getFloat("float", 0)) + .expect(2.1456775f), + TestSpec.whenAccessed(conf -> conf.getDouble("float", 0)).expect(closeTo(2.1456775, 0.0000001)), + TestSpec.whenAccessed(conf -> conf.getBoolean("float", true)) + .expectException( + "Unrecognized option for boolean: 2.1456776. Expected either true or false(case insensitive)"), + TestSpec.whenAccessed(conf -> conf.getString("float", "0")).expect(startsWith("2.145677")), + TestSpec.whenAccessed(conf -> conf.getBytes("float", EMPTY_BYTES)) + .expectException("Configuration cannot evaluate value 2.1456776 as a byte[] value"), + TestSpec.whenAccessed(conf -> conf.getClass( + "float", + ConfigurationConversionsTest.class, + ConfigurationConversionsTest.class.getClassLoader())) + .expectException("onfiguration cannot evaluate object of class class java.lang.Float as a class name"), + + // from double + TestSpec.whenAccessed(conf -> conf.getInteger("double", 0)) + .expectException("For input string: \"3.141592653589793\"", NumberFormatException.class), + TestSpec.whenAccessed(conf -> conf.getLong("double", 0)) + .expectException("For input string: \"3.141592653589793\"", NumberFormatException.class), + TestSpec.whenAccessed(conf -> conf.getFloat("double", 0)).expect(new IsCloseTo(3.141592f, 0.000001f)), + TestSpec.whenAccessed(conf -> conf.getDouble("double", 0)).expect(Math.PI), + TestSpec.whenAccessed(conf -> conf.getBoolean("double", true)) + .expectException( + "Unrecognized option for boolean: 3.141592653589793. Expected either true or false(case insensitive)"), + TestSpec.whenAccessed(conf -> conf.getString("double", "0")).expect(startsWith("3.1415926535")), + TestSpec.whenAccessed(conf -> conf.getBytes("double", EMPTY_BYTES)) + .expectException("Configuration cannot evaluate value 3.141592653589793 as a byte[] value"), + TestSpec.whenAccessed(conf -> conf.getClass( + "double", + ConfigurationConversionsTest.class, + ConfigurationConversionsTest.class.getClassLoader())) + .expectException("onfiguration cannot evaluate object of class class java.lang.Double as a class name"), + + // from negative double + TestSpec.whenAccessed(conf -> conf.getInteger("negative_double", 0)) + .expectException("For input string: \"-1.0\"", NumberFormatException.class), + TestSpec.whenAccessed(conf -> conf.getLong("negative_double", 0)) + .expectException("For input string: \"-1.0\"", NumberFormatException.class), + TestSpec.whenAccessed(conf -> conf.getFloat("negative_double", 0)) + .expect(new IsCloseTo(-1f, 0.000001f)), + TestSpec.whenAccessed(conf -> conf.getDouble("negative_double", 0)).expect(-1D), + TestSpec.whenAccessed(conf -> conf.getBoolean("negative_double", true)) + .expectException("Unrecognized option for boolean: -1.0. Expected either true or false(case insensitive)"), + TestSpec.whenAccessed(conf -> conf.getString("negative_double", "0")).expect(startsWith("-1")), + TestSpec.whenAccessed(conf -> conf.getBytes("negative_double", EMPTY_BYTES)) + .expectException("Configuration cannot evaluate value -1.0 as a byte[] value"), + TestSpec.whenAccessed(conf -> conf.getClass( + "negative_double", + ConfigurationConversionsTest.class, + ConfigurationConversionsTest.class.getClassLoader())) + .expectException("Configuration cannot evaluate object of class class java.lang.Double as a class name"), + + // from zero + TestSpec.whenAccessed(conf -> conf.getInteger("zero", 0)) + .expectException("For input string: \"0.0\"", NumberFormatException.class), + TestSpec.whenAccessed(conf -> conf.getLong("zero", 0)) + .expectException("For input string: \"0.0\"", NumberFormatException.class), + TestSpec.whenAccessed(conf -> conf.getFloat("zero", 0)).expect(new IsCloseTo(0f, 0.000001f)), + TestSpec.whenAccessed(conf -> conf.getDouble("zero", 0)).expect(0D), + TestSpec.whenAccessed(conf -> conf.getBoolean("zero", true)) + .expectException("Unrecognized option for boolean: 0.0. Expected either true or false(case insensitive)"), + TestSpec.whenAccessed(conf -> conf.getString("zero", "0")).expect(startsWith("0")), + TestSpec.whenAccessed(conf -> conf.getBytes("zero", EMPTY_BYTES)) + .expectException("Configuration cannot evaluate value 0.0 as a byte[] value"), + TestSpec.whenAccessed(conf -> conf.getClass( + "zero", + ConfigurationConversionsTest.class, + ConfigurationConversionsTest.class.getClassLoader())) + .expectException("Configuration cannot evaluate object of class class java.lang.Double as a class name"), + + // from too long double + TestSpec.whenAccessed(conf -> conf.getInteger("too_long_double", 0)) + .expectException("For input string: \"1.7976931348623157E308\"", NumberFormatException.class), + TestSpec.whenAccessed(conf -> conf.getLong("too_long_double", 0)) + .expectException("For input string: \"1.7976931348623157E308\"", NumberFormatException.class), + TestSpec.whenAccessed(conf -> conf.getFloat("too_long_double", 0)) + .expectException("Configuration value 1.7976931348623157E308 overflows/underflows the float type."), + TestSpec.whenAccessed(conf -> conf.getDouble("too_long_double", 0)).expect(TOO_LONG_DOUBLE), + TestSpec.whenAccessed(conf -> conf.getBoolean("too_long_double", true)) + .expectException("Unrecognized option for boolean: 1.7976931348623157E308. Expected either true or false(case insensitive)"), + TestSpec.whenAccessed(conf -> conf.getString("too_long_double", "0")) + .expect(String.valueOf(TOO_LONG_DOUBLE)), + TestSpec.whenAccessed(conf -> conf.getBytes("too_long_double", EMPTY_BYTES)) + .expectException("Configuration cannot evaluate value 1.7976931348623157E308 as a byte[] value"), + TestSpec.whenAccessed(conf -> conf.getClass( + "too_long_double", + ConfigurationConversionsTest.class, + ConfigurationConversionsTest.class.getClassLoader())) + .expectException("Configuration cannot evaluate object of class class java.lang.Double as a class name"), + + // from string + TestSpec.whenAccessed(conf -> conf.getInteger("string", 0)).expect(42), + TestSpec.whenAccessed(conf -> conf.getLong("string", 0)).expect(42L), + TestSpec.whenAccessed(conf -> conf.getFloat("string", 0)).expect(42f), + TestSpec.whenAccessed(conf -> conf.getDouble("string", 0)).expect(42.0), + TestSpec.whenAccessed(conf -> conf.getBoolean("string", true)) + .expectException("Unrecognized option for boolean: 42. Expected either true or false(case insensitive)"), + TestSpec.whenAccessed(conf -> conf.getString("string", "0")).expect("42"), + TestSpec.whenAccessed(conf -> conf.getBytes("string", EMPTY_BYTES)) + .expectException("Configuration cannot evaluate value 42 as a byte[] value"), + TestSpec.whenAccessed(conf -> conf.getClass( + "string", + ConfigurationConversionsTest.class, + ConfigurationConversionsTest.class.getClassLoader())) + .expectException("42", ClassNotFoundException.class), + + // from non convertible string + TestSpec.whenAccessed(conf -> conf.getInteger("non_convertible_string", 0)) + .expectException("For input string: \"bcdefg&&\"", NumberFormatException.class), + TestSpec.whenAccessed(conf -> conf.getLong("non_convertible_string", 0)) + .expectException("For input string: \"bcdefg&&\"", NumberFormatException.class), + TestSpec.whenAccessed(conf -> conf.getFloat("non_convertible_string", 0)) + .expectException("For input string: \"bcdefg&&\"", NumberFormatException.class), + TestSpec.whenAccessed(conf -> conf.getDouble("non_convertible_string", 0)) + .expectException("For input string: \"bcdefg&&\"", NumberFormatException.class), + TestSpec.whenAccessed(conf -> conf.getBoolean("non_convertible_string", true)) + .expectException("Unrecognized option for boolean: bcdefg&&. Expected either true or false(case insensitive)"), + TestSpec.whenAccessed(conf -> conf.getString("non_convertible_string", "0")).expect("bcdefg&&"), + TestSpec.whenAccessed(conf -> conf.getBytes("non_convertible_string", EMPTY_BYTES)) + .expectException("Configuration cannot evaluate value bcdefg&& as a byte[] value"), + TestSpec.whenAccessed(conf -> conf.getClass( + "non_convertible_string", + ConfigurationConversionsTest.class, + ConfigurationConversionsTest.class.getClassLoader())) + .expectException("bcdefg&&", ClassNotFoundException.class), + + // from boolean + TestSpec.whenAccessed(conf -> conf.getInteger("boolean", 0)) + .expectException("For input string: \"true\""), + TestSpec.whenAccessed(conf -> conf.getLong("boolean", 0)) + .expectException("For input string: \"true\""), + TestSpec.whenAccessed(conf -> conf.getFloat("boolean", 0)) + .expectException("For input string: \"true\""), + TestSpec.whenAccessed(conf -> conf.getDouble("boolean", 0)) + .expectException("For input string: \"true\""), + TestSpec.whenAccessed(conf -> conf.getBoolean("boolean", false)).expect(true), + TestSpec.whenAccessed(conf -> conf.getString("boolean", "0")).expect("true"), + TestSpec.whenAccessed(conf -> conf.getBytes("boolean", EMPTY_BYTES)) + .expectException("Configuration cannot evaluate value true as a byte[] value"), + TestSpec.whenAccessed(conf -> conf.getClass( + "boolean", + ConfigurationConversionsTest.class, + ConfigurationConversionsTest.class.getClassLoader())) + .expectException("Configuration cannot evaluate object of class class java.lang.Boolean as a class name") + ); + } + + @Parameterized.Parameter + public TestSpec testSpec; + + @Test + public void testConversions() throws Exception { + testSpec.getExpectedException().ifPresent(exception -> { + thrown.expect(testSpec.getExceptionClass()); + thrown.expectMessage(exception); + } + ); + + // workaround for type erasure + testSpec.assertConfiguration(pc); + } + + private static class IsCloseTo extends TypeSafeMatcher { + private final float delta; + private final float value; + + public IsCloseTo(float value, float error) { + this.delta = error; + this.value = value; + } + + public boolean matchesSafely(Float item) { + return this.actualDelta(item) <= 0.0D; + } + + public void describeMismatchSafely(Float item, Description mismatchDescription) { + mismatchDescription.appendValue(item).appendText(" differed by ").appendValue(this.actualDelta(item)); + } + + public void describeTo(Description description) { + description.appendText("a numeric value within ") + .appendValue(this.delta) + .appendText(" of ") + .appendValue(this.value); + } + + private double actualDelta(Float item) { + return Math.abs(item - this.value) - this.delta; + } + } + + private static class TestSpec { + private final ConfigurationAccessor configurationAccessor; + private Matcher matcher; + @Nullable private String expectedException = null; + @Nullable private Class exceptionClass; + + @FunctionalInterface + private interface ConfigurationAccessor { + T access(Configuration configuration) throws Exception; + } + + private TestSpec(ConfigurationAccessor configurationAccessor) { + this.configurationAccessor = configurationAccessor; + } + + public static TestSpec whenAccessed(ConfigurationAccessor configurationAccessor) { + return new TestSpec(configurationAccessor); + } + + public TestSpec expect(Matcher expected) { + this.matcher = expected; + return this; + } + + public TestSpec expect(T expected) { + this.matcher = equalTo(expected); + return this; + } + + public TestSpec expectException(String message) { + this.expectedException = message; + this.exceptionClass = IllegalArgumentException.class; + return this; + } + + public TestSpec expectException(String message, Class exceptionClass) { + this.expectedException = message; + this.exceptionClass = exceptionClass; + return this; + } + + public Optional getExpectedException() { + return Optional.ofNullable(expectedException); + } + + @Nullable + public Class getExceptionClass() { + return exceptionClass; + } + + void assertConfiguration(Configuration conf) throws Exception { + assertThat(configurationAccessor.access(conf), matcher); + } + } +} diff --git a/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationParsingInvalidFormatsTest.java b/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationParsingInvalidFormatsTest.java new file mode 100644 index 000000000000..dc96e44f5206 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationParsingInvalidFormatsTest.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.configuration; + +import org.apache.flink.util.TestLogger; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.time.Duration; +import java.util.Collections; + +/** + * Tests for reading configuration parameters with invalid formats. + */ +@RunWith(Parameterized.class) +public class ConfigurationParsingInvalidFormatsTest extends TestLogger { + @Parameterized.Parameters(name = "option: {0}, invalidString: {1}") + public static Object[][] getSpecs() { + return new Object[][]{ + new Object[]{ConfigOptions.key("int").intType().defaultValue(1), "ABC"}, + new Object[]{ConfigOptions.key("long").longType().defaultValue(1L), "ABC"}, + new Object[]{ConfigOptions.key("float").floatType().defaultValue(1F), "ABC"}, + new Object[]{ConfigOptions.key("double").doubleType().defaultValue(1D), "ABC"}, + new Object[]{ConfigOptions.key("boolean").booleanType().defaultValue(true), "ABC"}, + new Object[]{ConfigOptions.key("memory").memoryType().defaultValue(MemorySize.parse("1kB")), "ABC"}, + new Object[]{ConfigOptions.key("duration").durationType().defaultValue(Duration.ofSeconds(1)), "ABC"}, + new Object[]{ConfigOptions.key("enum").enumType(TestEnum.class).defaultValue(TestEnum.ENUM1), "ABC"}, + new Object[]{ConfigOptions.key("map").mapType().defaultValue(Collections.emptyMap()), "ABC"}, + new Object[]{ConfigOptions.key("list").intType().asList().defaultValues(1, 2), "A;B;C"}, + new Object[]{ConfigOptions.key("list").stringType().asList().defaultValues("A"), "'A;B;C"} + }; + } + + @Parameterized.Parameter + public ConfigOption option; + + @Parameterized.Parameter(value = 1) + public String invalidString; + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Test + public void testInvalidStringParsingWithGetOptional() { + Configuration configuration = new Configuration(); + configuration.setString(option.key(), invalidString); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage(String.format("Could not parse value '%s' for key '%s'", invalidString, option.key())); + configuration.getOptional(option); + } + + @Test + public void testInvalidStringParsingWithGet() { + Configuration configuration = new Configuration(); + configuration.setString(option.key(), invalidString); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage(String.format("Could not parse value '%s' for key '%s'", invalidString, option.key())); + configuration.get(option); + } + + private enum TestEnum { + ENUM1, + ENUM2 + } +} diff --git a/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java b/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java index 9727b44ee16a..0b1e61c83bc9 100644 --- a/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java +++ b/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java @@ -38,10 +38,6 @@ */ public class ConfigurationTest extends TestLogger { - private static final byte[] EMPTY_BYTES = new byte[0]; - private static final long TOO_LONG = Integer.MAX_VALUE + 10L; - private static final double TOO_LONG_DOUBLE = Double.MAX_VALUE; - /** * This test checks the serialization/deserialization of configuration objects. */ @@ -78,128 +74,6 @@ public void testConfigurationSerializationAndGetters() { } } - @Test - public void testConversions() { - try { - Configuration pc = new Configuration(); - - pc.setInteger("int", 5); - pc.setLong("long", 15); - pc.setLong("too_long", TOO_LONG); - pc.setFloat("float", 2.1456775f); - pc.setDouble("double", Math.PI); - pc.setDouble("negative_double", -1.0); - pc.setDouble("zero", 0.0); - pc.setDouble("too_long_double", TOO_LONG_DOUBLE); - pc.setString("string", "42"); - pc.setString("non_convertible_string", "bcdefg&&"); - pc.setBoolean("boolean", true); - - // as integer - assertEquals(5, pc.getInteger("int", 0)); - assertEquals(5L, pc.getLong("int", 0)); - assertEquals(5f, pc.getFloat("int", 0), 0.0); - assertEquals(5.0, pc.getDouble("int", 0), 0.0); - assertEquals(false, pc.getBoolean("int", true)); - assertEquals("5", pc.getString("int", "0")); - assertArrayEquals(EMPTY_BYTES, pc.getBytes("int", EMPTY_BYTES)); - - // as long - assertEquals(15, pc.getInteger("long", 0)); - assertEquals(15L, pc.getLong("long", 0)); - assertEquals(15f, pc.getFloat("long", 0), 0.0); - assertEquals(15.0, pc.getDouble("long", 0), 0.0); - assertEquals(false, pc.getBoolean("long", true)); - assertEquals("15", pc.getString("long", "0")); - assertArrayEquals(EMPTY_BYTES, pc.getBytes("long", EMPTY_BYTES)); - - // as too long - assertEquals(0, pc.getInteger("too_long", 0)); - assertEquals(TOO_LONG, pc.getLong("too_long", 0)); - assertEquals((float) TOO_LONG, pc.getFloat("too_long", 0), 10.0); - assertEquals((double) TOO_LONG, pc.getDouble("too_long", 0), 10.0); - assertEquals(false, pc.getBoolean("too_long", true)); - assertEquals(String.valueOf(TOO_LONG), pc.getString("too_long", "0")); - assertArrayEquals(EMPTY_BYTES, pc.getBytes("too_long", EMPTY_BYTES)); - - // as float - assertEquals(0, pc.getInteger("float", 0)); - assertEquals(0L, pc.getLong("float", 0)); - assertEquals(2.1456775f, pc.getFloat("float", 0), 0.0); - assertEquals(2.1456775, pc.getDouble("float", 0), 0.0000001); - assertEquals(false, pc.getBoolean("float", true)); - assertTrue(pc.getString("float", "0").startsWith("2.145677")); - assertArrayEquals(EMPTY_BYTES, pc.getBytes("float", EMPTY_BYTES)); - - // as double - assertEquals(0, pc.getInteger("double", 0)); - assertEquals(0L, pc.getLong("double", 0)); - assertEquals(3.141592f, pc.getFloat("double", 0), 0.000001); - assertEquals(Math.PI, pc.getDouble("double", 0), 0.0); - assertEquals(false, pc.getBoolean("double", true)); - assertTrue(pc.getString("double", "0").startsWith("3.1415926535")); - assertArrayEquals(EMPTY_BYTES, pc.getBytes("double", EMPTY_BYTES)); - - // as negative double - assertEquals(0, pc.getInteger("negative_double", 0)); - assertEquals(0L, pc.getLong("negative_double", 0)); - assertEquals(-1f, pc.getFloat("negative_double", 0), 0.000001); - assertEquals(-1, pc.getDouble("negative_double", 0), 0.0); - assertEquals(false, pc.getBoolean("negative_double", true)); - assertTrue(pc.getString("negative_double", "0").startsWith("-1")); - assertArrayEquals(EMPTY_BYTES, pc.getBytes("negative_double", EMPTY_BYTES)); - - // as zero - assertEquals(-1, pc.getInteger("zero", -1)); - assertEquals(-1L, pc.getLong("zero", -1)); - assertEquals(0f, pc.getFloat("zero", -1), 0.000001); - assertEquals(0.0, pc.getDouble("zero", -1), 0.0); - assertEquals(false, pc.getBoolean("zero", true)); - assertTrue(pc.getString("zero", "-1").startsWith("0")); - assertArrayEquals(EMPTY_BYTES, pc.getBytes("zero", EMPTY_BYTES)); - - // as too long double - assertEquals(0, pc.getInteger("too_long_double", 0)); - assertEquals(0L, pc.getLong("too_long_double", 0)); - assertEquals(0f, pc.getFloat("too_long_double", 0f), 0.000001); - assertEquals(TOO_LONG_DOUBLE, pc.getDouble("too_long_double", 0), 0.0); - assertEquals(false, pc.getBoolean("too_long_double", true)); - assertEquals(String.valueOf(TOO_LONG_DOUBLE), pc.getString("too_long_double", "0")); - assertArrayEquals(EMPTY_BYTES, pc.getBytes("too_long_double", EMPTY_BYTES)); - - // as string - assertEquals(42, pc.getInteger("string", 0)); - assertEquals(42L, pc.getLong("string", 0)); - assertEquals(42f, pc.getFloat("string", 0f), 0.000001); - assertEquals(42.0, pc.getDouble("string", 0), 0.0); - assertEquals(false, pc.getBoolean("string", true)); - assertEquals("42", pc.getString("string", "0")); - assertArrayEquals(EMPTY_BYTES, pc.getBytes("string", EMPTY_BYTES)); - - // as non convertible string - assertEquals(0, pc.getInteger("non_convertible_string", 0)); - assertEquals(0L, pc.getLong("non_convertible_string", 0)); - assertEquals(0f, pc.getFloat("non_convertible_string", 0f), 0.000001); - assertEquals(0.0, pc.getDouble("non_convertible_string", 0), 0.0); - assertEquals(false, pc.getBoolean("non_convertible_string", true)); - assertEquals("bcdefg&&", pc.getString("non_convertible_string", "0")); - assertArrayEquals(EMPTY_BYTES, pc.getBytes("non_convertible_string", EMPTY_BYTES)); - - // as boolean - assertEquals(0, pc.getInteger("boolean", 0)); - assertEquals(0L, pc.getLong("boolean", 0)); - assertEquals(0f, pc.getFloat("boolean", 0f), 0.000001); - assertEquals(0.0, pc.getDouble("boolean", 0), 0.0); - assertEquals(true, pc.getBoolean("boolean", false)); - assertEquals("true", pc.getString("boolean", "0")); - assertArrayEquals(EMPTY_BYTES, pc.getBytes("boolean", EMPTY_BYTES)); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - @Test public void testCopyConstructor() { try { diff --git a/flink-core/src/test/java/org/apache/flink/configuration/ReadableWritableConfigurationTest.java b/flink-core/src/test/java/org/apache/flink/configuration/ReadableWritableConfigurationTest.java new file mode 100644 index 000000000000..9f3e6db90d08 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/configuration/ReadableWritableConfigurationTest.java @@ -0,0 +1,265 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.java.tuple.Tuple2; +import org.apache.flink.util.Preconditions; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.time.Duration; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.stream.Collectors; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.junit.Assert.assertThat; + +/** + * Tests read access ({@link ReadableConfig}) to {@link Configuration}. + * There are 4 different test scenarios: + * + *

    + *
  1. Tests reading an object that is kept as an object (when set directly through + * {@link Configuration#set(ConfigOption, Object)}.
  2. + *
  3. Tests reading an object that was read from a config file, thus is stored as a string.
  4. + *
  5. Tests using the {@link ConfigOption#defaultValue()} if no key is present in the {@link Configuration}.
  6. + *
  7. Tests that the {@link ConfigOption#defaultValue()} is not used when calling + * {@link ReadableConfig#getOptional(ConfigOption)}.
  8. + *
+ */ +@RunWith(Parameterized.class) +public class ReadableWritableConfigurationTest { + @Parameterized.Parameters(name = "{0}") + public static Collection> getSpecs() { + return Arrays.asList( + + new TestSpec<>( + ConfigOptions.key("int") + .intType() + .defaultValue(-1)) + .valueEquals(12345, "12345") + .checkDefaultOverride(5), + + new TestSpec<>( + ConfigOptions.key("long") + .longType() + .defaultValue(-1L)) + .valueEquals(12345L, "12345") + .checkDefaultOverride(5L), + + new TestSpec<>( + ConfigOptions.key("float") + .floatType() + .defaultValue(0.01F)) + .valueEquals(0.003F, "0.003") + .checkDefaultOverride(1.23F), + + new TestSpec<>( + ConfigOptions.key("double") + .doubleType() + .defaultValue(0.01D)) + .valueEquals(0.003D, "0.003") + .checkDefaultOverride(1.23D), + + new TestSpec<>( + ConfigOptions.key("boolean") + .booleanType() + .defaultValue(false)) + .valueEquals(true, "true") + .checkDefaultOverride(true), + + new TestSpec<>( + ConfigOptions.key("list") + .intType() + .asList() + .defaultValues(-1, 2, 3)) + .valueEquals(Arrays.asList(1, 2, 3, 4, 5), "1;2;3;4;5") + .checkDefaultOverride(Arrays.asList(1, 2)), + + new TestSpec<>( + ConfigOptions.key("list") + .stringType() + .asList() + .defaultValues("A", "B", "C")) + .valueEquals(Arrays.asList("A;B", "C"), "'A;B';C") + .checkDefaultOverride(Collections.singletonList("C")), + + new TestSpec<>( + ConfigOptions.key("interval") + .durationType() + .defaultValue(Duration.ofHours(3))) + .valueEquals(Duration.ofMinutes(3), "3 min") + .checkDefaultOverride(Duration.ofSeconds(1)), + + new TestSpec<>( + ConfigOptions.key("memory") + .memoryType() + .defaultValue(new MemorySize(1024))) + .valueEquals(new MemorySize(1024 * 1024 * 1024), "1g") + .checkDefaultOverride(new MemorySize(2048)), + + new TestSpec<>( + ConfigOptions.key("properties") + .mapType() + .defaultValue(asMap(Collections.singletonList( + Tuple2.of("prop1", "value1") + )))) + .valueEquals(asMap(Arrays.asList( + Tuple2.of("key1", "value1"), + Tuple2.of("key2", "value2") + )), "key1:value1,key2:value2") + .checkDefaultOverride(Collections.emptyMap()), + + new TestSpec<>( + ConfigOptions.key("list") + .mapType() + .asList() + .defaultValues(asMap(Collections.singletonList( + Tuple2.of("prop1", "value1") + )))) + .valueEquals( + Arrays.asList( + asMap(Arrays.asList( + Tuple2.of("key1", "value1"), + Tuple2.of("key2", "value2")) + ), + asMap(Arrays.asList( + Tuple2.of("key3", "value3") + ))), + "key1:value1,key2:value2;key3:value3") + .checkDefaultOverride(Collections.emptyList()) + ); + } + + private static Map asMap(List> entries) { + return entries.stream().collect(Collectors.toMap( + t -> t.f0, + t -> t.f1 + )); + } + + @Parameterized.Parameter + public TestSpec testSpec; + + @Test + public void testGetOptionalFromObject() { + Configuration configuration = new Configuration(); + testSpec.setValue(configuration); + + Optional optional = configuration.getOptional(testSpec.getOption()); + assertThat(optional.get(), equalTo(testSpec.getValue())); + } + + @Test + public void testGetOptionalFromString() { + ConfigOption option = testSpec.getOption(); + Configuration configuration = new Configuration(); + configuration.setString(option.key(), testSpec.getStringValue()); + + Optional optional = configuration.getOptional(option); + assertThat(optional.get(), equalTo(testSpec.getValue())); + } + + @Test + public void testGetDefaultValue() { + Configuration configuration = new Configuration(); + + ConfigOption option = testSpec.getOption(); + Object value = configuration.get(option); + assertThat(value, equalTo(option.defaultValue())); + } + + @Test + @SuppressWarnings("unchecked") + public void testGetOptionalDefaultValueOverride() { + ReadableConfig configuration = new Configuration(); + + ConfigOption option = testSpec.getOption(); + Object value = ((Optional) configuration.getOptional(option)) + .orElse(testSpec.getDefaultValueOverride()); + assertThat(value, equalTo(testSpec.getDefaultValueOverride())); + } + + private static class TestSpec { + private final ConfigOption option; + private T value; + private String stringValue; + private T defaultValueOverride; + + private TestSpec(ConfigOption option) { + this.option = option; + } + + public TestSpec valueEquals(T objectValue, String stringValue) { + this.value = objectValue; + this.stringValue = stringValue; + return this; + } + + public TestSpec checkDefaultOverride(T defaultValueOverride) { + Preconditions.checkArgument( + !Objects.equals(defaultValueOverride, option.defaultValue()), + "Default value override should be different from the config option default."); + this.defaultValueOverride = defaultValueOverride; + return this; + } + + public ConfigOption getOption() { + return option; + } + + public T getValue() { + return value; + } + + public String getStringValue() { + return stringValue; + } + + public T getDefaultValueOverride() { + return defaultValueOverride; + } + + /** + * Workaround to set the value in the configuration. We cannot set in the test itself as the + * type of the TypeSpec is erased, because it used for parameterizing the test suite. + */ + public void setValue(Configuration configuration) { + configuration.set(option, value); + } + + @Override + public String toString() { + return "TestSpec{" + + "option=" + option + + ", value=" + value + + ", stringValue='" + stringValue + '\'' + + ", defaultValueOverride=" + defaultValueOverride + + '}'; + } + } +} diff --git a/flink-core/src/test/java/org/apache/flink/configuration/StructuredOptionsSplitterTest.java b/flink-core/src/test/java/org/apache/flink/configuration/StructuredOptionsSplitterTest.java new file mode 100644 index 000000000000..323e55dda13f --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/configuration/StructuredOptionsSplitterTest.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.configuration; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import javax.annotation.Nullable; + +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.junit.Assert.assertThat; + +/** + * Tests for {@link StructuredOptionsSplitter}. + */ +@RunWith(Parameterized.class) +public class StructuredOptionsSplitterTest { + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Parameterized.Parameters(name = "{0}") + public static Collection getSpecs() { + return Arrays.asList( + + // Use single quotes for quoting + TestSpec.split("'A;B';C", ';').expect("A;B", "C"), + TestSpec.split("'A;B';'C'", ';').expect("A;B", "C"), + TestSpec.split("A;B;C", ';').expect("A", "B", "C"), + TestSpec.split("'AB''D;B';C", ';').expect("AB'D;B", "C"), + TestSpec.split("A'BD;B';C", ';').expectException("Could not split string. Illegal quoting at position: 1"), + TestSpec.split("'AB'D;B;C", ';').expectException("Could not split string. Illegal quoting at position: 3"), + TestSpec.split("'A", ';').expectException("Could not split string. Quoting was not closed properly."), + TestSpec.split("C;'", ';').expectException("Could not split string. Quoting was not closed properly."), + + // Use double quotes for quoting + TestSpec.split("\"A;B\";C", ';').expect("A;B", "C"), + TestSpec.split("\"A;B\";\"C\"", ';').expect("A;B", "C"), + TestSpec.split("A;B;C", ';').expect("A", "B", "C"), + TestSpec.split("\"AB\"\"D;B\";C", ';').expect("AB\"D;B", "C"), + TestSpec.split("A\"BD;B\";C", ';') + .expectException("Could not split string. Illegal quoting at position: 1"), + TestSpec.split("\"AB\"D;B;C", ';') + .expectException("Could not split string. Illegal quoting at position: 3"), + TestSpec.split("\"A", ';').expectException("Could not split string. Quoting was not closed properly."), + TestSpec.split("C;\"", ';').expectException("Could not split string. Quoting was not closed properly."), + + // Mix different quoting + TestSpec.split("'AB\"D';B;C", ';').expect("AB\"D", "B", "C"), + TestSpec.split("'AB\"D;B';C", ';').expect("AB\"D;B", "C"), + TestSpec.split("'AB\"''D;B';C", ';').expect("AB\"'D;B", "C"), + TestSpec.split("\"AB'D\";B;C", ';').expect("AB'D", "B", "C"), + TestSpec.split("\"AB'D;B\";C", ';').expect("AB'D;B", "C"), + TestSpec.split("\"AB'\"\"D;B\";C", ';').expect("AB'\"D;B", "C"), + + // Use different delimiter + TestSpec.split("'A,B',C", ',').expect("A,B", "C"), + TestSpec.split("A,B,C", ',').expect("A", "B", "C"), + + // Whitespaces handling + TestSpec.split(" 'A;B' ; C ", ';').expect("A;B", "C"), + TestSpec.split(" A;B ; C ", ';').expect("A", "B", "C"), + TestSpec.split("'A;B' ;C A", ';').expect("A;B", "C A"), + TestSpec.split("' A ;B' ;' C'", ';').expect(" A ;B", " C") + ); + } + + @Parameterized.Parameter + public TestSpec testSpec; + + @Test + public void testParse() { + testSpec.getExpectedException().ifPresent(exception -> { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage(exception); + }); + List splits = StructuredOptionsSplitter.splitEscaped(testSpec.getString(), testSpec.getDelimiter()); + + assertThat(splits, equalTo(testSpec.getExpectedSplits())); + } + + private static class TestSpec { + private final String string; + private final char delimiter; + @Nullable private String expectedException = null; + private List expectedSplits = null; + + private TestSpec(String string, char delimiter) { + this.string = string; + this.delimiter = delimiter; + } + + public static TestSpec split(String string, char delimiter) { + return new TestSpec(string, delimiter); + } + + public TestSpec expect(String... splits) { + this.expectedSplits = Arrays.asList(splits); + return this; + } + + public TestSpec expectException(String message) { + this.expectedException = message; + return this; + } + + public String getString() { + return string; + } + + public char getDelimiter() { + return delimiter; + } + + public Optional getExpectedException() { + return Optional.ofNullable(expectedException); + } + + public List getExpectedSplits() { + return expectedSplits; + } + + @Override + public String toString() { + return String.format("str = [ %s ], del = '%s', expected = %s", + string, + delimiter, + getExpectedException() + .map(e -> String.format("Exception(%s)", e)) + .orElseGet(() -> expectedSplits.stream().collect(Collectors.joining("], [", "[", "]")))); + } + } +} diff --git a/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java b/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java index 8163c50add4a..ba2d6eb90a8d 100644 --- a/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java +++ b/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java @@ -74,7 +74,8 @@ public void testExceptionOnSet() { UnmodifiableConfiguration config = new UnmodifiableConfiguration(new Configuration()); for (Method m : clazz.getMethods()) { - if (m.getName().startsWith("set")) { + // ignore WritableConfig#set as it is covered in ReadableWritableConfigurationTest + if (m.getName().startsWith("set") && !m.getName().equals("set")) { Class keyClass = m.getParameterTypes()[0]; Class parameterClass = m.getParameterTypes()[1]; From 44c85e3190fac69b64cb21153e39c41b9a5b7227 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Wed, 16 Oct 2019 15:32:33 +0200 Subject: [PATCH 329/746] [FLINK-14490] Use ObjectIdentifier in TableOperations --- .../api/internal/TableEnvironmentImpl.java | 11 +- .../operations/CatalogQueryOperation.java | 12 +- .../CatalogSinkModifyOperation.java | 19 ++- .../operations/ddl/CreateTableOperation.java | 14 +- .../operations/ddl/DropTableOperation.java | 13 +- .../operations/SqlToOperationConverter.java | 42 +++-- .../planner/plan/QueryOperationConverter.java | 2 +- .../planner/delegation/PlannerBase.scala | 13 +- .../sqlexec/SqlToOperationConverterTest.java | 6 +- .../flink/table/api/batch/ExplainTest.xml | 24 +-- .../flink/table/api/stream/ExplainTest.xml | 40 ++--- .../plan/batch/sql/DagOptimizationTest.xml | 132 +++++++-------- .../table/planner/plan/batch/sql/SinkTest.xml | 12 +- .../plan/stream/sql/DagOptimizationTest.xml | 160 +++++++++--------- .../stream/sql/MiniBatchIntervalInferTest.xml | 12 +- .../planner/plan/stream/sql/SinkTest.xml | 48 +++--- .../table/planner/utils/TableTestBase.scala | 3 +- .../table/plan/QueryOperationConverter.java | 2 +- .../sqlexec/SqlToOperationConverter.java | 35 +++- .../table/api/internal/TableEnvImpl.scala | 17 +- .../flink/table/planner/StreamPlanner.scala | 12 +- .../sqlexec/SqlToOperationConverterTest.java | 12 +- 22 files changed, 348 insertions(+), 293 deletions(-) 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 d13dea35562b..fec7ee78e953 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 @@ -333,9 +333,10 @@ public void insertInto(Table table, String path, String... pathContinued) { List fullPath = new ArrayList<>(Arrays.asList(pathContinued)); fullPath.add(0, path); + ObjectIdentifier objectIdentifier = catalogManager.qualifyIdentifier(fullPath.toArray(new String[0])); List modifyOperations = Collections.singletonList( new CatalogSinkModifyOperation( - fullPath, + objectIdentifier, table.getQueryOperation())); if (isEagerOperationTranslation()) { @@ -366,15 +367,15 @@ public void sqlUpdate(String stmt) { } } else if (operation instanceof CreateTableOperation) { CreateTableOperation createTableOperation = (CreateTableOperation) operation; - ObjectIdentifier objectIdentifier = catalogManager.qualifyIdentifier(createTableOperation.getTablePath()); catalogManager.createTable( createTableOperation.getCatalogTable(), - objectIdentifier, + createTableOperation.getTableIdentifier(), createTableOperation.isIgnoreIfExists()); } else if (operation instanceof DropTableOperation) { DropTableOperation dropTableOperation = (DropTableOperation) operation; - ObjectIdentifier objectIdentifier = catalogManager.qualifyIdentifier(dropTableOperation.getTableName()); - catalogManager.dropTable(objectIdentifier, dropTableOperation.isIfExists()); + catalogManager.dropTable( + dropTableOperation.getTableIdentifier(), + dropTableOperation.isIfExists()); } else { throw new TableException( "Unsupported SQL query! sqlUpdate() only accepts a single SQL statements of " + diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/CatalogQueryOperation.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/CatalogQueryOperation.java index b5a51d7f1c3c..83145f084dba 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/CatalogQueryOperation.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/CatalogQueryOperation.java @@ -33,16 +33,16 @@ @Internal public class CatalogQueryOperation implements QueryOperation { - private final ObjectIdentifier objectIdentifier; + private final ObjectIdentifier tableIdentifier; private final TableSchema tableSchema; - public CatalogQueryOperation(ObjectIdentifier objectIdentifier, TableSchema tableSchema) { - this.objectIdentifier = objectIdentifier; + public CatalogQueryOperation(ObjectIdentifier tableIdentifier, TableSchema tableSchema) { + this.tableIdentifier = tableIdentifier; this.tableSchema = tableSchema; } - public ObjectIdentifier getObjectIdentifier() { - return objectIdentifier; + public ObjectIdentifier getTableIdentifier() { + return tableIdentifier; } @Override @@ -53,7 +53,7 @@ public TableSchema getTableSchema() { @Override public String asSummaryString() { Map args = new LinkedHashMap<>(); - args.put("identifier", objectIdentifier); + args.put("identifier", tableIdentifier); args.put("fields", tableSchema.getFieldNames()); return OperationUtils.formatWithChildren("CatalogTable", args, getChildren(), Operation::asSummaryString); diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/CatalogSinkModifyOperation.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/CatalogSinkModifyOperation.java index 1b9e4ab3a347..86e9a85e0259 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/CatalogSinkModifyOperation.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/CatalogSinkModifyOperation.java @@ -19,11 +19,11 @@ package org.apache.flink.table.operations; import org.apache.flink.annotation.Internal; +import org.apache.flink.table.catalog.ObjectIdentifier; import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; -import java.util.List; import java.util.Map; /** @@ -33,27 +33,28 @@ @Internal public class CatalogSinkModifyOperation implements ModifyOperation { + private final ObjectIdentifier tableIdentifier; private final Map staticPartitions; - private final List tablePath; private final QueryOperation child; private final boolean overwrite; - public CatalogSinkModifyOperation(List tablePath, QueryOperation child) { - this(tablePath, child, new HashMap<>(), false); + public CatalogSinkModifyOperation(ObjectIdentifier tableIdentifier, QueryOperation child) { + this(tableIdentifier, child, new HashMap<>(), false); } - public CatalogSinkModifyOperation(List tablePath, + public CatalogSinkModifyOperation( + ObjectIdentifier tableIdentifier, QueryOperation child, Map staticPartitions, boolean overwrite) { - this.tablePath = tablePath; + this.tableIdentifier = tableIdentifier; this.child = child; this.staticPartitions = staticPartitions; this.overwrite = overwrite; } - public List getTablePath() { - return tablePath; + public ObjectIdentifier getTableIdentifier() { + return tableIdentifier; } public Map getStaticPartitions() { @@ -77,7 +78,7 @@ public T accept(ModifyOperationVisitor visitor) { @Override public String asSummaryString() { Map params = new LinkedHashMap<>(); - params.put("tablePath", tablePath); + params.put("identifier", tableIdentifier); params.put("staticPartitions", staticPartitions); params.put("overwrite", overwrite); diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ddl/CreateTableOperation.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ddl/CreateTableOperation.java index 3c806f6b6683..9491d80532a3 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ddl/CreateTableOperation.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ddl/CreateTableOperation.java @@ -19,6 +19,7 @@ package org.apache.flink.table.operations.ddl; import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.OperationUtils; @@ -30,14 +31,15 @@ * Operation to describe a CREATE TABLE statement. */ public class CreateTableOperation implements CreateOperation { - private final String[] tablePath; + private final ObjectIdentifier tableIdentifier; private CatalogTable catalogTable; private boolean ignoreIfExists; - public CreateTableOperation(String[] tablePath, + public CreateTableOperation( + ObjectIdentifier tableIdentifier, CatalogTable catalogTable, boolean ignoreIfExists) { - this.tablePath = tablePath; + this.tableIdentifier = tableIdentifier; this.catalogTable = catalogTable; this.ignoreIfExists = ignoreIfExists; } @@ -46,8 +48,8 @@ public CatalogTable getCatalogTable() { return catalogTable; } - public String[] getTablePath() { - return tablePath; + public ObjectIdentifier getTableIdentifier() { + return tableIdentifier; } public boolean isIgnoreIfExists() { @@ -58,7 +60,7 @@ public boolean isIgnoreIfExists() { public String asSummaryString() { Map params = new LinkedHashMap<>(); params.put("catalogTable", catalogTable.toProperties()); - params.put("tablePath", tablePath); + params.put("identifier", tableIdentifier); params.put("ignoreIfExists", ignoreIfExists); return OperationUtils.formatWithChildren( diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ddl/DropTableOperation.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ddl/DropTableOperation.java index c9f67db7e3b3..99f15a98a290 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ddl/DropTableOperation.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ddl/DropTableOperation.java @@ -18,6 +18,7 @@ package org.apache.flink.table.operations.ddl; +import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.OperationUtils; @@ -29,16 +30,16 @@ * Operation to describe a DROP TABLE statement. */ public class DropTableOperation implements DropOperation { - private final String[] tableName; + private final ObjectIdentifier tableIdentifier; private final boolean ifExists; - public DropTableOperation(String[] tableName, boolean ifExists) { - this.tableName = tableName; + public DropTableOperation(ObjectIdentifier tableIdentifier, boolean ifExists) { + this.tableIdentifier = tableIdentifier; this.ifExists = ifExists; } - public String[] getTableName() { - return this.tableName; + public ObjectIdentifier getTableIdentifier() { + return this.tableIdentifier; } public boolean isIfExists() { @@ -48,7 +49,7 @@ public boolean isIfExists() { @Override public String asSummaryString() { Map params = new LinkedHashMap<>(); - params.put("tableName", tableName); + params.put("identifier", tableIdentifier); params.put("IfExists", ifExists); return OperationUtils.formatWithChildren( 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 d19c725aa590..9338fd03f5ef 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 @@ -25,8 +25,10 @@ import org.apache.flink.sql.parser.dml.RichSqlInsert; import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.CatalogManager; import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.CatalogTableImpl; +import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.operations.CatalogSinkModifyOperation; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.ddl.CreateTableOperation; @@ -60,11 +62,15 @@ */ public class SqlToOperationConverter { private FlinkPlannerImpl flinkPlanner; + private CatalogManager catalogManager; //~ Constructors ----------------------------------------------------------- - private SqlToOperationConverter(FlinkPlannerImpl flinkPlanner) { + private SqlToOperationConverter( + FlinkPlannerImpl flinkPlanner, + CatalogManager catalogManager) { this.flinkPlanner = flinkPlanner; + this.catalogManager = catalogManager; } /** @@ -72,16 +78,20 @@ private SqlToOperationConverter(FlinkPlannerImpl flinkPlanner) { * SqlNode will have it's implementation in the #convert(type) method whose 'type' argument * is subclass of {@code SqlNode}. * - * @param flinkPlanner FlinkPlannerImpl to convertCreateTable sql node to rel node - * @param sqlNode SqlNode to execute on + * @param flinkPlanner FlinkPlannerImpl to convertCreateTable sql node to rel node + * @param catalogManager CatalogManager to resolve full path for operations + * @param sqlNode SqlNode to execute on */ - public static Operation convert(FlinkPlannerImpl flinkPlanner, SqlNode sqlNode) { + public static Operation convert( + FlinkPlannerImpl flinkPlanner, + CatalogManager catalogManager, + SqlNode sqlNode) { // validate the query final SqlNode validated = flinkPlanner.validate(sqlNode); - SqlToOperationConverter converter = new SqlToOperationConverter(flinkPlanner); + SqlToOperationConverter converter = new SqlToOperationConverter(flinkPlanner, catalogManager); if (validated instanceof SqlCreateTable) { return converter.convertCreateTable((SqlCreateTable) validated); - } if (validated instanceof SqlDropTable) { + } else if (validated instanceof SqlDropTable) { return converter.convertDropTable((SqlDropTable) validated); } else if (validated instanceof RichSqlInsert) { return converter.convertSqlInsert((RichSqlInsert) validated); @@ -128,22 +138,34 @@ private Operation convertCreateTable(SqlCreateTable sqlCreateTable) { partitionKeys, properties, tableComment); - return new CreateTableOperation(sqlCreateTable.fullTableName(), catalogTable, + + ObjectIdentifier identifier = catalogManager.qualifyIdentifier(sqlCreateTable.fullTableName()); + + return new CreateTableOperation( + identifier, + catalogTable, sqlCreateTable.isIfNotExists()); } /** Convert DROP TABLE statement. */ private Operation convertDropTable(SqlDropTable sqlDropTable) { - return new DropTableOperation(sqlDropTable.fullTableName(), sqlDropTable.getIfExists()); + ObjectIdentifier identifier = catalogManager.qualifyIdentifier(sqlDropTable.fullTableName()); + + return new DropTableOperation(identifier, sqlDropTable.getIfExists()); } /** Convert insert into statement. */ private Operation convertSqlInsert(RichSqlInsert insert) { // get name of sink table List targetTablePath = ((SqlIdentifier) insert.getTargetTable()).names; + + ObjectIdentifier identifier = catalogManager.qualifyIdentifier(targetTablePath.toArray(new String[0])); + return new CatalogSinkModifyOperation( - targetTablePath, - (PlannerQueryOperation) SqlToOperationConverter.convert(flinkPlanner, + identifier, + (PlannerQueryOperation) SqlToOperationConverter.convert( + flinkPlanner, + catalogManager, insert.getSource()), insert.getStaticPartitionKVs(), insert.isOverwrite()); diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java index a6720eed8079..ac0d3b68957e 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java @@ -297,7 +297,7 @@ public RelNode visit(CalculatedQueryOperation calculatedTable) { @Override public RelNode visit(CatalogQueryOperation catalogTable) { - ObjectIdentifier objectIdentifier = catalogTable.getObjectIdentifier(); + ObjectIdentifier objectIdentifier = catalogTable.getTableIdentifier(); return relBuilder.scan( objectIdentifier.getCatalogName(), objectIdentifier.getDatabaseName(), diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala index ac9670e76b21..9725ed819021 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala @@ -126,11 +126,11 @@ abstract class PlannerBase( val parsed = planner.parse(stmt) parsed match { case insert: RichSqlInsert => - List(SqlToOperationConverter.convert(planner, insert)) + List(SqlToOperationConverter.convert(planner, catalogManager, insert)) case query if query.getKind.belongsTo(SqlKind.QUERY) => - List(SqlToOperationConverter.convert(planner, query)) + List(SqlToOperationConverter.convert(planner, catalogManager, query)) case ddl if ddl.getKind.belongsTo(SqlKind.DDL) => - List(SqlToOperationConverter.convert(planner, ddl)) + List(SqlToOperationConverter.convert(planner, catalogManager, ddl)) case _ => throw new TableException(s"Unsupported query: $stmt") } @@ -174,7 +174,7 @@ abstract class PlannerBase( case catalogSink: CatalogSinkModifyOperation => val input = getRelBuilder.queryOperation(modifyOperation.getChild).build() - val identifier = catalogManager.qualifyIdentifier(catalogSink.getTablePath: _*) + val identifier = catalogSink.getTableIdentifier getTableSink(identifier).map { case (table, sink) => TableSinkUtils.validateSink(catalogSink, identifier, sink, table.getPartitionKeys) sink match { @@ -190,10 +190,11 @@ abstract class PlannerBase( s"${classOf[OverwritableTableSink].getSimpleName} but actually got " + sink.getClass.getName) } - LogicalSink.create(input, sink, catalogSink.getTablePath.mkString("."), table) + LogicalSink.create(input, sink, identifier.toString, table) } match { case Some(sinkRel) => sinkRel - case None => throw new TableException(s"Sink ${catalogSink.getTablePath} does not exists") + case None => + throw new TableException(s"Sink ${catalogSink.getTableIdentifier} does not exists") } case outputConversion: OutputConversionModifyOperation => diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java index 9f872752b066..868246c3e681 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java @@ -167,7 +167,7 @@ public void testCreateTableWithMinusInOptionKey() { final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); SqlNode node = planner.parse(sql); assert node instanceof SqlCreateTable; - Operation operation = SqlToOperationConverter.convert(planner, node); + Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node); assert operation instanceof CreateTableOperation; CreateTableOperation op = (CreateTableOperation) operation; CatalogTable catalogTable = op.getCatalogTable(); @@ -325,7 +325,7 @@ public void testCreateTableWithFullDataTypes() { final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); SqlNode node = planner.parse(sql); assert node instanceof SqlCreateTable; - Operation operation = SqlToOperationConverter.convert(planner, node); + Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node); TableSchema schema = ((CreateTableOperation) operation).getCatalogTable().getSchema(); Object[] expectedDataTypes = testItems.stream().map(item -> item.expectedType).toArray(); assertArrayEquals(expectedDataTypes, schema.getFieldDataTypes()); @@ -347,7 +347,7 @@ private static TestItem createTestItem(Object... args) { private Operation parse(String sql, FlinkPlannerImpl planner) { SqlNode node = planner.parse(sql); - return SqlToOperationConverter.convert(planner, node); + return SqlToOperationConverter.convert(planner, catalogManager, node); } private FlinkPlannerImpl getPlannerBySqlDialect(SqlDialect sqlDialect) { diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/batch/ExplainTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/batch/ExplainTest.xml index 06d8b8812b19..0b24954dc9dd 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/batch/ExplainTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/batch/ExplainTest.xml @@ -283,14 +283,14 @@ Calc(select=[a, b, c, e, f]): rowcount = , cumulative cost = {rows, cpu, io, net ($1, 10)]) +- LogicalAggregate(group=[{0}], cnt=[COUNT()]) +- LogicalProject(a=[$0]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) -LogicalSink(name=[sink2], fields=[a, cnt]) +LogicalSink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a, cnt]) +- LogicalProject(a=[$0], cnt=[$1]) +- LogicalFilter(condition=[<($1, 10)]) +- LogicalAggregate(group=[{0}], cnt=[COUNT()]) @@ -304,11 +304,11 @@ HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count1$0) AS c +- Calc(select=[a]) +- BoundedStreamScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) -Sink(name=[sink1], fields=[a, cnt]) +Sink(name=[`default_catalog`.`default_database`.`sink1`], fields=[a, cnt]) +- Calc(select=[a, cnt], where=[>(cnt, 10)]) +- Reused(reference_id=[1]) -Sink(name=[sink2], fields=[a, cnt]) +Sink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a, cnt]) +- Calc(select=[a, cnt], where=[<(cnt, 10)]) +- Reused(reference_id=[1]) @@ -362,14 +362,14 @@ Sink(name=[sink2], fields=[a, cnt]) ($1, 10)]) +- LogicalAggregate(group=[{0}], cnt=[COUNT()]) +- LogicalProject(a=[$0]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) -LogicalSink(name=[sink2], fields=[a, cnt]) +LogicalSink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a, cnt]) +- LogicalProject(a=[$0], cnt=[$1]) +- LogicalFilter(condition=[<($1, 10)]) +- LogicalAggregate(group=[{0}], cnt=[COUNT()]) @@ -383,11 +383,11 @@ HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count1$0) AS c +- Calc(select=[a]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- BoundedStreamScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} -Sink(name=[sink1], fields=[a, cnt]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +Sink(name=[`default_catalog`.`default_database`.`sink1`], fields=[a, cnt]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- Calc(select=[a, cnt], where=[>(cnt, 10)]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- Reused(reference_id=[1]) -Sink(name=[sink2], fields=[a, cnt]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +Sink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a, cnt]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- Calc(select=[a, cnt], where=[<(cnt, 10)]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- Reused(reference_id=[1]) @@ -441,13 +441,13 @@ Sink(name=[sink2], fields=[a, cnt]): rowcount = , cumulative cost = {rows, cpu, ($0, 10)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) == Optimized Logical Plan == -Sink(name=[sink], fields=[a, b, c]) +Sink(name=[`default_catalog`.`default_database`.`sink`], fields=[a, b, c]) +- Calc(select=[a, b, c], where=[>(a, 10)]) +- BoundedStreamScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) @@ -477,13 +477,13 @@ Sink(name=[sink], fields=[a, b, c]) ($0, 10)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) == Optimized Logical Plan == -Sink(name=[sink], fields=[a, b, c]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +Sink(name=[`default_catalog`.`default_database`.`sink`], fields=[a, b, c]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- Calc(select=[a, b, c], where=[>(a, 10)]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- BoundedStreamScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml index 62ac33d4f799..46661c62c59f 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml @@ -345,14 +345,14 @@ Calc(select=[a, b, c, e, f], updateAsRetraction=[false], accMode=[Acc]): rowcoun ($1, 10)]) +- LogicalAggregate(group=[{0}], cnt=[COUNT()]) +- LogicalProject(a=[$0]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) -LogicalSink(name=[upsertSink2], fields=[a, cnt]) +LogicalSink(name=[`default_catalog`.`default_database`.`upsertSink2`], fields=[a, cnt]) +- LogicalProject(a=[$0], cnt=[$1]) +- LogicalFilter(condition=[<($1, 10)]) +- LogicalAggregate(group=[{0}], cnt=[COUNT()]) @@ -365,11 +365,11 @@ GroupAggregate(groupBy=[a], select=[a, COUNT(*) AS cnt], reuse_id=[1]) +- Calc(select=[a]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) -Sink(name=[upsertSink1], fields=[a, cnt]) +Sink(name=[`default_catalog`.`default_database`.`upsertSink1`], fields=[a, cnt]) +- Calc(select=[a, cnt], where=[>(cnt, 10)]) +- Reused(reference_id=[1]) -Sink(name=[upsertSink2], fields=[a, cnt]) +Sink(name=[`default_catalog`.`default_database`.`upsertSink2`], fields=[a, cnt]) +- Calc(select=[a, cnt], where=[<(cnt, 10)]) +- Reused(reference_id=[1]) @@ -427,14 +427,14 @@ Sink(name=[upsertSink2], fields=[a, cnt]) ($1, 10)]) +- LogicalAggregate(group=[{0}], cnt=[COUNT()]) +- LogicalProject(a=[$0]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) -LogicalSink(name=[upsertSink2], fields=[a, cnt]) +LogicalSink(name=[`default_catalog`.`default_database`.`upsertSink2`], fields=[a, cnt]) +- LogicalProject(a=[$0], cnt=[$1]) +- LogicalFilter(condition=[<($1, 10)]) +- LogicalAggregate(group=[{0}], cnt=[COUNT()]) @@ -447,11 +447,11 @@ GroupAggregate(groupBy=[a], select=[a, COUNT(*) AS cnt], updateAsRetraction=[fal +- Calc(select=[a], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- DataStreamScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} -Sink(name=[upsertSink1], fields=[a, cnt], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +Sink(name=[`default_catalog`.`default_database`.`upsertSink1`], fields=[a, cnt], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- Calc(select=[a, cnt], where=[>(cnt, 10)], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- Reused(reference_id=[1]) -Sink(name=[upsertSink2], fields=[a, cnt], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +Sink(name=[`default_catalog`.`default_database`.`upsertSink2`], fields=[a, cnt], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- Calc(select=[a, cnt], where=[<(cnt, 10)], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- Reused(reference_id=[1]) @@ -509,13 +509,13 @@ Sink(name=[upsertSink2], fields=[a, cnt], updateAsRetraction=[false], accMode=[A ($0, 10)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) == Optimized Logical Plan == -Sink(name=[appendSink], fields=[a, b, c]) +Sink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[a, b, c]) +- Calc(select=[a, b, c], where=[>(a, 10)]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) @@ -545,13 +545,13 @@ Sink(name=[appendSink], fields=[a, b, c]) ($0, 10)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) == Optimized Logical Plan == -Sink(name=[appendSink], fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +Sink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- Calc(select=[a, b, c], where=[>(a, 10)], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- DataStreamScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} @@ -635,7 +635,7 @@ SortLimit(orderBy=[a ASC], offset=[0], fetch=[5], updateAsRetraction=[false], ac =($0, 4), <($0, 6))]) +- LogicalAggregate(group=[{0}], cnt=[COUNT($1)]) @@ -62,7 +62,7 @@ TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [Tes Calc(select=[a, b, c], where=[LIKE(c, _UTF-16LE'%hello%')], reuse_id=[1]) +- Reused(reference_id=[2]) -Sink(name=[sink1], fields=[a, b, c]) +Sink(name=[`default_catalog`.`default_database`.`sink1`], fields=[a, b, c]) +- Reused(reference_id=[1]) HashAggregate(isMerge=[true], groupBy=[b], select=[b, Final_COUNT(count$0) AS cnt], reuse_id=[3]) @@ -74,11 +74,11 @@ HashAggregate(isMerge=[true], groupBy=[b], select=[b, Final_COUNT(count$0) AS cn +- Calc(select=[b, a], where=[LIKE(c, _UTF-16LE'%world%')]) +- Reused(reference_id=[2]) -Sink(name=[sink2], fields=[b, cnt]) +Sink(name=[`default_catalog`.`default_database`.`sink2`], fields=[b, cnt]) +- Calc(select=[b, cnt], where=[<(b, 4)]) +- Reused(reference_id=[3]) -Sink(name=[sink3], fields=[b, cnt]) +Sink(name=[`default_catalog`.`default_database`.`sink3`], fields=[b, cnt]) +- Calc(select=[b, cnt], where=[AND(>=(b, 4), <(b, 6))]) +- Reused(reference_id=[3]) ]]> @@ -87,7 +87,7 @@ Sink(name=[sink3], fields=[b, cnt]) @@ -187,7 +187,7 @@ Sink(name=[sink2], fields=[a, b1]) =($0, 0)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) -LogicalSink(name=[sink2], fields=[a1, b, c1]) +LogicalSink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a1, b, c1]) +- LogicalProject(a1=[$2], b=[$3], c1=[$1]) +- LogicalFilter(condition=[=($2, $0)]) +- LogicalJoin(condition=[true], joinType=[inner]) @@ -289,10 +289,10 @@ Calc(select=[a AS a1, b, c AS c2], reuse_id=[3]) +- Calc(select=[a AS a2, c], where=[>=(b, 5)]) +- Reused(reference_id=[2]) -Sink(name=[sink1], fields=[a1, b, c2]) +Sink(name=[`default_catalog`.`default_database`.`sink1`], fields=[a1, b, c2]) +- Reused(reference_id=[3]) -Sink(name=[sink2], fields=[a1, b, c1]) +Sink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a1, b, c1]) +- Calc(select=[a1, b, c1]) +- HashJoin(joinType=[InnerJoin], where=[=(a1, a3)], select=[a3, c1, a1, b], build=[left]) :- Exchange(distribution=[hash[a3]]) @@ -307,13 +307,13 @@ Sink(name=[sink2], fields=[a1, b, c1]) @@ -456,14 +456,14 @@ Sink(name=[sink3], fields=[a]) @@ -615,7 +615,7 @@ Sink(name=[sink2], fields=[a, sum_c, time]) ($0, 50)]) +- LogicalUnion(all=[true]) @@ -633,7 +633,7 @@ LogicalSink(name=[sink1], fields=[a, total_c]) +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) -LogicalSink(name=[sink2], fields=[a, total_c]) +LogicalSink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a, total_c]) +- LogicalProject(a=[$0], total_c=[$1]) +- LogicalFilter(condition=[<($0, 50)]) +- LogicalUnion(all=[true]) @@ -671,14 +671,14 @@ Calc(select=[a, total_c], reuse_id=[2]) Calc(select=[a, 0 AS total_c], where=[>=(b, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))], reuse_id=[3]) +- Reused(reference_id=[1]) -Sink(name=[sink1], fields=[a, total_c]) +Sink(name=[`default_catalog`.`default_database`.`sink1`], fields=[a, total_c]) +- Union(all=[true], union=[a, total_c]) :- Calc(select=[a, total_c], where=[>(a, 50)]) : +- Reused(reference_id=[2]) +- Calc(select=[a, CAST(total_c) AS total_c], where=[>(a, 50)]) +- Reused(reference_id=[3]) -Sink(name=[sink2], fields=[a, total_c]) +Sink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a, total_c]) +- Union(all=[true], union=[a, total_c]) :- Calc(select=[a, total_c], where=[<(a, 50)]) : +- Reused(reference_id=[2]) @@ -690,7 +690,7 @@ Sink(name=[sink2], fields=[a, total_c]) =($0, 4), <($0, 6))]) +- LogicalAggregate(group=[{0}], cnt=[COUNT($1)]) @@ -62,7 +62,7 @@ TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [Tes Calc(select=[a, b, c], where=[LIKE(c, _UTF-16LE'%hello%')], updateAsRetraction=[true], accMode=[Acc], reuse_id=[1]) +- Reused(reference_id=[2]) -Sink(name=[appendSink], fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc]) +Sink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc]) +- Reused(reference_id=[1]) GroupAggregate(groupBy=[b], select=[b, COUNT(a) AS cnt], updateAsRetraction=[true], accMode=[AccRetract], reuse_id=[3]) @@ -73,11 +73,11 @@ GroupAggregate(groupBy=[b], select=[b, COUNT(a) AS cnt], updateAsRetraction=[tru +- Calc(select=[b, a], where=[LIKE(c, _UTF-16LE'%world%')], updateAsRetraction=[true], accMode=[Acc]) +- Reused(reference_id=[2]) -Sink(name=[retractSink], fields=[b, cnt], updateAsRetraction=[true], accMode=[AccRetract]) +Sink(name=[`default_catalog`.`default_database`.`retractSink`], fields=[b, cnt], updateAsRetraction=[true], accMode=[AccRetract]) +- Calc(select=[b, cnt], where=[<(b, 4)], updateAsRetraction=[true], accMode=[AccRetract]) +- Reused(reference_id=[3]) -Sink(name=[upsertSink], fields=[b, cnt], updateAsRetraction=[false], accMode=[AccRetract]) +Sink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[b, cnt], updateAsRetraction=[false], accMode=[AccRetract]) +- Calc(select=[b, cnt], where=[AND(>=(b, 4), <(b, 6))], updateAsRetraction=[false], accMode=[AccRetract]) +- Reused(reference_id=[3]) ]]> @@ -86,7 +86,7 @@ Sink(name=[upsertSink], fields=[b, cnt], updateAsRetraction=[false], accMode=[Ac @@ -125,7 +125,7 @@ Sink(name=[appendSink2], fields=[total_min], updateAsRetraction=[false], accMode @@ -181,7 +181,7 @@ Sink(name=[appendSink2], fields=[a, b1], updateAsRetraction=[false], accMode=[Ac =($0, 0)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) -LogicalSink(name=[appendSink2], fields=[a1, b, c1]) +LogicalSink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a1, b, c1]) +- LogicalProject(a1=[$2], b=[$3], c1=[$1]) +- LogicalFilter(condition=[=($2, $0)]) +- LogicalJoin(condition=[true], joinType=[inner]) @@ -282,10 +282,10 @@ Calc(select=[a1, b, c AS c2], updateAsRetraction=[true], accMode=[Acc], reuse_id +- Calc(select=[a AS a2, c], where=[>=(b, 5)], updateAsRetraction=[true], accMode=[Acc]) +- Reused(reference_id=[2]) -Sink(name=[appendSink1], fields=[a1, b, c2], updateAsRetraction=[false], accMode=[Acc]) +Sink(name=[`default_catalog`.`default_database`.`appendSink1`], fields=[a1, b, c2], updateAsRetraction=[false], accMode=[Acc]) +- Reused(reference_id=[3]) -Sink(name=[appendSink2], fields=[a1, b, c1], updateAsRetraction=[false], accMode=[Acc]) +Sink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a1, b, c1], updateAsRetraction=[false], accMode=[Acc]) +- Calc(select=[a1, b, c1], updateAsRetraction=[false], accMode=[Acc]) +- Join(joinType=[InnerJoin], where=[=(a1, a3)], select=[a3, c1, a1, b, c2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], updateAsRetraction=[false], accMode=[Acc]) :- Exchange(distribution=[hash[a3]], updateAsRetraction=[true], accMode=[Acc]) @@ -299,13 +299,13 @@ Sink(name=[appendSink2], fields=[a1, b, c1], updateAsRetraction=[false], accMode @@ -331,7 +331,7 @@ Sink(name=[appendSink2], fields=[total_min], updateAsRetraction=[false], accMode ($0, 50)]) +- LogicalUnion(all=[true]) @@ -349,7 +349,7 @@ LogicalSink(name=[appendSink1], fields=[a, total_c]) +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) -LogicalSink(name=[appendSink2], fields=[a, total_c]) +LogicalSink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a, total_c]) +- LogicalProject(a=[$0], total_c=[$1]) +- LogicalFilter(condition=[<($0, 50)]) +- LogicalUnion(all=[true]) @@ -383,14 +383,14 @@ Calc(select=[a, total_c], updateAsRetraction=[false], accMode=[Acc], reuse_id=[2 Calc(select=[a, 0 AS total_c], where=[>=(b, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))], updateAsRetraction=[false], accMode=[Acc], reuse_id=[3]) +- Reused(reference_id=[1]) -Sink(name=[appendSink1], fields=[a, total_c], updateAsRetraction=[false], accMode=[Acc]) +Sink(name=[`default_catalog`.`default_database`.`appendSink1`], fields=[a, total_c], updateAsRetraction=[false], accMode=[Acc]) +- Union(all=[true], union=[a, total_c], updateAsRetraction=[false], accMode=[Acc]) :- Calc(select=[a, total_c], where=[>(a, 50)], updateAsRetraction=[false], accMode=[Acc]) : +- Reused(reference_id=[2]) +- Calc(select=[a, CAST(total_c) AS total_c], where=[>(a, 50)], updateAsRetraction=[false], accMode=[Acc]) +- Reused(reference_id=[3]) -Sink(name=[appendSink2], fields=[a, total_c], updateAsRetraction=[false], accMode=[Acc]) +Sink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a, total_c], updateAsRetraction=[false], accMode=[Acc]) +- Union(all=[true], union=[a, total_c], updateAsRetraction=[false], accMode=[Acc]) :- Calc(select=[a, total_c], where=[<(a, 50)], updateAsRetraction=[false], accMode=[Acc]) : +- Reused(reference_id=[2]) @@ -402,7 +402,7 @@ Sink(name=[appendSink2], fields=[a, total_c], updateAsRetraction=[false], accMod @@ -444,7 +444,7 @@ Sink(name=[retractSink], fields=[total_min], updateAsRetraction=[true], accMode= @@ -508,14 +508,14 @@ Sink(name=[appendSink3], fields=[a]) @@ -573,7 +573,7 @@ Sink(name=[upsertSink], fields=[total_min], updateAsRetraction=[false], accMode= @@ -624,12 +624,12 @@ Sink(name=[retractSink], fields=[total_min], updateAsRetraction=[true], accMode= =($1, 5)]) +- LogicalUnion(all=[true]) @@ -642,7 +642,7 @@ LogicalSink(name=[retractSink1], fields=[a, b, c]) +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) -LogicalSink(name=[retractSink2], fields=[b, cnt]) +LogicalSink(name=[`default_catalog`.`default_database`.`retractSink2`], fields=[b, cnt]) +- LogicalProject(b=[$0], cnt=[$1]) +- LogicalFilter(condition=[<($0, 4)]) +- LogicalAggregate(group=[{0}], cnt=[COUNT($1)]) @@ -657,7 +657,7 @@ LogicalSink(name=[retractSink2], fields=[b, cnt]) +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) -LogicalSink(name=[upsertSink], fields=[b, cnt]) +LogicalSink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[b, cnt]) +- LogicalProject(b=[$0], cnt=[$1]) +- LogicalFilter(condition=[<($0, 4)]) +- LogicalAggregate(group=[{0}], cnt=[COUNT($1)]) @@ -680,13 +680,13 @@ TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [Tes Calc(select=[a, b, c], where=[LIKE(c, _UTF-16LE'%hello%')], updateAsRetraction=[true], accMode=[Acc], reuse_id=[1]) +- Reused(reference_id=[2]) -Sink(name=[appendSink], fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc]) +Sink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc]) +- Reused(reference_id=[1]) Calc(select=[a, b, c], where=[LIKE(c, _UTF-16LE'%world%')], updateAsRetraction=[true], accMode=[Acc], reuse_id=[3]) +- Reused(reference_id=[2]) -Sink(name=[retractSink1], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc]) +Sink(name=[`default_catalog`.`default_database`.`retractSink1`], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc]) +- Union(all=[true], union=[a, b, c], updateAsRetraction=[true], accMode=[Acc]) :- Calc(select=[a, b, c], where=[>=(b, 5)], updateAsRetraction=[true], accMode=[Acc]) : +- Reused(reference_id=[1]) @@ -701,10 +701,10 @@ GroupAggregate(groupBy=[b], select=[b, COUNT(a) AS cnt], updateAsRetraction=[tru +- Calc(select=[b, a], where=[<(b, 4)], updateAsRetraction=[true], accMode=[Acc]) +- Reused(reference_id=[3]) -Sink(name=[retractSink2], fields=[b, cnt], updateAsRetraction=[true], accMode=[AccRetract]) +Sink(name=[`default_catalog`.`default_database`.`retractSink2`], fields=[b, cnt], updateAsRetraction=[true], accMode=[AccRetract]) +- Reused(reference_id=[4]) -Sink(name=[upsertSink], fields=[b, cnt], updateAsRetraction=[false], accMode=[AccRetract]) +Sink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[b, cnt], updateAsRetraction=[false], accMode=[AccRetract]) +- Reused(reference_id=[4]) ]]> @@ -712,7 +712,7 @@ Sink(name=[upsertSink], fields=[b, cnt], updateAsRetraction=[false], accMode=[Ac ($0, 6)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rank_num=[$3]) @@ -933,7 +933,7 @@ LogicalSink(name=[retractSink], fields=[a]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rank_num=[ROW_NUMBER() OVER (PARTITION BY $1 ORDER BY $2 DESC NULLS LAST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) -LogicalSink(name=[upsertSink], fields=[a, b]) +LogicalSink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[a, b]) +- LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[<($0, 6)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rank_num=[$3]) @@ -948,11 +948,11 @@ Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart= +- Exchange(distribution=[hash[b]], updateAsRetraction=[false], accMode=[Acc]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc]) -Sink(name=[retractSink], fields=[a], updateAsRetraction=[true], accMode=[AccRetract]) +Sink(name=[`default_catalog`.`default_database`.`retractSink`], fields=[a], updateAsRetraction=[true], accMode=[AccRetract]) +- Calc(select=[a], where=[>(a, 6)], updateAsRetraction=[true], accMode=[AccRetract]) +- Reused(reference_id=[1]) -Sink(name=[upsertSink], fields=[a, b], updateAsRetraction=[false], accMode=[AccRetract]) +Sink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[a, b], updateAsRetraction=[false], accMode=[AccRetract]) +- Calc(select=[a, b], where=[<(a, 6)], updateAsRetraction=[false], accMode=[AccRetract]) +- Reused(reference_id=[1]) ]]> @@ -961,7 +961,7 @@ Sink(name=[upsertSink], fields=[a, b], updateAsRetraction=[false], accMode=[AccR diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.xml index b82a3d542874..29f795f65d7b 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.xml @@ -218,7 +218,7 @@ Calc(select=[a, b]) =($0, 4), <($0, 6))]) +- LogicalAggregate(group=[{0}], cnt=[COUNT($1)]) @@ -63,13 +63,13 @@ GroupAggregate(groupBy=[b], select=[b, COUNT(a) AS cnt], updateAsRetraction=[tru +- Calc(select=[b, a], updateAsRetraction=[true], accMode=[Acc]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc]) -Sink(name=[upsertSink1], fields=[b, cnt], updateAsRetraction=[false], accMode=[Acc]) +Sink(name=[`default_catalog`.`default_database`.`upsertSink1`], fields=[b, cnt], updateAsRetraction=[false], accMode=[Acc]) +- GroupAggregate(groupBy=[cnt], select=[cnt, COUNT_RETRACT(b) AS frequency], updateAsRetraction=[false], accMode=[Acc]) +- Exchange(distribution=[hash[cnt]], updateAsRetraction=[true], accMode=[AccRetract]) +- Calc(select=[b, cnt], where=[<(b, 4)], updateAsRetraction=[true], accMode=[AccRetract]) +- Reused(reference_id=[1]) -Sink(name=[upsertSink2], fields=[b, cnt], updateAsRetraction=[false], accMode=[AccRetract]) +Sink(name=[`default_catalog`.`default_database`.`upsertSink2`], fields=[b, cnt], updateAsRetraction=[false], accMode=[AccRetract]) +- Calc(select=[b, cnt], where=[AND(>=(b, 4), <(b, 6))], updateAsRetraction=[false], accMode=[AccRetract]) +- Reused(reference_id=[1]) ]]> @@ -78,14 +78,14 @@ Sink(name=[upsertSink2], fields=[b, cnt], updateAsRetraction=[false], accMode=[A @@ -143,14 +143,14 @@ Sink(name=[upsertSink], fields=[total_min], updateAsRetraction=[false], accMode= =($0, 4), <($0, 6))]) +- LogicalAggregate(group=[{0}], cnt=[COUNT($1)]) @@ -165,11 +165,11 @@ GroupAggregate(groupBy=[b], select=[b, COUNT(a) AS cnt], updateAsRetraction=[tru +- Calc(select=[b, a], updateAsRetraction=[true], accMode=[Acc]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc]) -Sink(name=[retractSink], fields=[b, cnt], updateAsRetraction=[true], accMode=[AccRetract]) +Sink(name=[`default_catalog`.`default_database`.`retractSink`], fields=[b, cnt], updateAsRetraction=[true], accMode=[AccRetract]) +- Calc(select=[b, cnt], where=[<(b, 4)], updateAsRetraction=[true], accMode=[AccRetract]) +- Reused(reference_id=[1]) -Sink(name=[upsertSink], fields=[b, cnt], updateAsRetraction=[false], accMode=[AccRetract]) +Sink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[b, cnt], updateAsRetraction=[false], accMode=[AccRetract]) +- Calc(select=[b, cnt], where=[AND(>=(b, 4), <(b, 6))], updateAsRetraction=[false], accMode=[AccRetract]) +- Reused(reference_id=[1]) ]]> @@ -178,7 +178,7 @@ Sink(name=[upsertSink], fields=[b, cnt], updateAsRetraction=[false], accMode=[Ac RelNode visit(CalculatedQueryOperation calculatedTable) { @Override public RelNode visit(CatalogQueryOperation catalogTable) { - ObjectIdentifier objectIdentifier = catalogTable.getObjectIdentifier(); + ObjectIdentifier objectIdentifier = catalogTable.getTableIdentifier(); return relBuilder.scan( objectIdentifier.getCatalogName(), objectIdentifier.getDatabaseName(), 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 db8221254bd9..e678feb4db2d 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 @@ -27,8 +27,10 @@ import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.calcite.FlinkPlannerImpl; import org.apache.flink.table.calcite.FlinkTypeFactory; +import org.apache.flink.table.catalog.CatalogManager; import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.CatalogTableImpl; +import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.operations.CatalogSinkModifyOperation; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.PlannerQueryOperation; @@ -61,11 +63,15 @@ */ public class SqlToOperationConverter { private FlinkPlannerImpl flinkPlanner; + private CatalogManager catalogManager; //~ Constructors ----------------------------------------------------------- - private SqlToOperationConverter(FlinkPlannerImpl flinkPlanner) { + private SqlToOperationConverter( + FlinkPlannerImpl flinkPlanner, + CatalogManager catalogManager) { this.flinkPlanner = flinkPlanner; + this.catalogManager = catalogManager; } /** @@ -76,10 +82,13 @@ private SqlToOperationConverter(FlinkPlannerImpl flinkPlanner) { * @param flinkPlanner FlinkPlannerImpl to convert sql node to rel node * @param sqlNode SqlNode to execute on */ - public static Operation convert(FlinkPlannerImpl flinkPlanner, SqlNode sqlNode) { + public static Operation convert( + FlinkPlannerImpl flinkPlanner, + CatalogManager catalogManager, + SqlNode sqlNode) { // validate the query final SqlNode validated = flinkPlanner.validate(sqlNode); - SqlToOperationConverter converter = new SqlToOperationConverter(flinkPlanner); + SqlToOperationConverter converter = new SqlToOperationConverter(flinkPlanner, catalogManager); if (validated instanceof SqlCreateTable) { return converter.convertCreateTable((SqlCreateTable) validated); } if (validated instanceof SqlDropTable) { @@ -129,13 +138,20 @@ private Operation convertCreateTable(SqlCreateTable sqlCreateTable) { partitionKeys, properties, tableComment); - return new CreateTableOperation(sqlCreateTable.fullTableName(), catalogTable, + + ObjectIdentifier identifier = catalogManager.qualifyIdentifier(sqlCreateTable.fullTableName()); + + return new CreateTableOperation( + identifier, + catalogTable, sqlCreateTable.isIfNotExists()); } /** Convert DROP TABLE statement. */ private Operation convertDropTable(SqlDropTable sqlDropTable) { - return new DropTableOperation(sqlDropTable.fullTableName(), sqlDropTable.getIfExists()); + ObjectIdentifier identifier = catalogManager.qualifyIdentifier(sqlDropTable.fullTableName()); + + return new DropTableOperation(identifier, sqlDropTable.getIfExists()); } /** Fallback method for sql query. */ @@ -147,9 +163,14 @@ private Operation convertSqlQuery(SqlNode node) { private Operation convertSqlInsert(RichSqlInsert insert) { // get name of sink table List targetTablePath = ((SqlIdentifier) insert.getTargetTable()).names; + + ObjectIdentifier identifier = catalogManager.qualifyIdentifier(targetTablePath.toArray(new String[0])); + return new CatalogSinkModifyOperation( - targetTablePath, - (PlannerQueryOperation) SqlToOperationConverter.convert(flinkPlanner, + identifier, + (PlannerQueryOperation) SqlToOperationConverter.convert( + flinkPlanner, + catalogManager, insert.getSource()), insert.getStaticPartitionKVs(), insert.isOverwrite()); 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 ad14eca3c48d..43b37cd34acc 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 @@ -30,7 +30,7 @@ import org.apache.flink.table.expressions.resolver.lookups.TableReferenceLookup import org.apache.flink.table.factories.{TableFactoryService, TableFactoryUtil, TableSinkFactory} import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, TableFunction, UserDefinedAggregateFunction, _} import org.apache.flink.table.module.{Module, ModuleManager} -import org.apache.flink.table.operations.ddl.CreateTableOperation +import org.apache.flink.table.operations.ddl.{CreateTableOperation, DropTableOperation} import org.apache.flink.table.operations.utils.OperationTreeBuilder import org.apache.flink.table.operations.{CatalogQueryOperation, PlannerQueryOperation, TableSourceQueryOperation, _} import org.apache.flink.table.planner.PlanningConfigurationBuilder @@ -38,10 +38,12 @@ import org.apache.flink.table.sinks.{OverwritableTableSink, PartitionableTableSi import org.apache.flink.table.sources.TableSource import org.apache.flink.table.sqlexec.SqlToOperationConverter import org.apache.flink.table.util.JavaScalaConversionUtil + import org.apache.calcite.jdbc.CalciteSchemaBuilder.asRootSchema import org.apache.calcite.sql._ import org.apache.calcite.sql.parser.SqlParser import org.apache.calcite.tools.FrameworkConfig + import _root_.java.util.{Optional, HashMap => JHashMap, Map => JMap} import _root_.scala.collection.JavaConversions._ @@ -415,16 +417,19 @@ abstract class TableEnvImpl( targetTablePath.asScala:_*) case createTable: SqlCreateTable => val operation = SqlToOperationConverter - .convert(planner, createTable) + .convert(planner, catalogManager, createTable) .asInstanceOf[CreateTableOperation] - val objectIdentifier = catalogManager.qualifyIdentifier(operation.getTablePath: _*) catalogManager.createTable( operation.getCatalogTable, - objectIdentifier, + operation.getTableIdentifier, operation.isIgnoreIfExists) case dropTable: SqlDropTable => - val objectIdentifier = catalogManager.qualifyIdentifier(dropTable.fullTableName(): _*) - catalogManager.dropTable(objectIdentifier, dropTable.getIfExists) + val operation = SqlToOperationConverter + .convert(planner, catalogManager, dropTable) + .asInstanceOf[DropTableOperation] + catalogManager.dropTable( + operation.getTableIdentifier, + operation.isIfExists) case _ => throw new TableException( "Unsupported SQL query! sqlUpdate() only accepts SQL statements of " + diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala index c50d32b61595..c0821c0b2e4f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala @@ -110,9 +110,9 @@ class StreamPlanner( if (targetColumnList != null && insert.getTargetColumnList.size() != 0) { throw new ValidationException("Partial inserts are not supported") } - List(SqlToOperationConverter.convert(planner, insert)) + List(SqlToOperationConverter.convert(planner, catalogManager, insert)) case node if node.getKind.belongsTo(SqlKind.QUERY) || node.getKind.belongsTo(SqlKind.DDL) => - List(SqlToOperationConverter.convert(planner, parsed)).asJava + List(SqlToOperationConverter.convert(planner, catalogManager, parsed)) case _ => throw new TableException( "Unsupported SQL query! parse() only accepts SQL queries of type " + @@ -151,13 +151,12 @@ class StreamPlanner( writeToSink(s.getChild, s.getSink, unwrapQueryConfig) case catalogSink: CatalogSinkModifyOperation => - val identifier = catalogManager.qualifyIdentifier(catalogSink.getTablePath: _*) - getTableSink(identifier) + getTableSink(catalogSink.getTableIdentifier) .map(sink => { TableSinkUtils.validateSink( catalogSink.getStaticPartitions, catalogSink.getChild, - identifier, + catalogSink.getTableIdentifier, sink) // set static partitions if it is a partitioned sink sink match { @@ -177,7 +176,8 @@ class StreamPlanner( writeToSink(catalogSink.getChild, sink, unwrapQueryConfig) }) match { case Some(t) => t - case None => throw new TableException(s"Sink ${catalogSink.getTablePath} does not exists") + case None => + throw new TableException(s"Sink ${catalogSink.getTableIdentifier} does not exists") } case outputConversion: OutputConversionModifyOperation => diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java index 79f5470d557a..e5981b700313 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java @@ -128,7 +128,7 @@ public void testCreateTable() { final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); SqlNode node = planner.parse(sql); assert node instanceof SqlCreateTable; - Operation operation = SqlToOperationConverter.convert(planner, node); + Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node); assert operation instanceof CreateTableOperation; CreateTableOperation op = (CreateTableOperation) operation; CatalogTable catalogTable = op.getCatalogTable(); @@ -157,7 +157,7 @@ public void testCreateTableWithMinusInOptionKey() { final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); SqlNode node = planner.parse(sql); assert node instanceof SqlCreateTable; - Operation operation = SqlToOperationConverter.convert(planner, node); + Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node); assert operation instanceof CreateTableOperation; CreateTableOperation op = (CreateTableOperation) operation; CatalogTable catalogTable = op.getCatalogTable(); @@ -191,7 +191,7 @@ public void testCreateTableWithPkUniqueKeys() { final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); SqlNode node = planner.parse(sql); assert node instanceof SqlCreateTable; - SqlToOperationConverter.convert(planner, node); + SqlToOperationConverter.convert(planner, catalogManager, node); } @Test @@ -200,7 +200,7 @@ public void testSqlInsertWithStaticPartition() { FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.HIVE); SqlNode node = planner.parse(sql); assert node instanceof RichSqlInsert; - Operation operation = SqlToOperationConverter.convert(planner, node); + Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node); assert operation instanceof CatalogSinkModifyOperation; CatalogSinkModifyOperation sinkModifyOperation = (CatalogSinkModifyOperation) operation; final Map expectedStaticPartitions = new HashMap<>(); @@ -371,7 +371,7 @@ public void testCreateTableWithFullDataTypes() { final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); SqlNode node = planner.parse(sql); assert node instanceof SqlCreateTable; - Operation operation = SqlToOperationConverter.convert(planner, node); + Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node); TableSchema schema = ((CreateTableOperation) operation).getCatalogTable().getSchema(); Object[] expectedDataTypes = testItems.stream().map(item -> item.expectedType).toArray(); assertArrayEquals(expectedDataTypes, schema.getFieldDataTypes()); @@ -401,7 +401,7 @@ public void testCreateTableWithUnSupportedDataTypes() { assert node instanceof SqlCreateTable; expectedEx.expect(TableException.class); expectedEx.expectMessage(item.expectedError); - SqlToOperationConverter.convert(planner, node); + SqlToOperationConverter.convert(planner, catalogManager, node); } } From 54e69c37befd0d1512ad9848add406e837362d54 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Wed, 16 Oct 2019 16:44:31 +0200 Subject: [PATCH 330/746] [FLINK-14490][table-planner] Extract CalciteParser from FlinkPlannerImpl --- .../table/planner/calcite/CalciteParser.java | 71 +++++++++++++++++++ .../planner/delegation/PlannerContext.java | 10 +++ .../planner/calcite/FlinkPlannerImpl.scala | 39 +--------- .../planner/delegation/PlannerBase.scala | 9 ++- .../sqlexec/SqlToOperationConverterTest.java | 25 +++++-- .../utils/ExpressionTestBase.scala | 3 +- .../match/PatternTranslatorTestBase.scala | 3 +- .../flink/table/calcite/CalciteParser.java | 71 +++++++++++++++++++ .../planner/PlanningConfigurationBuilder.java | 10 +++ .../table/api/internal/TableEnvImpl.scala | 4 +- .../table/calcite/FlinkPlannerImpl.scala | 51 +------------ .../flink/table/planner/StreamPlanner.scala | 6 +- .../sqlexec/SqlToOperationConverterTest.java | 18 +++-- .../utils/ExpressionTestBase.scala | 7 +- 14 files changed, 219 insertions(+), 108 deletions(-) create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/calcite/CalciteParser.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/calcite/CalciteParser.java diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/calcite/CalciteParser.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/calcite/CalciteParser.java new file mode 100644 index 000000000000..f958003050a2 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/calcite/CalciteParser.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.table.planner.calcite; + +import org.apache.flink.table.api.SqlParserException; + +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.parser.SqlParseException; +import org.apache.calcite.sql.parser.SqlParser; + +/** + * Thin wrapper around {@link SqlParser} that does exception conversion and {@link SqlNode} casting. + */ +public class CalciteParser { + private final SqlParser.Config config; + + public CalciteParser(SqlParser.Config config) { + this.config = config; + } + + /** + * Parses a SQL statement into a {@link SqlNode}. The {@link SqlNode} is not yet validated. + * + * @param sql a sql string to parse + * @return a parsed sql node + * @throws SqlParserException if an exception is thrown when parsing the statement + */ + public SqlNode parse(String sql) { + try { + SqlParser parser = SqlParser.create(sql, config); + return parser.parseStmt(); + } catch (SqlParseException e) { + throw new SqlParserException("SQL parse failed. " + e.getMessage()); + } + } + + /** + * Parses a SQL string as an identifier into a {@link SqlIdentifier}. + * + * @param identifier a sql string to parse as an identifier + * @return a parsed sql node + * @throws SqlParserException if an exception is thrown when parsing the identifier + */ + public SqlIdentifier parseIdentifier(String identifier) { + try { + SqlParser parser = SqlParser.create(identifier, config); + SqlNode sqlNode = parser.parseExpression(); + return (SqlIdentifier) sqlNode; + } catch (Exception e) { + throw new SqlParserException(String.format( + "Invalid SQL identifier %s. All SQL keywords must be escaped.", identifier)); + } + } +} diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java index df093e1e2076..43cde6d12a97 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java @@ -27,6 +27,7 @@ import org.apache.flink.table.catalog.FunctionCatalog; import org.apache.flink.table.planner.calcite.CalciteConfig; import org.apache.flink.table.planner.calcite.CalciteConfig$; +import org.apache.flink.table.planner.calcite.CalciteParser; import org.apache.flink.table.planner.calcite.FlinkContextImpl; import org.apache.flink.table.planner.calcite.FlinkPlannerImpl; import org.apache.flink.table.planner.calcite.FlinkRelBuilder; @@ -150,6 +151,15 @@ public FlinkPlannerImpl createFlinkPlanner(String currentCatalog, String current cluster); } + /** + * Creates a configured instance of {@link CalciteParser}. + * + * @return configured calcite parser + */ + public CalciteParser createCalciteParser() { + return new CalciteParser(getSqlParserConfig()); + } + private FlinkCalciteCatalogReader createCatalogReader( boolean lenientCaseSensitivity, String currentCatalog, diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala index f85062fd4e3c..6e6d45ae0ade 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala @@ -19,9 +19,8 @@ package org.apache.flink.table.planner.calcite import org.apache.flink.sql.parser.ExtendedSqlNode -import org.apache.flink.table.api.{SqlParserException, TableException, ValidationException} +import org.apache.flink.table.api.{TableException, ValidationException} -import com.google.common.collect.ImmutableList import org.apache.calcite.config.NullCollation import org.apache.calcite.plan.RelOptTable.ViewExpander import org.apache.calcite.plan._ @@ -29,7 +28,6 @@ import org.apache.calcite.prepare.CalciteCatalogReader import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.{RelFieldCollation, RelRoot} import org.apache.calcite.sql.advise.{SqlAdvisor, SqlAdvisorValidator} -import org.apache.calcite.sql.parser.{SqlParser, SqlParseException => CSqlParseException} import org.apache.calcite.sql.validate.SqlValidator import org.apache.calcite.sql.{SqlKind, SqlNode, SqlOperatorTable} import org.apache.calcite.sql2rel.{RelDecorrelator, SqlRexConvertletTable, SqlToRelConverter} @@ -54,24 +52,13 @@ class FlinkPlannerImpl( cluster: RelOptCluster) { val operatorTable: SqlOperatorTable = config.getOperatorTable - /** Holds the trait definitions to be registered with planner. May be null. */ - val traitDefs: ImmutableList[RelTraitDef[_ <: RelTrait]] = config.getTraitDefs - val parserConfig: SqlParser.Config = config.getParserConfig + val parser: CalciteParser = new CalciteParser(config.getParserConfig) val convertletTable: SqlRexConvertletTable = config.getConvertletTable val sqlToRelConverterConfig: SqlToRelConverter.Config = config.getSqlToRelConverterConfig var validator: FlinkCalciteSqlValidator = _ var root: RelRoot = _ - private def ready() { - if (this.traitDefs != null) { - cluster.getPlanner.clearRelTraitDefs() - for (traitDef <- this.traitDefs) { - cluster.getPlanner.addRelTraitDef(traitDef) - } - } - } - def getCompletionHints(sql: String, cursor: Int): Array[String] = { val advisorValidator = new SqlAdvisorValidator( operatorTable, @@ -109,18 +96,6 @@ class FlinkPlannerImpl( validator } - def parse(sql: String): SqlNode = { - try { - ready() - val parser: SqlParser = SqlParser.create(sql, parserConfig) - val sqlNode: SqlNode = parser.parseStmt - sqlNode - } catch { - case e: CSqlParseException => - throw new SqlParserException(s"SQL parse failed. ${e.getMessage}", e) - } - } - def validate(sqlNode: SqlNode): SqlNode = { val catalogReader = catalogReaderSupplier.apply(false) // do pre-validate rewrite. @@ -179,15 +154,7 @@ class FlinkPlannerImpl( schemaPath: util.List[String], viewPath: util.List[String]): RelRoot = { - val parser: SqlParser = SqlParser.create(queryString, parserConfig) - var sqlNode: SqlNode = null - try { - sqlNode = parser.parseQuery - } - catch { - case e: CSqlParseException => - throw new SqlParserException(s"SQL parse failed. ${e.getMessage}", e) - } + val sqlNode: SqlNode = parser.parse(queryString) val catalogReader: CalciteCatalogReader = catalogReaderSupplier.apply(false) .withSchemaPath(schemaPath) val validator: SqlValidator = diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala index 9725ed819021..1329e3b1a8dc 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala @@ -82,7 +82,8 @@ abstract class PlannerBase( executor.asInstanceOf[ExecutorBase].setTableConfig(config) - private val plannerContext: PlannerContext = + @VisibleForTesting + private[flink] val plannerContext: PlannerContext = new PlannerContext( config, functionCatalog, @@ -122,8 +123,12 @@ abstract class PlannerBase( override def parse(stmt: String): util.List[Operation] = { val planner = createFlinkPlanner + // we do not cache the parser in order to use the most up to + // date configuration. Users might change parser configuration in TableConfig in between + // parsing statements + val parser = plannerContext.createCalciteParser() // parse the sql query - val parsed = planner.parse(stmt) + val parsed = parser.parse(stmt) parsed match { case insert: RichSqlInsert => List(SqlToOperationConverter.convert(planner, catalogManager, insert)) diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java index 868246c3e681..a3414780d423 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java @@ -37,6 +37,7 @@ import org.apache.flink.table.operations.CatalogSinkModifyOperation; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.ddl.CreateTableOperation; +import org.apache.flink.table.planner.calcite.CalciteParser; import org.apache.flink.table.planner.calcite.FlinkPlannerImpl; import org.apache.flink.table.planner.catalog.CatalogManagerCalciteSchema; import org.apache.flink.table.planner.delegation.PlannerContext; @@ -119,7 +120,8 @@ public void testCreateTable() { " 'kafka.topic' = 'log.test'\n" + ")\n"; FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); - Operation operation = parse(sql, planner); + final CalciteParser parser = getParserBySqlDialect(SqlDialect.DEFAULT); + Operation operation = parse(sql, planner, parser); assert operation instanceof CreateTableOperation; CreateTableOperation op = (CreateTableOperation) operation; CatalogTable catalogTable = op.getCatalogTable(); @@ -137,6 +139,7 @@ public void testCreateTable() { @Test(expected = SqlConversionException.class) public void testCreateTableWithPkUniqueKeys() { FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); + final CalciteParser parser = getParserBySqlDialect(SqlDialect.DEFAULT); final String sql = "CREATE TABLE tbl1 (\n" + " a bigint,\n" + " b varchar, \n" + @@ -150,7 +153,7 @@ public void testCreateTableWithPkUniqueKeys() { " 'connector' = 'kafka', \n" + " 'kafka.topic' = 'log.test'\n" + ")\n"; - parse(sql, planner); + parse(sql, planner, parser); } @Test @@ -165,7 +168,8 @@ public void testCreateTableWithMinusInOptionKey() { " 'a.b-c-d.e-f1231.g' = 'ada',\n" + " 'a.b-c-d.*' = 'adad')\n"; final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); - SqlNode node = planner.parse(sql); + final CalciteParser parser = getParserBySqlDialect(SqlDialect.DEFAULT); + SqlNode node = parser.parse(sql); assert node instanceof SqlCreateTable; Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node); assert operation instanceof CreateTableOperation; @@ -187,7 +191,8 @@ public void testCreateTableWithMinusInOptionKey() { public void testSqlInsertWithStaticPartition() { final String sql = "insert into t1 partition(a=1) select b, c, d from t2"; FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.HIVE); - Operation operation = parse(sql, planner); + final CalciteParser parser = getParserBySqlDialect(SqlDialect.HIVE); + Operation operation = parse(sql, planner, parser); assert operation instanceof CatalogSinkModifyOperation; CatalogSinkModifyOperation sinkModifyOperation = (CatalogSinkModifyOperation) operation; final Map expectedStaticPartitions = new HashMap<>(); @@ -323,7 +328,8 @@ public void testCreateTableWithFullDataTypes() { } final String sql = buffer.toString(); final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); - SqlNode node = planner.parse(sql); + final CalciteParser parser = getParserBySqlDialect(SqlDialect.DEFAULT); + SqlNode node = parser.parse(sql); assert node instanceof SqlCreateTable; Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node); TableSchema schema = ((CreateTableOperation) operation).getCatalogTable().getSchema(); @@ -345,8 +351,8 @@ private static TestItem createTestItem(Object... args) { return testItem; } - private Operation parse(String sql, FlinkPlannerImpl planner) { - SqlNode node = planner.parse(sql); + private Operation parse(String sql, FlinkPlannerImpl planner, CalciteParser parser) { + SqlNode node = parser.parse(sql); return SqlToOperationConverter.convert(planner, catalogManager, node); } @@ -356,6 +362,11 @@ private FlinkPlannerImpl getPlannerBySqlDialect(SqlDialect sqlDialect) { catalogManager.getCurrentDatabase()); } + private CalciteParser getParserBySqlDialect(SqlDialect sqlDialect) { + tableConfig.setSqlDialect(sqlDialect); + return plannerContext.createCalciteParser(); + } + //~ Inner Classes ---------------------------------------------------------- private static class TestItem { diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala index 909abcfc6c4d..36a29ae9b7e2 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala @@ -69,6 +69,7 @@ abstract class ExpressionTestBase { private val planner = tEnv.asInstanceOf[TableEnvironmentImpl].getPlanner.asInstanceOf[PlannerBase] private val relBuilder = planner.getRelBuilder private val calcitePlanner = planner.createFlinkPlanner + private val parser = planner.plannerContext.createCalciteParser() // setup test utils private val tableName = "testTable" @@ -179,7 +180,7 @@ abstract class ExpressionTestBase { private def addSqlTestExpr(sqlExpr: String, expected: String): Unit = { // create RelNode from SQL expression - val parsed = calcitePlanner.parse(s"SELECT $sqlExpr FROM $tableName") + val parsed = parser.parse(s"SELECT $sqlExpr FROM $tableName") val validated = calcitePlanner.validate(parsed) val converted = calcitePlanner.rel(validated).rel addTestExpr(converted, expected, sqlExpr) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/match/PatternTranslatorTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/match/PatternTranslatorTestBase.scala index bd5f47beaa28..211ce8930041 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/match/PatternTranslatorTestBase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/match/PatternTranslatorTestBase.scala @@ -55,6 +55,7 @@ abstract class PatternTranslatorTestBase extends TestLogger { private val tableName = "testTable" private val context = prepareContext(testTableTypeInfo) private val calcitePlanner: FlinkPlannerImpl = context._2.createFlinkPlanner + private val parser = context._2.plannerContext.createCalciteParser() private def prepareContext(typeInfo: TypeInformation[Row]) : (RelBuilder, PlannerBase, StreamExecutionEnvironment) = { @@ -80,7 +81,7 @@ abstract class PatternTranslatorTestBase extends TestLogger { def verifyPattern(matchRecognize: String, expected: Pattern[BaseRow, _ <: BaseRow]): Unit = { // create RelNode from SQL expression - val parsed = calcitePlanner.parse( + val parsed = parser.parse( s""" |SELECT * |FROM $tableName diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/calcite/CalciteParser.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/calcite/CalciteParser.java new file mode 100644 index 000000000000..d862267003ad --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/calcite/CalciteParser.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.table.calcite; + +import org.apache.flink.table.api.SqlParserException; + +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.parser.SqlParseException; +import org.apache.calcite.sql.parser.SqlParser; + +/** + * Thin wrapper around {@link SqlParser} that does exception conversion and {@link SqlNode} casting. + */ +public class CalciteParser { + private final SqlParser.Config config; + + public CalciteParser(SqlParser.Config config) { + this.config = config; + } + + /** + * Parses a SQL statement into a {@link SqlNode}. The {@link SqlNode} is not yet validated. + * + * @param sql a sql string to parse + * @return a parsed sql node + * @throws SqlParserException if an exception is thrown when parsing the statement + */ + public SqlNode parse(String sql) { + try { + SqlParser parser = SqlParser.create(sql, config); + return parser.parseStmt(); + } catch (SqlParseException e) { + throw new SqlParserException("SQL parse failed. " + e.getMessage()); + } + } + + /** + * Parses a SQL string as an identifier into a {@link SqlIdentifier}. + * + * @param identifier a sql string to parse as an identifier + * @return a parsed sql node + * @throws SqlParserException if an exception is thrown when parsing the identifier + */ + public SqlIdentifier parseIdentifier(String identifier) { + try { + SqlParser parser = SqlParser.create(identifier, config); + SqlNode sqlNode = parser.parseExpression(); + return (SqlIdentifier) sqlNode; + } catch (Exception e) { + throw new SqlParserException(String.format( + "Invalid SQL identifier %s. All SQL keywords must be escaped.", identifier)); + } + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/PlanningConfigurationBuilder.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/PlanningConfigurationBuilder.java index 1c137d157f80..bccf95dd5c43 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/PlanningConfigurationBuilder.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/PlanningConfigurationBuilder.java @@ -25,6 +25,7 @@ import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.calcite.CalciteConfig; +import org.apache.flink.table.calcite.CalciteParser; import org.apache.flink.table.calcite.FlinkPlannerImpl; import org.apache.flink.table.calcite.FlinkRelBuilder; import org.apache.flink.table.calcite.FlinkRelBuilderFactory; @@ -133,6 +134,15 @@ public FlinkPlannerImpl createFlinkPlanner(String currentCatalog, String current typeFactory); } + /** + * Creates a configured instance of {@link CalciteParser}. + * + * @return configured calcite parser + */ + public CalciteParser createCalciteParser() { + return new CalciteParser(getSqlParserConfig()); + } + /** Returns the Calcite {@link org.apache.calcite.plan.RelOptPlanner} that will be used. */ public RelOptPlanner getPlanner() { return planner; 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 43b37cd34acc..4b532fa80fa9 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 @@ -379,7 +379,7 @@ abstract class TableEnvImpl( override def sqlQuery(query: String): Table = { val planner = getFlinkPlanner // parse the sql query - val parsed = planner.parse(query) + val parsed = planningConfigurationBuilder.createCalciteParser().parse(query) if (null != parsed && parsed.getKind.belongsTo(SqlKind.QUERY)) { // validate the sql query val validated = planner.validate(parsed) @@ -396,7 +396,7 @@ abstract class TableEnvImpl( override def sqlUpdate(stmt: String): Unit = { val planner = getFlinkPlanner // parse the sql query - val parsed = planner.parse(stmt) + val parsed = planningConfigurationBuilder.createCalciteParser().parse(stmt) parsed match { case insert: RichSqlInsert => // validate the insert diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala index 285631645be1..73e9978fadc0 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala @@ -19,10 +19,9 @@ package org.apache.flink.table.calcite import org.apache.flink.sql.parser.ExtendedSqlNode -import org.apache.flink.table.api.{SqlParserException, TableException, ValidationException} +import org.apache.flink.table.api.{TableException, ValidationException} import org.apache.flink.table.catalog.CatalogReader -import com.google.common.collect.ImmutableList import org.apache.calcite.plan.RelOptTable.ViewExpander import org.apache.calcite.plan._ import org.apache.calcite.prepare.CalciteCatalogReader @@ -30,9 +29,7 @@ import org.apache.calcite.rel.RelRoot import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.RelFactories import org.apache.calcite.rex.RexBuilder -import org.apache.calcite.schema.SchemaPlus import org.apache.calcite.sql.advise.{SqlAdvisor, SqlAdvisorValidator} -import org.apache.calcite.sql.parser.{SqlParser, SqlParseException => CSqlParseException} import org.apache.calcite.sql.validate.SqlValidator import org.apache.calcite.sql.{SqlKind, SqlNode, SqlOperatorTable} import org.apache.calcite.sql2rel.{RelDecorrelator, SqlRexConvertletTable, SqlToRelConverter} @@ -57,24 +54,13 @@ class FlinkPlannerImpl( val typeFactory: FlinkTypeFactory) { val operatorTable: SqlOperatorTable = config.getOperatorTable - /** Holds the trait definitions to be registered with planner. May be null. */ - val traitDefs: ImmutableList[RelTraitDef[_ <: RelTrait]] = config.getTraitDefs - val parserConfig: SqlParser.Config = config.getParserConfig val convertletTable: SqlRexConvertletTable = config.getConvertletTable val sqlToRelConverterConfig: SqlToRelConverter.Config = config.getSqlToRelConverterConfig + val parser = new CalciteParser(config) var validator: FlinkCalciteSqlValidator = _ var root: RelRoot = _ - private def ready() { - if (this.traitDefs != null) { - planner.clearRelTraitDefs() - for (traitDef <- this.traitDefs) { - planner.addRelTraitDef(traitDef) - } - } - } - def getCompletionHints(sql: String, cursor: Int): Array[String] = { val advisorValidator = new SqlAdvisorValidator( operatorTable, @@ -111,18 +97,6 @@ class FlinkPlannerImpl( validator } - def parse(sql: String): SqlNode = { - try { - ready() - val parser: SqlParser = SqlParser.create(sql, parserConfig) - val sqlNode: SqlNode = parser.parseStmt - sqlNode - } catch { - case e: CSqlParseException => - throw new SqlParserException(s"SQL parse failed. ${e.getMessage}", e) - } - } - def validate(sqlNode: SqlNode): SqlNode = { val catalogReader = catalogReaderSupplier.apply(false) // do pre-validate rewrite. @@ -185,15 +159,7 @@ class FlinkPlannerImpl( schemaPath: util.List[String], viewPath: util.List[String]): RelRoot = { - val parser: SqlParser = SqlParser.create(queryString, parserConfig) - var sqlNode: SqlNode = null - try { - sqlNode = parser.parseQuery - } - catch { - case e: CSqlParseException => - throw new SqlParserException(s"SQL parse failed. ${e.getMessage}", e) - } + val sqlNode = parser.parse(queryString) val catalogReader: CalciteCatalogReader = catalogReaderSupplier.apply(false) .withSchemaPath(schemaPath) val validator: SqlValidator = @@ -227,14 +193,3 @@ class FlinkPlannerImpl( RelFactories.LOGICAL_BUILDER.create(relOptCluster, relOptSchema) } } - -object FlinkPlannerImpl { - private def rootSchema(schema: SchemaPlus): SchemaPlus = { - if (schema.getParentSchema == null) { - schema - } - else { - rootSchema(schema.getParentSchema) - } - } -} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala index c0821c0b2e4f..f18a80452eba 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala @@ -101,8 +101,10 @@ class StreamPlanner( override def parse(stmt: String): JList[Operation] = { val planner = getFlinkPlanner - // parse the sql query - val parsed = planner.parse(stmt) + // parse the sql query, we do not cache the parser in order to use the most up to + // date configuration. Users might change parser configuration in TableConfig in between + // parsing statements + val parsed = planningConfigurationBuilder.createCalciteParser().parse(stmt) parsed match { case insert: RichSqlInsert => diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java index e5981b700313..266e78a95943 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java @@ -26,6 +26,7 @@ import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.Types; +import org.apache.flink.table.calcite.CalciteParser; import org.apache.flink.table.calcite.FlinkPlannerImpl; import org.apache.flink.table.catalog.Catalog; import org.apache.flink.table.catalog.CatalogManager; @@ -126,7 +127,7 @@ public void testCreateTable() { " 'kafka.topic' = 'log.test'\n" + ")\n"; final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); - SqlNode node = planner.parse(sql); + SqlNode node = getParserBySqlDialect(SqlDialect.DEFAULT).parse(sql); assert node instanceof SqlCreateTable; Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node); assert operation instanceof CreateTableOperation; @@ -155,7 +156,7 @@ public void testCreateTableWithMinusInOptionKey() { " 'a.b-c-d.e-f1231.g' = 'ada',\n" + " 'a.b-c-d.*' = 'adad')\n"; final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); - SqlNode node = planner.parse(sql); + SqlNode node = getParserBySqlDialect(SqlDialect.DEFAULT).parse(sql); assert node instanceof SqlCreateTable; Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node); assert operation instanceof CreateTableOperation; @@ -189,7 +190,7 @@ public void testCreateTableWithPkUniqueKeys() { " 'kafka.topic' = 'log.test'\n" + ")\n"; final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); - SqlNode node = planner.parse(sql); + SqlNode node = getParserBySqlDialect(SqlDialect.DEFAULT).parse(sql); assert node instanceof SqlCreateTable; SqlToOperationConverter.convert(planner, catalogManager, node); } @@ -198,7 +199,7 @@ public void testCreateTableWithPkUniqueKeys() { public void testSqlInsertWithStaticPartition() { final String sql = "insert into t1 partition(a=1) select b, c, d from t2"; FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.HIVE); - SqlNode node = planner.parse(sql); + SqlNode node = getParserBySqlDialect(SqlDialect.HIVE).parse(sql); assert node instanceof RichSqlInsert; Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node); assert operation instanceof CatalogSinkModifyOperation; @@ -369,7 +370,7 @@ public void testCreateTableWithFullDataTypes() { } final String sql = buffer.toString(); final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); - SqlNode node = planner.parse(sql); + SqlNode node = getParserBySqlDialect(SqlDialect.DEFAULT).parse(sql); assert node instanceof SqlCreateTable; Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node); TableSchema schema = ((CreateTableOperation) operation).getCatalogTable().getSchema(); @@ -397,7 +398,7 @@ public void testCreateTableWithUnSupportedDataTypes() { final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); for (TestItem item : testItems) { String sql = String.format(sqlTemplate, item.testExpr); - SqlNode node = planner.parse(sql); + SqlNode node = getParserBySqlDialect(SqlDialect.DEFAULT).parse(sql); assert node instanceof SqlCreateTable; expectedEx.expect(TableException.class); expectedEx.expectMessage(item.expectedError); @@ -419,6 +420,11 @@ private static TestItem createTestItem(Object... args) { return testItem; } + private CalciteParser getParserBySqlDialect(SqlDialect sqlDialect) { + tableConfig.setSqlDialect(sqlDialect); + return planningConfigurationBuilder.createCalciteParser(); + } + private FlinkPlannerImpl getPlannerBySqlDialect(SqlDialect sqlDialect) { tableConfig.setSqlDialect(sqlDialect); return planningConfigurationBuilder.createFlinkPlanner(catalogManager.getCurrentCatalog(), diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala index 081fee6be332..2512fad38ec0 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala @@ -20,7 +20,6 @@ package org.apache.flink.table.expressions.utils import java.util import java.util.concurrent.Future - import org.apache.calcite.plan.hep.{HepMatchOrder, HepPlanner, HepProgramBuilder} import org.apache.calcite.rel.RelNode import org.apache.calcite.rex.RexNode @@ -41,13 +40,14 @@ import org.apache.flink.table.api.scala.BatchTableEnvironment import org.apache.flink.table.api.TableConfig import org.apache.flink.table.api.internal.TableEnvImpl import org.apache.flink.table.api.scala.internal.BatchTableEnvironmentImpl -import org.apache.flink.table.calcite.FlinkRelBuilder +import org.apache.flink.table.calcite.{CalciteParser, FlinkRelBuilder} import org.apache.flink.table.codegen.{Compiler, FunctionCodeGenerator, GeneratedFunction} import org.apache.flink.table.expressions.{Expression, ExpressionParser} import org.apache.flink.table.functions.ScalarFunction import org.apache.flink.table.plan.nodes.dataset.{DataSetCalc, DataSetScan} import org.apache.flink.table.plan.rules.FlinkRuleSets import org.apache.flink.types.Row + import org.junit.Assert._ import org.junit.{After, Before} import org.mockito.Mockito._ @@ -179,7 +179,8 @@ abstract class ExpressionTestBase { private def addSqlTestExpr(sqlExpr: String, expected: String): Unit = { // create RelNode from SQL expression - val parsed = planner.parse(s"SELECT $sqlExpr FROM $tableName") + val parsed = new CalciteParser(config) + .parse(s"SELECT $sqlExpr FROM $tableName") val validated = planner.validate(parsed) val converted = planner.rel(validated).rel From 0fe3fe1fdcc0de2c6f57fb22e906d0d14d9b7823 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Wed, 16 Oct 2019 17:54:00 +0200 Subject: [PATCH 331/746] [FLINK-14490][table-api] Introduce UnresolvedIdentifier --- .../api/internal/TableEnvironmentImpl.java | 15 ++- .../flink/table/catalog/CatalogManager.java | 53 ++------ .../table/catalog/UnresolvedIdentifier.java | 127 ++++++++++++++++++ .../operations/SqlToOperationConverter.java | 10 +- .../table/planner/utils/TableTestBase.scala | 5 +- .../sqlexec/SqlToOperationConverter.java | 10 +- .../table/api/internal/TableEnvImpl.scala | 20 +-- .../table/catalog/PathResolutionTest.java | 3 +- 8 files changed, 180 insertions(+), 63 deletions(-) create mode 100644 flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/UnresolvedIdentifier.java 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 fec7ee78e953..a1d91a02e308 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 @@ -37,6 +37,7 @@ import org.apache.flink.table.catalog.GenericInMemoryCatalog; import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.QueryOperationCatalogView; +import org.apache.flink.table.catalog.UnresolvedIdentifier; import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; import org.apache.flink.table.delegation.Executor; import org.apache.flink.table.delegation.ExecutorFactory; @@ -233,7 +234,7 @@ public Table scan(String... tablePath) { } private Optional scanInternal(String... tablePath) { - ObjectIdentifier objectIdentifier = catalogManager.qualifyIdentifier(tablePath); + ObjectIdentifier objectIdentifier = catalogManager.qualifyIdentifier(UnresolvedIdentifier.of(tablePath)); return catalogManager.getTable(objectIdentifier) .map(t -> new CatalogQueryOperation(objectIdentifier, t.getSchema())); } @@ -333,7 +334,8 @@ public void insertInto(Table table, String path, String... pathContinued) { List fullPath = new ArrayList<>(Arrays.asList(pathContinued)); fullPath.add(0, path); - ObjectIdentifier objectIdentifier = catalogManager.qualifyIdentifier(fullPath.toArray(new String[0])); + ObjectIdentifier objectIdentifier = catalogManager.qualifyIdentifier( + UnresolvedIdentifier.of(fullPath.toArray(new String[0]))); List modifyOperations = Collections.singletonList( new CatalogSinkModifyOperation( objectIdentifier, @@ -452,9 +454,10 @@ private void buffer(List modifyOperations) { private ObjectIdentifier getTemporaryObjectIdentifier(String name) { return catalogManager.qualifyIdentifier( - catalogManager.getBuiltInCatalogName(), - catalogManager.getBuiltInDatabaseName(), - name); + UnresolvedIdentifier.of( + catalogManager.getBuiltInCatalogName(), + catalogManager.getBuiltInDatabaseName(), + name)); } private void registerTableSourceInternal(String name, TableSource tableSource) { @@ -517,7 +520,7 @@ private void registerTableSinkInternal(String name, TableSink tableSink) { } private Optional getCatalogTable(String... name) { - return catalogManager.getTable(catalogManager.qualifyIdentifier(name)); + return catalogManager.getTable(catalogManager.qualifyIdentifier(UnresolvedIdentifier.of(name))); } protected TableImpl createTable(QueryOperation tableOperation) { diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java index 564770978f69..d7340ad8394e 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java @@ -31,7 +31,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Arrays; import java.util.LinkedHashMap; import java.util.Map; import java.util.Optional; @@ -116,7 +115,7 @@ public Set getCatalogs() { * Gets the current catalog that will be used when resolving table path. * * @return the current catalog - * @see CatalogManager#qualifyIdentifier(String...) + * @see CatalogManager#qualifyIdentifier(UnresolvedIdentifier) */ public String getCurrentCatalog() { return currentCatalogName; @@ -127,7 +126,7 @@ public String getCurrentCatalog() { * * @param catalogName catalog name to set as current catalog * @throws CatalogNotExistException thrown if the catalog doesn't exist - * @see CatalogManager#qualifyIdentifier(String...) + * @see CatalogManager#qualifyIdentifier(UnresolvedIdentifier) */ public void setCurrentCatalog(String catalogName) throws CatalogNotExistException { checkArgument(!StringUtils.isNullOrWhitespaceOnly(catalogName), "Catalog name cannot be null or empty."); @@ -152,7 +151,7 @@ public void setCurrentCatalog(String catalogName) throws CatalogNotExistExceptio * Gets the current database name that will be used when resolving table path. * * @return the current database - * @see CatalogManager#qualifyIdentifier(String...) + * @see CatalogManager#qualifyIdentifier(UnresolvedIdentifier) */ public String getCurrentDatabase() { return currentDatabaseName; @@ -164,7 +163,7 @@ public String getCurrentDatabase() { * * @param databaseName database name to set as current database name * @throws CatalogException thrown if the database doesn't exist in the current catalog - * @see CatalogManager#qualifyIdentifier(String...) + * @see CatalogManager#qualifyIdentifier(UnresolvedIdentifier) * @see CatalogManager#setCurrentCatalog(String) */ public void setCurrentDatabase(String databaseName) { @@ -210,7 +209,7 @@ public String getBuiltInDatabaseName() { /** * Retrieves a fully qualified table. If the path is not yet fully qualified use - * {@link #qualifyIdentifier(String...)} first. + * {@link #qualifyIdentifier(UnresolvedIdentifier)} first. * * @param objectIdentifier full path of the table to retrieve * @return table that the path points to. @@ -232,42 +231,16 @@ public Optional getTable(ObjectIdentifier objectIdentifier) { /** * Returns the full name of the given table path, this name may be padded - * with current catalog/database name based on the {@code paths} length. + * with current catalog/database name based on the {@code identifier's} length. * - * @param path Table path whose format can be "catalog.db.table", "db.table" or "table" - * @return An array of complete table path + * @param identifier an unresolved identifier + * @return a fully qualified object identifier */ - public ObjectIdentifier qualifyIdentifier(String... path) { - if (path == null) { - throw new ValidationException("Table paths can not be null!"); - } - if (path.length < 1 || path.length > 3) { - throw new ValidationException("Table paths length must be " + - "between 1(inclusive) and 3(inclusive)"); - } - if (Arrays.stream(path).anyMatch(StringUtils::isNullOrWhitespaceOnly)) { - throw new ValidationException("Table paths contain null or " + - "while-space-only string"); - } - - String catalogName; - String dbName; - String tableName; - if (path.length == 3) { - catalogName = path[0]; - dbName = path[1]; - tableName = path[2]; - } else if (path.length == 2) { - catalogName = getCurrentCatalog(); - dbName = path[0]; - tableName = path[1]; - } else { - catalogName = getCurrentCatalog(); - dbName = getCurrentDatabase(); - tableName = path[0]; - } - - return ObjectIdentifier.of(catalogName, dbName, tableName); + public ObjectIdentifier qualifyIdentifier(UnresolvedIdentifier identifier) { + return ObjectIdentifier.of( + identifier.getCatalogName().orElseGet(this::getCurrentCatalog), + identifier.getDatabaseName().orElseGet(this::getCurrentDatabase), + identifier.getObjectName()); } /** diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/UnresolvedIdentifier.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/UnresolvedIdentifier.java new file mode 100644 index 000000000000..f6e9cc15391f --- /dev/null +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/UnresolvedIdentifier.java @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.utils.EncodingUtils; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StringUtils; + +import javax.annotation.Nullable; + +import java.util.Arrays; +import java.util.Objects; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Identifier of an object, such as table, view, function or type in a catalog. This identifier + * cannot be used directly to access an object in a {@link CatalogManager}, but has to be first + * fully resolved into {@link ObjectIdentifier}. + */ +@Internal +public class UnresolvedIdentifier { + + private final String catalogName; + + private final String databaseName; + + private final String objectName; + + /** + * Constructs an {@link UnresolvedIdentifier} from an array of identifier segments. + * The length of the path must be between 1 (only object name) and 3 (fully qualified + * identifier with catalog, database and object name). + * + * @param path array of identifier segments + * @return an identifier that must be resolved before accessing an object from a {@link CatalogManager} + */ + public static UnresolvedIdentifier of(String... path) { + if (path == null) { + throw new ValidationException("Object identifier can not be null!"); + } + if (path.length < 1 || path.length > 3) { + throw new ValidationException("Object identifier must consist of 1 to 3 parts."); + } + if (Arrays.stream(path).anyMatch(StringUtils::isNullOrWhitespaceOnly)) { + throw new ValidationException("Parts of the object identifier are null or whitespace-only."); + } + + if (path.length == 3) { + return new UnresolvedIdentifier(path[0], path[1], path[2]); + } else if (path.length == 2) { + return new UnresolvedIdentifier(null, path[0], path[1]); + } else { + return new UnresolvedIdentifier(null, null, path[0]); + } + } + + private UnresolvedIdentifier( + @Nullable String catalogName, + @Nullable String databaseName, + String objectName) { + this.catalogName = catalogName; + this.databaseName = databaseName; + this.objectName = Preconditions.checkNotNull(objectName, "Object name must not be null."); + } + + public Optional getCatalogName() { + return Optional.ofNullable(catalogName); + } + + public Optional getDatabaseName() { + return Optional.ofNullable(databaseName); + } + + public String getObjectName() { + return objectName; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + UnresolvedIdentifier that = (UnresolvedIdentifier) o; + return catalogName.equals(that.catalogName) && + databaseName.equals(that.databaseName) && + objectName.equals(that.objectName); + } + + @Override + public int hashCode() { + return Objects.hash(catalogName, databaseName, objectName); + } + + @Override + public String toString() { + return Stream.of( + catalogName, + databaseName, + objectName + ).filter(Objects::nonNull) + .map(EncodingUtils::escapeIdentifier) + .collect(Collectors.joining(".")); + } +} 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 9338fd03f5ef..55ba4713d66b 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 @@ -29,6 +29,7 @@ import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.CatalogTableImpl; import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.UnresolvedIdentifier; import org.apache.flink.table.operations.CatalogSinkModifyOperation; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.ddl.CreateTableOperation; @@ -139,7 +140,8 @@ private Operation convertCreateTable(SqlCreateTable sqlCreateTable) { properties, tableComment); - ObjectIdentifier identifier = catalogManager.qualifyIdentifier(sqlCreateTable.fullTableName()); + UnresolvedIdentifier unresolvedIdentifier = UnresolvedIdentifier.of(sqlCreateTable.fullTableName()); + ObjectIdentifier identifier = catalogManager.qualifyIdentifier(unresolvedIdentifier); return new CreateTableOperation( identifier, @@ -149,7 +151,8 @@ private Operation convertCreateTable(SqlCreateTable sqlCreateTable) { /** Convert DROP TABLE statement. */ private Operation convertDropTable(SqlDropTable sqlDropTable) { - ObjectIdentifier identifier = catalogManager.qualifyIdentifier(sqlDropTable.fullTableName()); + UnresolvedIdentifier unresolvedIdentifier = UnresolvedIdentifier.of(sqlDropTable.fullTableName()); + ObjectIdentifier identifier = catalogManager.qualifyIdentifier(unresolvedIdentifier); return new DropTableOperation(identifier, sqlDropTable.getIfExists()); } @@ -159,7 +162,8 @@ private Operation convertSqlInsert(RichSqlInsert insert) { // get name of sink table List targetTablePath = ((SqlIdentifier) insert.getTargetTable()).names; - ObjectIdentifier identifier = catalogManager.qualifyIdentifier(targetTablePath.toArray(new String[0])); + UnresolvedIdentifier unresolvedIdentifier = UnresolvedIdentifier.of(targetTablePath.toArray(new String[0])); + ObjectIdentifier identifier = catalogManager.qualifyIdentifier(unresolvedIdentifier); return new CatalogSinkModifyOperation( identifier, 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 6ac3bdb5f14e..b8643a3b3178 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 @@ -33,7 +33,7 @@ import org.apache.flink.table.api.java.internal.{StreamTableEnvironmentImpl => J import org.apache.flink.table.api.java.{StreamTableEnvironment => JavaStreamTableEnv} import org.apache.flink.table.api.scala.internal.{StreamTableEnvironmentImpl => ScalaStreamTableEnvImpl} import org.apache.flink.table.api.scala.{StreamTableEnvironment => ScalaStreamTableEnv} -import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog, GenericInMemoryCatalog} +import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog, GenericInMemoryCatalog, UnresolvedIdentifier} import org.apache.flink.table.dataformat.BaseRow import org.apache.flink.table.delegation.{Executor, ExecutorFactory, PlannerFactory} import org.apache.flink.table.expressions.Expression @@ -956,7 +956,8 @@ class TestingTableEnvironment private( override def insertInto(table: Table, path: String, pathContinued: String*): Unit = { val fullPath = List(path) ++ pathContinued.toList - val identifier = catalogManager.qualifyIdentifier(fullPath: _*) + val unresolvedIdentifier = UnresolvedIdentifier.of(fullPath:_*) + val identifier = catalogManager.qualifyIdentifier(unresolvedIdentifier) val modifyOperations = List(new CatalogSinkModifyOperation(identifier, table.getQueryOperation)) if (isEagerOperationTranslation) { 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 e678feb4db2d..cf506f996fa6 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 @@ -31,6 +31,7 @@ import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.CatalogTableImpl; import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.UnresolvedIdentifier; import org.apache.flink.table.operations.CatalogSinkModifyOperation; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.PlannerQueryOperation; @@ -139,7 +140,8 @@ private Operation convertCreateTable(SqlCreateTable sqlCreateTable) { properties, tableComment); - ObjectIdentifier identifier = catalogManager.qualifyIdentifier(sqlCreateTable.fullTableName()); + UnresolvedIdentifier unresolvedIdentifier = UnresolvedIdentifier.of(sqlCreateTable.fullTableName()); + ObjectIdentifier identifier = catalogManager.qualifyIdentifier(unresolvedIdentifier); return new CreateTableOperation( identifier, @@ -149,7 +151,8 @@ private Operation convertCreateTable(SqlCreateTable sqlCreateTable) { /** Convert DROP TABLE statement. */ private Operation convertDropTable(SqlDropTable sqlDropTable) { - ObjectIdentifier identifier = catalogManager.qualifyIdentifier(sqlDropTable.fullTableName()); + UnresolvedIdentifier unresolvedIdentifier = UnresolvedIdentifier.of(sqlDropTable.fullTableName()); + ObjectIdentifier identifier = catalogManager.qualifyIdentifier(unresolvedIdentifier); return new DropTableOperation(identifier, sqlDropTable.getIfExists()); } @@ -164,7 +167,8 @@ private Operation convertSqlInsert(RichSqlInsert insert) { // get name of sink table List targetTablePath = ((SqlIdentifier) insert.getTargetTable()).names; - ObjectIdentifier identifier = catalogManager.qualifyIdentifier(targetTablePath.toArray(new String[0])); + UnresolvedIdentifier unresolvedIdentifier = UnresolvedIdentifier.of(targetTablePath.toArray(new String[0])); + ObjectIdentifier identifier = catalogManager.qualifyIdentifier(unresolvedIdentifier); return new CatalogSinkModifyOperation( identifier, 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 4b532fa80fa9..31a42da326eb 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 @@ -46,8 +46,8 @@ import org.apache.calcite.tools.FrameworkConfig import _root_.java.util.{Optional, HashMap => JHashMap, Map => JMap} -import _root_.scala.collection.JavaConversions._ import _root_.scala.collection.JavaConverters._ +import _root_.scala.collection.JavaConversions._ /** * The abstract base class for the implementation of batch TableEnvironment. @@ -320,10 +320,11 @@ abstract class TableEnvImpl( private def getTemporaryObjectIdentifier(name: String): ObjectIdentifier = { catalogManager.qualifyIdentifier( - catalogManager.getBuiltInCatalogName, - catalogManager.getBuiltInDatabaseName, - name - ) + UnresolvedIdentifier.of( + catalogManager.getBuiltInCatalogName, + catalogManager.getBuiltInDatabaseName, + name + )) } @throws[TableException] @@ -335,7 +336,8 @@ abstract class TableEnvImpl( } private[flink] def scanInternal(tablePath: Array[String]): Option[CatalogQueryOperation] = { - val objectIdentifier = catalogManager.qualifyIdentifier(tablePath: _*) + val unresolvedIdentifier = UnresolvedIdentifier.of(tablePath: _*) + val objectIdentifier = catalogManager.qualifyIdentifier(unresolvedIdentifier) JavaScalaConversionUtil.toScala(catalogManager.getTable(objectIdentifier)) .map(t => new CatalogQueryOperation(objectIdentifier, t.getSchema)) } @@ -477,7 +479,8 @@ abstract class TableEnvImpl( insertOptions: InsertOptions, sinkTablePath: String*): Unit = { - val objectIdentifier = catalogManager.qualifyIdentifier(sinkTablePath: _*) + val unresolvedIdentifier = UnresolvedIdentifier.of(sinkTablePath: _*) + val objectIdentifier = catalogManager.qualifyIdentifier(unresolvedIdentifier) getTableSink(objectIdentifier) match { @@ -540,7 +543,8 @@ abstract class TableEnvImpl( } protected def getCatalogTable(name: String*): Option[CatalogBaseTable] = { - val objectIdentifier = catalogManager.qualifyIdentifier(name: _*) + val unresolvedIdentifier = UnresolvedIdentifier.of(name: _*) + val objectIdentifier = catalogManager.qualifyIdentifier(unresolvedIdentifier) JavaScalaConversionUtil.toScala(catalogManager.getTable(objectIdentifier)) } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/PathResolutionTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/PathResolutionTest.java index 703f99ea2e67..0142ebafa523 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/PathResolutionTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/PathResolutionTest.java @@ -168,7 +168,8 @@ public void testTableApiPathResolution() { testSpec.getDefaultCatalog().ifPresent(catalogManager::setCurrentCatalog); testSpec.getDefaultDatabase().ifPresent(catalogManager::setCurrentDatabase); - ObjectIdentifier identifier = catalogManager.qualifyIdentifier(lookupPath.toArray(new String[0])); + UnresolvedIdentifier unresolvedIdentifier = UnresolvedIdentifier.of(lookupPath.toArray(new String[0])); + ObjectIdentifier identifier = catalogManager.qualifyIdentifier(unresolvedIdentifier); assertThat( Arrays.asList(identifier.getCatalogName(), identifier.getDatabaseName(), identifier.getObjectName()), CoreMatchers.equalTo(testSpec.getExpectedPath())); From 96a62d997eec419646cdc8c8735f9aa940584da4 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Wed, 16 Oct 2019 17:16:45 +0200 Subject: [PATCH 332/746] [FLINK-14490][table-planner] Extract parsing logic from Planner interface --- .../flink-table-api-java-bridge/pom.xml | 8 ++ .../StreamTableEnvironmentImplTest.java | 37 +----- .../api/internal/TableEnvironmentImpl.java | 7 +- .../apache/flink/table/delegation/Parser.java | 57 +++++++++ .../flink/table/delegation/Planner.java | 19 +-- .../apache/flink/table/utils/ParserMock.java | 40 +++++++ .../apache/flink/table/utils/PlannerMock.java | 5 +- .../flink-table-api-scala-bridge/pom.xml | 8 ++ .../StreamTableEnvironmentImplTest.scala | 26 ++--- .../table/planner/delegation/ParserImpl.java | 77 +++++++++++++ .../operations/SqlToOperationConverter.java | 26 +++-- .../planner/delegation/PlannerBase.scala | 41 +++---- .../sqlexec/SqlToOperationConverterTest.java | 6 +- .../table/planner/utils/TableTestBase.scala | 6 +- .../flink/table/planner/ParserImpl.java | 81 +++++++++++++ .../sqlexec/SqlToOperationConverter.java | 31 +++-- .../table/api/internal/TableEnvImpl.scala | 109 +++++++++--------- .../table/calcite/FlinkPlannerImpl.scala | 2 +- .../flink/table/planner/StreamPlanner.scala | 42 +++---- .../sqlexec/SqlToOperationConverterTest.java | 8 +- .../utils/ExpressionTestBase.scala | 2 +- .../match/PatternTranslatorTestBase.scala | 3 +- 22 files changed, 429 insertions(+), 212 deletions(-) create mode 100644 flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/delegation/Parser.java create mode 100644 flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/ParserMock.java create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/ParserImpl.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/ParserImpl.java diff --git a/flink-table/flink-table-api-java-bridge/pom.xml b/flink-table/flink-table-api-java-bridge/pom.xml index ea43eb15e1d7..79e24d057206 100644 --- a/flink-table/flink-table-api-java-bridge/pom.xml +++ b/flink-table/flink-table-api-java-bridge/pom.xml @@ -52,5 +52,13 @@ under the License. flink-streaming-java_${scala.binary.version} ${project.version} + + + org.apache.flink + flink-table-api-java + ${project.version} + test-jar + test + diff --git a/flink-table/flink-table-api-java-bridge/src/test/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImplTest.java b/flink-table/flink-table-api-java-bridge/src/test/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImplTest.java index 6b1724de54ca..1a0cf5a2b308 100644 --- a/flink-table/flink-table-api-java-bridge/src/test/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImplTest.java +++ b/flink-table/flink-table-api-java-bridge/src/test/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImplTest.java @@ -18,7 +18,6 @@ package org.apache.flink.table.api.java.internal; -import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.dag.Transformation; import org.apache.flink.streaming.api.datastream.DataStreamSource; @@ -28,11 +27,10 @@ import org.apache.flink.table.catalog.CatalogManager; import org.apache.flink.table.catalog.FunctionCatalog; import org.apache.flink.table.catalog.GenericInMemoryCatalog; -import org.apache.flink.table.delegation.Executor; -import org.apache.flink.table.delegation.Planner; import org.apache.flink.table.module.ModuleManager; import org.apache.flink.table.operations.ModifyOperation; -import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.utils.ExecutorMock; +import org.apache.flink.table.utils.PlannerMock; import org.apache.flink.types.Row; import org.junit.Test; @@ -101,48 +99,21 @@ private StreamTableEnvironmentImpl getStreamTableEnvironment( new TableConfig(), env, new TestPlanner(elements.getTransformation()), - executor, + new ExecutorMock(), true ); } - private static class TestPlanner implements Planner { + private static class TestPlanner extends PlannerMock { private final Transformation transformation; private TestPlanner(Transformation transformation) { this.transformation = transformation; } - @Override - public List parse(String statement) { - throw new AssertionError("Should not be called"); - } - @Override public List> translate(List modifyOperations) { return Collections.singletonList(transformation); } - - @Override - public String explain(List operations, boolean extended) { - throw new AssertionError("Should not be called"); - } - - @Override - public String[] getCompletionHints(String statement, int position) { - throw new AssertionError("Should not be called"); - } } - - private final Executor executor = new Executor() { - @Override - public void apply(List> transformations) { - - } - - @Override - public JobExecutionResult execute(String jobName) throws Exception { - throw new AssertionError("Should not be called"); - } - }; } 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 a1d91a02e308..d31fbe7cd674 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 @@ -41,6 +41,7 @@ import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; import org.apache.flink.table.delegation.Executor; import org.apache.flink.table.delegation.ExecutorFactory; +import org.apache.flink.table.delegation.Parser; import org.apache.flink.table.delegation.Planner; import org.apache.flink.table.delegation.PlannerFactory; import org.apache.flink.table.descriptors.ConnectTableDescriptor; @@ -93,6 +94,7 @@ public class TableEnvironmentImpl implements TableEnvironment { protected final Executor execEnv; protected final FunctionCatalog functionCatalog; protected final Planner planner; + protected final Parser parser; protected TableEnvironmentImpl( CatalogManager catalogManager, @@ -110,6 +112,7 @@ protected TableEnvironmentImpl( this.functionCatalog = functionCatalog; this.planner = planner; + this.parser = planner.getParser(); this.operationTreeBuilder = OperationTreeBuilder.create( functionCatalog, path -> { @@ -311,7 +314,7 @@ public String[] getCompletionHints(String statement, int position) { @Override public Table sqlQuery(String query) { - List operations = planner.parse(query); + List operations = parser.parse(query); if (operations.size() != 1) { throw new ValidationException( @@ -350,7 +353,7 @@ public void insertInto(Table table, String path, String... pathContinued) { @Override public void sqlUpdate(String stmt) { - List operations = planner.parse(stmt); + List operations = parser.parse(stmt); if (operations.size() != 1) { throw new TableException( diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/delegation/Parser.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/delegation/Parser.java new file mode 100644 index 000000000000..c1df477167ea --- /dev/null +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/delegation/Parser.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.delegation; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.catalog.UnresolvedIdentifier; +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.operations.QueryOperation; + +import java.util.List; + +/** + * Provides methods for parsing SQL objects from a SQL string. + */ +@Internal +public interface Parser { + + /** + * Entry point for parsing SQL queries expressed as a String. + * + *

Note:If the created {@link Operation} is a {@link QueryOperation} + * it must be in a form that will be understood by the + * {@link Planner#translate(List)} method. + * + *

The produced Operation trees should already be validated. + * + * @param statement the SQL statement to evaluate + * @return parsed queries as trees of relational {@link Operation}s + * @throws org.apache.flink.table.api.SqlParserException when failed to parse the statement + */ + List parse(String statement); + + /** + * Entry point for parsing SQL identifiers expressed as a String. + * + * @param identifier the SQL identifier to parse + * @return parsed identifier + * @throws org.apache.flink.table.api.SqlParserException when failed to parse the identifier + */ + UnresolvedIdentifier parseIdentifier(String identifier); +} diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/delegation/Planner.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/delegation/Planner.java index a6d0e8ebb7ce..5bb9266ed115 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/delegation/Planner.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/delegation/Planner.java @@ -29,8 +29,8 @@ /** * This interface serves two purposes: *

    - *
  • SQL parser - transforms a SQL string into a Table API specific tree of - * {@link Operation}s
  • + *
  • SQL parser via {@link #getParser()} - transforms a SQL string into a Table API specific objects + * e.g. tree of {@link Operation}s
  • *
  • relational planner - provides a way to plan, optimize and transform tree of * {@link ModifyOperation} into a runnable form ({@link Transformation})
  • *
. @@ -43,7 +43,7 @@ * of {@link Planner#translate(List)} will strip any execution configuration from * the DataStream information. * - *

All Tables referenced in either {@link Planner#parse(String)} or + *

All Tables referenced in either {@link Parser#parse(String)} or * {@link Planner#translate(List)} should be previously registered in a * {@link org.apache.flink.table.catalog.CatalogManager}, which will be provided during * instantiation of the {@link Planner}. @@ -52,18 +52,11 @@ public interface Planner { /** - * Entry point for parsing sql queries expressed as a String. + * Retrieves a {@link Parser} that provides methods for parsing a SQL string. * - *

Note:If the created {@link Operation} is a {@link QueryOperation} - * it must be in a form that will be understood by the - * {@link Planner#translate(List)} method. - * - *

The produced Operation trees should already be validated. - * - * @param statement the sql statement to evaluate - * @return parsed queries as trees of relational {@link Operation}s + * @return initialized {@link Parser} */ - List parse(String statement); + Parser getParser(); /** * Converts a relational tree of {@link ModifyOperation}s into a set of runnable diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/ParserMock.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/ParserMock.java new file mode 100644 index 000000000000..9dc48de8cd2b --- /dev/null +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/ParserMock.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.table.utils; + +import org.apache.flink.table.catalog.UnresolvedIdentifier; +import org.apache.flink.table.delegation.Parser; +import org.apache.flink.table.operations.Operation; + +import java.util.List; + +/** + * Mocks {@link Parser} for tests. + */ +public class ParserMock implements Parser { + @Override + public List parse(String statement) { + return null; + } + + @Override + public UnresolvedIdentifier parseIdentifier(String identifier) { + return null; + } +} diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/PlannerMock.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/PlannerMock.java index 8dee10c4be61..92f50ff62427 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/PlannerMock.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/PlannerMock.java @@ -19,6 +19,7 @@ package org.apache.flink.table.utils; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.table.delegation.Parser; import org.apache.flink.table.delegation.Planner; import org.apache.flink.table.operations.ModifyOperation; import org.apache.flink.table.operations.Operation; @@ -31,8 +32,8 @@ public class PlannerMock implements Planner { @Override - public List parse(String statement) { - return null; + public Parser getParser() { + return new ParserMock(); } @Override diff --git a/flink-table/flink-table-api-scala-bridge/pom.xml b/flink-table/flink-table-api-scala-bridge/pom.xml index 9a1df4aa7bbe..f0ae087cc3af 100644 --- a/flink-table/flink-table-api-scala-bridge/pom.xml +++ b/flink-table/flink-table-api-scala-bridge/pom.xml @@ -52,6 +52,14 @@ under the License. flink-streaming-scala_${scala.binary.version} ${project.version} + + + org.apache.flink + flink-table-api-java + ${project.version} + test-jar + test + diff --git a/flink-table/flink-table-api-scala-bridge/src/test/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImplTest.scala b/flink-table/flink-table-api-scala-bridge/src/test/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImplTest.scala index cf36ed29db08..e238ddc81718 100644 --- a/flink-table/flink-table-api-scala-bridge/src/test/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImplTest.scala +++ b/flink-table/flink-table-api-scala-bridge/src/test/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImplTest.scala @@ -23,15 +23,16 @@ import org.apache.flink.api.dag.Transformation import org.apache.flink.streaming.api.scala.{StreamExecutionEnvironment, _} import org.apache.flink.table.api.TableConfig import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog, GenericInMemoryCatalog} -import org.apache.flink.table.delegation.{Executor, Planner} -import org.apache.flink.table.operations.{ModifyOperation, Operation} +import org.apache.flink.table.module.ModuleManager +import org.apache.flink.table.operations.ModifyOperation +import org.apache.flink.table.utils.{ExecutorMock, PlannerMock} import org.apache.flink.types.Row + import org.hamcrest.CoreMatchers.equalTo import org.junit.Assert.assertThat import org.junit.Test -import java.util.{Collections, List => JList} -import org.apache.flink.table.module.ModuleManager +import java.util.{Collections, List => JList} /** * Tests for [[StreamTableEnvironmentImpl]]. @@ -91,28 +92,15 @@ class StreamTableEnvironmentImplTest { new TableConfig, env, new TestPlanner(elements.javaStream.getTransformation), - executor, + new ExecutorMock, true) } - private class TestPlanner(transformation: Transformation[_]) extends Planner { - override def parse(statement: String) = throw new AssertionError("Should not be called") - + private class TestPlanner(transformation: Transformation[_]) extends PlannerMock { override def translate(modifyOperations: JList[ModifyOperation]) : JList[Transformation[_]] = { Collections.singletonList(transformation) } - - override def explain(operations: JList[Operation], extended: Boolean) = - throw new AssertionError("Should not be called") - - override def getCompletionHints(statement: String, position: Int) = - throw new AssertionError("Should not be called") } - private val executor = new Executor() { - override def apply(transformations: JList[Transformation[_]]): Unit = {} - - override def execute(jobName: String) = throw new AssertionError("Should not be called") - } } diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/ParserImpl.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/ParserImpl.java new file mode 100644 index 000000000000..878cb761d135 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/ParserImpl.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.delegation; + +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.catalog.CatalogManager; +import org.apache.flink.table.catalog.UnresolvedIdentifier; +import org.apache.flink.table.delegation.Parser; +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.planner.calcite.CalciteParser; +import org.apache.flink.table.planner.calcite.FlinkPlannerImpl; +import org.apache.flink.table.planner.operations.SqlToOperationConverter; + +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlNode; + +import java.util.Collections; +import java.util.List; +import java.util.function.Supplier; + +/** + * Implementation of {@link Parser} that uses Calcite. + */ +public class ParserImpl implements Parser { + + private final CatalogManager catalogManager; + + // we use supplier pattern here in order to use the most up to + // date configuration. Users might change the parser configuration in a TableConfig in between + // multiple statements parsing + private final Supplier validatorSupplier; + private final Supplier calciteParserSupplier; + + public ParserImpl( + CatalogManager catalogManager, + Supplier validatorSupplier, + Supplier calciteParserSupplier) { + this.catalogManager = catalogManager; + this.validatorSupplier = validatorSupplier; + this.calciteParserSupplier = calciteParserSupplier; + } + + @Override + public List parse(String statement) { + CalciteParser parser = calciteParserSupplier.get(); + FlinkPlannerImpl planner = validatorSupplier.get(); + // parse the sql query + SqlNode parsed = parser.parse(statement); + + Operation operation = SqlToOperationConverter.convert(planner, catalogManager, parsed) + .orElseThrow(() -> new TableException("Unsupported query: " + statement)); + return Collections.singletonList(operation); + } + + @Override + public UnresolvedIdentifier parseIdentifier(String identifier) { + CalciteParser parser = calciteParserSupplier.get(); + SqlIdentifier sqlIdentifier = parser.parseIdentifier(identifier); + return UnresolvedIdentifier.of(sqlIdentifier.names.toArray(new String[0])); + } +} 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 55ba4713d66b..596aa492e5a9 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 @@ -48,6 +48,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.stream.Collectors; /** @@ -83,7 +84,7 @@ private SqlToOperationConverter( * @param catalogManager CatalogManager to resolve full path for operations * @param sqlNode SqlNode to execute on */ - public static Operation convert( + public static Optional convert( FlinkPlannerImpl flinkPlanner, CatalogManager catalogManager, SqlNode sqlNode) { @@ -91,16 +92,15 @@ public static Operation convert( final SqlNode validated = flinkPlanner.validate(sqlNode); SqlToOperationConverter converter = new SqlToOperationConverter(flinkPlanner, catalogManager); if (validated instanceof SqlCreateTable) { - return converter.convertCreateTable((SqlCreateTable) validated); + return Optional.of(converter.convertCreateTable((SqlCreateTable) validated)); } else if (validated instanceof SqlDropTable) { - return converter.convertDropTable((SqlDropTable) validated); + return Optional.of(converter.convertDropTable((SqlDropTable) validated)); } else if (validated instanceof RichSqlInsert) { - return converter.convertSqlInsert((RichSqlInsert) validated); + return Optional.of(converter.convertSqlInsert((RichSqlInsert) validated)); } else if (validated.getKind().belongsTo(SqlKind.QUERY)) { - return converter.convertSqlQuery(validated); + return Optional.of(converter.convertSqlQuery(validated)); } else { - throw new TableException("Unsupported node type " - + validated.getClass().getSimpleName()); + return Optional.empty(); } } @@ -165,12 +165,16 @@ private Operation convertSqlInsert(RichSqlInsert insert) { UnresolvedIdentifier unresolvedIdentifier = UnresolvedIdentifier.of(targetTablePath.toArray(new String[0])); ObjectIdentifier identifier = catalogManager.qualifyIdentifier(unresolvedIdentifier); + PlannerQueryOperation query = (PlannerQueryOperation) SqlToOperationConverter.convert( + flinkPlanner, + catalogManager, + insert.getSource()) + .orElseThrow(() -> new TableException( + "Unsupported node type " + insert.getSource().getClass().getSimpleName())); + return new CatalogSinkModifyOperation( identifier, - (PlannerQueryOperation) SqlToOperationConverter.convert( - flinkPlanner, - catalogManager, - insert.getSource()), + query, insert.getStaticPartitionKVs(), insert.isOverwrite()); } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala index 1329e3b1a8dc..0807fe53dec7 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala @@ -21,19 +21,17 @@ package org.apache.flink.table.planner.delegation import org.apache.flink.annotation.VisibleForTesting import org.apache.flink.api.dag.Transformation import org.apache.flink.configuration.Configuration -import org.apache.flink.sql.parser.dml.RichSqlInsert import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import org.apache.flink.table.api.config.ExecutionConfigOptions import org.apache.flink.table.api.{TableConfig, TableEnvironment, TableException} import org.apache.flink.table.catalog._ -import org.apache.flink.table.delegation.{Executor, Planner} +import org.apache.flink.table.delegation.{Executor, Parser, Planner} import org.apache.flink.table.factories.{TableFactoryService, TableFactoryUtil, TableSinkFactory} import org.apache.flink.table.operations.OutputConversionModifyOperation.UpdateMode import org.apache.flink.table.operations._ -import org.apache.flink.table.planner.calcite.{FlinkPlannerImpl, FlinkRelBuilder, FlinkTypeFactory} +import org.apache.flink.table.planner.calcite.{CalciteParser, FlinkPlannerImpl, FlinkRelBuilder, FlinkTypeFactory} import org.apache.flink.table.planner.catalog.CatalogManagerCalciteSchema import org.apache.flink.table.planner.expressions.PlannerTypeInferenceUtilImpl -import org.apache.flink.table.planner.operations.SqlToOperationConverter import org.apache.flink.table.planner.plan.nodes.calcite.LogicalSink import org.apache.flink.table.planner.plan.nodes.exec.ExecNode import org.apache.flink.table.planner.plan.nodes.physical.FlinkPhysicalRel @@ -48,10 +46,10 @@ import org.apache.flink.table.types.utils.LegacyTypeInfoDataTypeConverter import org.apache.calcite.jdbc.CalciteSchemaBuilder.asRootSchema import org.apache.calcite.plan.{RelTrait, RelTraitDef} import org.apache.calcite.rel.RelNode -import org.apache.calcite.sql.SqlKind import org.apache.calcite.tools.FrameworkConfig import java.util +import java.util.function.{Supplier => JSupplier} import _root_.scala.collection.JavaConversions._ @@ -91,6 +89,19 @@ abstract class PlannerBase( getTraitDefs.toList ) + private val parser: Parser = new ParserImpl( + catalogManager, + new JSupplier[FlinkPlannerImpl] { + override def get(): FlinkPlannerImpl = createFlinkPlanner + }, + // we do not cache the parser in order to use the most up to + // date configuration. Users might change parser configuration in TableConfig in between + // parsing statements + new JSupplier[CalciteParser] { + override def get(): CalciteParser = plannerContext.createCalciteParser() + } + ) + /** Returns the [[FlinkRelBuilder]] of this TableEnvironment. */ private[flink] def getRelBuilder: FlinkRelBuilder = { val currentCatalogName = catalogManager.getCurrentCatalog @@ -121,25 +132,7 @@ abstract class PlannerBase( executor.asInstanceOf[ExecutorBase].getExecutionEnvironment } - override def parse(stmt: String): util.List[Operation] = { - val planner = createFlinkPlanner - // we do not cache the parser in order to use the most up to - // date configuration. Users might change parser configuration in TableConfig in between - // parsing statements - val parser = plannerContext.createCalciteParser() - // parse the sql query - val parsed = parser.parse(stmt) - parsed match { - case insert: RichSqlInsert => - List(SqlToOperationConverter.convert(planner, catalogManager, insert)) - case query if query.getKind.belongsTo(SqlKind.QUERY) => - List(SqlToOperationConverter.convert(planner, catalogManager, query)) - case ddl if ddl.getKind.belongsTo(SqlKind.DDL) => - List(SqlToOperationConverter.convert(planner, catalogManager, ddl)) - case _ => - throw new TableException(s"Unsupported query: $stmt") - } - } + override def getParser: Parser = parser override def translate( modifyOperations: util.List[ModifyOperation]): util.List[Transformation[_]] = { diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java index a3414780d423..5c11d89725d9 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java @@ -171,7 +171,7 @@ public void testCreateTableWithMinusInOptionKey() { final CalciteParser parser = getParserBySqlDialect(SqlDialect.DEFAULT); SqlNode node = parser.parse(sql); assert node instanceof SqlCreateTable; - Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node); + Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node).get(); assert operation instanceof CreateTableOperation; CreateTableOperation op = (CreateTableOperation) operation; CatalogTable catalogTable = op.getCatalogTable(); @@ -331,7 +331,7 @@ public void testCreateTableWithFullDataTypes() { final CalciteParser parser = getParserBySqlDialect(SqlDialect.DEFAULT); SqlNode node = parser.parse(sql); assert node instanceof SqlCreateTable; - Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node); + Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node).get(); TableSchema schema = ((CreateTableOperation) operation).getCatalogTable().getSchema(); Object[] expectedDataTypes = testItems.stream().map(item -> item.expectedType).toArray(); assertArrayEquals(expectedDataTypes, schema.getFieldDataTypes()); @@ -353,7 +353,7 @@ private static TestItem createTestItem(Object... args) { private Operation parse(String sql, FlinkPlannerImpl planner, CalciteParser parser) { SqlNode node = parser.parse(sql); - return SqlToOperationConverter.convert(planner, catalogManager, node); + return SqlToOperationConverter.convert(planner, catalogManager, node).get(); } private FlinkPlannerImpl getPlannerBySqlDialect(SqlDialect sqlDialect) { 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 b8643a3b3178..760f41faebae 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 @@ -38,14 +38,14 @@ import org.apache.flink.table.dataformat.BaseRow import org.apache.flink.table.delegation.{Executor, ExecutorFactory, PlannerFactory} import org.apache.flink.table.expressions.Expression import org.apache.flink.table.factories.ComponentFactoryService -import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, TableAggregateFunction, TableFunction, UserDefinedAggregateFunction, UserFunctionsTypeHelper} +import org.apache.flink.table.functions._ import org.apache.flink.table.operations.{CatalogSinkModifyOperation, ModifyOperation, QueryOperation} import org.apache.flink.table.planner.calcite.CalciteConfig import org.apache.flink.table.planner.delegation.PlannerBase import org.apache.flink.table.planner.operations.{DataStreamQueryOperation, PlannerQueryOperation, RichTableSourceQueryOperation} import org.apache.flink.table.planner.plan.nodes.calcite.LogicalWatermarkAssigner import org.apache.flink.table.planner.plan.nodes.exec.ExecNode -import org.apache.flink.table.planner.plan.optimize.program.{BatchOptimizeContext, FlinkBatchProgram, FlinkChainedProgram, FlinkStreamProgram, StreamOptimizeContext} +import org.apache.flink.table.planner.plan.optimize.program._ import org.apache.flink.table.planner.plan.stats.FlinkStatistic import org.apache.flink.table.planner.plan.utils.{ExecNodePlanDumper, FlinkRelOptUtil} import org.apache.flink.table.planner.runtime.utils.{TestingAppendTableSink, TestingRetractTableSink, TestingUpsertTableSink} @@ -968,7 +968,7 @@ class TestingTableEnvironment private( } override def sqlUpdate(stmt: String): Unit = { - val operations = planner.parse(stmt) + val operations = parser.parse(stmt) if (operations.size != 1) { throw new TableException( "Unsupported SQL query! sqlUpdate() only accepts a single SQL statement of type INSERT.") diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/ParserImpl.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/ParserImpl.java new file mode 100644 index 000000000000..92f0670da81b --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/ParserImpl.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.planner; + +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.calcite.CalciteParser; +import org.apache.flink.table.calcite.FlinkPlannerImpl; +import org.apache.flink.table.catalog.CatalogManager; +import org.apache.flink.table.catalog.UnresolvedIdentifier; +import org.apache.flink.table.delegation.Parser; +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.sqlexec.SqlToOperationConverter; + +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlNode; + +import java.util.Collections; +import java.util.List; +import java.util.function.Supplier; + +/** + * Implementation of {@link Parser} that uses Calcite. + */ +public class ParserImpl implements Parser { + + private final CatalogManager catalogManager; + + // we use supplier pattern here in order to use the most up to + // date configuration. Users might change the parser configuration in a TableConfig in between + // multiple statements parsing + private final Supplier validatorSupplier; + private final Supplier calciteParserSupplier; + + public ParserImpl( + CatalogManager catalogManager, + Supplier validatorSupplier, + Supplier calciteParserSupplier) { + this.catalogManager = catalogManager; + this.validatorSupplier = validatorSupplier; + this.calciteParserSupplier = calciteParserSupplier; + } + + @Override + public List parse(String statement) { + CalciteParser parser = calciteParserSupplier.get(); + FlinkPlannerImpl planner = validatorSupplier.get(); + // parse the sql query + SqlNode parsed = parser.parse(statement); + + Operation operation = SqlToOperationConverter.convert(planner, catalogManager, parsed) + .orElseThrow(() -> new TableException( + "Unsupported SQL query! parse() only accepts SQL queries of type " + + "SELECT, UNION, INTERSECT, EXCEPT, VALUES, ORDER_BY or INSERT;" + + "and SQL DDLs of type " + + "CREATE TABLE")); + return Collections.singletonList(operation); + } + + @Override + public UnresolvedIdentifier parseIdentifier(String identifier) { + CalciteParser parser = calciteParserSupplier.get(); + SqlIdentifier sqlIdentifier = parser.parseIdentifier(identifier); + return UnresolvedIdentifier.of(sqlIdentifier.names.toArray(new String[0])); + } +} 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 cf506f996fa6..6c0af99a5e34 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 @@ -25,6 +25,7 @@ import org.apache.flink.sql.parser.dml.RichSqlInsert; import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.calcite.FlinkPlannerImpl; import org.apache.flink.table.calcite.FlinkTypeFactory; import org.apache.flink.table.catalog.CatalogManager; @@ -49,6 +50,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.stream.Collectors; /** @@ -83,7 +85,7 @@ private SqlToOperationConverter( * @param flinkPlanner FlinkPlannerImpl to convert sql node to rel node * @param sqlNode SqlNode to execute on */ - public static Operation convert( + public static Optional convert( FlinkPlannerImpl flinkPlanner, CatalogManager catalogManager, SqlNode sqlNode) { @@ -91,16 +93,19 @@ public static Operation convert( final SqlNode validated = flinkPlanner.validate(sqlNode); SqlToOperationConverter converter = new SqlToOperationConverter(flinkPlanner, catalogManager); if (validated instanceof SqlCreateTable) { - return converter.convertCreateTable((SqlCreateTable) validated); + return Optional.of(converter.convertCreateTable((SqlCreateTable) validated)); } if (validated instanceof SqlDropTable) { - return converter.convertDropTable((SqlDropTable) validated); + return Optional.of(converter.convertDropTable((SqlDropTable) validated)); } else if (validated instanceof RichSqlInsert) { - return converter.convertSqlInsert((RichSqlInsert) validated); + SqlNodeList targetColumnList = ((RichSqlInsert) validated).getTargetColumnList(); + if (targetColumnList != null && targetColumnList.size() != 0) { + throw new ValidationException("Partial inserts are not supported"); + } + return Optional.of(converter.convertSqlInsert((RichSqlInsert) validated)); } else if (validated.getKind().belongsTo(SqlKind.QUERY)) { - return converter.convertSqlQuery(validated); + return Optional.of(converter.convertSqlQuery(validated)); } else { - throw new TableException("Unsupported node type " - + validated.getClass().getSimpleName()); + return Optional.empty(); } } @@ -170,12 +175,16 @@ private Operation convertSqlInsert(RichSqlInsert insert) { UnresolvedIdentifier unresolvedIdentifier = UnresolvedIdentifier.of(targetTablePath.toArray(new String[0])); ObjectIdentifier identifier = catalogManager.qualifyIdentifier(unresolvedIdentifier); + PlannerQueryOperation query = (PlannerQueryOperation) SqlToOperationConverter.convert( + flinkPlanner, + catalogManager, + insert.getSource()) + .orElseThrow(() -> new TableException( + "Unsupported node type " + insert.getSource().getClass().getSimpleName())); + return new CatalogSinkModifyOperation( identifier, - (PlannerQueryOperation) SqlToOperationConverter.convert( - flinkPlanner, - catalogManager, - insert.getSource()), + query, insert.getStaticPartitionKVs(), insert.isOverwrite()); } 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 31a42da326eb..f42d77f977d4 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 @@ -20,11 +20,10 @@ package org.apache.flink.table.api.internal import org.apache.flink.annotation.VisibleForTesting import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.sql.parser.ddl.{SqlCreateTable, SqlDropTable} -import org.apache.flink.sql.parser.dml.RichSqlInsert import org.apache.flink.table.api._ -import org.apache.flink.table.calcite.{FlinkPlannerImpl, FlinkRelBuilder} +import org.apache.flink.table.calcite.{CalciteParser, FlinkPlannerImpl, FlinkRelBuilder} import org.apache.flink.table.catalog._ +import org.apache.flink.table.delegation.Parser import org.apache.flink.table.expressions._ import org.apache.flink.table.expressions.resolver.lookups.TableReferenceLookup import org.apache.flink.table.factories.{TableFactoryService, TableFactoryUtil, TableSinkFactory} @@ -32,18 +31,17 @@ import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, Tabl import org.apache.flink.table.module.{Module, ModuleManager} import org.apache.flink.table.operations.ddl.{CreateTableOperation, DropTableOperation} import org.apache.flink.table.operations.utils.OperationTreeBuilder -import org.apache.flink.table.operations.{CatalogQueryOperation, PlannerQueryOperation, TableSourceQueryOperation, _} -import org.apache.flink.table.planner.PlanningConfigurationBuilder +import org.apache.flink.table.operations.{CatalogQueryOperation, TableSourceQueryOperation, _} +import org.apache.flink.table.planner.{ParserImpl, PlanningConfigurationBuilder} import org.apache.flink.table.sinks.{OverwritableTableSink, PartitionableTableSink, TableSink, TableSinkUtils} import org.apache.flink.table.sources.TableSource -import org.apache.flink.table.sqlexec.SqlToOperationConverter import org.apache.flink.table.util.JavaScalaConversionUtil import org.apache.calcite.jdbc.CalciteSchemaBuilder.asRootSchema -import org.apache.calcite.sql._ import org.apache.calcite.sql.parser.SqlParser import org.apache.calcite.tools.FrameworkConfig +import _root_.java.util.function.{Supplier => JSupplier} import _root_.java.util.{Optional, HashMap => JHashMap, Map => JMap} import _root_.scala.collection.JavaConverters._ @@ -92,6 +90,19 @@ abstract class TableEnvImpl( asRootSchema(new CatalogManagerCalciteSchema(catalogManager, isStreamingMode)), expressionBridge) + private val parser: Parser = new ParserImpl( + catalogManager, + new JSupplier[FlinkPlannerImpl] { + override def get(): FlinkPlannerImpl = getFlinkPlanner + }, + // we do not cache the parser in order to use the most up to + // date configuration. Users might change parser configuration in TableConfig in between + // parsing statements + new JSupplier[CalciteParser] { + override def get(): CalciteParser = planningConfigurationBuilder.createCalciteParser() + } + ) + def getConfig: TableConfig = config private def isStreamingMode: Boolean = this match { @@ -379,63 +390,47 @@ abstract class TableEnvImpl( } override def sqlQuery(query: String): Table = { - val planner = getFlinkPlanner - // parse the sql query - val parsed = planningConfigurationBuilder.createCalciteParser().parse(query) - if (null != parsed && parsed.getKind.belongsTo(SqlKind.QUERY)) { - // validate the sql query - val validated = planner.validate(parsed) - // transform to a relational tree - val relational = planner.rel(validated) - createTable(new PlannerQueryOperation(relational.rel)) - } else { - throw new TableException( - "Unsupported SQL query! sqlQuery() only accepts SQL queries of type " + + val operations = parser.parse(query) + + if (operations.size != 1) throw new ValidationException( + "Unsupported SQL query! sqlQuery() only accepts a single SQL query.") + + operations.get(0) match { + case op: QueryOperation if !op.isInstanceOf[ModifyOperation] => + createTable(op) + case _ => throw new ValidationException( + "Unsupported SQL query! sqlQuery() only accepts a single SQL query of type " + "SELECT, UNION, INTERSECT, EXCEPT, VALUES, and ORDER_BY.") } } override def sqlUpdate(stmt: String): Unit = { - val planner = getFlinkPlanner - // parse the sql query - val parsed = planningConfigurationBuilder.createCalciteParser().parse(stmt) - parsed match { - case insert: RichSqlInsert => - // validate the insert - val validatedInsert = planner.validate(insert).asInstanceOf[RichSqlInsert] - // we do not validate the row type for sql insert now, so validate the source - // separately. - val validatedQuery = planner.validate(validatedInsert.getSource) - - val tableOperation = new PlannerQueryOperation(planner.rel(validatedQuery).rel) - // get query result as Table - val queryResult = createTable(tableOperation) - - // get name of sink table - val targetTablePath = insert.getTargetTable.asInstanceOf[SqlIdentifier].names - - // insert query result into sink table - insertInto(queryResult, InsertOptions(insert.getStaticPartitionKVs, insert.isOverwrite), - targetTablePath.asScala:_*) - case createTable: SqlCreateTable => - val operation = SqlToOperationConverter - .convert(planner, catalogManager, createTable) - .asInstanceOf[CreateTableOperation] + val operations = parser.parse(stmt) + + if (operations.size != 1) throw new TableException( + "Unsupported SQL query! sqlUpdate() only accepts a single SQL statement of type " + + "INSERT, CREATE TABLE, DROP TABLE") + + operations.get(0) match { + case op: CatalogSinkModifyOperation => + insertInto( + createTable(op.getChild), + InsertOptions(op.getStaticPartitions, op.isOverwrite), + op.getTableIdentifier.getCatalogName, + op.getTableIdentifier.getDatabaseName, + op.getTableIdentifier.getObjectName) + case createTableOperation: CreateTableOperation => catalogManager.createTable( - operation.getCatalogTable, - operation.getTableIdentifier, - operation.isIgnoreIfExists) - case dropTable: SqlDropTable => - val operation = SqlToOperationConverter - .convert(planner, catalogManager, dropTable) - .asInstanceOf[DropTableOperation] + createTableOperation.getCatalogTable, + createTableOperation.getTableIdentifier, + createTableOperation.isIgnoreIfExists) + case dropTableOperation: DropTableOperation => catalogManager.dropTable( - operation.getTableIdentifier, - operation.isIfExists) - case _ => - throw new TableException( - "Unsupported SQL query! sqlUpdate() only accepts SQL statements of " + - "type INSERT, CREATE TABLE, DROP TABLE.") + dropTableOperation.getTableIdentifier, + dropTableOperation.isIfExists) + case _ => throw new TableException( + "Unsupported SQL query! sqlUpdate() only accepts a single SQL statements of " + + "type INSERT, CREATE TABLE, DROP TABLE") } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala index 73e9978fadc0..8ce6569582e8 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala @@ -56,7 +56,7 @@ class FlinkPlannerImpl( val operatorTable: SqlOperatorTable = config.getOperatorTable val convertletTable: SqlRexConvertletTable = config.getConvertletTable val sqlToRelConverterConfig: SqlToRelConverter.Config = config.getSqlToRelConverterConfig - val parser = new CalciteParser(config) + val parser = new CalciteParser(config.getParserConfig) var validator: FlinkCalciteSqlValidator = _ var root: RelRoot = _ diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala index f18a80452eba..0b2243194e2a 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala @@ -21,13 +21,12 @@ import org.apache.flink.annotation.VisibleForTesting import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.dag.Transformation import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2} -import org.apache.flink.sql.parser.dml.RichSqlInsert import org.apache.flink.streaming.api.datastream.{DataStream, DataStreamSink} import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import org.apache.flink.table.api._ -import org.apache.flink.table.calcite.{CalciteConfig, FlinkPlannerImpl, FlinkRelBuilder, FlinkTypeFactory} +import org.apache.flink.table.calcite._ import org.apache.flink.table.catalog.{CatalogManager, CatalogManagerCalciteSchema, CatalogTable, ConnectorCatalogTable, _} -import org.apache.flink.table.delegation.{Executor, Planner} +import org.apache.flink.table.delegation.{Executor, Parser, Planner} import org.apache.flink.table.executor.StreamExecutor import org.apache.flink.table.explain.PlanJsonParser import org.apache.flink.table.expressions.{ExpressionBridge, PlannerExpression, PlannerExpressionConverter, PlannerTypeInferenceUtilImpl} @@ -39,7 +38,6 @@ import org.apache.flink.table.plan.nodes.datastream.DataStreamRel import org.apache.flink.table.plan.util.UpdatingPlanChecker import org.apache.flink.table.runtime.types.CRow import org.apache.flink.table.sinks._ -import org.apache.flink.table.sqlexec.SqlToOperationConverter import org.apache.flink.table.types.utils.TypeConversions import org.apache.flink.table.util.JavaScalaConversionUtil @@ -47,11 +45,11 @@ import org.apache.calcite.jdbc.CalciteSchema import org.apache.calcite.jdbc.CalciteSchemaBuilder.asRootSchema import org.apache.calcite.plan.RelOptUtil import org.apache.calcite.rel.RelNode -import org.apache.calcite.sql.SqlKind import _root_.java.lang.{Boolean => JBool} import _root_.java.util -import _root_.java.util.{Objects, List => JList} +import _root_.java.util.Objects +import _root_.java.util.function.{Supplier => JSupplier} import _root_.scala.collection.JavaConversions._ import _root_.scala.collection.JavaConverters._ @@ -99,30 +97,20 @@ class StreamPlanner( .orElse(CalciteConfig.DEFAULT), planningConfigurationBuilder) - override def parse(stmt: String): JList[Operation] = { - val planner = getFlinkPlanner - // parse the sql query, we do not cache the parser in order to use the most up to + private val parser: Parser = new ParserImpl( + catalogManager, + // we do not cache the parser in order to use the most up to // date configuration. Users might change parser configuration in TableConfig in between // parsing statements - val parsed = planningConfigurationBuilder.createCalciteParser().parse(stmt) - - parsed match { - case insert: RichSqlInsert => - val targetColumnList = insert.getTargetColumnList - if (targetColumnList != null && insert.getTargetColumnList.size() != 0) { - throw new ValidationException("Partial inserts are not supported") - } - List(SqlToOperationConverter.convert(planner, catalogManager, insert)) - case node if node.getKind.belongsTo(SqlKind.QUERY) || node.getKind.belongsTo(SqlKind.DDL) => - List(SqlToOperationConverter.convert(planner, catalogManager, parsed)) - case _ => - throw new TableException( - "Unsupported SQL query! parse() only accepts SQL queries of type " + - "SELECT, UNION, INTERSECT, EXCEPT, VALUES, ORDER_BY or INSERT;" + - "and SQL DDLs of type " + - "CREATE TABLE") + new JSupplier[FlinkPlannerImpl] { + override def get(): FlinkPlannerImpl = getFlinkPlanner + }, + new JSupplier[CalciteParser] { + override def get(): CalciteParser = planningConfigurationBuilder.createCalciteParser() } - } + ) + + override def getParser: Parser = parser override def translate(tableOperations: util.List[ModifyOperation]) : util.List[Transformation[_]] = { diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java index 266e78a95943..21a399f2781c 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java @@ -129,7 +129,7 @@ public void testCreateTable() { final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); SqlNode node = getParserBySqlDialect(SqlDialect.DEFAULT).parse(sql); assert node instanceof SqlCreateTable; - Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node); + Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node).get(); assert operation instanceof CreateTableOperation; CreateTableOperation op = (CreateTableOperation) operation; CatalogTable catalogTable = op.getCatalogTable(); @@ -158,7 +158,7 @@ public void testCreateTableWithMinusInOptionKey() { final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); SqlNode node = getParserBySqlDialect(SqlDialect.DEFAULT).parse(sql); assert node instanceof SqlCreateTable; - Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node); + Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node).get(); assert operation instanceof CreateTableOperation; CreateTableOperation op = (CreateTableOperation) operation; CatalogTable catalogTable = op.getCatalogTable(); @@ -201,7 +201,7 @@ public void testSqlInsertWithStaticPartition() { FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.HIVE); SqlNode node = getParserBySqlDialect(SqlDialect.HIVE).parse(sql); assert node instanceof RichSqlInsert; - Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node); + Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node).get(); assert operation instanceof CatalogSinkModifyOperation; CatalogSinkModifyOperation sinkModifyOperation = (CatalogSinkModifyOperation) operation; final Map expectedStaticPartitions = new HashMap<>(); @@ -372,7 +372,7 @@ public void testCreateTableWithFullDataTypes() { final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); SqlNode node = getParserBySqlDialect(SqlDialect.DEFAULT).parse(sql); assert node instanceof SqlCreateTable; - Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node); + Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node).get(); TableSchema schema = ((CreateTableOperation) operation).getCatalogTable().getSchema(); Object[] expectedDataTypes = testItems.stream().map(item -> item.expectedType).toArray(); assertArrayEquals(expectedDataTypes, schema.getFieldDataTypes()); diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala index 2512fad38ec0..180c83321f4b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala @@ -179,7 +179,7 @@ abstract class ExpressionTestBase { private def addSqlTestExpr(sqlExpr: String, expected: String): Unit = { // create RelNode from SQL expression - val parsed = new CalciteParser(config) + val parsed = new CalciteParser(context._2.getParserConfig) .parse(s"SELECT $sqlExpr FROM $tableName") val validated = planner.validate(parsed) val converted = planner.rel(validated).rel diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/match/PatternTranslatorTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/match/PatternTranslatorTestBase.scala index cb7d85a935f1..6cb549f1b112 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/match/PatternTranslatorTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/match/PatternTranslatorTestBase.scala @@ -31,6 +31,7 @@ import org.apache.flink.table.plan.nodes.datastream.{DataStreamMatch, DataStream import org.apache.flink.table.planner.StreamPlanner import org.apache.flink.types.Row import org.apache.flink.util.TestLogger + import org.junit.Assert._ import org.junit.rules.ExpectedException import org.junit.{ComparisonFailure, Rule} @@ -68,7 +69,7 @@ abstract class PatternTranslatorTestBase extends TestLogger{ def verifyPattern(matchRecognize: String, expected: Pattern[Row, _ <: Row]): Unit = { // create RelNode from SQL expression - val parsed = context._3.parse( + val parsed = context._3.getParser.parse( s""" |SELECT * |FROM $tableName From c0f4b7fad10ac49c72c3a70c3550c6a490f4bcc9 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Wed, 16 Oct 2019 18:11:41 +0200 Subject: [PATCH 333/746] [FLINK-14490][table] Introduced temporary tables to CatalogManager --- .../gateway/local/LocalExecutorITCase.java | 8 +- .../flink/table/api/TableEnvironment.java | 25 +- .../api/internal/TableEnvironmentImpl.java | 59 ++-- .../flink/table/catalog/CatalogManager.java | 255 ++++++++++++++++-- .../flink/table/api/TableEnvironmentTest.java | 13 +- .../planner/catalog/CatalogCalciteSchema.java | 14 +- .../catalog/CatalogManagerCalciteSchema.java | 10 +- .../catalog/DatabaseCalciteSchema.java | 201 ++++++++------ .../planner/delegation/PlannerBase.scala | 11 +- .../catalog/CatalogStatisticsTest.java | 11 +- .../table/api/TableEnvironmentTest.scala | 4 +- .../table/catalog/CatalogCalciteSchema.java | 13 +- .../catalog/CatalogManagerCalciteSchema.java | 10 +- .../table/catalog/DatabaseCalciteSchema.java | 88 +++--- .../table/api/internal/TableEnvImpl.scala | 56 ++-- .../flink/table/planner/StreamPlanner.scala | 3 +- .../table/catalog/CatalogManagerTest.java | 148 +++++++++- .../catalog/CatalogStructureBuilder.java | 137 ++++++++-- .../catalog/DatabaseCalciteSchemaTest.java | 3 +- .../table/catalog/PathResolutionTest.java | 48 +++- .../table/utils/MockTableEnvironment.scala | 4 + 21 files changed, 862 insertions(+), 259 deletions(-) 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 24799e047838..f4c80aa6c2aa 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 @@ -145,13 +145,13 @@ public void testValidateSession() throws Exception { List actualTables = executor.listTables(session); List expectedTables = Arrays.asList( + "AdditionalView1", + "AdditionalView2", "TableNumber1", "TableNumber2", "TableSourceSink", "TestView1", - "TestView2", - "AdditionalView1", - "AdditionalView2"); + "TestView2"); assertEquals(expectedTables, actualTables); session.removeView("AdditionalView1"); @@ -183,8 +183,8 @@ public void testListCatalogs() throws Exception { final List actualCatalogs = executor.listCatalogs(session); final List expectedCatalogs = Arrays.asList( - "default_catalog", "catalog1", + "default_catalog", "simple-catalog"); assertEquals(expectedCatalogs, actualCatalogs); } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java index 344e7ffe796d..5b5c14657351 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java @@ -272,12 +272,35 @@ static TableEnvironment create(EnvironmentSettings settings) { String[] listDatabases(); /** - * Gets the names of all tables registered in the current database of the current catalog. + * Gets the names of all tables available in the current namespace (the current database of the current catalog). + * It returns both temporary and permanent tables and views. * * @return A list of the names of all registered tables in the current database of the current catalog. + * @see #listTemporaryTables() + * @see #listTemporaryViews() */ String[] listTables(); + /** + * Gets the names of all temporary tables and views available in the current namespace (the current + * database of the current catalog). + * + * @return A list of the names of all registered temporary tables and views in the current database + * of the current catalog. + * @see #listTables() + */ + String[] listTemporaryTables(); + + /** + * Gets the names of all temporary views available in the current namespace (the current + * database of the current catalog). + * + * @return A list of the names of all registered temporary views in the current database + * of the current catalog. + * @see #listTables() + */ + String[] listTemporaryViews(); + /** * Gets the names of all user defined functions registered in this environment. */ 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 d31fbe7cd674..a2d7bf4fb270 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 @@ -38,7 +38,6 @@ import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.QueryOperationCatalogView; import org.apache.flink.table.catalog.UnresolvedIdentifier; -import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; import org.apache.flink.table.delegation.Executor; import org.apache.flink.table.delegation.ExecutorFactory; import org.apache.flink.table.delegation.Parser; @@ -199,7 +198,7 @@ public void registerTable(String name, Table table) { } CatalogBaseTable tableTable = new QueryOperationCatalogView(table.getQueryOperation()); - catalogManager.createTable(tableTable, getTemporaryObjectIdentifier(name), false); + catalogManager.createTemporaryTable(tableTable, getTemporaryObjectIdentifier(name), false); } @Override @@ -239,7 +238,7 @@ public Table scan(String... tablePath) { private Optional scanInternal(String... tablePath) { ObjectIdentifier objectIdentifier = catalogManager.qualifyIdentifier(UnresolvedIdentifier.of(tablePath)); return catalogManager.getTable(objectIdentifier) - .map(t -> new CatalogQueryOperation(objectIdentifier, t.getSchema())); + .map(t -> new CatalogQueryOperation(objectIdentifier, t.getTable().getSchema())); } @Override @@ -249,7 +248,10 @@ public ConnectTableDescriptor connect(ConnectorDescriptor connectorDescriptor) { @Override public String[] listCatalogs() { - return catalogManager.getCatalogs().toArray(new String[0]); + return catalogManager.listCatalogs() + .stream() + .sorted() + .toArray(String[]::new); } @Override @@ -267,17 +269,26 @@ public String[] listDatabases() { @Override public String[] listTables() { - String currentCatalogName = catalogManager.getCurrentCatalog(); - Optional currentCatalog = catalogManager.getCatalog(currentCatalogName); - - return currentCatalog.map(catalog -> { - try { - return catalog.listTables(catalogManager.getCurrentDatabase()).toArray(new String[0]); - } catch (DatabaseNotExistException e) { - throw new ValidationException("Current database does not exist", e); - } - }).orElseThrow(() -> - new TableException(String.format("The current catalog %s does not exist.", currentCatalogName))); + return catalogManager.listTables() + .stream() + .sorted() + .toArray(String[]::new); + } + + @Override + public String[] listTemporaryTables() { + return catalogManager.listTemporaryTables() + .stream() + .sorted() + .toArray(String[]::new); + } + + @Override + public String[] listTemporaryViews() { + return catalogManager.listTemporaryViews() + .stream() + .sorted() + .toArray(String[]::new); } @Override @@ -465,7 +476,7 @@ private ObjectIdentifier getTemporaryObjectIdentifier(String name) { private void registerTableSourceInternal(String name, TableSource tableSource) { validateTableSource(tableSource); - Optional table = getCatalogTable( + Optional table = getTemporaryTable( catalogManager.getBuiltInCatalogName(), catalogManager.getBuiltInDatabaseName(), name); @@ -482,7 +493,7 @@ private void registerTableSourceInternal(String name, TableSource tableSource tableSource, sourceSinkTable.getTableSink().get(), !IS_STREAM_TABLE); - catalogManager.alterTable(sourceAndSink, getTemporaryObjectIdentifier(name), false); + catalogManager.createTemporaryTable(sourceAndSink, getTemporaryObjectIdentifier(name), true); } } else { throw new ValidationException(String.format( @@ -490,12 +501,12 @@ private void registerTableSourceInternal(String name, TableSource tableSource } } else { ConnectorCatalogTable source = ConnectorCatalogTable.source(tableSource, !IS_STREAM_TABLE); - catalogManager.createTable(source, getTemporaryObjectIdentifier(name), false); + catalogManager.createTemporaryTable(source, getTemporaryObjectIdentifier(name), false); } } private void registerTableSinkInternal(String name, TableSink tableSink) { - Optional table = getCatalogTable( + Optional table = getTemporaryTable( catalogManager.getBuiltInCatalogName(), catalogManager.getBuiltInDatabaseName(), name); @@ -510,7 +521,7 @@ private void registerTableSinkInternal(String name, TableSink tableSink) { // wrapper contains only sink (not source) ConnectorCatalogTable sourceAndSink = ConnectorCatalogTable .sourceAndSink(sourceSinkTable.getTableSource().get(), tableSink, !IS_STREAM_TABLE); - catalogManager.alterTable(sourceAndSink, getTemporaryObjectIdentifier(name), false); + catalogManager.createTemporaryTable(sourceAndSink, getTemporaryObjectIdentifier(name), true); } } else { throw new ValidationException(String.format( @@ -518,12 +529,14 @@ private void registerTableSinkInternal(String name, TableSink tableSink) { } } else { ConnectorCatalogTable sink = ConnectorCatalogTable.sink(tableSink, !IS_STREAM_TABLE); - catalogManager.createTable(sink, getTemporaryObjectIdentifier(name), false); + catalogManager.createTemporaryTable(sink, getTemporaryObjectIdentifier(name), false); } } - private Optional getCatalogTable(String... name) { - return catalogManager.getTable(catalogManager.qualifyIdentifier(UnresolvedIdentifier.of(name))); + private Optional getTemporaryTable(String... name) { + return catalogManager.getTable(catalogManager.qualifyIdentifier(UnresolvedIdentifier.of(name))) + .filter(CatalogManager.TableLookupResult::isTemporary) + .map(CatalogManager.TableLookupResult::getTable); } protected TableImpl createTable(QueryOperation tableOperation) { diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java index d7340ad8394e..81d0ad9079ec 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java @@ -31,10 +31,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Collections; +import java.util.HashMap; import java.util.LinkedHashMap; import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; import static java.lang.String.format; import static org.apache.flink.util.Preconditions.checkArgument; @@ -51,6 +55,10 @@ public class CatalogManager { // A map between names and catalogs. private Map catalogs; + // Those tables take precedence over corresponding permanent tables, thus they shadow + // tables coming from catalogs. + private Map temporaryTables; + // The name of the current catalog and database private String currentCatalogName; @@ -69,6 +77,7 @@ public CatalogManager(String defaultCatalogName, Catalog defaultCatalog) { this.currentCatalogName = defaultCatalogName; this.currentDatabaseName = defaultCatalog.getDefaultDatabase(); + this.temporaryTables = new HashMap<>(); // right now the default catalog is always the built-in one this.builtInCatalogName = defaultCatalogName; } @@ -102,15 +111,6 @@ public Optional getCatalog(String catalogName) { return Optional.ofNullable(catalogs.get(catalogName)); } - /** - * Retrieves names of all registered catalogs. - * - * @return a set of names of registered catalogs - */ - public Set getCatalogs() { - return catalogs.keySet(); - } - /** * Gets the current catalog that will be used when resolving table path. * @@ -207,6 +207,37 @@ public String getBuiltInDatabaseName() { return catalogs.get(getBuiltInCatalogName()).getDefaultDatabase(); } + /** + * Result of a lookup for a table through {@link #getTable(ObjectIdentifier)}. It combines the + * {@link CatalogBaseTable} with additional information such as if the table is a temporary table or comes + * from the catalog. + */ + public static class TableLookupResult { + private final boolean isTemporary; + private final CatalogBaseTable table; + + private static TableLookupResult temporary(CatalogBaseTable table) { + return new TableLookupResult(true, table); + } + + private static TableLookupResult permanent(CatalogBaseTable table) { + return new TableLookupResult(false, table); + } + + private TableLookupResult(boolean isTemporary, CatalogBaseTable table) { + this.isTemporary = isTemporary; + this.table = table; + } + + public boolean isTemporary() { + return isTemporary; + } + + public CatalogBaseTable getTable() { + return table; + } + } + /** * Retrieves a fully qualified table. If the path is not yet fully qualified use * {@link #qualifyIdentifier(UnresolvedIdentifier)} first. @@ -214,21 +245,186 @@ public String getBuiltInDatabaseName() { * @param objectIdentifier full path of the table to retrieve * @return table that the path points to. */ - public Optional getTable(ObjectIdentifier objectIdentifier) { + public Optional getTable(ObjectIdentifier objectIdentifier) { try { - Catalog currentCatalog = catalogs.get(objectIdentifier.getCatalogName()); - ObjectPath objectPath = new ObjectPath( - objectIdentifier.getDatabaseName(), - objectIdentifier.getObjectName()); - - if (currentCatalog != null && currentCatalog.tableExists(objectPath)) { - return Optional.of(currentCatalog.getTable(objectPath)); + CatalogBaseTable temporaryTable = temporaryTables.get(objectIdentifier); + if (temporaryTable != null) { + return Optional.of(TableLookupResult.temporary(temporaryTable)); + } else { + return getPermanentTable(objectIdentifier); } } catch (TableNotExistException ignored) { } return Optional.empty(); } + private Optional getPermanentTable(ObjectIdentifier objectIdentifier) + throws TableNotExistException { + Catalog currentCatalog = catalogs.get(objectIdentifier.getCatalogName()); + ObjectPath objectPath = objectIdentifier.toObjectPath(); + + if (currentCatalog != null && currentCatalog.tableExists(objectPath)) { + return Optional.of(TableLookupResult.permanent(currentCatalog.getTable(objectPath))); + } + return Optional.empty(); + } + + /** + * Retrieves names of all registered catalogs. + * + * @return a set of names of registered catalogs + */ + public Set listCatalogs() { + return Collections.unmodifiableSet(catalogs.keySet()); + } + + /** + * Returns an array of names of all tables (tables and views, both temporary and permanent) + * registered in the namespace of the current catalog and database. + * + * @return names of all registered tables + */ + public Set listTables() { + return listTables(getCurrentCatalog(), getCurrentDatabase()); + } + + /** + * Returns an array of names of all tables (tables and views, both temporary and permanent) + * registered in the namespace of the current catalog and database. + * + * @return names of all registered tables + */ + public Set listTables(String catalogName, String databaseName) { + Catalog currentCatalog = catalogs.get(getCurrentCatalog()); + + try { + return Stream.concat( + currentCatalog.listTables(getCurrentDatabase()).stream(), + listTemporaryTablesInternal(catalogName, databaseName).map(e -> e.getKey().getObjectName()) + ).collect(Collectors.toSet()); + } catch (DatabaseNotExistException e) { + throw new ValidationException("Current database does not exist", e); + } + } + + /** + * Returns an array of names of temporary tables registered in the namespace of the current + * catalog and database. + * + * @return names of registered temporary tables + */ + public Set listTemporaryTables() { + return listTemporaryTablesInternal(getCurrentCatalog(), getCurrentDatabase()) + .map(e -> e.getKey().getObjectName()) + .collect(Collectors.toSet()); + } + + /** + * Returns an array of names of temporary views registered in the namespace of the current + * catalog and database. + * + * @return names of registered temporary views + */ + public Set listTemporaryViews() { + return listTemporaryTablesInternal(getCurrentCatalog(), getCurrentDatabase()) + .filter(e -> e.getValue() instanceof CatalogView) + .map(e -> e.getKey().getObjectName()) + .collect(Collectors.toSet()); + } + + private Stream> listTemporaryTablesInternal( + String catalogName, + String databaseName) { + return temporaryTables + .entrySet() + .stream() + .filter(e -> { + ObjectIdentifier identifier = e.getKey(); + return identifier.getCatalogName().equals(catalogName) && + identifier.getDatabaseName().equals(databaseName); + }); + } + + /** + * Lists all available schemas in the root of the catalog manager. It is not equivalent to listing all catalogs + * as it includes also different catalog parts of the temporary objects. + * + * NOTE:It is primarily used for interacting with Calcite's schema. + * + * @return list of schemas in the root of catalog manager + */ + public Set listSchemas() { + return Stream.concat( + catalogs.keySet().stream(), + temporaryTables.keySet().stream().map(ObjectIdentifier::getCatalogName) + ).collect(Collectors.toSet()); + } + + /** + * Lists all available schemas in the given catalog. It is not equivalent to listing databases within + * the given catalog as it includes also different database parts of the temporary objects identifiers. + * + * NOTE:It is primarily used for interacting with Calcite's schema. + * + * @param catalogName filter for the catalog part of the schema + * @return list of schemas with the given prefix + */ + public Set listSchemas(String catalogName) { + return Stream.concat( + Optional.ofNullable(catalogs.get(catalogName)) + .map(Catalog::listDatabases) + .orElse(Collections.emptyList()) + .stream(), + temporaryTables.keySet() + .stream() + .filter(i -> i.getCatalogName().equals(catalogName)) + .map(ObjectIdentifier::getDatabaseName) + ).collect(Collectors.toSet()); + } + + /** + * Checks if there is a catalog with given name or is there a temporary object registered within a + * given catalog. + * + * NOTE:It is primarily used for interacting with Calcite's schema. + * + * @param catalogName filter for the catalog part of the schema + * @return true if a subschema exists + */ + public boolean schemaExists(String catalogName) { + return getCatalog(catalogName).isPresent() || + temporaryTables.keySet() + .stream() + .anyMatch(i -> i.getCatalogName().equals(catalogName)); + } + + /** + * Checks if there is a database with given name in a given catalog or is there a temporary + * object registered within a given catalog and database. + * + * NOTE:It is primarily used for interacting with Calcite's schema. + * + * @param catalogName filter for the catalog part of the schema + * @param databaseName filter for the database part of the schema + * @return true if a subschema exists + */ + public boolean schemaExists(String catalogName, String databaseName) { + return temporaryDatabaseExists(catalogName, databaseName) || permanentDatabaseExists(catalogName, databaseName); + } + + private boolean temporaryDatabaseExists(String catalogName, String databaseName) { + return temporaryTables + .keySet() + .stream() + .anyMatch(i -> i.getCatalogName().equals(catalogName) && i.getDatabaseName().equals(databaseName)); + } + + private boolean permanentDatabaseExists(String catalogName, String databaseName) { + return getCatalog(catalogName) + .map(c -> c.databaseExists(databaseName)) + .orElse(false); + } + /** * Returns the full name of the given table path, this name may be padded * with current catalog/database name based on the {@code identifier's} length. @@ -258,13 +454,34 @@ public void createTable(CatalogBaseTable table, ObjectIdentifier objectIdentifie "CreateTable"); } + /** + * Creates a temporary table in a given fully qualified path. + * + * @param table The table to put in the given path. + * @param objectIdentifier The fully qualified path where to put the table. + * @param replace controls what happens if a table exists in the given path, + * if true the table is replaced, an exception will be thrown otherwise + */ + public void createTemporaryTable( + CatalogBaseTable table, + ObjectIdentifier objectIdentifier, + boolean replace) { + temporaryTables.compute(objectIdentifier, (k, v) -> { + if (v != null && !replace) { + throw new ValidationException(String.format("Temporary table %s already exists", objectIdentifier)); + } else { + return table; + } + }); + } + /** * Alters a table in a given fully qualified path. * * @param table The table to put in the given path * @param objectIdentifier The fully qualified path where to alter the table. * @param ignoreIfNotExists If false exception will be thrown if the table or database or catalog to be altered - * does not exist. + * does not exist. */ public void alterTable(CatalogBaseTable table, ObjectIdentifier objectIdentifier, boolean ignoreIfNotExists) { execute( @@ -279,7 +496,7 @@ public void alterTable(CatalogBaseTable table, ObjectIdentifier objectIdentifier * * @param objectIdentifier The fully qualified path of the table to drop. * @param ignoreIfNotExists If false exception will be thrown if the table or database or catalog to be altered - * does not exist. + * does not exist. */ public void dropTable(ObjectIdentifier objectIdentifier, boolean ignoreIfNotExists) { execute( diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/api/TableEnvironmentTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/api/TableEnvironmentTest.java index 2c98c21bfb61..017e204d65c7 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/api/TableEnvironmentTest.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/api/TableEnvironmentTest.java @@ -18,10 +18,10 @@ package org.apache.flink.table.api; -import org.apache.flink.table.catalog.Catalog; import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogManager; import org.apache.flink.table.catalog.ConnectorCatalogTable; -import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.descriptors.Schema; import org.apache.flink.table.utils.ConnectorDescriptorMock; import org.apache.flink.table.utils.FormatDescriptorMock; @@ -51,12 +51,15 @@ public void testConnect() throws Exception { .inAppendMode() .registerTableSource("my_table"); - final Catalog catalog = tableEnv.getCatalog(EnvironmentSettings.DEFAULT_BUILTIN_CATALOG) + CatalogManager.TableLookupResult lookupResult = tableEnv.catalogManager.getTable(ObjectIdentifier.of( + EnvironmentSettings.DEFAULT_BUILTIN_CATALOG, + EnvironmentSettings.DEFAULT_BUILTIN_DATABASE, + "my_table")) .orElseThrow(AssertionError::new); - final CatalogBaseTable table = catalog - .getTable(new ObjectPath(EnvironmentSettings.DEFAULT_BUILTIN_DATABASE, "my_table")); + assertThat(lookupResult.isTemporary(), equalTo(true)); + CatalogBaseTable table = lookupResult.getTable(); assertThat( table.getSchema(), equalTo( diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/CatalogCalciteSchema.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/CatalogCalciteSchema.java index d53a40261f14..f869f46d6b43 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/CatalogCalciteSchema.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/CatalogCalciteSchema.java @@ -19,7 +19,7 @@ package org.apache.flink.table.planner.catalog; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.catalog.CatalogManager; import org.apache.calcite.linq4j.tree.Expression; import org.apache.calcite.schema.Schema; @@ -39,13 +39,13 @@ public class CatalogCalciteSchema extends FlinkSchema { private final String catalogName; - private final Catalog catalog; + private final CatalogManager catalogManager; // Flag that tells if the current planner should work in a batch or streaming mode. private final boolean isStreamingMode; - public CatalogCalciteSchema(String catalogName, Catalog catalog, boolean isStreamingMode) { + public CatalogCalciteSchema(String catalogName, CatalogManager catalog, boolean isStreamingMode) { this.catalogName = catalogName; - this.catalog = catalog; + this.catalogManager = catalog; this.isStreamingMode = isStreamingMode; } @@ -57,8 +57,8 @@ public CatalogCalciteSchema(String catalogName, Catalog catalog, boolean isStrea */ @Override public Schema getSubSchema(String schemaName) { - if (catalog.databaseExists(schemaName)) { - return new DatabaseCalciteSchema(schemaName, catalogName, catalog, isStreamingMode); + if (catalogManager.schemaExists(catalogName, schemaName)) { + return new DatabaseCalciteSchema(schemaName, catalogName, catalogManager, isStreamingMode); } else { return null; } @@ -66,7 +66,7 @@ public Schema getSubSchema(String schemaName) { @Override public Set getSubSchemaNames() { - return new HashSet<>(catalog.listDatabases()); + return catalogManager.listSchemas(catalogName); } @Override diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/CatalogManagerCalciteSchema.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/CatalogManagerCalciteSchema.java index 7dd5da3b8e0d..0afd2a89d5ee 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/CatalogManagerCalciteSchema.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/CatalogManagerCalciteSchema.java @@ -62,14 +62,16 @@ public Set getTableNames() { @Override public Schema getSubSchema(String name) { - return catalogManager.getCatalog(name) - .map(catalog -> new CatalogCalciteSchema(name, catalog, isStreamingMode)) - .orElse(null); + if (catalogManager.schemaExists(name)) { + return new CatalogCalciteSchema(name, catalogManager, isStreamingMode); + } else { + return null; + } } @Override public Set getSubSchemaNames() { - return new HashSet<>(catalogManager.getCatalogs()); + return new HashSet<>(catalogManager.listCatalogs()); } @Override diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/DatabaseCalciteSchema.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/DatabaseCalciteSchema.java index ade4b46d6848..233c190f7ec6 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/DatabaseCalciteSchema.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/DatabaseCalciteSchema.java @@ -21,12 +21,12 @@ import org.apache.flink.table.api.TableException; import org.apache.flink.table.catalog.Catalog; import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogManager; import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.ConnectorCatalogTable; +import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.ObjectPath; import org.apache.flink.table.catalog.QueryOperationCatalogView; -import org.apache.flink.table.catalog.exceptions.CatalogException; -import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; import org.apache.flink.table.catalog.exceptions.TableNotExistException; import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; import org.apache.flink.table.catalog.stats.CatalogTableStatistics; @@ -50,6 +50,8 @@ import org.apache.calcite.schema.Schemas; import org.apache.calcite.schema.Table; +import javax.annotation.Nullable; + import java.util.Arrays; import java.util.HashSet; import java.util.List; @@ -66,109 +68,154 @@ class DatabaseCalciteSchema extends FlinkSchema { private final String databaseName; private final String catalogName; - private final Catalog catalog; + private final CatalogManager catalogManager; // Flag that tells if the current planner should work in a batch or streaming mode. private final boolean isStreamingMode; - public DatabaseCalciteSchema(String databaseName, String catalogName, Catalog catalog, boolean isStreamingMode) { + public DatabaseCalciteSchema(String databaseName, String catalogName, CatalogManager catalog, boolean isStreamingMode) { this.databaseName = databaseName; this.catalogName = catalogName; - this.catalog = catalog; + this.catalogManager = catalog; this.isStreamingMode = isStreamingMode; } @Override public Table getTable(String tableName) { - ObjectPath tablePath = new ObjectPath(databaseName, tableName); + ObjectIdentifier identifier = ObjectIdentifier.of(catalogName, databaseName, tableName); + return catalogManager.getTable(identifier) + .map(result -> { + CatalogBaseTable table = result.getTable(); + if (result.isTemporary()) { + return convertTemporaryTable(new ObjectPath(databaseName, tableName), table); + } else { + return convertPermanentTable( + identifier.toObjectPath(), + table, + catalogManager.getCatalog(catalogName) + .flatMap(Catalog::getTableFactory) + .orElse(null) + ); + } + }) + .orElse(null); + } - try { - if (!catalog.tableExists(tablePath)) { - return null; + private Table convertPermanentTable( + ObjectPath tablePath, + CatalogBaseTable table, + @Nullable TableFactory tableFactory) { + // TODO supports GenericCatalogView + if (table instanceof QueryOperationCatalogView) { + return convertQueryOperationView(tablePath, (QueryOperationCatalogView) table); + } else if (table instanceof ConnectorCatalogTable) { + ConnectorCatalogTable connectorTable = (ConnectorCatalogTable) table; + if ((connectorTable).getTableSource().isPresent()) { + TableStats tableStats = extractTableStats(connectorTable, tablePath); + return convertSourceTable(connectorTable, tableStats); + } else { + return convertSinkTable(connectorTable); } + } else if (table instanceof CatalogTable) { + return convertCatalogTable(tablePath, (CatalogTable) table, tableFactory); + } else { + throw new TableException("Unsupported table type: " + table); + } + } - CatalogBaseTable table = catalog.getTable(tablePath); - - // TODO supports GenericCatalogView - if (table instanceof QueryOperationCatalogView) { - QueryOperationCatalogView view = (QueryOperationCatalogView) table; - QueryOperation operation = view.getQueryOperation(); - if (operation instanceof DataStreamQueryOperation) { - List qualifiedName = Arrays.asList(catalogName, databaseName, tableName); - ((DataStreamQueryOperation) operation).setQualifiedName(qualifiedName); - } else if (operation instanceof RichTableSourceQueryOperation) { - List qualifiedName = Arrays.asList(catalogName, databaseName, tableName); - ((RichTableSourceQueryOperation) operation).setQualifiedName(qualifiedName); - } - return QueryOperationCatalogViewTable.createCalciteTable(view); - } else if (table instanceof ConnectorCatalogTable) { - return convertConnectorTable((ConnectorCatalogTable) table, tablePath); - } else if (table instanceof CatalogTable) { - return convertCatalogTable(tablePath, (CatalogTable) table); + private Table convertTemporaryTable( + ObjectPath tablePath, + CatalogBaseTable table) { + // TODO supports GenericCatalogView + if (table instanceof QueryOperationCatalogView) { + return convertQueryOperationView(tablePath, (QueryOperationCatalogView) table); + } else if (table instanceof ConnectorCatalogTable) { + ConnectorCatalogTable connectorTable = (ConnectorCatalogTable) table; + if ((connectorTable).getTableSource().isPresent()) { + return convertSourceTable(connectorTable, TableStats.UNKNOWN); } else { - throw new TableException("Unsupported table type: " + table); + return convertSinkTable(connectorTable); } - } catch (TableNotExistException | CatalogException e) { - // TableNotExistException should never happen, because we are checking it exists - // via catalog.tableExists - throw new TableException(format( - "A failure occurred when accessing table. Table path [%s, %s, %s]", - catalogName, - databaseName, - tableName), e); + } else if (table instanceof CatalogTable) { + return convertCatalogTable(tablePath, (CatalogTable) table, null); + } else { + throw new TableException("Unsupported table type: " + table); + } + } + + private Table convertQueryOperationView(ObjectPath tablePath, QueryOperationCatalogView table) { + QueryOperation operation = table.getQueryOperation(); + if (operation instanceof DataStreamQueryOperation) { + List qualifiedName = Arrays.asList(catalogName, databaseName, tablePath.getObjectName()); + ((DataStreamQueryOperation) operation).setQualifiedName(qualifiedName); + } else if (operation instanceof RichTableSourceQueryOperation) { + List qualifiedName = Arrays.asList(catalogName, databaseName, tablePath.getObjectName()); + ((RichTableSourceQueryOperation) operation).setQualifiedName(qualifiedName); + } + return QueryOperationCatalogViewTable.createCalciteTable(table); + } + + private Table convertSinkTable(ConnectorCatalogTable table) { + Optional tableSinkTable = table.getTableSink() + .map(tableSink -> new TableSinkTable<>( + tableSink, + FlinkStatistic.UNKNOWN())); + if (tableSinkTable.isPresent()) { + return tableSinkTable.get(); + } else { + throw new TableException("Cannot convert a connector table " + + "without either source or sink."); } } - private Table convertConnectorTable( + private Table convertSourceTable( ConnectorCatalogTable table, - ObjectPath tablePath) throws TableNotExistException { - if (table.getTableSource().isPresent()) { - TableSource tableSource = table.getTableSource().get(); - if (!(tableSource instanceof StreamTableSource || - tableSource instanceof LookupableTableSource)) { - throw new TableException( - "Only StreamTableSource and LookupableTableSource can be used in Blink planner."); - } - if (!isStreamingMode && tableSource instanceof StreamTableSource && - !((StreamTableSource) tableSource).isBounded()) { - throw new TableException("Only bounded StreamTableSource can be used in batch mode."); - } + TableStats tableStats) { + TableSource tableSource = table.getTableSource().get(); + if (!(tableSource instanceof StreamTableSource || + tableSource instanceof LookupableTableSource)) { + throw new TableException( + "Only StreamTableSource and LookupableTableSource can be used in Blink planner."); + } + if (!isStreamingMode && tableSource instanceof StreamTableSource && + !((StreamTableSource) tableSource).isBounded()) { + throw new TableException("Only bounded StreamTableSource can be used in batch mode."); + } + + return new TableSourceTable<>( + tableSource, + isStreamingMode, + FlinkStatistic.builder().tableStats(tableStats).build(), + null); + } - TableStats tableStats = TableStats.UNKNOWN; + private TableStats extractTableStats(ConnectorCatalogTable table, ObjectPath tablePath) { + TableStats tableStats = TableStats.UNKNOWN; + try { // TODO supports stats for partitionable table if (!table.isPartitioned()) { + Catalog catalog = catalogManager.getCatalog(catalogName).get(); CatalogTableStatistics tableStatistics = catalog.getTableStatistics(tablePath); CatalogColumnStatistics columnStatistics = catalog.getTableColumnStatistics(tablePath); tableStats = convertToTableStats(tableStatistics, columnStatistics); } - return new TableSourceTable<>( - tableSource, - isStreamingMode, - FlinkStatistic.builder().tableStats(tableStats).build(), - null); - } else { - Optional tableSinkTable = table.getTableSink() - .map(tableSink -> new TableSinkTable<>( - tableSink, - FlinkStatistic.UNKNOWN())); - if (tableSinkTable.isPresent()) { - return tableSinkTable.get(); - } else { - throw new TableException("Cannot convert a connector table " + - "without either source or sink."); - } + return tableStats; + } catch (TableNotExistException e) { + throw new TableException(format( + "Could not access table partitions for table: [%s, %s, %s]", + catalogName, + databaseName, + tablePath.getObjectName()), e); } } - private Table convertCatalogTable(ObjectPath tablePath, CatalogTable table) { + private Table convertCatalogTable(ObjectPath tablePath, CatalogTable table, @Nullable TableFactory tableFactory) { TableSource tableSource; - Optional tableFactory = catalog.getTableFactory(); - if (tableFactory.isPresent()) { - TableFactory tf = tableFactory.get(); - if (tf instanceof TableSourceFactory) { - tableSource = ((TableSourceFactory) tf).createTableSource(tablePath, table); + if (tableFactory != null) { + if (tableFactory instanceof TableSourceFactory) { + tableSource = ((TableSourceFactory) tableFactory).createTableSource(tablePath, table); } else { - throw new TableException(String.format("Cannot query a sink-only table. TableFactory provided by catalog %s must implement TableSourceFactory", - catalog.getClass())); + throw new TableException( + "Cannot query a sink-only table. TableFactory provided by catalog must implement TableSourceFactory"); } } else { tableSource = TableFactoryUtil.findAndCreateTableSource(table); @@ -188,11 +235,7 @@ private Table convertCatalogTable(ObjectPath tablePath, CatalogTable table) { @Override public Set getTableNames() { - try { - return new HashSet<>(catalog.listTables(databaseName)); - } catch (DatabaseNotExistException e) { - throw new CatalogException(e); - } + return catalogManager.listTables(catalogName, databaseName); } @Override diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala index 0807fe53dec7..691910b47692 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala @@ -251,9 +251,10 @@ abstract class PlannerBase( */ protected def translateToPlan(execNodes: util.List[ExecNode[_, _]]): util.List[Transformation[_]] - private def getTableSink( - tableIdentifier: ObjectIdentifier): Option[(CatalogTable, TableSink[_])] = { - JavaScalaConversionUtil.toScala(catalogManager.getTable(tableIdentifier)) match { + private def getTableSink(objectIdentifier: ObjectIdentifier) + : Option[(CatalogTable, TableSink[_])] = { + JavaScalaConversionUtil.toScala(catalogManager.getTable(objectIdentifier)) + .map(_.getTable) match { case Some(s) if s.isInstanceOf[ConnectorCatalogTable[_, _]] => val table = s.asInstanceOf[ConnectorCatalogTable[_, _]] JavaScalaConversionUtil.toScala(table.getTableSink) match { @@ -262,10 +263,10 @@ abstract class PlannerBase( } case Some(s) if s.isInstanceOf[CatalogTable] => - val catalog = catalogManager.getCatalog(tableIdentifier.getCatalogName) + val catalog = catalogManager.getCatalog(objectIdentifier.getCatalogName) val table = s.asInstanceOf[CatalogTable] if (catalog.isPresent && catalog.get().getTableFactory.isPresent) { - val objectPath = tableIdentifier.toObjectPath + val objectPath = objectIdentifier.toObjectPath val sink = TableFactoryUtil.createTableSinkForCatalogTable( catalog.get(), table, diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/catalog/CatalogStatisticsTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/catalog/CatalogStatisticsTest.java index 7bd663235ab6..c3d2317972f8 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/catalog/CatalogStatisticsTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/catalog/CatalogStatisticsTest.java @@ -24,6 +24,7 @@ import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.catalog.ConnectorCatalogTable; import org.apache.flink.table.catalog.ObjectPath; import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataBase; @@ -61,10 +62,16 @@ public class CatalogStatisticsTest { public void testGetStatsFromCatalog() throws Exception { EnvironmentSettings settings = EnvironmentSettings.newInstance().useBlinkPlanner().inBatchMode().build(); TableEnvironment tEnv = TableEnvironment.create(settings); - tEnv.registerTableSource("T1", new TestTableSource(true, tableSchema)); - tEnv.registerTableSource("T2", new TestTableSource(true, tableSchema)); Catalog catalog = tEnv.getCatalog(tEnv.getCurrentCatalog()).orElse(null); assertNotNull(catalog); + catalog.createTable( + ObjectPath.fromString("default_database.T1"), + ConnectorCatalogTable.source(new TestTableSource(true, tableSchema), true), + false); + catalog.createTable( + ObjectPath.fromString("default_database.T2"), + ConnectorCatalogTable.source(new TestTableSource(true, tableSchema), true), + false); catalog.alterTableStatistics(ObjectPath.fromString("default_database.T1"), new CatalogTableStatistics(100, 10, 1000L, 2000L), true); diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala index cf02c1b21ec8..185c4ca48abf 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala @@ -29,7 +29,6 @@ import org.junit.Assert.assertEquals import org.junit.rules.ExpectedException import org.junit.{Rule, Test} - class TableEnvironmentTest { // used for accurate exception information checking. @@ -60,7 +59,8 @@ class TableEnvironmentTest { // register on a conflict name thrown.expect(classOf[ValidationException]) - thrown.expectMessage("Could not execute CreateTable in path") + thrown.expectMessage( + "Temporary table `default_catalog`.`default_database`.`MyTable` already exists") tableEnv.registerDataStream("MyTable", env.fromElements[(Int, Long)]()) } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/CatalogCalciteSchema.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/CatalogCalciteSchema.java index 9b50b9191a11..5be5a7968344 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/CatalogCalciteSchema.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/CatalogCalciteSchema.java @@ -43,12 +43,12 @@ public class CatalogCalciteSchema implements Schema { private final boolean isStreamingMode; private final String catalogName; - private final Catalog catalog; + private final CatalogManager catalogManager; - public CatalogCalciteSchema(boolean isStreamingMode, String catalogName, Catalog catalog) { + public CatalogCalciteSchema(boolean isStreamingMode, String catalogName, CatalogManager catalogManager) { this.isStreamingMode = isStreamingMode; this.catalogName = catalogName; - this.catalog = catalog; + this.catalogManager = catalogManager; } /** @@ -59,9 +59,8 @@ public CatalogCalciteSchema(boolean isStreamingMode, String catalogName, Catalog */ @Override public Schema getSubSchema(String schemaName) { - - if (catalog.databaseExists(schemaName)) { - return new DatabaseCalciteSchema(isStreamingMode, schemaName, catalogName, catalog); + if (catalogManager.schemaExists(catalogName, schemaName)) { + return new DatabaseCalciteSchema(isStreamingMode, schemaName, catalogName, catalogManager); } else { return null; } @@ -69,7 +68,7 @@ public Schema getSubSchema(String schemaName) { @Override public Set getSubSchemaNames() { - return new HashSet<>(catalog.listDatabases()); + return catalogManager.listSchemas(catalogName); } @Override diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/CatalogManagerCalciteSchema.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/CatalogManagerCalciteSchema.java index ef1aafc4aa7a..e7376fc71f45 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/CatalogManagerCalciteSchema.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/CatalogManagerCalciteSchema.java @@ -82,14 +82,16 @@ public Set getFunctionNames() { @Override public Schema getSubSchema(String name) { - return catalogManager.getCatalog(name) - .map(catalog -> new CatalogCalciteSchema(isStreamingMode, name, catalog)) - .orElse(null); + if (catalogManager.schemaExists(name)) { + return new CatalogCalciteSchema(isStreamingMode, name, catalogManager); + } else { + return null; + } } @Override public Set getSubSchemaNames() { - return catalogManager.getCatalogs(); + return catalogManager.listSchemas(); } @Override diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/DatabaseCalciteSchema.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/DatabaseCalciteSchema.java index 76df4bb3b442..72e89b1ec8dd 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/DatabaseCalciteSchema.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/DatabaseCalciteSchema.java @@ -19,9 +19,6 @@ package org.apache.flink.table.catalog; import org.apache.flink.table.api.TableException; -import org.apache.flink.table.catalog.exceptions.CatalogException; -import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; -import org.apache.flink.table.catalog.exceptions.TableNotExistException; import org.apache.flink.table.factories.TableFactory; import org.apache.flink.table.factories.TableFactoryUtil; import org.apache.flink.table.factories.TableSourceFactory; @@ -40,63 +37,62 @@ import org.apache.calcite.schema.Schemas; import org.apache.calcite.schema.Table; +import javax.annotation.Nullable; + import java.util.Collection; import java.util.HashSet; import java.util.Optional; import java.util.Set; -import static java.lang.String.format; - /** * A mapping between Flink catalog's database and Calcite's schema. * Tables are registered as tables in the schema. */ class DatabaseCalciteSchema implements Schema { private final boolean isStreamingMode; - private final String databaseName; private final String catalogName; - private final Catalog catalog; + private final String databaseName; + private final CatalogManager catalogManager; public DatabaseCalciteSchema( boolean isStreamingMode, String databaseName, String catalogName, - Catalog catalog) { + CatalogManager catalogManager) { this.isStreamingMode = isStreamingMode; this.databaseName = databaseName; this.catalogName = catalogName; - this.catalog = catalog; + this.catalogManager = catalogManager; } @Override public Table getTable(String tableName) { + ObjectIdentifier identifier = ObjectIdentifier.of(catalogName, databaseName, tableName); + return catalogManager.getTable(identifier) + .map(result -> { + CatalogBaseTable table = result.getTable(); + final TableFactory tableFactory; + if (result.isTemporary()) { + tableFactory = null; + } else { + tableFactory = catalogManager.getCatalog(catalogName) + .flatMap(Catalog::getTableFactory) + .orElse(null); + } + return convertTable(identifier.toObjectPath(), table, tableFactory); + }) + .orElse(null); + } - ObjectPath tablePath = new ObjectPath(databaseName, tableName); - - try { - if (!catalog.tableExists(tablePath)) { - return null; - } - - CatalogBaseTable table = catalog.getTable(tablePath); - - if (table instanceof QueryOperationCatalogView) { - return QueryOperationCatalogViewTable.createCalciteTable(((QueryOperationCatalogView) table)); - } else if (table instanceof ConnectorCatalogTable) { - return convertConnectorTable((ConnectorCatalogTable) table); - } else if (table instanceof CatalogTable) { - return convertCatalogTable(tablePath, (CatalogTable) table); - } else { - throw new TableException("Unsupported table type: " + table); - } - } catch (TableNotExistException | CatalogException e) { - // TableNotExistException should never happen, because we are checking it exists - // via catalog.tableExists - throw new TableException(format( - "A failure occurred when accessing table. Table path [%s, %s, %s]", - catalogName, - databaseName, - tableName), e); + private Table convertTable(ObjectPath tablePath, CatalogBaseTable table, @Nullable TableFactory tableFactory) { + if (table instanceof QueryOperationCatalogView) { + return QueryOperationCatalogViewTable.createCalciteTable(((QueryOperationCatalogView) table)); + } else if (table instanceof ConnectorCatalogTable) { + return convertConnectorTable((ConnectorCatalogTable) table); + } else if (table instanceof CatalogTable) { + return convertCatalogTable(tablePath, (CatalogTable) table, tableFactory); + } else { + throw new TableException("Unsupported table type: " + table); } } @@ -122,16 +118,14 @@ private Table convertConnectorTable(ConnectorCatalogTable table) { } } - private Table convertCatalogTable(ObjectPath tablePath, CatalogTable table) { - TableSource tableSource; - Optional tableFactory = catalog.getTableFactory(); - if (tableFactory.isPresent()) { - TableFactory tf = tableFactory.get(); - if (tf instanceof TableSourceFactory) { - tableSource = ((TableSourceFactory) tf).createTableSource(tablePath, table); + private Table convertCatalogTable(ObjectPath tablePath, CatalogTable table, @Nullable TableFactory tableFactory) { + final TableSource tableSource; + if (tableFactory != null) { + if (tableFactory instanceof TableSourceFactory) { + tableSource = ((TableSourceFactory) tableFactory).createTableSource(tablePath, table); } else { - throw new TableException(String.format("Cannot query a sink-only table. TableFactory provided by catalog %s must implement TableSourceFactory", - catalog.getClass())); + throw new TableException( + "Cannot query a sink-only table. TableFactory provided by catalog must implement TableSourceFactory"); } } else { tableSource = TableFactoryUtil.findAndCreateTableSource(table); @@ -153,11 +147,7 @@ private Table convertCatalogTable(ObjectPath tablePath, CatalogTable table) { @Override public Set getTableNames() { - try { - return new HashSet<>(catalog.listTables(databaseName)); - } catch (DatabaseNotExistException e) { - throw new CatalogException(e); - } + return catalogManager.listTables(catalogName, databaseName); } @Override 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 f42d77f977d4..fe8c0d706e4c 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 @@ -203,7 +203,7 @@ abstract class TableEnvImpl( } val view = new QueryOperationCatalogView(table.getQueryOperation) - catalogManager.createTable(view, getTemporaryObjectIdentifier(name), false) + catalogManager.createTemporaryTable(view, getTemporaryObjectIdentifier(name), false) } override def registerTableSource(name: String, tableSource: TableSource[_]): Unit = { @@ -271,7 +271,7 @@ abstract class TableEnvImpl( tableSource: TableSource[_]) : Unit = { // register - getCatalogTable( + getTemporaryTable( catalogManager.getBuiltInCatalogName, catalogManager.getBuiltInDatabaseName, name) match { @@ -287,13 +287,16 @@ abstract class TableEnvImpl( tableSource, table.getTableSink.get, isBatchTable) - catalogManager.alterTable(sourceAndSink, getTemporaryObjectIdentifier(name), false) + catalogManager.createTemporaryTable( + sourceAndSink, + getTemporaryObjectIdentifier(name), + true) } // no table is registered case _ => val source = ConnectorCatalogTable.source(tableSource, isBatchTable) - catalogManager.createTable(source, getTemporaryObjectIdentifier(name), false) + catalogManager.createTemporaryTable(source, getTemporaryObjectIdentifier(name), false) } } @@ -302,7 +305,7 @@ abstract class TableEnvImpl( tableSink: TableSink[_]) : Unit = { // check if a table (source or sink) is registered - getCatalogTable( + getTemporaryTable( catalogManager.getBuiltInCatalogName, catalogManager.getBuiltInDatabaseName, name) match { @@ -319,13 +322,16 @@ abstract class TableEnvImpl( table.getTableSource.get, tableSink, isBatchTable) - catalogManager.alterTable(sourceAndSink, getTemporaryObjectIdentifier(name), false) + catalogManager.createTemporaryTable( + sourceAndSink, + getTemporaryObjectIdentifier(name), + true) } // no table is registered case _ => val sink = ConnectorCatalogTable.sink(tableSink, isBatchTable) - catalogManager.createTable(sink, getTemporaryObjectIdentifier(name), false) + catalogManager.createTemporaryTable(sink, getTemporaryObjectIdentifier(name), false) } } @@ -350,7 +356,7 @@ abstract class TableEnvImpl( val unresolvedIdentifier = UnresolvedIdentifier.of(tablePath: _*) val objectIdentifier = catalogManager.qualifyIdentifier(unresolvedIdentifier) JavaScalaConversionUtil.toScala(catalogManager.getTable(objectIdentifier)) - .map(t => new CatalogQueryOperation(objectIdentifier, t.getSchema)) + .map(t => new CatalogQueryOperation(objectIdentifier, t.getTable.getSchema)) } override def listModules(): Array[String] = { @@ -358,7 +364,10 @@ abstract class TableEnvImpl( } override def listCatalogs(): Array[String] = { - catalogManager.getCatalogs.asScala.toArray + catalogManager.listCatalogs + .asScala + .toArray + .sorted } override def listDatabases(): Array[String] = { @@ -369,13 +378,21 @@ abstract class TableEnvImpl( } override def listTables(): Array[String] = { - val currentCatalogName = catalogManager.getCurrentCatalog - val currentCatalog = catalogManager.getCatalog(currentCatalogName) - JavaScalaConversionUtil.toScala(currentCatalog) match { - case Some(catalog) => catalog.listTables(catalogManager.getCurrentDatabase).asScala.toArray - case None => - throw new TableException(s"The current catalog ($currentCatalogName) does not exist.") - } + catalogManager.listTables().asScala + .toArray + .sorted + } + + override def listTemporaryTables(): Array[String] = { + catalogManager.listTemporaryTables().asScala + .toArray + .sorted + } + + override def listTemporaryViews(): Array[String] = { + catalogManager.listTemporaryViews().asScala + .toArray + .sorted } override def listUserDefinedFunctions(): Array[String] = functionCatalog.getUserDefinedFunctions @@ -510,7 +527,8 @@ abstract class TableEnvImpl( } private def getTableSink(objectIdentifier: ObjectIdentifier): Option[TableSink[_]] = { - JavaScalaConversionUtil.toScala(catalogManager.getTable(objectIdentifier)) match { + JavaScalaConversionUtil.toScala(catalogManager.getTable(objectIdentifier)) + .map(_.getTable) match { case Some(s) if s.isInstanceOf[ConnectorCatalogTable[_, _]] => JavaScalaConversionUtil @@ -537,10 +555,12 @@ abstract class TableEnvImpl( } } - protected def getCatalogTable(name: String*): Option[CatalogBaseTable] = { + protected def getTemporaryTable(name: String*): Option[CatalogBaseTable] = { val unresolvedIdentifier = UnresolvedIdentifier.of(name: _*) val objectIdentifier = catalogManager.qualifyIdentifier(unresolvedIdentifier) JavaScalaConversionUtil.toScala(catalogManager.getTable(objectIdentifier)) + .filter(_.isTemporary) + .map(_.getTable) } /** Returns the [[FlinkRelBuilder]] of this TableEnvironment. */ diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala index 0b2243194e2a..4c54ec600b41 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala @@ -421,7 +421,8 @@ class StreamPlanner( } private def getTableSink(objectIdentifier: ObjectIdentifier): Option[TableSink[_]] = { - JavaScalaConversionUtil.toScala(catalogManager.getTable(objectIdentifier)) match { + JavaScalaConversionUtil.toScala(catalogManager.getTable(objectIdentifier)) + .map(_.getTable) match { case Some(s) if s.isInstanceOf[ConnectorCatalogTable[_, _]] => JavaScalaConversionUtil.toScala(s.asInstanceOf[ConnectorCatalogTable[_, _]].getTableSink) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/CatalogManagerTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/CatalogManagerTest.java index 14f818a79687..0c23905fc8d4 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/CatalogManagerTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/CatalogManagerTest.java @@ -18,6 +18,7 @@ package org.apache.flink.table.catalog; +import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.exceptions.CatalogException; import org.apache.flink.util.TestLogger; @@ -25,10 +26,18 @@ import org.junit.Test; import org.junit.rules.ExpectedException; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.table.catalog.CatalogStructureBuilder.BUILTIN_CATALOG_NAME; import static org.apache.flink.table.catalog.CatalogStructureBuilder.database; import static org.apache.flink.table.catalog.CatalogStructureBuilder.root; +import static org.apache.flink.table.catalog.CatalogStructureBuilder.table; +import static org.hamcrest.CoreMatchers.equalTo; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; /** @@ -50,13 +59,13 @@ public void testRegisterCatalog() throws Exception { database(BUILTIN_DEFAULT_DATABASE_NAME)) .build(); - assertEquals(1, manager.getCatalogs().size()); - assertFalse(manager.getCatalogs().contains(TEST_CATALOG_NAME)); + assertEquals(1, manager.listCatalogs().size()); + assertFalse(manager.listCatalogs().contains(TEST_CATALOG_NAME)); manager.registerCatalog(TEST_CATALOG_NAME, new GenericInMemoryCatalog(TEST_CATALOG_NAME)); - assertEquals(2, manager.getCatalogs().size()); - assertTrue(manager.getCatalogs().contains(TEST_CATALOG_NAME)); + assertEquals(2, manager.listCatalogs().size()); + assertTrue(manager.listCatalogs().contains(TEST_CATALOG_NAME)); } @Test @@ -91,6 +100,133 @@ public void testRegisterCatalogWithExistingName() throws Exception { manager.registerCatalog(TEST_CATALOG_NAME, new GenericInMemoryCatalog(TEST_CATALOG_NAME)); } + @Test + public void testReplaceTemporaryTable() throws Exception { + ObjectIdentifier tempIdentifier = ObjectIdentifier.of( + BUILTIN_CATALOG_NAME, + BUILTIN_DEFAULT_DATABASE_NAME, + "temp"); + CatalogManager manager = root() + .builtin( + database(BUILTIN_DEFAULT_DATABASE_NAME)) + .temporaryTable(tempIdentifier) + .build(); + + CatalogTest.TestTable table = new CatalogTest.TestTable(); + manager.createTemporaryTable(table, tempIdentifier, true); + assertThat(manager.getTable(tempIdentifier).get().isTemporary(), equalTo(true)); + assertThat(manager.getTable(tempIdentifier).get().getTable(), equalTo(table)); + } + + @Test + public void testTemporaryTableExists() throws Exception { + ObjectIdentifier tempIdentifier = ObjectIdentifier.of( + BUILTIN_CATALOG_NAME, + BUILTIN_DEFAULT_DATABASE_NAME, + "temp"); + CatalogManager manager = root() + .builtin( + database(BUILTIN_DEFAULT_DATABASE_NAME)) + .temporaryTable(tempIdentifier) + .build(); + + thrown.expect(ValidationException.class); + thrown.expectMessage(String.format("Temporary table %s already exists", tempIdentifier)); + manager.createTemporaryTable(new CatalogTest.TestTable(), tempIdentifier, false); + } + + @Test + public void testListTables() throws Exception { + ObjectIdentifier identifier1 = ObjectIdentifier.of(TEST_CATALOG_NAME, TEST_CATALOG_DEFAULT_DB_NAME, "test1"); + ObjectIdentifier identifier2 = ObjectIdentifier.of( + BUILTIN_CATALOG_NAME, + BUILTIN_DEFAULT_DATABASE_NAME, + "test2"); + ObjectIdentifier viewIdentifier = ObjectIdentifier.of( + BUILTIN_CATALOG_NAME, + BUILTIN_DEFAULT_DATABASE_NAME, + "testView"); + CatalogManager manager = root() + .builtin( + database(BUILTIN_DEFAULT_DATABASE_NAME, table("test_in_builtin"))) + .catalog(TEST_CATALOG_NAME, database(TEST_CATALOG_DEFAULT_DB_NAME, table("test_in_catalog"))) + .temporaryTable(identifier1) + .temporaryTable(identifier2) + .temporaryView(viewIdentifier, "SELECT * FROM none") + .build(); + + manager.setCurrentCatalog(BUILTIN_CATALOG_NAME); + manager.setCurrentDatabase(BUILTIN_DEFAULT_DATABASE_NAME); + + assertThat( + manager.listTables(), + equalTo( + setOf( + "test2", + "testView", + "test_in_builtin" + ))); + } + + @Test + public void testListTemporaryTables() throws Exception { + ObjectIdentifier identifier1 = ObjectIdentifier.of(TEST_CATALOG_NAME, TEST_CATALOG_DEFAULT_DB_NAME, "test1"); + ObjectIdentifier identifier2 = ObjectIdentifier.of( + BUILTIN_CATALOG_NAME, + BUILTIN_DEFAULT_DATABASE_NAME, + "test2"); + ObjectIdentifier viewIdentifier = ObjectIdentifier.of( + BUILTIN_CATALOG_NAME, + BUILTIN_DEFAULT_DATABASE_NAME, + "testView"); + CatalogManager manager = root() + .builtin( + database(BUILTIN_DEFAULT_DATABASE_NAME, table("test_in_builtin"))) + .catalog(TEST_CATALOG_NAME, database(TEST_CATALOG_DEFAULT_DB_NAME, table("test_in_catalog"))) + .temporaryTable(identifier1) + .temporaryTable(identifier2) + .temporaryView(viewIdentifier, "SELECT * FROM none") + .build(); + + manager.setCurrentCatalog(BUILTIN_CATALOG_NAME); + manager.setCurrentDatabase(BUILTIN_DEFAULT_DATABASE_NAME); + + assertThat( + manager.listTemporaryTables(), + equalTo( + setOf( + "test2", + "testView" + ))); + } + + @Test + public void testListTemporaryViews() throws Exception { + ObjectIdentifier tableIdentifier = ObjectIdentifier.of( + TEST_CATALOG_NAME, + TEST_CATALOG_DEFAULT_DB_NAME, + "table"); + ObjectIdentifier identifier1 = ObjectIdentifier.of(TEST_CATALOG_NAME, TEST_CATALOG_DEFAULT_DB_NAME, "test1"); + ObjectIdentifier identifier2 = ObjectIdentifier.of( + BUILTIN_CATALOG_NAME, + BUILTIN_DEFAULT_DATABASE_NAME, + "test2"); + + CatalogManager manager = root() + .builtin( + database(BUILTIN_DEFAULT_DATABASE_NAME, table("test_in_builtin"))) + .catalog(TEST_CATALOG_NAME, database(TEST_CATALOG_DEFAULT_DB_NAME, table("test_in_catalog"))) + .temporaryTable(tableIdentifier) + .temporaryView(identifier1, "SELECT * FROM none") + .temporaryView(identifier2, "SELECT * FROM none") + .build(); + + manager.setCurrentCatalog(TEST_CATALOG_NAME); + manager.setCurrentDatabase(TEST_CATALOG_DEFAULT_DB_NAME); + + assertThat(manager.listTemporaryViews(), equalTo(setOf("test1"))); + } + @Test public void testSetNonExistingCurrentCatalog() throws Exception { thrown.expect(CatalogException.class); @@ -109,4 +245,8 @@ public void testSetNonExistingCurrentDatabase() throws Exception { // This catalog does not exist in the builtin catalog manager.setCurrentDatabase("nonexistent"); } + + private Set setOf(String... element) { + return Stream.of(element).collect(Collectors.toSet()); + } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/CatalogStructureBuilder.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/CatalogStructureBuilder.java index 384251f0408f..5141802888de 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/CatalogStructureBuilder.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/CatalogStructureBuilder.java @@ -26,8 +26,11 @@ import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.types.Row; +import java.util.Collections; import java.util.HashMap; +import java.util.Map; import java.util.Objects; +import java.util.Optional; /** * Utility classes to construct a {@link CatalogManager} with a given structure. @@ -51,6 +54,7 @@ * table("tab1"), * table("tab2") * ) + * .temporaryTable(ObjectIdentifier.of("cat1", "default", "tab1")) * ).build(); * } */ @@ -80,11 +84,30 @@ public CatalogStructureBuilder builtin(DatabaseBuilder defaultDb, DatabaseBuilde return this; } + public CatalogStructureBuilder temporaryTable(ObjectIdentifier path) { + this.catalogManager.createTemporaryTable(new TestTable(path.toString(), true), path, false); + return this; + } + + public CatalogStructureBuilder temporaryView(ObjectIdentifier path, String query) { + this.catalogManager.createTemporaryTable( + new TestView( + query, + query, + TableSchema.builder().build(), + Collections.emptyMap(), + "", + true, + path.toString()), + path, + false); + return this; + } + public CatalogStructureBuilder catalog( String name, DatabaseBuilder defaultDatabase, DatabaseBuilder... databases) throws Exception { - GenericInMemoryCatalog catalog = buildCatalog(name, defaultDatabase, databases); catalogManager.registerCatalog(name, catalog); @@ -158,33 +181,46 @@ public String getName() { } public TestTable build(String path) { - return new TestTable(path + "." + name); + return new TestTable(path + "." + name, false); } } - private static class TestTable extends ConnectorCatalogTable { + /** + * A test {@link CatalogTable}. + */ + public static class TestTable extends ConnectorCatalogTable { private final String fullyQualifiedPath; + private final boolean isTemporary; - private static final StreamTableSource tableSource = new StreamTableSource() { - @Override - public DataStream getDataStream(StreamExecutionEnvironment execEnv) { - return null; - } + public boolean isTemporary() { + return isTemporary; + } - @Override - public TypeInformation getReturnType() { - return Types.ROW(); - } + private TestTable(String fullyQualifiedPath, boolean isTemporary) { + super(new StreamTableSource() { + @Override + public DataStream getDataStream(StreamExecutionEnvironment execEnv) { + return null; + } - @Override - public TableSchema getTableSchema() { - return TableSchema.builder().build(); - } - }; + @Override + public TypeInformation getReturnType() { + return Types.ROW(); + } + + @Override + public TableSchema getTableSchema() { + return TableSchema.builder().build(); + } + + @Override + public String explainSource() { + return String.format("isTemporary=[%s]", isTemporary); + } + }, null, TableSchema.builder().build(), false); - private TestTable(String fullyQualifiedPath) { - super(tableSource, null, tableSource.getTableSchema(), false); this.fullyQualifiedPath = fullyQualifiedPath; + this.isTemporary = isTemporary; } @Override @@ -196,12 +232,71 @@ public boolean equals(Object o) { return false; } TestTable testTable = (TestTable) o; - return Objects.equals(fullyQualifiedPath, testTable.fullyQualifiedPath); + return Objects.equals(fullyQualifiedPath, testTable.fullyQualifiedPath) && + Objects.equals(isTemporary, testTable.isTemporary); + } + + @Override + public int hashCode() { + return Objects.hash(fullyQualifiedPath, isTemporary); + } + } + + /** + * A test {@link CatalogView}. + */ + public static class TestView extends AbstractCatalogView { + private final boolean isTemporary; + private final String fullyQualifiedPath; + + public boolean isTemporary() { + return isTemporary; + } + + private TestView( + String originalQuery, + String expandedQuery, + TableSchema schema, + Map properties, + String comment, + boolean isTemporary, + String fullyQualifiedPath) { + super(originalQuery, expandedQuery, schema, properties, comment); + this.isTemporary = isTemporary; + this.fullyQualifiedPath = fullyQualifiedPath; + } + + @Override + public CatalogBaseTable copy() { + return this; + } + + @Override + public Optional getDescription() { + return Optional.empty(); + } + + @Override + public Optional getDetailedDescription() { + return Optional.empty(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TestView testView = (TestView) o; + return isTemporary == testView.isTemporary && + Objects.equals(fullyQualifiedPath, testView.fullyQualifiedPath); } @Override public int hashCode() { - return Objects.hash(fullyQualifiedPath); + return Objects.hash(isTemporary, fullyQualifiedPath); } } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/DatabaseCalciteSchemaTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/DatabaseCalciteSchemaTest.java index ea447bb16592..ba06d1c93bf1 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/DatabaseCalciteSchemaTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/DatabaseCalciteSchemaTest.java @@ -48,10 +48,11 @@ public class DatabaseCalciteSchemaTest { @Test public void testCatalogTable() throws TableAlreadyExistException, DatabaseNotExistException { GenericInMemoryCatalog catalog = new GenericInMemoryCatalog(catalogName, databaseName); + CatalogManager catalogManager = new CatalogManager(catalogName, catalog); DatabaseCalciteSchema calciteSchema = new DatabaseCalciteSchema(true, databaseName, catalogName, - catalog); + catalogManager); catalog.createTable(new ObjectPath(databaseName, tableName), new TestCatalogBaseTable(), false); Table table = calciteSchema.getTable(tableName); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/PathResolutionTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/PathResolutionTest.java index 0142ebafa523..c2e323f07333 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/PathResolutionTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/PathResolutionTest.java @@ -43,6 +43,7 @@ import static org.apache.flink.table.catalog.CatalogStructureBuilder.root; import static org.apache.flink.table.catalog.CatalogStructureBuilder.table; import static org.apache.flink.table.catalog.PathResolutionTest.TestSpec.testSpec; +import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.assertThat; /** @@ -110,7 +111,13 @@ public static List testData() throws Exception { .withCatalogManager(catalogWithSpecialCharacters()) .tableApiLookupPath("default db", "tab 1") .sqlLookupPath("`default db`.`tab 1`") - .expectPath(BUILTIN_CATALOG_NAME, "default db", "tab 1") + .expectPath(BUILTIN_CATALOG_NAME, "default db", "tab 1"), + + testSpec("shadowingWithTemporaryTable") + .withCatalogManager(catalogWithTemporaryObjects()) + .tableApiLookupPath("cat1", "db1", "tab1") + .sqlLookupPath("cat1.db1.tab1") + .expectTemporaryPath("cat1", "db1", "tab1") ); } @@ -139,6 +146,22 @@ private static CatalogManager simpleCatalog() throws Exception { ).build(); } + private static CatalogManager catalogWithTemporaryObjects() throws Exception { + return root() + .builtin( + database("default") + ) + .catalog( + "cat1", + database( + "db1", + table("tab1") + ) + ) + .temporaryTable(ObjectIdentifier.of("cat1", "db1", "tab1")) + .build(); + } + private static CatalogManager catalogWithSpecialCharacters() throws Exception { return root() .builtin( @@ -170,9 +193,13 @@ public void testTableApiPathResolution() { UnresolvedIdentifier unresolvedIdentifier = UnresolvedIdentifier.of(lookupPath.toArray(new String[0])); ObjectIdentifier identifier = catalogManager.qualifyIdentifier(unresolvedIdentifier); + assertThat( Arrays.asList(identifier.getCatalogName(), identifier.getDatabaseName(), identifier.getObjectName()), CoreMatchers.equalTo(testSpec.getExpectedPath())); + Optional tableLookup = catalogManager.getTable(identifier); + assertThat(tableLookup.isPresent(), is(true)); + assertThat(tableLookup.get().isTemporary(), is(testSpec.isTemporaryObject())); } @Test @@ -186,8 +213,9 @@ public void testStreamSqlPathResolution() { util.verifyJavaSql( format("SELECT * FROM %s", testSpec.getSqlPathToLookup()), format( - "StreamTableSourceScan(table=[[%s]], fields=[], source=[()])", - String.join(", ", testSpec.getExpectedPath())) + "StreamTableSourceScan(table=[[%s]], fields=[], source=[isTemporary=[%s]])", + String.join(", ", testSpec.getExpectedPath()), + testSpec.isTemporaryObject()) ); } @@ -197,6 +225,7 @@ static class TestSpec { private String sqlPathToLookup; private List tableApiLookupPath; private List expectedPath; + private boolean isTemporaryObject = false; private String defaultCatalog; private String defaultDatabase; private CatalogManager catalogManager; @@ -238,6 +267,11 @@ public TestSpec expectPath(String... expectedPath) { return this; } + public TestSpec expectTemporaryPath(String... expectedPath) { + this.isTemporaryObject = true; + return expectPath(expectedPath); + } + public TestSpec withDefaultPath(String defaultCatalog) { this.defaultCatalog = defaultCatalog; return this; @@ -273,6 +307,10 @@ public Optional getDefaultDatabase() { return Optional.ofNullable(defaultDatabase); } + public boolean isTemporaryObject() { + return isTemporaryObject; + } + @Override public String toString() { @@ -287,6 +325,10 @@ public String toString() { properties.add("defaultDatabase: " + defaultDatabase); } + if (isTemporaryObject) { + properties.add("temporary: true"); + } + properties.add("sqlPath: " + sqlPathToLookup); properties.add("tableApiPath: " + tableApiLookupPath); properties.add("expectedPath: " + expectedPath); diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala index b0f11f3b6079..d693b33c6f2d 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala @@ -101,4 +101,8 @@ class MockTableEnvironment extends TableEnvironment { override def loadModule(moduleName: String, module: Module): Unit = ??? override def unloadModule(moduleName: String): Unit = ??? + + override def listTemporaryTables(): Array[String] = ??? + + override def listTemporaryViews(): Array[String] = ??? } From 0c24cecf8b49b44de0f0bc443c8d36ff12c39edc Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Mon, 21 Oct 2019 10:38:18 +0200 Subject: [PATCH 334/746] [FLINK-14490][table-api] Add createTemporaryView --- .../table/api/java/BatchTableEnvironment.java | 78 +++++++++++++-- .../api/java/StreamTableEnvironment.java | 98 +++++++++++++++---- .../internal/StreamTableEnvironmentImpl.java | 14 ++- .../flink/table/api/TableEnvironment.java | 43 +++++++- .../api/internal/TableEnvironmentImpl.java | 31 ++++-- .../api/scala/BatchTableEnvironment.scala | 68 ++++++++++++- .../api/scala/StreamTableEnvironment.scala | 81 ++++++++++++--- .../internal/StreamTableEnvironmentImpl.scala | 12 +++ .../table/api/internal/TableEnvImpl.scala | 25 ++++- .../internal/BatchTableEnvironmentImpl.scala | 13 +++ .../internal/BatchTableEnvironmentImpl.scala | 13 +++ .../table/utils/MockTableEnvironment.scala | 4 + 12 files changed, 418 insertions(+), 62 deletions(-) diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/BatchTableEnvironment.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/BatchTableEnvironment.java index 38d0063c4788..4ffd117e95af 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/BatchTableEnvironment.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/BatchTableEnvironment.java @@ -109,24 +109,50 @@ public interface BatchTableEnvironment extends TableEnvironment { Table fromDataSet(DataSet dataSet, String fields); /** - * Registers the given {@link DataSet} as table in the - * {@link TableEnvironment}'s catalog. - * Registered tables can be referenced in SQL queries. + * Creates a view from the given {@link DataSet}. + * Registered views can be referenced in SQL queries. * - * The field names of the {@link Table} are automatically derived from the type of the{@link DataSet}. + *

The field names of the {@link Table} are automatically derived + * from the type of the {@link DataSet}. + * + *

The view is registered in the namespace of the current catalog and database. To register the view in + * a different catalog use {@link #createTemporaryView(String, DataSet)}. + * + *

Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. * * @param name The name under which the {@link DataSet} is registered in the catalog. * @param dataSet The {@link DataSet} to register. * @param The type of the {@link DataSet} to register. + * @deprecated use {@link #createTemporaryView(String, DataSet)} */ + @Deprecated void registerDataSet(String name, DataSet dataSet); /** - * Registers the given {@link DataSet} as table with specified field names in the - * {@link TableEnvironment}'s catalog. - * Registered tables can be referenced in SQL queries. + * Creates a view from the given {@link DataSet} in a given path. + * Registered views can be referenced in SQL queries. * - * Example: + *

The field names of the {@link Table} are automatically derived + * from the type of the {@link DataSet}. + * + *

Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * + * @param path The path under which the view is created. + * See also the {@link TableEnvironment} class description for the format of the path. + * @param dataSet The {@link DataSet} out of which to create the view. + * @param The type of the {@link DataSet}. + */ + void createTemporaryView(String path, DataSet dataSet); + + /** + * Creates a view from the given {@link DataSet} in a given path with specified field names. + * Registered views can be referenced in SQL queries. + * + *

Example: * *

 	 * {@code
@@ -135,13 +161,47 @@ public interface BatchTableEnvironment extends TableEnvironment {
 	 * }
 	 * 
* + *

The view is registered in the namespace of the current catalog and database. To register the view in + * a different catalog use {@link #createTemporaryView(String, DataSet)}. + * + *

Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * * @param name The name under which the {@link DataSet} is registered in the catalog. * @param dataSet The {@link DataSet} to register. - * @param fields The field names of the registered table. + * @param fields The field names of the registered view. * @param The type of the {@link DataSet} to register. + * @deprecated use {@link #createTemporaryView(String, DataSet, String)} */ + @Deprecated void registerDataSet(String name, DataSet dataSet, String fields); + /** + * Creates a view from the given {@link DataSet} in a given path with specified field names. + * Registered views can be referenced in SQL queries. + * + *

Example: + * + *

+	 * {@code
+	 *   DataSet> set = ...
+	 *   tableEnv.createTemporaryView("cat.db.myTable", set, "a, b");
+	 * }
+	 * 
+ * + *

Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * + * @param path The path under which the view is created. + * See also the {@link TableEnvironment} class description for the format of the path. + * @param dataSet The {@link DataSet} out of which to create the view. + * @param fields The field names of the registered view. + * @param The type of the {@link DataSet}. + */ + void createTemporaryView(String path, DataSet dataSet, String fields); + /** * Converts the given {@link Table} into a {@link DataSet} of a specified type. * diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/StreamTableEnvironment.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/StreamTableEnvironment.java index a18f3d4226d1..5bf322996eb9 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/StreamTableEnvironment.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/StreamTableEnvironment.java @@ -39,7 +39,7 @@ import org.apache.flink.table.sinks.TableSink; /** - * This table environment is the entry point and central context for creating Table & SQL + * This table environment is the entry point and central context for creating Table and SQL * API programs that integrate with the Java-specific {@link DataStream} API. * *

It is unified for bounded and unbounded data processing. @@ -60,7 +60,7 @@ public interface StreamTableEnvironment extends TableEnvironment { /** - * Creates a table environment that is the entry point and central context for creating Table & SQL + * Creates a table environment that is the entry point and central context for creating Table and SQL * API programs that integrate with the Java-specific {@link DataStream} API. * *

It is unified for bounded and unbounded data processing. @@ -86,7 +86,7 @@ static StreamTableEnvironment create(StreamExecutionEnvironment executionEnviron } /** - * Creates a table environment that is the entry point and central context for creating Table & SQL + * Creates a table environment that is the entry point and central context for creating Table and SQL * API programs that integrate with the Java-specific {@link DataStream} API. * *

It is unified for bounded and unbounded data processing. @@ -117,7 +117,7 @@ static StreamTableEnvironment create( } /** - * Creates a table environment that is the entry point and central context for creating Table & SQL + * Creates a table environment that is the entry point and central context for creating Table and SQL * API programs that integrate with the Java-specific {@link DataStream} API. * *

It is unified for bounded and unbounded data processing. @@ -164,7 +164,7 @@ static StreamTableEnvironment create(StreamExecutionEnvironment executionEnviron * @param name The name under which the function is registered. * @param aggregateFunction The AggregateFunction to register. * @param The type of the output value. - * @tparam ACC The type of aggregate accumulator. + * @param The type of aggregate accumulator. */ void registerFunction(String name, AggregateFunction aggregateFunction); @@ -175,7 +175,7 @@ static StreamTableEnvironment create(StreamExecutionEnvironment executionEnviron * @param name The name under which the function is registered. * @param tableAggregateFunction The TableAggregateFunction to register. * @param The type of the output value. - * @tparam ACC The type of aggregate accumulator. + * @param The type of aggregate accumulator. */ void registerFunction(String name, TableAggregateFunction tableAggregateFunction); @@ -194,7 +194,7 @@ static StreamTableEnvironment create(StreamExecutionEnvironment executionEnviron /** * Converts the given {@link DataStream} into a {@link Table} with specified field names. * - * Example: + *

Example: * *

 	 * {@code
@@ -211,43 +211,103 @@ static StreamTableEnvironment create(StreamExecutionEnvironment executionEnviron
 	 Table fromDataStream(DataStream dataStream, String fields);
 
 	/**
-	 * Registers the given {@link DataStream} as table in the {@link TableEnvironment}'s catalog.
-	 * Registered tables can be referenced in SQL queries.
+	 * Creates a view from the given {@link DataStream}.
+	 * Registered views can be referenced in SQL queries.
 	 *
-	 * The field names of the {@link Table} are automatically derived
+	 * 

The field names of the {@link Table} are automatically derived * from the type of the {@link DataStream}. * + *

The view is registered in the namespace of the current catalog and database. To register the view in + * a different catalog use {@link #createTemporaryView(String, DataStream)}. + * + *

Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * * @param name The name under which the {@link DataStream} is registered in the catalog. * @param dataStream The {@link DataStream} to register. * @param The type of the {@link DataStream} to register. + * @deprecated use {@link #createTemporaryView(String, DataStream)} */ + @Deprecated void registerDataStream(String name, DataStream dataStream); /** - * Registers the given {@link DataStream} as table with specified field names in the - * {@link TableEnvironment}'s catalog. - * Registered tables can be referenced in SQL queries. + * Creates a view from the given {@link DataStream} in a given path. + * Registered views can be referenced in SQL queries. + * + *

The field names of the {@link Table} are automatically derived + * from the type of the {@link DataStream}. * - * Example: + *

Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * + * @param path The path under which the {@link DataStream} is created. + * See also the {@link TableEnvironment} class description for the format of the path. + * @param dataStream The {@link DataStream} out of which to create the view. + * @param The type of the {@link DataStream}. + */ + void createTemporaryView(String path, DataStream dataStream); + + /** + * Creates a view from the given {@link DataStream} in a given path with specified field names. + * Registered views can be referenced in SQL queries. + * + *

Example: * *

 	 * {@code
-	 *   DataStream> set = ...
-	 *   tableEnv.registerDataStream("myTable", set, "a, b")
+	 *   DataStream> stream = ...
+	 *   tableEnv.registerDataStream("myTable", stream, "a, b")
 	 * }
 	 * 
* + *

The view is registered in the namespace of the current catalog and database. To register the view in + * a different catalog use {@link #createTemporaryView(String, DataStream)}. + * + *

Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * * @param name The name under which the {@link DataStream} is registered in the catalog. * @param dataStream The {@link DataStream} to register. - * @param fields The field names of the registered table. + * @param fields The field names of the registered view. * @param The type of the {@link DataStream} to register. + * @deprecated use {@link #createTemporaryView(String, DataStream, String)} */ + @Deprecated void registerDataStream(String name, DataStream dataStream, String fields); + /** + * Creates a view from the given {@link DataStream} in a given path with specified field names. + * Registered views can be referenced in SQL queries. + * + *

Example: + * + *

+	 * {@code
+	 *   DataStream> stream = ...
+	 *   tableEnv.createTemporaryView("cat.db.myTable", stream, "a, b")
+	 * }
+	 * 
+ * + *

Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * + * @param path The path under which the {@link DataStream} is created. + * See also the {@link TableEnvironment} class description for the format of the path. + * @param dataStream The {@link DataStream} out of which to create the view. + * @param fields The field names of the created view. + * @param The type of the {@link DataStream}. + */ + void createTemporaryView(String path, DataStream dataStream, String fields); + /** * Converts the given {@link Table} into an append {@link DataStream} of a specified type. * - * The {@link Table} must only have insert (append) changes. If the {@link Table} is also modified + *

The {@link Table} must only have insert (append) changes. If the {@link Table} is also modified * by update or delete changes, the conversion will fail. * *

The fields of the {@link Table} are mapped to {@link DataStream} fields as follows: @@ -267,7 +327,7 @@ static StreamTableEnvironment create(StreamExecutionEnvironment executionEnviron /** * Converts the given {@link Table} into an append {@link DataStream} of a specified type. * - * The {@link Table} must only have insert (append) changes. If the {@link Table} is also modified + *

The {@link Table} must only have insert (append) changes. If the {@link Table} is also modified * by update or delete changes, the conversion will fail. * *

The fields of the {@link Table} are mapped to {@link DataStream} fields as follows: 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 4addf4fe5e18..962b0d3cac7c 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 @@ -209,12 +209,22 @@ public Table fromDataStream(DataStream dataStream, String fields) { @Override public void registerDataStream(String name, DataStream dataStream) { - registerTable(name, fromDataStream(dataStream)); + createTemporaryView(name, dataStream); + } + + @Override + public void createTemporaryView(String path, DataStream dataStream) { + createTemporaryView(path, fromDataStream(dataStream)); } @Override public void registerDataStream(String name, DataStream dataStream, String fields) { - registerTable(name, fromDataStream(dataStream, fields)); + createTemporaryView(name, dataStream, fields); + } + + @Override + public void createTemporaryView(String path, DataStream dataStream, String fields) { + createTemporaryView(path, fromDataStream(dataStream, fields)); } @Override diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java index 5b5c14657351..8cc02f8d6a94 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java @@ -36,7 +36,7 @@ import java.util.Optional; /** - * A table environment is the base class, entry point, and central context for creating Table & SQL + * A table environment is the base class, entry point, and central context for creating Table and SQL * API programs. * *

It is unified both on a language level for all JVM-based languages (i.e. there is no distinction @@ -50,6 +50,14 @@ *

  • Offering further configuration options.
  • * * + *

    The path in methods such as {@link #createTemporaryView(String, Table)} should be a proper SQL identifier. + * The syntax is following [[catalog-name.]database-name.]object-name, where the catalog name and database are + * optional. For path resolution see {@link #useCatalog(String)} and {@link #useDatabase(String)}. All keywords + * or other special characters need to be escaped. + * + *

    Example: `cat.1`.`db`.`Table` resolves to an object named 'Table' (table is a reserved keyword, thus must + * be escaped) in a catalog named 'cat.1' and database named 'db'. + * *

    Note: This environment is meant for pure table programs. If you would like to convert from or to * other Flink APIs, it might be necessary to use one of the available language-specific table environments * in the corresponding bridging modules. @@ -58,7 +66,7 @@ public interface TableEnvironment { /** - * Creates a table environment that is the entry point and central context for creating Table & SQL + * Creates a table environment that is the entry point and central context for creating Table and SQL * API programs. * *

    It is unified both on a language level for all JVM-based languages (i.e. there is no distinction @@ -133,15 +141,38 @@ static TableEnvironment create(EnvironmentSettings settings) { * Registers a {@link Table} under a unique name in the TableEnvironment's catalog. * Registered tables can be referenced in SQL queries. * + *

    Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * * @param name The name under which the table will be registered. * @param table The table to register. + * @deprecated use {@link #createTemporaryView(String, Table)} */ + @Deprecated void registerTable(String name, Table table); + /** + * Registers a {@link Table} API object as a temporary view similar to SQL temporary views. + * + *

    Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * + * @param path The path under which the view will be registered. + * See also the {@link TableEnvironment} class description for the format of the path. + * @param view The view to register. + */ + void createTemporaryView(String path, Table view); + /** * Registers an external {@link TableSource} in this {@link TableEnvironment}'s catalog. * Registered tables can be referenced in SQL queries. * + *

    Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * * @param name The name under which the {@link TableSource} is registered. * @param tableSource The {@link TableSource} to register. */ @@ -152,6 +183,10 @@ static TableEnvironment create(EnvironmentSettings settings) { * {@link TableEnvironment}'s catalog. * Registered sink tables can be referenced in SQL DML statements. * + *

    Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * * @param name The name under which the {@link TableSink} is registered. * @param fieldNames The field names to register with the {@link TableSink}. * @param fieldTypes The field types to register with the {@link TableSink}. @@ -166,6 +201,10 @@ static TableEnvironment create(EnvironmentSettings settings) { * this {@link TableEnvironment}'s catalog. * Registered sink tables can be referenced in SQL DML statements. * + *

    Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * * @param name The name under which the {@link TableSink} is registered. * @param configuredSink The configured {@link TableSink} to register. */ 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 a2d7bf4fb270..d16f0891f6c2 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 @@ -192,13 +192,26 @@ public void registerFunction(String name, ScalarFunction function) { @Override public void registerTable(String name, Table table) { - if (((TableImpl) table).getTableEnvironment() != this) { + UnresolvedIdentifier identifier = UnresolvedIdentifier.of(name); + createTemporaryView(identifier, table); + } + + @Override + public void createTemporaryView(String path, Table view) { + UnresolvedIdentifier identifier = parser.parseIdentifier(path); + createTemporaryView(identifier, view); + } + + private void createTemporaryView(UnresolvedIdentifier identifier, Table view) { + if (((TableImpl) view).getTableEnvironment() != this) { throw new TableException( - "Only tables that belong to this TableEnvironment can be registered."); + "Only table API objects that belong to this TableEnvironment can be registered."); } - CatalogBaseTable tableTable = new QueryOperationCatalogView(table.getQueryOperation()); - catalogManager.createTemporaryTable(tableTable, getTemporaryObjectIdentifier(name), false); + CatalogBaseTable tableTable = new QueryOperationCatalogView(view.getQueryOperation()); + + ObjectIdentifier tableIdentifier = catalogManager.qualifyIdentifier(identifier); + catalogManager.createTemporaryTable(tableTable, tableIdentifier, false); } @Override @@ -236,9 +249,9 @@ public Table scan(String... tablePath) { } private Optional scanInternal(String... tablePath) { - ObjectIdentifier objectIdentifier = catalogManager.qualifyIdentifier(UnresolvedIdentifier.of(tablePath)); - return catalogManager.getTable(objectIdentifier) - .map(t -> new CatalogQueryOperation(objectIdentifier, t.getTable().getSchema())); + ObjectIdentifier tableIdentifier = catalogManager.qualifyIdentifier(UnresolvedIdentifier.of(tablePath)); + return catalogManager.getTable(tableIdentifier) + .map(t -> new CatalogQueryOperation(tableIdentifier, t.getTable().getSchema())); } @Override @@ -348,11 +361,11 @@ public void insertInto(Table table, String path, String... pathContinued) { List fullPath = new ArrayList<>(Arrays.asList(pathContinued)); fullPath.add(0, path); - ObjectIdentifier objectIdentifier = catalogManager.qualifyIdentifier( + ObjectIdentifier tableIdentifier = catalogManager.qualifyIdentifier( UnresolvedIdentifier.of(fullPath.toArray(new String[0]))); List modifyOperations = Collections.singletonList( new CatalogSinkModifyOperation( - objectIdentifier, + tableIdentifier, table.getQueryOperation())); if (isEagerOperationTranslation()) { diff --git a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/BatchTableEnvironment.scala b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/BatchTableEnvironment.scala index c57b0351e990..6790dc647c6d 100644 --- a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/BatchTableEnvironment.scala +++ b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/BatchTableEnvironment.scala @@ -95,23 +95,48 @@ trait BatchTableEnvironment extends TableEnvironment { def fromDataSet[T](dataSet: DataSet[T], fields: Expression*): Table /** - * Registers the given [[DataSet]] as table in the - * [[TableEnvironment]]'s catalog. - * Registered tables can be referenced in SQL queries. + * Creates a view from the given [[DataSet]]. + * Registered views can be referenced in SQL queries. * * The field names of the [[Table]] are automatically derived from the type of the [[DataSet]]. * + * The view is registered in the namespace of the current catalog and database. To register the + * view in a different catalog use [[createTemporaryView]]. + * + * Temporary objects can shadow permanent ones. If a permanent object in a given path exists, + * it will be inaccessible in the current session. To make the permanent object available again + * you can drop the corresponding temporary object. + * * @param name The name under which the [[DataSet]] is registered in the catalog. * @param dataSet The [[DataSet]] to register. * @tparam T The type of the [[DataSet]] to register. + * @deprecated use [[createTemporaryView]] */ + @deprecated def registerDataSet[T](name: String, dataSet: DataSet[T]): Unit /** - * Registers the given [[DataSet]] as table with specified field names in the - * [[TableEnvironment]]'s catalog. + * Creates a view from the given [[DataSet]] in a given path. * Registered tables can be referenced in SQL queries. * + * The field names of the [[Table]] are automatically derived + * from the type of the [[DataSet]]. + * + * Temporary objects can shadow permanent ones. If a permanent object in a given path exists, + * it will be inaccessible in the current session. To make the permanent object available again + * you can drop the corresponding temporary object. + * + * @param path The path under which the [[DataSet]] is created. + * See also the [[TableEnvironment]] class description for the format of the path. + * @param dataSet The [[DataSet]] out of which to create the view. + * @tparam T The type of the [[DataSet]]. + */ + def createTemporaryView[T](path: String, dataSet: DataSet[T]): Unit + + /** + * Creates a view from the given [[DataSet]] in a given path with specified field names. + * Registered views can be referenced in SQL queries. + * * Example: * * {{{ @@ -119,13 +144,46 @@ trait BatchTableEnvironment extends TableEnvironment { * tableEnv.registerDataSet("myTable", set, 'a, 'b) * }}} * + * The view is registered in the namespace of the current catalog and database. To register the + * view in a different catalog use [[createTemporaryView]]. + * + * Temporary objects can shadow permanent ones. If a permanent object in a given path exists, + * it will be inaccessible in the current session. To make the permanent object available again + * you can drop the corresponding temporary object. + * * @param name The name under which the [[DataSet]] is registered in the catalog. * @param dataSet The [[DataSet]] to register. * @param fields The field names of the registered table. * @tparam T The type of the [[DataSet]] to register. + * @deprecated use [[createTemporaryView]] */ + @deprecated def registerDataSet[T](name: String, dataSet: DataSet[T], fields: Expression*): Unit + /** + * Creates a view from the given [[DataSet]] in a given path with specified field names. + * Registered views can be referenced in SQL queries. + * + * Example: + * + * {{{ + * val set: DataSet[(String, Long)] = ... + * tableEnv.createTemporaryView("cat.db.myTable", set, 'a, 'b) + * }}} + * + * Temporary objects can shadow permanent ones. If a permanent object in a given path exists, + * it will be inaccessible in the current session. To make the permanent object available again + * you can drop the corresponding temporary object. + * + * @param path The path under which the [[DataSet]] is created. + * See also the [[TableEnvironment]] class description for the format of the + * path. + * @param dataSet The [[DataSet]] out of which to create the view. + * @param fields The field names of the created view. + * @tparam T The type of the [[DataSet]]. + */ + def createTemporaryView[T](path: String, dataSet: DataSet[T], fields: Expression*): Unit + /** * Converts the given [[Table]] into a [[DataSet]] of a specified type. * diff --git a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala index db8825163092..1ed49754ede0 100644 --- a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala +++ b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala @@ -29,7 +29,7 @@ import org.apache.flink.table.functions.{AggregateFunction, TableAggregateFuncti import org.apache.flink.table.sinks.TableSink /** - * This table environment is the entry point and central context for creating Table & SQL + * This table environment is the entry point and central context for creating Table and SQL * API programs that integrate with the Scala-specific [[DataStream]] API. * * It is unified for bounded and unbounded data processing. @@ -113,38 +113,95 @@ trait StreamTableEnvironment extends TableEnvironment { def fromDataStream[T](dataStream: DataStream[T], fields: Expression*): Table /** - * Registers the given [[DataStream]] as table in the - * [[TableEnvironment]]'s catalog. - * Registered tables can be referenced in SQL queries. + * Creates a view from the given [[DataStream]]. + * Registered views can be referenced in SQL queries. * * The field names of the [[Table]] are automatically derived * from the type of the [[DataStream]]. * + * The view is registered in the namespace of the current catalog and database. To register the + * view in a different catalog use [[createTemporaryView]]. + * + * Temporary objects can shadow permanent ones. If a permanent object in a given path exists, + * it will be inaccessible in the current session. To make the permanent object available again + * you can drop the corresponding temporary object. + * * @param name The name under which the [[DataStream]] is registered in the catalog. * @param dataStream The [[DataStream]] to register. * @tparam T The type of the [[DataStream]] to register. + * @deprecated use [[createTemporaryView]] */ + @deprecated def registerDataStream[T](name: String, dataStream: DataStream[T]): Unit /** - * Registers the given [[DataStream]] as table with specified field names in the - * [[TableEnvironment]]'s catalog. + * Creates a view from the given [[DataStream]] in a given path. * Registered tables can be referenced in SQL queries. * + * The field names of the [[Table]] are automatically derived + * from the type of the [[DataStream]]. + * + * Temporary objects can shadow permanent ones. If a permanent object in a given path exists, + * it will be inaccessible in the current session. To make the permanent object available again + * you can drop the corresponding temporary object. + * + * @param path The path under which the [[DataStream]] is created. + * See also the [[TableEnvironment]] class description for the format of the path. + * @param dataStream The [[DataStream]] out of which to create the view. + * @tparam T The type of the [[DataStream]]. + */ + def createTemporaryView[T](path: String, dataStream: DataStream[T]): Unit + + /** + * Creates a view from the given [[DataStream]] in a given path with specified field names. + * Registered views can be referenced in SQL queries. + * * Example: * * {{{ - * val set: DataStream[(String, Long)] = ... - * tableEnv.registerDataStream("myTable", set, 'a, 'b) + * val stream: DataStream[(String, Long)] = ... + * tableEnv.registerDataStream("myTable", stream, 'a, 'b) * }}} * + * The view is registered in the namespace of the current catalog and database. To register the + * view in a different catalog use [[createTemporaryView]]. + * + * Temporary objects can shadow permanent ones. If a permanent object in a given path exists, + * it will be inaccessible in the current session. To make the permanent object available again + * you can drop the corresponding temporary object. + * * @param name The name under which the [[DataStream]] is registered in the catalog. * @param dataStream The [[DataStream]] to register. - * @param fields The field names of the registered table. + * @param fields The field names of the registered view. * @tparam T The type of the [[DataStream]] to register. + * @deprecated use [[createTemporaryView]] */ + @deprecated def registerDataStream[T](name: String, dataStream: DataStream[T], fields: Expression*): Unit + /** + * Creates a view from the given [[DataStream]] in a given path with specified field names. + * Registered views can be referenced in SQL queries. + * + * Example: + * + * {{{ + * val stream: DataStream[(String, Long)] = ... + * tableEnv.createTemporaryView("cat.db.myTable", stream, 'a, 'b) + * }}} + * + * Temporary objects can shadow permanent ones. If a permanent object in a given path exists, + * it will be inaccessible in the current session. To make the permanent object available again + * you can drop the corresponding temporary object. + * + * @param path The path under which the [[DataStream]] is created. + * See also the [[TableEnvironment]] class description for the format of the path. + * @param dataStream The [[DataStream]] out of which to create the view. + * @param fields The field names of the created view. + * @tparam T The type of the [[DataStream]]. + */ + def createTemporaryView[T](path: String, dataStream: DataStream[T], fields: Expression*): Unit + /** * Converts the given [[Table]] into an append [[DataStream]] of a specified type. * @@ -313,7 +370,7 @@ trait StreamTableEnvironment extends TableEnvironment { object StreamTableEnvironment { /** - * Creates a table environment that is the entry point and central context for creating Table & + * Creates a table environment that is the entry point and central context for creating Table and * SQL API programs that integrate with the Scala-specific [[DataStream]] API. * * It is unified for bounded and unbounded data processing. @@ -339,7 +396,7 @@ object StreamTableEnvironment { } /** - * Creates a table environment that is the entry point and central context for creating Table & + * Creates a table environment that is the entry point and central context for creating Table and * SQL API programs that integrate with the Scala-specific [[DataStream]] API. * * It is unified for bounded and unbounded data processing. @@ -367,7 +424,7 @@ object StreamTableEnvironment { } /** - * Creates a table environment that is the entry point and central context for creating Table & + * Creates a table environment that is the entry point and central context for creating Table and * SQL API programs that integrate with the Scala-specific [[DataStream]] API. * * It is unified for bounded and unbounded data processing. 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 97da79868eef..ca77bf19552a 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 @@ -266,6 +266,18 @@ class StreamTableEnvironmentImpl ( Time.milliseconds(queryConfig.getMinIdleStateRetentionTime), Time.milliseconds(queryConfig.getMaxIdleStateRetentionTime)) insertInto(table, sinkPath, sinkPathContinued: _*) + + override def createTemporaryView[T]( + path: String, + dataStream: DataStream[T]): Unit = { + createTemporaryView(path, fromDataStream(dataStream)) + } + + override def createTemporaryView[T]( + path: String, + dataStream: DataStream[T], + fields: Expression*): Unit = { + createTemporaryView(path, fromDataStream(dataStream, fields: _*)) } } 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 fe8c0d706e4c..ca99fc23ec6e 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 @@ -195,15 +195,32 @@ abstract class TableEnvImpl( } override def registerTable(name: String, table: Table): Unit = { + createTemporaryView(UnresolvedIdentifier.of(name), table) + } + + protected def parseIdentifier(identifier: String): UnresolvedIdentifier = { + val parser = planningConfigurationBuilder.createCalciteParser() + UnresolvedIdentifier.of(parser.parseIdentifier(identifier).names: _*) + } + override def createTemporaryView(path: String, view: Table): Unit = { + val identifier = parseIdentifier(path) + createTemporaryView(identifier, view) + } + + private def createTemporaryView(identifier: UnresolvedIdentifier, view: Table): Unit = { // check that table belongs to this table environment - if (table.asInstanceOf[TableImpl].getTableEnvironment != this) { + if (view.asInstanceOf[TableImpl].getTableEnvironment != this) { throw new TableException( - "Only tables that belong to this TableEnvironment can be registered.") + "Only table API objects that belong to this TableEnvironment can be registered.") } - val view = new QueryOperationCatalogView(table.getQueryOperation) - catalogManager.createTemporaryTable(view, getTemporaryObjectIdentifier(name), false) + val objectIdentifier = catalogManager.qualifyIdentifier(identifier) + + catalogManager.createTemporaryTable( + new QueryOperationCatalogView(view.getQueryOperation), + objectIdentifier, + false) } override def registerTableSource(name: String, tableSource: TableSource[_]): Unit = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/java/internal/BatchTableEnvironmentImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/java/internal/BatchTableEnvironmentImpl.scala index 971ba3718ac1..f79a3ba9555a 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/java/internal/BatchTableEnvironmentImpl.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/java/internal/BatchTableEnvironmentImpl.scala @@ -69,6 +69,19 @@ class BatchTableEnvironmentImpl( registerTable(name, fromDataSet(dataSet, fields)) } + override def createTemporaryView[T]( + path: String, + dataSet: DataSet[T]): Unit = { + createTemporaryView(path, fromDataSet(dataSet)) + } + + override def createTemporaryView[T]( + path: String, + dataSet: DataSet[T], + fields: String): Unit = { + createTemporaryView(path, fromDataSet(dataSet, fields)) + } + override def toDataSet[T](table: Table, clazz: Class[T]): DataSet[T] = { // Use the default query config. translate[T](table)(TypeExtractor.createTypeInfo(clazz)) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/scala/internal/BatchTableEnvironmentImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/scala/internal/BatchTableEnvironmentImpl.scala index 7324a4bbccd3..1f0a4a032974 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/scala/internal/BatchTableEnvironmentImpl.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/scala/internal/BatchTableEnvironmentImpl.scala @@ -92,5 +92,18 @@ class BatchTableEnvironmentImpl( queryConfig: BatchQueryConfig, sinkPath: String, sinkPathContinued: String*): Unit = insertInto(table, sinkPath, sinkPathContinued: _*) + + override def createTemporaryView[T]( + path: String, + dataSet: DataSet[T]): Unit = { + createTemporaryView(path, fromDataSet(dataSet)) + } + + override def createTemporaryView[T]( + path: String, + dataSet: DataSet[T], + fields: Expression*): Unit = { + createTemporaryView(path, fromDataSet(dataSet, fields: _*)) + } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala index d693b33c6f2d..776473edf841 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala @@ -102,6 +102,10 @@ class MockTableEnvironment extends TableEnvironment { override def unloadModule(moduleName: String): Unit = ??? + override def createTemporaryView( + path: String, + view: Table): Unit = ??? + override def listTemporaryTables(): Array[String] = ??? override def listTemporaryViews(): Array[String] = ??? From 8d40df4ffb30dd7661bd70f181c17c6089238c9f Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Mon, 21 Oct 2019 10:43:25 +0200 Subject: [PATCH 335/746] [FLINK-14490][table-api] Add drop temporary tables --- .../flink/table/api/TableEnvironment.java | 20 +++++++++ .../api/internal/TableEnvironmentImpl.java | 12 ++++++ .../flink/table/catalog/CatalogManager.java | 41 ++++++++++++++++++ .../table/api/internal/TableEnvImpl.scala | 12 ++++++ .../table/catalog/CatalogManagerTest.java | 43 +++++++++++++++++++ .../table/utils/MockTableEnvironment.scala | 5 +++ 6 files changed, 133 insertions(+) diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java index 8cc02f8d6a94..d1b17d5c6ec5 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java @@ -350,6 +350,26 @@ static TableEnvironment create(EnvironmentSettings settings) { */ String[] listFunctions(); + /** + * Drops a temporary table registered in the given path. + * + *

    If a permanent table with a given path exists, it will be used + * from now on for any queries that reference this path. + * + * @return true if a table existed in the given path and was removed + */ + boolean dropTemporaryTable(String path); + + /** + * Drops a temporary view registered in the given path. + * + *

    If a permanent table or view with a given path exists, it will be used + * from now on for any queries that reference this path. + * + * @return true if a view existed in the given path and was removed + */ + boolean dropTemporaryView(String path); + /** * Returns the AST of the specified Table API and SQL queries and the execution plan to compute * the result of the given {@link Table}. 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 d16f0891f6c2..3d2d8bc988e9 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 @@ -304,6 +304,18 @@ public String[] listTemporaryViews() { .toArray(String[]::new); } + @Override + public boolean dropTemporaryTable(String path) { + UnresolvedIdentifier unresolvedIdentifier = parser.parseIdentifier(path); + return catalogManager.dropTemporaryTable(unresolvedIdentifier); + } + + @Override + public boolean dropTemporaryView(String path) { + UnresolvedIdentifier unresolvedIdentifier = parser.parseIdentifier(path); + return catalogManager.dropTemporaryView(unresolvedIdentifier); + } + @Override public String[] listUserDefinedFunctions() { return functionCatalog.getUserDefinedFunctions(); diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java index 81d0ad9079ec..f50c00ff27ad 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java @@ -37,6 +37,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -475,6 +476,41 @@ public void createTemporaryTable( }); } + /** + * Qualifies the given {@link UnresolvedIdentifier} with current catalog & database and + * removes a temporary table registered with this path if it exists. + * + * @param identifier potentially unresolved identifier + * @return true if a table with a given identifier existed and was removed, false otherwise + */ + public boolean dropTemporaryTable(UnresolvedIdentifier identifier) { + return dropTemporaryTableInternal(identifier, (table) -> table instanceof CatalogTable); + } + + /** + * Qualifies the given {@link UnresolvedIdentifier} with current catalog & database and + * removes a temporary view registered with this path if it exists. + * + * @param identifier potentially unresolved identifier + * @return true if a view with a given identifier existed and was removed, false otherwise + */ + public boolean dropTemporaryView(UnresolvedIdentifier identifier) { + return dropTemporaryTableInternal(identifier, (table) -> table instanceof CatalogView); + } + + private boolean dropTemporaryTableInternal( + UnresolvedIdentifier unresolvedIdentifier, + Predicate filter) { + ObjectIdentifier objectIdentifier = qualifyIdentifier(unresolvedIdentifier); + CatalogBaseTable catalogBaseTable = temporaryTables.get(objectIdentifier); + if (filter.test(catalogBaseTable)) { + temporaryTables.remove(objectIdentifier); + return true; + } else { + return false; + } + } + /** * Alters a table in a given fully qualified path. * @@ -499,6 +535,11 @@ public void alterTable(CatalogBaseTable table, ObjectIdentifier objectIdentifier * does not exist. */ public void dropTable(ObjectIdentifier objectIdentifier, boolean ignoreIfNotExists) { + if (temporaryTables.containsKey(objectIdentifier)) { + throw new ValidationException(String.format( + "Temporary table with identifier '%s' exists. Drop it first before removing the permanent table.", + objectIdentifier)); + } execute( (catalog, path) -> catalog.dropTable(path, ignoreIfNotExists), objectIdentifier, 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 ca99fc23ec6e..232cbefc9d89 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 @@ -412,6 +412,18 @@ abstract class TableEnvImpl( .sorted } + override def dropTemporaryTable(path: String): Boolean = { + val parser = planningConfigurationBuilder.createCalciteParser() + val unresolvedIdentifier = UnresolvedIdentifier.of(parser.parseIdentifier(path).names: _*) + catalogManager.dropTemporaryTable(unresolvedIdentifier) + } + + override def dropTemporaryView(path: String): Boolean = { + val parser = planningConfigurationBuilder.createCalciteParser() + val unresolvedIdentifier = UnresolvedIdentifier.of(parser.parseIdentifier(path).names: _*) + catalogManager.dropTemporaryView(unresolvedIdentifier) + } + override def listUserDefinedFunctions(): Array[String] = functionCatalog.getUserDefinedFunctions override def listFunctions(): Array[String] = functionCatalog.getFunctions diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/CatalogManagerTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/CatalogManagerTest.java index 0c23905fc8d4..42ccef1f2263 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/CatalogManagerTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/CatalogManagerTest.java @@ -35,6 +35,7 @@ import static org.apache.flink.table.catalog.CatalogStructureBuilder.root; import static org.apache.flink.table.catalog.CatalogStructureBuilder.table; import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; @@ -135,6 +136,48 @@ public void testTemporaryTableExists() throws Exception { manager.createTemporaryTable(new CatalogTest.TestTable(), tempIdentifier, false); } + @Test + public void testDropTableWhenTemporaryTableExists() throws Exception { + ObjectIdentifier identifier = ObjectIdentifier.of(BUILTIN_CATALOG_NAME, BUILTIN_DEFAULT_DATABASE_NAME, "test"); + CatalogManager manager = root() + .builtin( + database(BUILTIN_DEFAULT_DATABASE_NAME, table("test"))) + .temporaryTable(identifier) + .build(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("Temporary table with identifier '`builtin`.`default`.`test`' exists." + + " Drop it first before removing the permanent table."); + manager.dropTable(identifier, false); + + } + + @Test + public void testDropTemporaryNonExistingTable() throws Exception { + CatalogManager manager = root() + .builtin( + database(BUILTIN_DEFAULT_DATABASE_NAME, table("test"))) + .build(); + + boolean dropped = manager.dropTemporaryTable(UnresolvedIdentifier.of("test")); + + assertThat(dropped, is(false)); + } + + @Test + public void testDropTemporaryTable() throws Exception { + ObjectIdentifier identifier = ObjectIdentifier.of(BUILTIN_CATALOG_NAME, BUILTIN_DEFAULT_DATABASE_NAME, "test"); + CatalogManager manager = root() + .builtin( + database(BUILTIN_DEFAULT_DATABASE_NAME, table("test"))) + .temporaryTable(identifier) + .build(); + + boolean dropped = manager.dropTemporaryTable(UnresolvedIdentifier.of("test")); + + assertThat(dropped, is(true)); + } + @Test public void testListTables() throws Exception { ObjectIdentifier identifier1 = ObjectIdentifier.of(TEST_CATALOG_NAME, TEST_CATALOG_DEFAULT_DB_NAME, "test1"); diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala index 776473edf841..6a4e8e6ced73 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala @@ -109,4 +109,9 @@ class MockTableEnvironment extends TableEnvironment { override def listTemporaryTables(): Array[String] = ??? override def listTemporaryViews(): Array[String] = ??? + + override def dropTemporaryTable(path: String): Boolean = ??? + + override def dropTemporaryView(path: String): Boolean = ??? + } From b286330997944bfc449694a6a7392b2f46d44801 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Mon, 21 Oct 2019 10:48:20 +0200 Subject: [PATCH 336/746] [FLINK-14490][table-api] Add from method --- .../flink/table/api/TableEnvironment.java | 41 +++++++++++++++++++ .../api/internal/TableEnvironmentImpl.java | 37 +++++++++++++---- .../table/api/TableEnvironmentTest.scala | 2 +- .../table/api/internal/TableEnvImpl.scala | 35 +++++++++++++--- .../table/utils/MockTableEnvironment.scala | 2 + 5 files changed, 103 insertions(+), 14 deletions(-) diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java index d1b17d5c6ec5..7ab7feb0dd10 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java @@ -239,9 +239,50 @@ static TableEnvironment create(EnvironmentSettings settings) { * @return The resulting {@link Table}. * @see TableEnvironment#useCatalog(String) * @see TableEnvironment#useDatabase(String) + * @deprecated use {@link #from(String)} */ + @Deprecated Table scan(String... tablePath); + /** + * Reads a registered table and returns the resulting {@link Table}. + * + *

    A table to scan must be registered in the {@link TableEnvironment}. + * + *

    See the documentation of {@link TableEnvironment#useDatabase(String)} or + * {@link TableEnvironment#useCatalog(String)} for the rules on the path resolution. + * + *

    Examples: + * + *

    Reading a table from default catalog and database. + *

    +	 * {@code
    +	 *   Table tab = tableEnv.from("tableName");
    +	 * }
    +	 * 
    + * + *

    Reading a table from a registered catalog. + *

    +	 * {@code
    +	 *   Table tab = tableEnv.from("catalogName.dbName.tableName");
    +	 * }
    +	 * 
    + * + *

    Reading a table from a registered catalog with escaping. ({@code Table} is a reserved keyword). + * Dots in e.g. a database name also must be escaped. + *

    +	 * {@code
    +	 *   Table tab = tableEnv.from("catalogName.`db.Name`.`Table`");
    +	 * }
    +	 * 
    + * + * @param path The path of a table API object to scan. + * @return Either a table or virtual table (=view). + * @see TableEnvironment#useCatalog(String) + * @see TableEnvironment#useDatabase(String) + */ + Table from(String path); + /** * Writes the {@link Table} to a {@link TableSink} that was registered under the specified name. * 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 3d2d8bc988e9..da957915d042 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 @@ -24,6 +24,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.dag.Transformation; import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.SqlParserException; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableEnvironment; @@ -115,8 +116,17 @@ protected TableEnvironmentImpl( this.operationTreeBuilder = OperationTreeBuilder.create( functionCatalog, path -> { - Optional catalogTableOperation = scanInternal(path); - return catalogTableOperation.map(tableOperation -> new TableReferenceExpression(path, tableOperation)); + try { + UnresolvedIdentifier unresolvedIdentifier = parser.parseIdentifier(path); + Optional catalogQueryOperation = scanInternal(unresolvedIdentifier); + return catalogQueryOperation.map(t -> new TableReferenceExpression(path, t)); + } catch (SqlParserException ex) { + // The TableLookup is used during resolution of expressions and it actually might not be an + // identifier of a table. It might be a reference to some other object such as column, local + // reference etc. This method should return empty optional in such cases to fallback for other + // identifiers resolution. + return Optional.empty(); + } }, isStreamingMode ); @@ -242,14 +252,27 @@ public void registerTableSink(String name, TableSink configuredSink) { @Override public Table scan(String... tablePath) { - return scanInternal(tablePath).map(this::createTable) + UnresolvedIdentifier unresolvedIdentifier = UnresolvedIdentifier.of(tablePath); + return scanInternal(unresolvedIdentifier) + .map(this::createTable) + .orElseThrow(() -> new ValidationException(String.format( + "Table %s was not found.", + unresolvedIdentifier))); + } + + @Override + public Table from(String path) { + UnresolvedIdentifier unresolvedIdentifier = parser.parseIdentifier(path); + return scanInternal(unresolvedIdentifier) + .map(this::createTable) .orElseThrow(() -> new ValidationException(String.format( - "Table '%s' was not found.", - String.join(".", tablePath)))); + "Table %s was not found.", + unresolvedIdentifier))); } - private Optional scanInternal(String... tablePath) { - ObjectIdentifier tableIdentifier = catalogManager.qualifyIdentifier(UnresolvedIdentifier.of(tablePath)); + private Optional scanInternal(UnresolvedIdentifier identifier) { + ObjectIdentifier tableIdentifier = catalogManager.qualifyIdentifier(identifier); + return catalogManager.getTable(tableIdentifier) .map(t -> new CatalogQueryOperation(tableIdentifier, t.getTable().getSchema())); } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala index 185c4ca48abf..f4ea542800fc 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala @@ -43,7 +43,7 @@ class TableEnvironmentTest { @Test def testScanNonExistTable(): Unit = { thrown.expect(classOf[ValidationException]) - thrown.expectMessage("Table 'MyTable' was not found") + thrown.expectMessage("Table `MyTable` was not found") tableEnv.scan("MyTable") } 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 232cbefc9d89..ec2ce01fa347 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 @@ -46,6 +46,7 @@ import _root_.java.util.{Optional, HashMap => JHashMap, Map => JMap} import _root_.scala.collection.JavaConverters._ import _root_.scala.collection.JavaConversions._ +import _root_.scala.util.Try /** * The abstract base class for the implementation of batch TableEnvironment. @@ -73,7 +74,18 @@ abstract class TableEnvImpl( new TableReferenceLookup { override def lookupTable(name: String): Optional[TableReferenceExpression] = { JavaScalaConversionUtil - .toJava(scanInternal(Array(name)).map(t => new TableReferenceExpression(name, t))) + .toJava( + // The TableLookup is used during resolution of expressions and it actually might not + // be an identifier of a table. It might be a reference to some other object such as + // column, local reference etc. This method should return empty optional in such cases + // to fallback for other identifiers resolution. + Try({ + val unresolvedIdentifier = UnresolvedIdentifier.of(name) + scanInternal(unresolvedIdentifier) + .map(t => new TableReferenceExpression(name, t)) + }) + .toOption + .flatten) } } } @@ -363,15 +375,26 @@ abstract class TableEnvImpl( @throws[TableException] override def scan(tablePath: String*): Table = { - scanInternal(tablePath.toArray) match { + val unresolvedIdentifier = UnresolvedIdentifier.of(tablePath: _*) + scanInternal(unresolvedIdentifier) match { case Some(table) => createTable(table) - case None => throw new TableException(s"Table '${tablePath.mkString(".")}' was not found.") + case None => throw new TableException(s"Table '$unresolvedIdentifier' was not found.") } } - private[flink] def scanInternal(tablePath: Array[String]): Option[CatalogQueryOperation] = { - val unresolvedIdentifier = UnresolvedIdentifier.of(tablePath: _*) - val objectIdentifier = catalogManager.qualifyIdentifier(unresolvedIdentifier) + override def from(path: String): Table = { + val parser = planningConfigurationBuilder.createCalciteParser() + val unresolvedIdentifier = UnresolvedIdentifier.of(parser.parseIdentifier(path).names: _*) + scanInternal(unresolvedIdentifier) match { + case Some(table) => createTable(table) + case None => throw new TableException(s"Table '$unresolvedIdentifier' was not found.") + } + } + + private[flink] def scanInternal(identifier: UnresolvedIdentifier) + : Option[CatalogQueryOperation] = { + val objectIdentifier: ObjectIdentifier = catalogManager.qualifyIdentifier(identifier) + JavaScalaConversionUtil.toScala(catalogManager.getTable(objectIdentifier)) .map(t => new CatalogQueryOperation(objectIdentifier, t.getTable.getSchema)) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala index 6a4e8e6ced73..c11c1466c729 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala @@ -110,6 +110,8 @@ class MockTableEnvironment extends TableEnvironment { override def listTemporaryViews(): Array[String] = ??? + override def from(path: String): Table = ??? + override def dropTemporaryTable(path: String): Boolean = ??? override def dropTemporaryView(path: String): Boolean = ??? From fe966d424cc921751b5ec755e691b6ad2cf6b78a Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Mon, 21 Oct 2019 10:50:32 +0200 Subject: [PATCH 337/746] [FLINK-14490][table-api] Rework insertInto method --- .../connectors/hive/HiveTableSinkTest.java | 6 +- .../table/catalog/hive/HiveTestUtils.java | 4 +- .../addons/hbase/HBaseConnectorITCase.java | 4 +- flink-python/pyflink/table/table.py | 13 +-- .../tests/test_environment_completeness.py | 8 +- .../client/gateway/local/LocalExecutor.java | 3 - .../table/api/java/BatchTableEnvironment.java | 2 + .../api/java/StreamTableEnvironment.java | 4 +- .../org/apache/flink/table/api/Table.java | 12 +- .../flink/table/api/TableEnvironment.java | 19 +++- .../table/api/internal/Registration.java | 60 ++++++++++ .../api/internal/TableEnvironmentImpl.java | 106 +++++++++++------- .../flink/table/api/internal/TableImpl.java | 4 +- .../descriptors/BatchTableDescriptor.java | 6 +- .../descriptors/ConnectTableDescriptor.java | 76 ++++++++++++- .../descriptors/StreamTableDescriptor.java | 6 +- .../api/scala/BatchTableEnvironment.scala | 2 + .../api/scala/StreamTableEnvironment.scala | 4 +- .../internal/StreamTableEnvironmentImpl.scala | 1 + .../validation/TableSinkValidationTest.scala | 2 +- .../runtime/batch/sql/join/JoinITCase.scala | 5 +- .../runtime/stream/sql/CalcITCase.scala | 2 +- .../runtime/stream/sql/CorrelateITCase.scala | 14 +-- .../stream/sql/Limit0RemoveITCase.scala | 16 +-- .../runtime/stream/sql/RankITCase.scala | 38 +++---- .../runtime/stream/sql/UnnestITCase.scala | 2 +- .../stream/sql/WindowAggregateITCase.scala | 2 +- .../stream/table/AggregateITCase.scala | 2 +- .../runtime/stream/table/JoinITCase.scala | 12 +- .../runtime/utils/BatchTableEnvUtil.scala | 7 +- .../planner/runtime/utils/TableUtil.scala | 8 +- .../table/planner/utils/TableTestBase.scala | 7 +- .../api/internal/BatchTableEnvImpl.scala | 30 ++++- .../table/api/internal/TableEnvImpl.scala | 92 ++++++++------- .../flink/table/api/TableSourceTest.scala | 35 +++++- .../table/utils/MockTableEnvironment.scala | 2 + 36 files changed, 412 insertions(+), 204 deletions(-) create mode 100644 flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/Registration.java diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkTest.java index 51a56fb4edd6..63263002cbdc 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkTest.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkTest.java @@ -96,7 +96,7 @@ public void testInsertIntoNonPartitionTable() throws Exception { tableEnv.registerTable("src", src); tableEnv.registerCatalog("hive", hiveCatalog); - tableEnv.sqlQuery("select * from src").insertInto("hive", "default", "dest"); + tableEnv.sqlQuery("select * from src").insertInto("hive.`default`.dest"); tableEnv.execute("mytest"); verifyWrittenData(toWrite, hiveShell.executeQuery("select * from " + tblName)); @@ -138,7 +138,7 @@ public void testWriteComplexType() throws Exception { tableEnv.registerTable("complexSrc", src); tableEnv.registerCatalog("hive", hiveCatalog); - tableEnv.sqlQuery("select * from complexSrc").insertInto("hive", "default", "dest"); + tableEnv.sqlQuery("select * from complexSrc").insertInto("hive.`default`.dest"); tableEnv.execute("mytest"); List result = hiveShell.executeQuery("select * from " + tblName); @@ -177,7 +177,7 @@ public void testWriteNestedComplexType() throws Exception { Table src = tableEnv.fromTableSource(new CollectionTableSource(toWrite, rowTypeInfo)); tableEnv.registerTable("nestedSrc", src); tableEnv.registerCatalog("hive", hiveCatalog); - tableEnv.sqlQuery("select * from nestedSrc").insertInto("hive", "default", "dest"); + tableEnv.sqlQuery("select * from nestedSrc").insertInto("hive.`default`.dest"); tableEnv.execute("mytest"); List result = hiveShell.executeQuery("select * from " + tblName); diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java index e274633472a5..2e06d519288c 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java @@ -133,9 +133,7 @@ public static List collectTable(TableEnvironment tableEnv, Table table) thr sink.init(serializer, id); String sinkName = UUID.randomUUID().toString(); tableEnv.registerTableSink(sinkName, sink); - final String builtInCatalogName = EnvironmentSettings.DEFAULT_BUILTIN_CATALOG; - final String builtInDBName = EnvironmentSettings.DEFAULT_BUILTIN_DATABASE; - tableEnv.insertInto(table, builtInCatalogName, builtInDBName, sinkName); + tableEnv.insertInto(table, sinkName); JobExecutionResult result = tableEnv.execute("collect-table"); ArrayList data = result.getAccumulatorResult(id); return SerializedListAccumulator.deserializeList(data, serializer); diff --git a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseConnectorITCase.java b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseConnectorITCase.java index d7231ab6142f..dfa40a0d9b9e 100644 --- a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseConnectorITCase.java +++ b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseConnectorITCase.java @@ -28,7 +28,6 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo; 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.TableConfig; import org.apache.flink.table.api.TableEnvironment; @@ -564,8 +563,7 @@ private List collectBatchResult(Table table) throws Exception { schema.getFieldNames(), types.toArray(new TypeInformation[0])); return JavaScalaConversionUtil.toJava( BatchTableEnvUtil.collect( - t.getTableEnvironment(), table, configuredSink, Option.apply("JOB"), - EnvironmentSettings.DEFAULT_BUILTIN_CATALOG, EnvironmentSettings.DEFAULT_BUILTIN_DATABASE)); + t.getTableEnvironment(), table, configuredSink, Option.apply("JOB"))); } } diff --git a/flink-python/pyflink/table/table.py b/flink-python/pyflink/table/table.py index 4ea3ec70b5aa..2feaf525c1e5 100644 --- a/flink-python/pyflink/table/table.py +++ b/flink-python/pyflink/table/table.py @@ -608,7 +608,7 @@ def drop_columns(self, fields): """ return Table(self._j_table.dropColumns(fields)) - def insert_into(self, table_path, *table_path_continued): + def insert_into(self, table_path): """ Writes the :class:`Table` to a :class:`TableSink` that was registered under the specified name. For the path resolution algorithm see @@ -619,15 +619,10 @@ def insert_into(self, table_path, *table_path_continued): >>> tab.insert_into("sink") - :param table_path: The first part of the path of the registered :class:`TableSink` to which - the :class:`Table` is written. This is to ensure at least the name of the - :class:`Table` is provided. - :param table_path_continued: The remaining part of the path of the registered - :class:`TableSink` to which the :class:`Table` is written. + :param table_path: The path of the registered :class:`TableSink` to which + the :class:`Table` is written. """ - gateway = get_gateway() - j_table_path = to_jarray(gateway.jvm.String, table_path_continued) - self._j_table.insertInto(table_path, j_table_path) + self._j_table.insertInto(table_path) def get_schema(self): """ diff --git a/flink-python/pyflink/table/tests/test_environment_completeness.py b/flink-python/pyflink/table/tests/test_environment_completeness.py index d50bebbb4a3e..4ff3d29273c1 100644 --- a/flink-python/pyflink/table/tests/test_environment_completeness.py +++ b/flink-python/pyflink/table/tests/test_environment_completeness.py @@ -48,7 +48,13 @@ def excluded_methods(cls): 'getCompletionHints', 'create', 'loadModule', - 'unloadModule'} + 'unloadModule', + 'listTemporaryTables', + 'createTemporaryView', + 'dropTemporaryTable', + 'listTemporaryViews', + 'from', + 'dropTemporaryView'} if __name__ == '__main__': 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 aff7e7daff00..605a3661849e 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 @@ -32,7 +32,6 @@ import org.apache.flink.core.fs.Path; import org.apache.flink.core.plugin.PluginUtils; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.QueryConfig; import org.apache.flink.table.api.StreamQueryConfig; import org.apache.flink.table.api.Table; @@ -494,8 +493,6 @@ private ResultDescriptor executeQueryInternal(ExecutionContext context, S envInst.getTableEnvironment().registerTableSink(jobName, result.getTableSink()); table.insertInto( envInst.getQueryConfig(), - EnvironmentSettings.DEFAULT_BUILTIN_CATALOG, - EnvironmentSettings.DEFAULT_BUILTIN_DATABASE, jobName); return null; }); diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/BatchTableEnvironment.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/BatchTableEnvironment.java index 4ffd117e95af..42214c85ed0b 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/BatchTableEnvironment.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/BatchTableEnvironment.java @@ -309,7 +309,9 @@ public interface BatchTableEnvironment extends TableEnvironment { * written. This is to ensure at least the name of the {@link TableSink} is provided. * @param sinkPathContinued The remaining part of the path of the registered {@link TableSink} to which the * {@link Table} is written. + * @deprecated use {@link #insertInto(String, Table)} */ + @Deprecated void insertInto(Table table, BatchQueryConfig queryConfig, String sinkPath, String... sinkPathContinued); /** diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/StreamTableEnvironment.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/StreamTableEnvironment.java index 5bf322996eb9..fd0051780f75 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/StreamTableEnvironment.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/StreamTableEnvironment.java @@ -503,7 +503,7 @@ static StreamTableEnvironment create(StreamExecutionEnvironment executionEnviron * .field("count", "DECIMAL") * .field("proc-time", "TIMESTAMP").proctime()) * .inAppendMode() - * .registerSource("MyTable") + * .createTemporaryTable("MyTable") * } *
    * @@ -549,7 +549,9 @@ static StreamTableEnvironment create(StreamExecutionEnvironment executionEnviron * written. This is to ensure at least the name of the {@link TableSink} is provided. * @param sinkPathContinued The remaining part of the path of the registered {@link TableSink} to which the * {@link Table} is written. + * @deprecated use {@link #insertInto(String, Table)} */ + @Deprecated void insertInto(Table table, StreamQueryConfig queryConfig, String sinkPath, String... sinkPathContinued); /** diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Table.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Table.java index 70350faa83ec..c696c65ca31d 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Table.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Table.java @@ -821,12 +821,10 @@ public interface Table { * {@code org.apache.flink.table.sinks.RetractStreamTableSink}, or an * {@code org.apache.flink.table.sinks.UpsertStreamTableSink}. * - * @param tablePath The first part of the path of the registered {@link TableSink} to which the {@link Table} is - * written. This is to ensure at least the name of the {@link TableSink} is provided. - * @param tablePathContinued The remaining part of the path of the registered {@link TableSink} to which the - * {@link Table} is written. + * @param tablePath The path of the registered {@link TableSink} to which the {@link Table} is + * written. */ - void insertInto(String tablePath, String... tablePathContinued); + void insertInto(String tablePath); /** * Writes the {@link Table} to a {@link TableSink} that was registered under the specified name @@ -840,7 +838,7 @@ public interface Table { * * @param tableName The name of the {@link TableSink} to which the {@link Table} is written. * @param conf The {@link QueryConfig} to use. - * @deprecated use {@link #insertInto(QueryConfig, String, String...)} + * @deprecated use {@link #insertInto(String)} */ @Deprecated void insertInto(String tableName, QueryConfig conf); @@ -860,7 +858,9 @@ public interface Table { * written. This is to ensure at least the name of the {@link TableSink} is provided. * @param tablePathContinued The remaining part of the path of the registered {@link TableSink} to which the * {@link Table} is written. + * @deprecated use {@link #insertInto(String)} */ + @Deprecated void insertInto(QueryConfig conf, String tablePath, String... tablePathContinued); /** diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java index 7ab7feb0dd10..a0a59fab5830 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java @@ -175,7 +175,9 @@ static TableEnvironment create(EnvironmentSettings settings) { * * @param name The name under which the {@link TableSource} is registered. * @param tableSource The {@link TableSource} to register. + * @deprecated Use {@link #connect(ConnectorDescriptor)} instead. */ + @Deprecated void registerTableSource(String name, TableSource tableSource); /** @@ -191,7 +193,7 @@ static TableEnvironment create(EnvironmentSettings settings) { * @param fieldNames The field names to register with the {@link TableSink}. * @param fieldTypes The field types to register with the {@link TableSink}. * @param tableSink The {@link TableSink} to register. - * @deprecated Use {@link #registerTableSink(String, TableSink)} instead. + * @deprecated Use {@link #connect(ConnectorDescriptor)} instead. */ @Deprecated void registerTableSink(String name, String[] fieldNames, TypeInformation[] fieldTypes, TableSink tableSink); @@ -207,7 +209,9 @@ static TableEnvironment create(EnvironmentSettings settings) { * * @param name The name under which the {@link TableSink} is registered. * @param configuredSink The configured {@link TableSink} to register. + * @deprecated Use {@link #connect(ConnectorDescriptor)} instead. */ + @Deprecated void registerTableSink(String name, TableSink configuredSink); /** @@ -294,9 +298,22 @@ static TableEnvironment create(EnvironmentSettings settings) { * written. This is to ensure at least the name of the {@link TableSink} is provided. * @param sinkPathContinued The remaining part of the path of the registered {@link TableSink} to which the * {@link Table} is written. + * @deprecated use {@link #insertInto(String, Table)} */ + @Deprecated void insertInto(Table table, String sinkPath, String... sinkPathContinued); + /** + * Instructs to write the content of a {@link Table} API object into a table. + * + *

    See the documentation of {@link TableEnvironment#useDatabase(String)} or + * {@link TableEnvironment#useCatalog(String)} for the rules on the path resolution. + * + * @param targetPath The path of the registered {@link TableSink} to which the {@link Table} is written. + * @param table The Table to write to the sink. + */ + void insertInto(String targetPath, Table table); + /** * Creates a table source and/or table sink from a descriptor. * diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/Registration.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/Registration.java new file mode 100644 index 000000000000..f2bc4eca8458 --- /dev/null +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/Registration.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.api.internal; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.table.sources.TableSource; + +/** + * A way to register a table in a {@link TableEnvironment} that this descriptor originates from. + */ +@Internal +public interface Registration { + /** + * Registers an external {@link TableSource} in this {@link TableEnvironment}'s catalog. + * Registered tables can be referenced in SQL queries. + * + * @param name The name under which the {@link TableSource} is registered. + * @param tableSource The {@link TableSource} to register. + * @see TableEnvironment#registerTableSource(String, TableSource) + */ + void createTableSource(String name, TableSource tableSource); + + /** + * Registers an external {@link TableSink} with already configured field names and field types in + * this {@link TableEnvironment}'s catalog. + * Registered sink tables can be referenced in SQL DML statements. + * + * @param name The name under which the {@link TableSink} is registered. + * @param tableSink The configured {@link TableSink} to register. + * @see TableEnvironment#registerTableSink(String, TableSink) + */ + void createTableSink(String name, TableSink tableSink); + + /** + * Creates a temporary table in a given path. + * + * @param path Path where to register the given table + * @param table table to register + */ + void createTemporaryTable(String path, CatalogBaseTable table); +} 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 da957915d042..10459d07346b 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 @@ -96,6 +96,29 @@ public class TableEnvironmentImpl implements TableEnvironment { protected final Planner planner; protected final Parser parser; + /** + * Provides necessary methods for {@link ConnectTableDescriptor}. + */ + private final Registration registration = new Registration() { + @Override + public void createTemporaryTable(String path, CatalogBaseTable table) { + UnresolvedIdentifier unresolvedIdentifier = parser.parseIdentifier(path); + ObjectIdentifier objectIdentifier = catalogManager.qualifyIdentifier( + unresolvedIdentifier); + catalogManager.createTemporaryTable(table, objectIdentifier, false); + } + + @Override + public void createTableSource(String name, TableSource tableSource) { + registerTableSource(name, tableSource); + } + + @Override + public void createTableSink(String name, TableSink tableSink) { + registerTableSink(name, tableSink); + } + }; + protected TableEnvironmentImpl( CatalogManager catalogManager, ModuleManager moduleManager, @@ -270,6 +293,36 @@ public Table from(String path) { unresolvedIdentifier))); } + @Override + public void insertInto(String targetPath, Table table) { + UnresolvedIdentifier unresolvedIdentifier = parser.parseIdentifier(targetPath); + insertIntoInternal(unresolvedIdentifier, table); + } + + @Override + public void insertInto(Table table, String sinkPath, String... sinkPathContinued) { + List fullPath = new ArrayList<>(Arrays.asList(sinkPathContinued)); + fullPath.add(0, sinkPath); + UnresolvedIdentifier unresolvedIdentifier = UnresolvedIdentifier.of(fullPath.toArray(new String[0])); + + insertIntoInternal(unresolvedIdentifier, table); + } + + private void insertIntoInternal(UnresolvedIdentifier unresolvedIdentifier, Table table) { + ObjectIdentifier objectIdentifier = catalogManager.qualifyIdentifier(unresolvedIdentifier); + + List modifyOperations = Collections.singletonList( + new CatalogSinkModifyOperation( + objectIdentifier, + table.getQueryOperation())); + + if (isEagerOperationTranslation()) { + translate(modifyOperations); + } else { + buffer(modifyOperations); + } + } + private Optional scanInternal(UnresolvedIdentifier identifier) { ObjectIdentifier tableIdentifier = catalogManager.qualifyIdentifier(identifier); @@ -279,7 +332,7 @@ private Optional scanInternal(UnresolvedIdentifier identi @Override public ConnectTableDescriptor connect(ConnectorDescriptor connectorDescriptor) { - return new StreamTableDescriptor(this, connectorDescriptor); + return new StreamTableDescriptor(registration, connectorDescriptor); } @Override @@ -391,25 +444,6 @@ public Table sqlQuery(String query) { } } - @Override - public void insertInto(Table table, String path, String... pathContinued) { - List fullPath = new ArrayList<>(Arrays.asList(pathContinued)); - fullPath.add(0, path); - - ObjectIdentifier tableIdentifier = catalogManager.qualifyIdentifier( - UnresolvedIdentifier.of(fullPath.toArray(new String[0]))); - List modifyOperations = Collections.singletonList( - new CatalogSinkModifyOperation( - tableIdentifier, - table.getQueryOperation())); - - if (isEagerOperationTranslation()) { - translate(modifyOperations); - } else { - buffer(modifyOperations); - } - } - @Override public void sqlUpdate(String stmt) { List operations = parser.parse(stmt); @@ -514,20 +548,10 @@ private void buffer(List modifyOperations) { bufferedModifyOperations.addAll(modifyOperations); } - private ObjectIdentifier getTemporaryObjectIdentifier(String name) { - return catalogManager.qualifyIdentifier( - UnresolvedIdentifier.of( - catalogManager.getBuiltInCatalogName(), - catalogManager.getBuiltInDatabaseName(), - name)); - } - private void registerTableSourceInternal(String name, TableSource tableSource) { validateTableSource(tableSource); - Optional table = getTemporaryTable( - catalogManager.getBuiltInCatalogName(), - catalogManager.getBuiltInDatabaseName(), - name); + ObjectIdentifier objectIdentifier = catalogManager.qualifyIdentifier(UnresolvedIdentifier.of(name)); + Optional table = getTemporaryTable(objectIdentifier); if (table.isPresent()) { if (table.get() instanceof ConnectorCatalogTable) { @@ -541,7 +565,7 @@ private void registerTableSourceInternal(String name, TableSource tableSource tableSource, sourceSinkTable.getTableSink().get(), !IS_STREAM_TABLE); - catalogManager.createTemporaryTable(sourceAndSink, getTemporaryObjectIdentifier(name), true); + catalogManager.createTemporaryTable(sourceAndSink, objectIdentifier, true); } } else { throw new ValidationException(String.format( @@ -549,15 +573,13 @@ private void registerTableSourceInternal(String name, TableSource tableSource } } else { ConnectorCatalogTable source = ConnectorCatalogTable.source(tableSource, !IS_STREAM_TABLE); - catalogManager.createTemporaryTable(source, getTemporaryObjectIdentifier(name), false); + catalogManager.createTemporaryTable(source, objectIdentifier, false); } } private void registerTableSinkInternal(String name, TableSink tableSink) { - Optional table = getTemporaryTable( - catalogManager.getBuiltInCatalogName(), - catalogManager.getBuiltInDatabaseName(), - name); + ObjectIdentifier objectIdentifier = catalogManager.qualifyIdentifier(UnresolvedIdentifier.of(name)); + Optional table = getTemporaryTable(objectIdentifier); if (table.isPresent()) { if (table.get() instanceof ConnectorCatalogTable) { @@ -569,7 +591,7 @@ private void registerTableSinkInternal(String name, TableSink tableSink) { // wrapper contains only sink (not source) ConnectorCatalogTable sourceAndSink = ConnectorCatalogTable .sourceAndSink(sourceSinkTable.getTableSource().get(), tableSink, !IS_STREAM_TABLE); - catalogManager.createTemporaryTable(sourceAndSink, getTemporaryObjectIdentifier(name), true); + catalogManager.createTemporaryTable(sourceAndSink, objectIdentifier, true); } } else { throw new ValidationException(String.format( @@ -577,12 +599,12 @@ private void registerTableSinkInternal(String name, TableSink tableSink) { } } else { ConnectorCatalogTable sink = ConnectorCatalogTable.sink(tableSink, !IS_STREAM_TABLE); - catalogManager.createTemporaryTable(sink, getTemporaryObjectIdentifier(name), false); + catalogManager.createTemporaryTable(sink, objectIdentifier, false); } } - private Optional getTemporaryTable(String... name) { - return catalogManager.getTable(catalogManager.qualifyIdentifier(UnresolvedIdentifier.of(name))) + private Optional getTemporaryTable(ObjectIdentifier identifier) { + return catalogManager.getTable(identifier) .filter(CatalogManager.TableLookupResult::isTemporary) .map(CatalogManager.TableLookupResult::getTable); } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableImpl.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableImpl.java index 41e483bb38c5..2b545abd739d 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableImpl.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableImpl.java @@ -407,8 +407,8 @@ public Table fetch(int fetch) { } @Override - public void insertInto(String tablePath, String... tablePathContinued) { - tableEnvironment.insertInto(this, tablePath, tablePathContinued); + public void insertInto(String tablePath) { + tableEnvironment.insertInto(tablePath, this); } @Override diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/descriptors/BatchTableDescriptor.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/descriptors/BatchTableDescriptor.java index cd0180117ee3..faaeae48db43 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/descriptors/BatchTableDescriptor.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/descriptors/BatchTableDescriptor.java @@ -19,7 +19,7 @@ package org.apache.flink.table.descriptors; import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.internal.Registration; /** * Describes a table connected from a batch environment. @@ -30,8 +30,8 @@ @PublicEvolving public final class BatchTableDescriptor extends ConnectTableDescriptor { - public BatchTableDescriptor(TableEnvironment tableEnv, ConnectorDescriptor connectorDescriptor) { - super(tableEnv, connectorDescriptor); + public BatchTableDescriptor(Registration registration, ConnectorDescriptor connectorDescriptor) { + super(registration, connectorDescriptor); } @Override diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/descriptors/ConnectTableDescriptor.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/descriptors/ConnectTableDescriptor.java index fabc5882e6c7..4ec3f9621407 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/descriptors/ConnectTableDescriptor.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/descriptors/ConnectTableDescriptor.java @@ -20,6 +20,10 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.internal.Registration; +import org.apache.flink.table.catalog.CatalogTableImpl; import org.apache.flink.table.factories.TableFactoryUtil; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.table.sources.TableSource; @@ -28,6 +32,7 @@ import javax.annotation.Nullable; import java.util.Collections; +import java.util.HashMap; import java.util.Map; /** @@ -39,13 +44,13 @@ public abstract class ConnectTableDescriptor extends TableDescriptor { - private final TableEnvironment tableEnv; + private final Registration registration; private @Nullable Schema schemaDescriptor; - public ConnectTableDescriptor(TableEnvironment tableEnv, ConnectorDescriptor connectorDescriptor) { + public ConnectTableDescriptor(Registration registration, ConnectorDescriptor connectorDescriptor) { super(connectorDescriptor); - this.tableEnv = tableEnv; + this.registration = registration; } /** @@ -60,37 +65,98 @@ public ConnectTableDescriptor withSchema(Schema schema) { * Searches for the specified table source, configures it accordingly, and registers it as * a table under the given name. * + *

    Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * * @param name table name to be registered in the table environment + * @deprecated use {@link #createTemporaryTable(String)} */ + @Deprecated public void registerTableSource(String name) { Preconditions.checkNotNull(name); TableSource tableSource = TableFactoryUtil.findAndCreateTableSource(this); - tableEnv.registerTableSource(name, tableSource); + registration.createTableSource(name, tableSource); } /** * Searches for the specified table sink, configures it accordingly, and registers it as * a table under the given name. * + *

    Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * * @param name table name to be registered in the table environment + * @deprecated use {@link #createTemporaryTable(String)} */ + @Deprecated public void registerTableSink(String name) { Preconditions.checkNotNull(name); TableSink tableSink = TableFactoryUtil.findAndCreateTableSink(this); - tableEnv.registerTableSink(name, tableSink); + registration.createTableSink(name, tableSink); } /** * Searches for the specified table source and sink, configures them accordingly, and registers * them as a table under the given name. * + *

    Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * * @param name table name to be registered in the table environment + * @deprecated use {@link #createTemporaryTable(String)} */ + @Deprecated public void registerTableSourceAndSink(String name) { registerTableSource(name); registerTableSink(name); } + /** + * Registers the table described by underlying properties in a given path. + * + *

    There is no distinction between source and sink at the descriptor level anymore as this + * method does not perform actual class lookup. It only stores the underlying properties. The + * actual source/sink lookup is performed when the table is used. + * + *

    Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * + *

    NOTE: The schema must be explicitly defined. + * + * @param path path where to register the temporary table + */ + public void createTemporaryTable(String path) { + if (schemaDescriptor == null) { + throw new TableException( + "Table schema must be explicitly defined. To derive schema from the underlying connector" + + " use registerTableSource/registerTableSink/registerTableSourceAndSink."); + } + + Map schemaProperties = schemaDescriptor.toProperties(); + TableSchema tableSchema = getTableSchema(schemaProperties); + + Map properties = new HashMap<>(toProperties()); + schemaProperties.keySet().forEach(properties::remove); + + CatalogTableImpl catalogTable = new CatalogTableImpl( + tableSchema, + properties, + "" + ); + + registration.createTemporaryTable(path, catalogTable); + } + + private TableSchema getTableSchema(Map schemaProperties) { + DescriptorProperties properties = new DescriptorProperties(); + properties.putProperties(schemaProperties); + return properties.getTableSchema(Schema.SCHEMA); + } + @Override protected Map additionalProperties() { if (schemaDescriptor != null) { diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/descriptors/StreamTableDescriptor.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/descriptors/StreamTableDescriptor.java index 8e67959571d0..65d17171d25e 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/descriptors/StreamTableDescriptor.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/descriptors/StreamTableDescriptor.java @@ -19,7 +19,7 @@ package org.apache.flink.table.descriptors; import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.internal.Registration; /** * Describes a table connected from a streaming environment. @@ -30,8 +30,8 @@ @PublicEvolving public final class StreamTableDescriptor extends ConnectTableDescriptor { - public StreamTableDescriptor(TableEnvironment tableEnv, ConnectorDescriptor connectorDescriptor) { - super(tableEnv, connectorDescriptor); + public StreamTableDescriptor(Registration registration, ConnectorDescriptor connectorDescriptor) { + super(registration, connectorDescriptor); } @Override diff --git a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/BatchTableEnvironment.scala b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/BatchTableEnvironment.scala index 6790dc647c6d..d7bd0ca99150 100644 --- a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/BatchTableEnvironment.scala +++ b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/BatchTableEnvironment.scala @@ -251,7 +251,9 @@ trait BatchTableEnvironment extends TableEnvironment { * name of the [[TableSink]] is provided. * @param sinkPathContinued The remaining part of the path of the registered [[TableSink]] to * which the [[Table]] is written. + * @deprecated use `TableEnvironment#insertInto(String, Table)` */ + @deprecated def insertInto( table: Table, queryConfig: BatchQueryConfig, diff --git a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala index 1ed49754ede0..d40b21a0ded0 100644 --- a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala +++ b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala @@ -305,7 +305,9 @@ trait StreamTableEnvironment extends TableEnvironment { * of the [[TableSink]] is provided. * @param sinkPathContinued The remaining part of the path of the registered [[TableSink]] to * which the [[Table]] is written. + * @deprecated use `TableEnvironment#insertInto(String, Table)` */ + @deprecated def insertInto( table: Table, queryConfig: StreamQueryConfig, @@ -359,7 +361,7 @@ trait StreamTableEnvironment extends TableEnvironment { * .field("count", "DECIMAL") * .field("proc-time", "TIMESTAMP").proctime()) * .inAppendMode() - * .registerSource("MyTable") + * .createTemporaryTable("MyTable") * }}} * * @param connectorDescriptor connector descriptor describing the external system 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 ca77bf19552a..9a1435c3a784 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 @@ -266,6 +266,7 @@ class StreamTableEnvironmentImpl ( Time.milliseconds(queryConfig.getMinIdleStateRetentionTime), Time.milliseconds(queryConfig.getMaxIdleStateRetentionTime)) insertInto(table, sinkPath, sinkPathContinued: _*) + } override def createTemporaryView[T]( path: String, diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/TableSinkValidationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/TableSinkValidationTest.scala index 16d7ff13c4f9..1b8cc44c1f40 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/TableSinkValidationTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/TableSinkValidationTest.scala @@ -63,7 +63,7 @@ class TableSinkValidationTest extends TableTestBase { val schema = result.getSchema sink.configure(schema.getFieldNames, schema.getFieldTypes) tEnv.registerTableSink("testSink", sink) - tEnv.insertInto(result, "testSink") + tEnv.insertInto("testSink", result) // must fail because table is updating table without full key env.execute() } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/JoinITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/JoinITCase.scala index 7f5b2c8507f7..702c00ba35c0 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/JoinITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/JoinITCase.scala @@ -100,10 +100,7 @@ class JoinITCase(expectedJoinType: JoinType) extends BatchTestBase { if (expectedJoinType == HashJoin) { val sink = (new CollectRowTableSink).configure(Array("c"), Array(Types.STRING)) tEnv.registerTableSink("collect", sink) - tEnv.insertInto( - tEnv.sqlQuery("SELECT c FROM SmallTable3, Table5 WHERE b = e"), - "collect" - ) + tEnv.insertInto("collect", tEnv.sqlQuery("SELECT c FROM SmallTable3, Table5 WHERE b = e")) var haveTwoOp = false env.getStreamGraph.getAllOperatorFactory.foreach(o => o.f1 match { diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CalcITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CalcITCase.scala index 8861ed0f33fc..fbb37fa68548 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CalcITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CalcITCase.scala @@ -216,7 +216,7 @@ class CalcITCase extends StreamingTestBase { val result = tEnv.sqlQuery(sqlQuery) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink()) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") val expected = List("0,0,0", "1,1,1", "2,2,2") diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CorrelateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CorrelateITCase.scala index 91f792208f35..e60a61cfb178 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CorrelateITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CorrelateITCase.scala @@ -239,7 +239,7 @@ class CorrelateITCase extends StreamingTestBase { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") val expected = List("1,2,,null", "1,3,,null") @@ -261,7 +261,7 @@ class CorrelateITCase extends StreamingTestBase { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") val expected = List("3018-06-10", "2018-06-03", "2018-06-01", "2018-06-02") @@ -283,7 +283,7 @@ class CorrelateITCase extends StreamingTestBase { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") val expected = List("1,3018-06-10", "1,2018-06-03", "1,2018-06-01", "1,2018-06-02") @@ -304,7 +304,7 @@ class CorrelateITCase extends StreamingTestBase { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") val expected = List("a") @@ -325,7 +325,7 @@ class CorrelateITCase extends StreamingTestBase { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") // output two null @@ -347,7 +347,7 @@ class CorrelateITCase extends StreamingTestBase { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") val expected = List("1,a") @@ -368,7 +368,7 @@ class CorrelateITCase extends StreamingTestBase { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") val expected = List("2,null", "3,null") diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/Limit0RemoveITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/Limit0RemoveITCase.scala index 824907d84c93..b3c1e9e1ac2d 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/Limit0RemoveITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/Limit0RemoveITCase.scala @@ -38,7 +38,7 @@ class Limit0RemoveITCase extends StreamingTestBase() { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink()) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") assertEquals(0, sink.getAppendResults.size) @@ -55,7 +55,7 @@ class Limit0RemoveITCase extends StreamingTestBase() { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink()) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") assertEquals(0, sink.getAppendResults.size) @@ -72,7 +72,7 @@ class Limit0RemoveITCase extends StreamingTestBase() { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink()) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") assertEquals(0, sink.getAppendResults.size) @@ -93,7 +93,7 @@ class Limit0RemoveITCase extends StreamingTestBase() { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink()) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") assertEquals(0, sink.getAppendResults.size) @@ -114,7 +114,7 @@ class Limit0RemoveITCase extends StreamingTestBase() { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink()) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") val expected = Seq("1", "2", "3", "4", "5", "6") @@ -136,7 +136,7 @@ class Limit0RemoveITCase extends StreamingTestBase() { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingRetractTableSink()) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") assertEquals(0, sink.getRawResults.size) @@ -157,7 +157,7 @@ class Limit0RemoveITCase extends StreamingTestBase() { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingRetractTableSink()) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") val expected = Seq("1", "2", "3", "4", "5", "6") @@ -179,7 +179,7 @@ class Limit0RemoveITCase extends StreamingTestBase() { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink()) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") assertEquals(0, sink.getAppendResults.size) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/RankITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/RankITCase.scala index a422c0658c1f..c2f0fb552fd0 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/RankITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/RankITCase.scala @@ -131,7 +131,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val expected = List( @@ -194,7 +194,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val updatedExpected = List( @@ -257,7 +257,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val updatedExpected = List( @@ -303,7 +303,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val updatedExpected = List( @@ -357,7 +357,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val updatedExpected = List( @@ -402,7 +402,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val updatedExpected = List( @@ -537,7 +537,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val expected = List( @@ -596,7 +596,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val expected = List( @@ -656,7 +656,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val expected = List( @@ -718,7 +718,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val expected = List( @@ -789,7 +789,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val expected = List( @@ -853,7 +853,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val expected = List( @@ -933,7 +933,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val expected = List( @@ -1006,7 +1006,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val expected = List( @@ -1073,7 +1073,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val expected = List( @@ -1202,7 +1202,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema1.getFieldNames, schema1 .getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink1", sink1) - tEnv.insertInto(table1, "MySink1") + tEnv.insertInto("MySink1", table1) tEnv.execute("test") val table2 = tEnv.sqlQuery( @@ -1219,7 +1219,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema2.getFieldNames, schema2 .getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink2", sink2) - tEnv.insertInto(table2, "MySink2") + tEnv.insertInto("MySink2", table2) tEnv.execute("test") val expected1 = List( @@ -1276,7 +1276,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema1.getFieldNames, schema1.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink1", sink1) - tEnv.insertInto(table1, "MySink1") + tEnv.insertInto("MySink1", table1) tEnv.execute("test") val table2 = tEnv.sqlQuery( @@ -1293,7 +1293,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema2.getFieldNames, schema2.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink2", sink2) - tEnv.insertInto(table2, "MySink2") + tEnv.insertInto("MySink2", table2) tEnv.execute("test") val expected1 = List( diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala index 38a891ae3590..2cd326a394f8 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala @@ -265,7 +265,7 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo Array("a", "b", "v"), Array(Types.INT, Types.LONG, Types.STRING)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") val expected = List("1,11,10", "1,11,11", "2,22,20", "3,33,30", "3,33,31") diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowAggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowAggregateITCase.scala index 077fdfdc0918..d195f0748166 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowAggregateITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowAggregateITCase.scala @@ -202,7 +202,7 @@ class WindowAggregateITCase(mode: StateBackendMode) val sink = new TestingUpsertTableSink(Array(0, 1)).configure(fieldNames, fieldTypes) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") val expected = Seq( diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/AggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/AggregateITCase.scala index 758f6537d297..78e3ce0c85ab 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/AggregateITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/AggregateITCase.scala @@ -308,7 +308,7 @@ class AggregateITCase(mode: StateBackendMode) extends StreamingWithStateTestBase Array[String]("c", "bMax"), Array[TypeInformation[_]](Types.STRING, Types.LONG)) tEnv.registerTableSink("testSink", tableSink) - tEnv.insertInto(t, "testSink") + tEnv.insertInto("testSink", t) tEnv.execute("test") val expected = List("A,1", "B,2", "C,3") diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/JoinITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/JoinITCase.scala index 306b5597c7c3..f1bd82728051 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/JoinITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/JoinITCase.scala @@ -135,7 +135,7 @@ class JoinITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode Array[TypeInformation[_]](Types.INT, Types.LONG, Types.LONG)) tEnv.registerTableSink("upsertSink", sink) - tEnv.insertInto(t, "upsertSink") + tEnv.insertInto("upsertSink", t) tEnv.execute("test") val expected = Seq("0,1,1", "1,2,3", "2,1,1", "3,1,1", "4,1,1", "5,2,3", "6,0,1") @@ -186,7 +186,7 @@ class JoinITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode .join(rightTable, 'a === 'bb && ('a < 4 || 'a > 4)) .select('a, 'b, 'c, 'd) tEnv.registerTableSink("retractSink", sink) - tEnv.insertInto(t, "retractSink") + tEnv.insertInto("retractSink", t) tEnv.execute("test") val expected = Seq("1,1,1,1", "1,1,1,1", "1,1,1,1", "1,1,1,1", "2,2,2,2", "3,3,3,3", @@ -793,7 +793,7 @@ class JoinITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode val sink = new TestingUpsertTableSink(Array(0, 2)) .configure(schema.getFieldNames, schema.getFieldTypes) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(t, "MySink") + tEnv.insertInto("MySink", t) tEnv.execute("test") val expected = Seq("1,5,1,2") @@ -874,7 +874,7 @@ class JoinITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode val sink = new TestingRetractTableSink().configure( schema.getFieldNames, schema.getFieldTypes) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(t, "MySink") + tEnv.insertInto("MySink", t) tEnv.execute("test") val expected = Seq("1,4,1,2", "1,5,1,2") @@ -1062,7 +1062,7 @@ class JoinITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode val sink = new TestingUpsertTableSink(Array(0, 1)) .configure(schema.getFieldNames, schema.getFieldTypes) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(t, "MySink") + tEnv.insertInto("MySink", t) tEnv.execute("test") val expected = Seq("0,1,1", "1,2,3", "2,1,1", "3,1,1", "4,1,1", "5,2,3", "6,0,1") @@ -1345,7 +1345,7 @@ class JoinITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode val sink = new TestingUpsertTableSink(Array(0, 1, 2)) .configure(schema.getFieldNames, schema.getFieldTypes) tEnv.registerTableSink("sinkTests", sink) - tEnv.insertInto(t, "sinkTests") + tEnv.insertInto("sinkTests", t) tEnv.execute("test") val expected = Seq("4,1,1,1") diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTableEnvUtil.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTableEnvUtil.scala index 7305f5396ab2..ea678b9a4f0d 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTableEnvUtil.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTableEnvUtil.scala @@ -33,6 +33,7 @@ import org.apache.flink.table.planner.sinks.CollectTableSink import org.apache.flink.table.planner.utils.TableTestUtil import org.apache.flink.table.types.utils.TypeConversions.fromDataTypeToLegacyInfo import org.apache.flink.util.AbstractID + import _root_.java.util.{UUID, ArrayList => JArrayList} import _root_.scala.collection.JavaConversions._ @@ -45,9 +46,7 @@ object BatchTableEnvUtil { tEnv: TableEnvironment, table: Table, sink: CollectTableSink[T], - jobName: Option[String], - builtInCatalogName: String, - builtInDBName: String): Seq[T] = { + jobName: Option[String]): Seq[T] = { val typeSerializer = fromDataTypeToLegacyInfo(sink.getConsumedDataType) .asInstanceOf[TypeInformation[T]] .createSerializer(tEnv.asInstanceOf[TableEnvironmentImpl] @@ -56,7 +55,7 @@ object BatchTableEnvUtil { sink.init(typeSerializer.asInstanceOf[TypeSerializer[T]], id) val sinkName = UUID.randomUUID().toString tEnv.registerTableSink(sinkName, sink) - tEnv.insertInto(table, builtInCatalogName, builtInDBName, sinkName) + tEnv.insertInto(s"`$sinkName`", table) val res = tEnv.execute("test") val accResult: JArrayList[Array[Byte]] = res.getAccumulatorResult(id) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TableUtil.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TableUtil.scala index 6b98d2cf8b0b..1202e224e5af 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TableUtil.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TableUtil.scala @@ -19,7 +19,6 @@ package org.apache.flink.table.planner.runtime.utils import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.table.api.EnvironmentSettings import org.apache.flink.table.api.internal.TableImpl import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.schema.TimeIndicatorRelDataType @@ -53,9 +52,7 @@ object TableUtil { new CollectTableSink(_ => t.asInstanceOf[TypeInformation[T]]), Option(jobName)) def collectSink[T]( - table: TableImpl, sink: CollectTableSink[T], jobName: Option[String] = None, - builtInCatalogName: String = EnvironmentSettings.DEFAULT_BUILTIN_CATALOG, - builtInDBName: String = EnvironmentSettings.DEFAULT_BUILTIN_DATABASE): Seq[T] = { + table: TableImpl, sink: CollectTableSink[T], jobName: Option[String] = None): Seq[T] = { // get schema information of table val relNode = TableTestUtil.toRelNode(table) val rowType = relNode.getRowType @@ -74,8 +71,7 @@ object TableUtil { val configuredSink = sink.configure( fieldNames, fieldTypes.map(TypeInfoLogicalTypeConverter.fromLogicalTypeToTypeInfo)) BatchTableEnvUtil.collect(table.getTableEnvironment, - table, configuredSink.asInstanceOf[CollectTableSink[T]], jobName, - builtInCatalogName, builtInDBName) + table, configuredSink.asInstanceOf[CollectTableSink[T]], jobName) } } 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 760f41faebae..c55cd32bb725 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 @@ -137,7 +137,7 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) def writeToSink(table: Table, sink: TableSink[_], sinkName: String): Unit = { val tableEnv = getTableEnv tableEnv.registerTableSink(sinkName, sink) - tableEnv.insertInto(table, sinkName) + tableEnv.insertInto(sinkName, table) } /** @@ -954,9 +954,8 @@ class TestingTableEnvironment private( ) } - override def insertInto(table: Table, path: String, pathContinued: String*): Unit = { - val fullPath = List(path) ++ pathContinued.toList - val unresolvedIdentifier = UnresolvedIdentifier.of(fullPath:_*) + override def insertInto(path: String, table: Table): Unit = { + val unresolvedIdentifier = parser.parseIdentifier(path) val identifier = catalogManager.qualifyIdentifier(unresolvedIdentifier) val modifyOperations = List(new CatalogSinkModifyOperation(identifier, table.getQueryOperation)) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala index 789948a66f30..815cbb4fe057 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala @@ -18,8 +18,6 @@ package org.apache.flink.table.api.internal -import org.apache.calcite.plan.RelOptUtil -import org.apache.calcite.rel.RelNode import org.apache.flink.api.common.JobExecutionResult import org.apache.flink.api.common.functions.MapFunction import org.apache.flink.api.common.typeinfo.TypeInformation @@ -28,8 +26,8 @@ import org.apache.flink.api.java.typeutils.GenericTypeInfo import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} import org.apache.flink.table.api._ import org.apache.flink.table.calcite.{CalciteConfig, FlinkTypeFactory} -import org.apache.flink.table.catalog.CatalogManager -import org.apache.flink.table.descriptors.{BatchTableDescriptor, ConnectorDescriptor} +import org.apache.flink.table.catalog.{CatalogBaseTable, CatalogManager} +import org.apache.flink.table.descriptors.{BatchTableDescriptor, ConnectTableDescriptor, ConnectorDescriptor} import org.apache.flink.table.explain.PlanJsonParser import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor import org.apache.flink.table.expressions.{Expression, UnresolvedCallExpression} @@ -48,6 +46,9 @@ import org.apache.flink.table.typeutils.FieldInfoUtils.{getFieldsInfo, validateI import org.apache.flink.table.utils.TableConnectorUtils import org.apache.flink.types.Row +import org.apache.calcite.plan.RelOptUtil +import org.apache.calcite.rel.RelNode + import _root_.scala.collection.JavaConverters._ /** @@ -68,6 +69,25 @@ abstract class BatchTableEnvImpl( planningConfigurationBuilder ) + /** + * Provides necessary methods for [[ConnectTableDescriptor]]. + */ + private val registration = new Registration() { + override def createTemporaryTable(path: String, table: CatalogBaseTable): Unit = { + val unresolvedIdentifier = parseIdentifier(path) + val objectIdentifier = catalogManager.qualifyIdentifier(unresolvedIdentifier) + catalogManager.createTemporaryTable(table, objectIdentifier, false) + } + + override def createTableSource(name: String, tableSource: TableSource[_]): Unit = { + registerTableSource(name, tableSource) + } + + override def createTableSink(name: String, tableSource: TableSink[_]): Unit = { + registerTableSink(name, tableSource) + } + } + /** * Registers an internal [[BatchTableSource]] in this [[TableEnvImpl]]'s catalog without * name checking. Registered tables can be referenced in SQL queries. @@ -93,7 +113,7 @@ abstract class BatchTableEnvImpl( } def connect(connectorDescriptor: ConnectorDescriptor): BatchTableDescriptor = { - new BatchTableDescriptor(this, connectorDescriptor) + new BatchTableDescriptor(registration, connectorDescriptor) } /** 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 ec2ce01fa347..ca2ecccd170d 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 @@ -296,13 +296,13 @@ abstract class TableEnvImpl( protected def validateTableSink(tableSink: TableSink[_]): Unit private def registerTableSourceInternal( - name: String, - tableSource: TableSource[_]) - : Unit = { - // register - getTemporaryTable( - catalogManager.getBuiltInCatalogName, - catalogManager.getBuiltInDatabaseName, name) match { + name: String, + tableSource: TableSource[_]) + : Unit = { + val unresolvedIdentifier = UnresolvedIdentifier.of(name) + val objectIdentifier = catalogManager.qualifyIdentifier(unresolvedIdentifier) + // check if a table (source or sink) is registered + getTemporaryTable(objectIdentifier) match { // check if a table (source or sink) is registered case Some(table: ConnectorCatalogTable[_, _]) => @@ -318,26 +318,25 @@ abstract class TableEnvImpl( isBatchTable) catalogManager.createTemporaryTable( sourceAndSink, - getTemporaryObjectIdentifier(name), + objectIdentifier, true) } // no table is registered case _ => val source = ConnectorCatalogTable.source(tableSource, isBatchTable) - catalogManager.createTemporaryTable(source, getTemporaryObjectIdentifier(name), false) + catalogManager.createTemporaryTable(source, objectIdentifier, false) } } private def registerTableSinkInternal( - name: String, - tableSink: TableSink[_]) - : Unit = { + name: String, + tableSink: TableSink[_]) + : Unit = { + val unresolvedIdentifier = UnresolvedIdentifier.of(name) + val objectIdentifier = catalogManager.qualifyIdentifier(unresolvedIdentifier) // check if a table (source or sink) is registered - getTemporaryTable( - catalogManager.getBuiltInCatalogName, - catalogManager.getBuiltInDatabaseName, - name) match { + getTemporaryTable(objectIdentifier) match { // table source and/or sink is registered case Some(table: ConnectorCatalogTable[_, _]) => @@ -353,26 +352,17 @@ abstract class TableEnvImpl( isBatchTable) catalogManager.createTemporaryTable( sourceAndSink, - getTemporaryObjectIdentifier(name), + objectIdentifier, true) } // no table is registered case _ => val sink = ConnectorCatalogTable.sink(tableSink, isBatchTable) - catalogManager.createTemporaryTable(sink, getTemporaryObjectIdentifier(name), false) + catalogManager.createTemporaryTable(sink, objectIdentifier, false) } } - private def getTemporaryObjectIdentifier(name: String): ObjectIdentifier = { - catalogManager.qualifyIdentifier( - UnresolvedIdentifier.of( - catalogManager.getBuiltInCatalogName, - catalogManager.getBuiltInDatabaseName, - name - )) - } - @throws[TableException] override def scan(tablePath: String*): Table = { val unresolvedIdentifier = UnresolvedIdentifier.of(tablePath: _*) @@ -485,9 +475,7 @@ abstract class TableEnvImpl( insertInto( createTable(op.getChild), InsertOptions(op.getStaticPartitions, op.isOverwrite), - op.getTableIdentifier.getCatalogName, - op.getTableIdentifier.getDatabaseName, - op.getTableIdentifier.getObjectName) + op.getTableIdentifier) case createTableOperation: CreateTableOperation => catalogManager.createTable( createTableOperation.getCatalogTable, @@ -520,14 +508,26 @@ abstract class TableEnvImpl( */ private[flink] def writeToSink[T](table: Table, sink: TableSink[T]): Unit + override def insertInto(path: String, table: Table): Unit = { + val parser = planningConfigurationBuilder.createCalciteParser() + val unresolvedIdentifier = UnresolvedIdentifier.of(parser.parseIdentifier(path).names: _*) + val objectIdentifier: ObjectIdentifier = catalogManager.qualifyIdentifier(unresolvedIdentifier) + insertInto( + table, + InsertOptions(new JHashMap[String, String](), overwrite = false), + objectIdentifier) + } + override def insertInto( - table: Table, - path: String, - pathContinued: String*): Unit = { + table: Table, + sinkPath: String, + sinkPathContinued: String*): Unit = { + val unresolvedIdentifier = UnresolvedIdentifier.of(sinkPath +: sinkPathContinued: _*) + val objectIdentifier = catalogManager.qualifyIdentifier(unresolvedIdentifier) insertInto( table, - InsertOptions(new JHashMap[String, String](), false), - path +: pathContinued: _*) + InsertOptions(new JHashMap[String, String](), overwrite = false), + objectIdentifier) } /** Insert options for executing sql insert. **/ @@ -537,26 +537,24 @@ abstract class TableEnvImpl( * Writes the [[Table]] to a [[TableSink]] that was registered under the specified name. * * @param table The table to write to the TableSink. - * @param sinkTablePath The name of the registered TableSink. + * @param sinkIdentifier The name of the registered TableSink. */ - private def insertInto(table: Table, + private def insertInto( + table: Table, insertOptions: InsertOptions, - sinkTablePath: String*): Unit = { - - val unresolvedIdentifier = UnresolvedIdentifier.of(sinkTablePath: _*) - val objectIdentifier = catalogManager.qualifyIdentifier(unresolvedIdentifier) + sinkIdentifier: ObjectIdentifier): Unit = { - getTableSink(objectIdentifier) match { + getTableSink(sinkIdentifier) match { case None => - throw new TableException(s"No table was registered under the name $sinkTablePath.") + throw new TableException(s"No table was registered under the name $sinkIdentifier.") case Some(tableSink) => // validate schema of source table and table sink TableSinkUtils.validateSink( insertOptions.staticPartitions, table.getQueryOperation, - objectIdentifier, + sinkIdentifier, tableSink) // set static partitions if it is a partitioned table sink tableSink match { @@ -607,10 +605,8 @@ abstract class TableEnvImpl( } } - protected def getTemporaryTable(name: String*): Option[CatalogBaseTable] = { - val unresolvedIdentifier = UnresolvedIdentifier.of(name: _*) - val objectIdentifier = catalogManager.qualifyIdentifier(unresolvedIdentifier) - JavaScalaConversionUtil.toScala(catalogManager.getTable(objectIdentifier)) + protected def getTemporaryTable(identifier: ObjectIdentifier): Option[CatalogBaseTable] = { + JavaScalaConversionUtil.toScala(catalogManager.getTable(identifier)) .filter(_.isTemporary) .map(_.getTable) } 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 5b7736442e26..b91f0d656026 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 @@ -18,17 +18,21 @@ package org.apache.flink.table.api +import _root_.java.util.{HashMap => JHashMap} +import _root_.java.util.{Map => JMap} import _root_.java.sql.{Date, Time, Timestamp} - import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, SqlTimeTypeInfo, TypeInformation} import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.table.api.scala._ +import org.apache.flink.table.descriptors.{ConnectorDescriptor, Schema} +import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR import org.apache.flink.table.expressions.utils._ import org.apache.flink.table.runtime.utils.CommonTestData import org.apache.flink.table.sources.{CsvTableSource, TableSource} import org.apache.flink.table.utils.TableTestUtil._ import org.apache.flink.table.utils.{TableTestBase, TestFilterableTableSource} import org.apache.flink.types.Row + import org.junit.{Assert, Test} class TableSourceTest extends TableTestBase { @@ -362,6 +366,34 @@ class TableSourceTest extends TableTestBase { util.verifyTable(result, expected) } + @Test + def testConnectToTableWithProperties(): Unit = { + val util = streamTestUtil() + val tableEnv = util.tableEnv + + val path = "cat.db.tab1" + tableEnv.connect(new ConnectorDescriptor("COLLECTION", 1, false) { + override protected def toConnectorProperties: JMap[String, String] = { + val context = new JHashMap[String, String]() + context.put(CONNECTOR, "COLLECTION") + context + } + }).withSchema( + new Schema() + .schema(TableSchema.builder() + .field("id", DataTypes.INT()) + .field("name", DataTypes.STRING()) + .build()) + ).createTemporaryTable(path) + + val result = tableEnv.from(path) + + val expected = "StreamTableSourceScan(table=[[cat, db, tab1]], fields=[id, name], " + + "source=[CollectionTableSource(id, name)])" + + util.verifyTable(result, expected) + } + // csv builder @Test @@ -490,7 +522,6 @@ class TableSourceTest extends TableTestBase { (tableSource, "filterableTable") } - def csvTable: (CsvTableSource, String) = { val csvTable = CommonTestData.getCsvTableSource val tableName = "csvTable" diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala index c11c1466c729..bec0cebd3f7b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala @@ -91,6 +91,8 @@ class MockTableEnvironment extends TableEnvironment { override def useDatabase(databaseName: String): Unit = ??? + override def insertInto(sinkPath: String, table: Table): Unit = ??? + override def insertInto( table: Table, sinkPath: String, From f71ce33c2e6de60a2b3ab671e121a851b7b07b59 Mon Sep 17 00:00:00 2001 From: liyafan82 Date: Fri, 10 May 2019 18:28:36 +0800 Subject: [PATCH 338/746] [FLINK-12223][Runtime]HeapMemorySegment.getArray should return null after being freed --- .../apache/flink/core/memory/HeapMemorySegment.java | 2 +- .../flink/core/memory/HeapMemorySegmentTest.java | 10 ++++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/HeapMemorySegment.java b/flink-core/src/main/java/org/apache/flink/core/memory/HeapMemorySegment.java index 96c013abb075..ed4d00526d6f 100644 --- a/flink-core/src/main/java/org/apache/flink/core/memory/HeapMemorySegment.java +++ b/flink-core/src/main/java/org/apache/flink/core/memory/HeapMemorySegment.java @@ -95,7 +95,7 @@ public ByteBuffer wrap(int offset, int length) { * @return The byte array that backs this memory segment, or null, if the segment has been freed. */ public byte[] getArray() { - return this.heapMemory; + return this.memory; } // ------------------------------------------------------------------------ diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/HeapMemorySegmentTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/HeapMemorySegmentTest.java index 90176fd85fe3..211c408c3972 100644 --- a/flink-core/src/test/java/org/apache/flink/core/memory/HeapMemorySegmentTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/memory/HeapMemorySegmentTest.java @@ -26,6 +26,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; /** @@ -67,4 +68,13 @@ public void testHeapSegmentSpecifics() { assertEquals(3, buf2.position()); assertEquals(7, buf2.limit()); } + + @Test + public void testGetArrayAfterFree() { + final byte[] buffer = new byte[100]; + HeapMemorySegment seg = new HeapMemorySegment(buffer); + + seg.free(); + assertNull(seg.getArray()); + } } From d5bb07301427aa0b3a1976447162c5aefc71b044 Mon Sep 17 00:00:00 2001 From: liyafan82 Date: Fri, 10 May 2019 18:34:08 +0800 Subject: [PATCH 339/746] [hotfix] Fix typos in MemorySegment java docs --- .../java/org/apache/flink/core/memory/MemorySegment.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java index 0331736a437a..d66b75d4d9df 100644 --- a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java +++ b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java @@ -271,7 +271,7 @@ public long getAddress() { /** * Wraps the chunk of the underlying memory located between offset and - * length in a NIO ByteBuffer. The ByteBuffer has the full segment as capacity + * offset + length in a NIO ByteBuffer. The ByteBuffer has the full segment as capacity * and the offset and length parameters set the buffers position and limit. * * @param offset The offset in the memory segment. @@ -1273,7 +1273,7 @@ else if (target.address > target.addressLimit) { /** * Bulk copy method. Copies {@code numBytes} bytes to target unsafe object and pointer. - * NOTE: This is a unsafe method, no check here, please be carefully. + * NOTE: This is an unsafe method, no check here, please be careful. * * @param offset The position where the bytes are started to be read from in this memory segment. * @param target The unsafe memory to copy the bytes to. @@ -1295,7 +1295,7 @@ public final void copyToUnsafe(int offset, Object target, int targetPointer, int /** * Bulk copy method. Copies {@code numBytes} bytes from source unsafe object and pointer. - * NOTE: This is a unsafe method, no check here, please be carefully. + * NOTE: This is an unsafe method, no check here, please be careful. * * @param offset The position where the bytes are started to be write in this memory segment. * @param source The unsafe memory to copy the bytes from. From 11ef25319484baf8e00e5ab3bafc7e4c95416891 Mon Sep 17 00:00:00 2001 From: liyafan82 Date: Wed, 23 Oct 2019 18:33:55 +0800 Subject: [PATCH 340/746] [FLINK-12289][flink-runtime]Fix typos in Memory manager --- .../java/org/apache/flink/runtime/memory/MemoryManager.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java index a6bc31135eaf..738412b6490d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java @@ -176,7 +176,7 @@ public boolean isShutdown() { } /** - * Checks if the memory manager all memory available. + * Checks if the memory manager's memory is completely available (nothing allocated at the moment). * * @return True, if the memory manager is empty and valid, false if it is not empty or corrupted. */ @@ -328,8 +328,7 @@ public void release(MemorySegment segment) { * the memory pool of its type, increasing its available limit for the later allocations. * * @param segments The segments to be released. - * @throws NullPointerException Thrown, if the given collection is null. - * @throws IllegalArgumentException Thrown, id the segments are of an incompatible type. + * @throws IllegalArgumentException Thrown, if the segments are of an incompatible type. */ public void release(Collection segments) { if (segments == null) { From 6134cbc7a2c63b6f0c602614000596818be13cdb Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 29 Oct 2019 14:05:25 +0100 Subject: [PATCH 341/746] [FLINK-14318][travis] Compile with a single thread Multi-threaded compilation can deadlock builds with later shade-plugin versions. --- tools/travis/nightly.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/travis/nightly.sh b/tools/travis/nightly.sh index 433b925db47c..0fb918e4d22d 100755 --- a/tools/travis/nightly.sh +++ b/tools/travis/nightly.sh @@ -38,7 +38,7 @@ LOG4J_PROPERTIES=${HERE}/../log4j-travis.properties MVN_LOGGING_OPTIONS="-Dlog.dir=${ARTIFACTS_DIR} -Dlog4j.configuration=file://$LOG4J_PROPERTIES -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn" MVN_COMMON_OPTIONS="-nsu -B -Dflink.forkCount=2 -Dflink.forkCountTestPackage=2 -Dfast -Pskip-webui-build" -MVN_COMPILE_OPTIONS="-T1C -DskipTests" +MVN_COMPILE_OPTIONS="-DskipTests" cp tools/travis/splits/* flink-end-to-end-tests From 5d92eb1b5038a106437629473342771dd42ebcd2 Mon Sep 17 00:00:00 2001 From: hehuiyuan <471627698@qq.com> Date: Tue, 29 Oct 2019 22:18:13 +0800 Subject: [PATCH 342/746] [FLINK-13756][table] Fix javadoc for findAndCreateTableSource method in TableFactoryUtil class --- .../java/org/apache/flink/table/factories/TableFactoryUtil.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/factories/TableFactoryUtil.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/factories/TableFactoryUtil.java index 715d5434ac4b..fe4009e30f21 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/factories/TableFactoryUtil.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/factories/TableFactoryUtil.java @@ -86,7 +86,7 @@ public static TableSink findAndCreateTableSink(CatalogTable table) { } /** - * Returns a table sink matching the {@link org.apache.flink.table.catalog.CatalogTable}. + * Returns a table source matching the {@link org.apache.flink.table.catalog.CatalogTable}. */ public static TableSource findAndCreateTableSource(CatalogTable table) { return findAndCreateTableSource(table.toProperties()); From 4571b319e7452ec86325a6c389e6d1bd5008fb0e Mon Sep 17 00:00:00 2001 From: Aitozi <1059789585@qq.com> Date: Tue, 29 Oct 2019 22:23:56 +0800 Subject: [PATCH 343/746] [FLINK-12492][cep] Minor optimize the cep operator by avoiding unnecessary copy --- .../java/org/apache/flink/cep/operator/CepOperator.java | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CepOperator.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CepOperator.java index ca74e126b385..fe95c6dbbd05 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CepOperator.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CepOperator.java @@ -292,12 +292,7 @@ private void bufferEvent(IN event, long currentTime) throws Exception { elementsForTimestamp = new ArrayList<>(); } - if (getExecutionConfig().isObjectReuseEnabled()) { - // copy the StreamRecord so that it cannot be changed - elementsForTimestamp.add(inputSerializer.copy(event)); - } else { - elementsForTimestamp.add(event); - } + elementsForTimestamp.add(event); elementQueueState.put(currentTime, elementsForTimestamp); } From f79dda8ff1791e28743d2ea8723b980c2968acf3 Mon Sep 17 00:00:00 2001 From: Ling Wang Date: Tue, 29 Oct 2019 09:34:36 -0500 Subject: [PATCH 344/746] [FLINK-12092][docs] Clarify when onTimer(...) in ProcessFunction is called. --- docs/dev/stream/operators/process_function.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/dev/stream/operators/process_function.md b/docs/dev/stream/operators/process_function.md index 79212311ad9c..bcf954ccb9cc 100644 --- a/docs/dev/stream/operators/process_function.md +++ b/docs/dev/stream/operators/process_function.md @@ -44,8 +44,7 @@ For fault-tolerant state, the `ProcessFunction` gives access to Flink's [keyed s The timers allow applications to react to changes in processing time and in [event time]({{ site.baseurl }}/dev/event_time.html). Every call to the function `processElement(...)` gets a `Context` object which gives access to the element's event time timestamp, and to the *TimerService*. The `TimerService` can be used to register callbacks for future -event-/processing-time instants. When a timer's particular time is reached, the `onTimer(...)` method is -called. During that call, all states are again scoped to the key with which the timer was created, allowing +event-/processing-time instants. With event-time timers, the `onTimer(...)` method is called when the current watermark is advanced up to or beyond the timestamp of the timer, while with processing-time timers, `onTimer(...)` is called when wall clock time reaches the specified time. During that call, all states are again scoped to the key with which the timer was created, allowing timers to manipulate keyed state. Note If you want to access keyed state and timers you have From 6378f82cdef490385f3c89e865902eca64531c8a Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Tue, 29 Oct 2019 23:39:08 +0800 Subject: [PATCH 345/746] [FLINK-10435][yarn] Fix client sporadically hangs after Ctrl + C This closes #10010 --- .../apache/flink/yarn/YarnClusterDescriptor.java | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 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 26d641798bf4..18dcb3c891ca 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 @@ -1031,7 +1031,7 @@ private ApplicationReport startAppMaster( setApplicationTags(appContext); // add a hook to clean up in case deployment fails - Thread deploymentFailureHook = new DeploymentFailureHook(yarnClient, yarnApplication, yarnFilesDir); + Thread deploymentFailureHook = new DeploymentFailureHook(yarnApplication, yarnFilesDir); Runtime.getRuntime().addShutdownHook(deploymentFailureHook); LOG.info("Submitting application master " + appId); yarnClient.submitApplication(appContext); @@ -1241,7 +1241,6 @@ private void failSessionDuringDeployment(YarnClient yarnClient, YarnClientApplic // 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 { @@ -1526,16 +1525,22 @@ private class DeploymentFailureHook extends Thread { private final YarnClientApplication yarnApplication; private final Path yarnFilesDir; - DeploymentFailureHook(YarnClient yarnClient, YarnClientApplication yarnApplication, Path yarnFilesDir) { - this.yarnClient = Preconditions.checkNotNull(yarnClient); + DeploymentFailureHook(YarnClientApplication yarnApplication, Path yarnFilesDir) { this.yarnApplication = Preconditions.checkNotNull(yarnApplication); this.yarnFilesDir = Preconditions.checkNotNull(yarnFilesDir); + + // A new yarn client need to be created in shutdown hook in order to avoid + // the yarn client has been closed by YarnClusterDescriptor. + this.yarnClient = YarnClient.createYarnClient(); + this.yarnClient.init(yarnConfiguration); } @Override public void run() { LOG.info("Cancelling deployment from Deployment Failure Hook"); + yarnClient.start(); failSessionDuringDeployment(yarnClient, yarnApplication); + yarnClient.stop(); LOG.info("Deleting files in {}.", yarnFilesDir); try { FileSystem fs = FileSystem.get(yarnConfiguration); From 759b8cb95cf5dad6ea0494df0c8800429ab94b64 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Tue, 29 Oct 2019 15:46:32 +0800 Subject: [PATCH 346/746] [FLINK-14061][runtime] Introduce managed memory fractions to StreamConfig This closes #10024. --- .../streaming/api/graph/StreamConfig.java | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) 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 5384541030d3..e614399940a8 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 @@ -94,6 +94,9 @@ public class StreamConfig implements Serializable { private static final String TIME_CHARACTERISTIC = "timechar"; + private static final String MANAGED_MEMORY_FRACTION_ON_HEAP = "managedMemFractionOnHeap"; + private static final String MANAGED_MEMORY_FRACTION_OFF_HEAP = "managedMemFractionOffHeap"; + // ------------------------------------------------------------------------ // Default Values // ------------------------------------------------------------------------ @@ -101,6 +104,7 @@ public class StreamConfig implements Serializable { private static final long DEFAULT_TIMEOUT = 100; private static final CheckpointingMode DEFAULT_CHECKPOINTING_MODE = CheckpointingMode.EXACTLY_ONCE; + private static final double DEFAULT_MANAGED_MEMORY_FRACTION = 0.0; // ------------------------------------------------------------------------ // Config @@ -128,6 +132,22 @@ public Integer getVertexID() { return config.getInteger(VERTEX_NAME, -1); } + public void setManagedMemoryFractionOnHeap(double managedMemFractionOnHeap) { + config.setDouble(MANAGED_MEMORY_FRACTION_ON_HEAP, managedMemFractionOnHeap); + } + + public double getManagedMemoryFractionOnHeap() { + return config.getDouble(MANAGED_MEMORY_FRACTION_ON_HEAP, DEFAULT_MANAGED_MEMORY_FRACTION); + } + + public void setManagedMemoryFractionOffHeap(double managedMemFractionOffHeap) { + config.setDouble(MANAGED_MEMORY_FRACTION_OFF_HEAP, managedMemFractionOffHeap); + } + + public double getManagedMemoryFractionOffHeap() { + return config.getDouble(MANAGED_MEMORY_FRACTION_OFF_HEAP, DEFAULT_MANAGED_MEMORY_FRACTION); + } + public void setTimeCharacteristic(TimeCharacteristic characteristic) { config.setInteger(TIME_CHARACTERISTIC, characteristic.ordinal()); } From 6cc09ac6ed822e2b3d753b27c621f5122d353673 Mon Sep 17 00:00:00 2001 From: Rui Li Date: Fri, 25 Oct 2019 19:13:55 +0800 Subject: [PATCH 347/746] [FLINK-14526][hive] Support Hive version 1.1.0 and 1.1.1 To support Hive 1.1.0 and 1.1.1. This closes #9995. --- flink-connectors/flink-connector-hive/pom.xml | 23 ++ .../flink/table/catalog/hive/HiveCatalog.java | 13 +- .../table/catalog/hive/client/HiveShim.java | 27 +- .../catalog/hive/client/HiveShimLoader.java | 8 + .../catalog/hive/client/HiveShimV110.java | 297 ++++++++++++++++++ .../catalog/hive/client/HiveShimV111.java | 26 ++ .../catalog/hive/client/HiveShimV120.java | 218 ++++++------- .../hive/util/HiveReflectionUtils.java | 28 +- .../catalog/hive/util/HiveStatsUtil.java | 48 +-- .../table/catalog/hive/util/HiveTypeUtil.java | 9 + .../table/functions/hive/HiveGenericUDF.java | 10 +- .../table/functions/hive/HiveGenericUDTF.java | 2 +- .../hive/conversion/HiveInspectors.java | 125 ++++---- .../WritableHiveObjectConversion.java | 36 +++ .../FlinkStandaloneHiveServerContext.java | 3 + .../connectors/hive/HiveRunnerShimLoader.java | 2 + .../connectors/hive/HiveTableSourceTest.java | 58 ++-- .../hive/TableEnvHiveConnectorTest.java | 17 +- .../hive/HiveCatalogHiveMetadataTest.java | 33 +- .../catalog/hive/HiveCatalogTestBase.java | 11 + .../table/catalog/hive/HiveTestUtils.java | 109 +++++++ .../functions/hive/HiveGenericUDFTest.java | 10 +- .../functions/hive/HiveGenericUDTFTest.java | 12 +- 23 files changed, 829 insertions(+), 296 deletions(-) create mode 100644 flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV110.java create mode 100644 flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV111.java create mode 100644 flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/conversion/WritableHiveObjectConversion.java diff --git a/flink-connectors/flink-connector-hive/pom.xml b/flink-connectors/flink-connector-hive/pom.xml index 857affefc2e5..e20bdebfce7d 100644 --- a/flink-connectors/flink-connector-hive/pom.xml +++ b/flink-connectors/flink-connector-hive/pom.xml @@ -415,6 +415,10 @@ under the License. org.apache.hive hive-hcatalog-core + + org.apache.hive.hcatalog + hive-webhcat-java-client + org.apache.tez tez-common @@ -601,9 +605,20 @@ under the License. hadoop-mapreduce-client-core org.apache.hadoop + + org.apache.hadoop + hadoop-yarn-server-resourcemanager + + + org.apache.hive.hcatalog + hive-webhcat-java-client + ${hive.version} + test + + org.apache.flink @@ -652,6 +667,14 @@ under the License. + + hive-1.1.1 + + 1.1.1 + 2.6.5 + 3.1.1 + + hive-1.2.1 diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java index 1e66551d3a1b..49ff7441c167 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java @@ -1120,6 +1120,10 @@ private boolean isTablePartitioned(Table hiveTable) { public void alterTableStatistics(ObjectPath tablePath, CatalogTableStatistics tableStatistics, boolean ignoreIfNotExists) throws TableNotExistException, CatalogException { try { Table hiveTable = getHiveTable(tablePath); + // the stats we put in table parameters will be overridden by HMS in older Hive versions, so error out + if (!isTablePartitioned(hiveTable) && hiveVersion.compareTo("1.2.1") < 0) { + throw new CatalogException("Alter table stats is not supported in Hive version " + hiveVersion); + } // Set table stats if (compareAndUpdateStatisticsProperties(tableStatistics, hiveTable.getParameters())) { client.alter_table(tablePath.getDatabaseName(), tablePath.getObjectName(), hiveTable); @@ -1139,7 +1143,7 @@ public void alterTableColumnStatistics(ObjectPath tablePath, CatalogColumnStatis Table hiveTable = getHiveTable(tablePath); // Set table column stats. This only works for non-partitioned tables. if (!isTablePartitioned(hiveTable)) { - client.updateTableColumnStatistics(HiveStatsUtil.createTableColumnStats(hiveTable, columnStatistics.getColumnStatisticsData())); + client.updateTableColumnStatistics(HiveStatsUtil.createTableColumnStats(hiveTable, columnStatistics.getColumnStatisticsData(), hiveVersion)); } else { throw new TablePartitionedException(getName(), tablePath); } @@ -1204,7 +1208,8 @@ public void alterPartitionColumnStatistics(ObjectPath tablePath, CatalogPartitio Partition hivePartition = getHivePartition(tablePath, partitionSpec); Table hiveTable = getHiveTable(tablePath); String partName = getPartitionName(tablePath, partitionSpec, hiveTable); - client.updatePartitionColumnStatistics(HiveStatsUtil.createPartitionColumnStats(hivePartition, partName, columnStatistics.getColumnStatisticsData())); + client.updatePartitionColumnStatistics(HiveStatsUtil.createPartitionColumnStats( + hivePartition, partName, columnStatistics.getColumnStatisticsData(), hiveVersion)); } catch (TableNotExistException | PartitionSpecInvalidException e) { if (!ignoreIfNotExists) { throw new PartitionNotExistException(getName(), tablePath, partitionSpec, e); @@ -1243,7 +1248,7 @@ public CatalogColumnStatistics getTableColumnStatistics(ObjectPath tablePath) th if (!isTablePartitioned(hiveTable)) { List columnStatisticsObjs = client.getTableColumnStatistics( hiveTable.getDbName(), hiveTable.getTableName(), getFieldNames(hiveTable.getSd().getCols())); - return new CatalogColumnStatistics(HiveStatsUtil.createCatalogColumnStats(columnStatisticsObjs)); + return new CatalogColumnStatistics(HiveStatsUtil.createCatalogColumnStats(columnStatisticsObjs, hiveVersion)); } else { // TableColumnStats of partitioned table is unknown, the behavior is same as HIVE return CatalogColumnStatistics.UNKNOWN; @@ -1280,7 +1285,7 @@ public CatalogColumnStatistics getPartitionColumnStatistics(ObjectPath tablePath getFieldNames(partition.getSd().getCols())); List columnStatisticsObjs = partitionColumnStatistics.get(partName); if (columnStatisticsObjs != null && !columnStatisticsObjs.isEmpty()) { - return new CatalogColumnStatistics(HiveStatsUtil.createCatalogColumnStats(columnStatisticsObjs)); + return new CatalogColumnStatistics(HiveStatsUtil.createCatalogColumnStats(columnStatisticsObjs, hiveVersion)); } else { return CatalogColumnStatistics.UNKNOWN; } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShim.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShim.java index 1f39d844ed2a..2111099afb83 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShim.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShim.java @@ -18,12 +18,15 @@ package org.apache.flink.table.catalog.hive.client; +import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataDate; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData; import org.apache.hadoop.hive.metastore.api.Function; import org.apache.hadoop.hive.metastore.api.InvalidOperationException; import org.apache.hadoop.hive.metastore.api.MetaException; @@ -33,16 +36,18 @@ import org.apache.hadoop.hive.metastore.api.UnknownDBException; import org.apache.hadoop.hive.ql.udf.generic.SimpleGenericUDAFParameterInfo; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; import org.apache.thrift.TException; import java.io.IOException; +import java.io.Serializable; import java.util.List; import java.util.Map; /** * A shim layer to support different versions of Hive. */ -public interface HiveShim { +public interface HiveShim extends Serializable { /** * Create a Hive Metastore client based on the given HiveConf object. @@ -159,4 +164,24 @@ SimpleGenericUDAFParameterInfo createUDAFParameterInfo(ObjectInspector[] params, */ void makeSpecFromName(Map partSpec, Path currPath); + /** + * Get ObjectInspector for a constant value. + */ + ObjectInspector getObjectInspectorForConstant(PrimitiveTypeInfo primitiveTypeInfo, Object value); + + /** + * Generate Hive ColumnStatisticsData from Flink CatalogColumnStatisticsDataDate for DATE columns. + */ + ColumnStatisticsData toHiveDateColStats(CatalogColumnStatisticsDataDate flinkDateColStats); + + /** + * Whether a Hive ColumnStatisticsData is for DATE columns. + */ + boolean isDateStats(ColumnStatisticsData colStatsData); + + /** + * Generate Flink CatalogColumnStatisticsDataDate from Hive ColumnStatisticsData for DATE columns. + */ + CatalogColumnStatisticsDataDate toFlinkDateColStats(ColumnStatisticsData hiveDateColStats); + } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimLoader.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimLoader.java index b3e3903385a7..1645d081a113 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimLoader.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimLoader.java @@ -32,6 +32,8 @@ */ public class HiveShimLoader { + public static final String HIVE_VERSION_V1_1_0 = "1.1.0"; + public static final String HIVE_VERSION_V1_1_1 = "1.1.1"; public static final String HIVE_VERSION_V1_2_0 = "1.2.0"; public static final String HIVE_VERSION_V1_2_1 = "1.2.1"; public static final String HIVE_VERSION_V1_2_2 = "1.2.2"; @@ -60,6 +62,12 @@ private HiveShimLoader() { public static HiveShim loadHiveShim(String version) { return hiveShims.computeIfAbsent(version, (v) -> { + if (v.startsWith(HIVE_VERSION_V1_1_0)) { + return new HiveShimV110(); + } + if (v.startsWith(HIVE_VERSION_V1_1_1)) { + return new HiveShimV111(); + } if (v.startsWith(HIVE_VERSION_V1_2_0)) { return new HiveShimV120(); } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV110.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV110.java new file mode 100644 index 000000000000..91fd80af9a9c --- /dev/null +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV110.java @@ -0,0 +1,297 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.catalog.hive.client; + +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.hive.util.HiveReflectionUtils; +import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataDate; +import org.apache.flink.table.functions.hive.FlinkHiveUDFException; +import org.apache.flink.table.functions.hive.conversion.HiveInspectors; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.FileUtils; +import org.apache.hadoop.hive.common.HiveStatsUtils; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.Warehouse; +import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData; +import org.apache.hadoop.hive.metastore.api.Function; +import org.apache.hadoop.hive.metastore.api.InvalidOperationException; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.api.UnknownDBException; +import org.apache.hadoop.hive.ql.udf.generic.SimpleGenericUDAFParameterInfo; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantBinaryObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantBooleanObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantByteObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantDateObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantDoubleObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantFloatObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantHiveCharObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantHiveDecimalObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantHiveVarcharObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantIntObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantLongObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantShortObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantStringObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantTimestampObjectInspector; +import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo; +import org.apache.thrift.TException; + +import java.io.IOException; +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * Shim for Hive version 1.1.0. + */ +public class HiveShimV110 implements HiveShim { + + @Override + public IMetaStoreClient getHiveMetastoreClient(HiveConf hiveConf) { + try { + return new HiveMetaStoreClient(hiveConf); + } catch (MetaException ex) { + throw new CatalogException("Failed to create Hive Metastore client", ex); + } + } + + @Override + // 1.x client doesn't support filtering tables by type, so here we need to get all tables and filter by ourselves + public List getViews(IMetaStoreClient client, String databaseName) throws UnknownDBException, TException { + // We don't have to use reflection here because client.getAllTables(String) is supposed to be there for + // all versions. + List tableNames = client.getAllTables(databaseName); + List views = new ArrayList<>(); + for (String name : tableNames) { + Table table = client.getTable(databaseName, name); + String viewDef = table.getViewOriginalText(); + if (viewDef != null && !viewDef.isEmpty()) { + views.add(table.getTableName()); + } + } + return views; + } + + @Override + public Function getFunction(IMetaStoreClient client, String dbName, String functionName) throws NoSuchObjectException, TException { + try { + // hive-1.x doesn't throw NoSuchObjectException if function doesn't exist, instead it throws a MetaException + return client.getFunction(dbName, functionName); + } catch (MetaException e) { + // need to check the cause and message of this MetaException to decide whether it should actually be a NoSuchObjectException + if (e.getCause() instanceof NoSuchObjectException) { + throw (NoSuchObjectException) e.getCause(); + } + if (e.getMessage().startsWith(NoSuchObjectException.class.getSimpleName())) { + throw new NoSuchObjectException(e.getMessage()); + } + throw e; + } + } + + @Override + public boolean moveToTrash(FileSystem fs, Path path, Configuration conf, boolean purge) throws IOException { + try { + Method method = FileUtils.class.getDeclaredMethod("moveToTrash", FileSystem.class, Path.class, Configuration.class); + return (boolean) method.invoke(null, fs, path, conf); + } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) { + throw new IOException("Failed to move " + path + " to trash", e); + } + } + + @Override + public void alterTable(IMetaStoreClient client, String databaseName, String tableName, Table table) throws InvalidOperationException, MetaException, TException { + client.alter_table(databaseName, tableName, table); + } + + @Override + public void alterPartition(IMetaStoreClient client, String databaseName, String tableName, Partition partition) + throws InvalidOperationException, MetaException, TException { + String errorMsg = "Failed to alter partition for table %s in database %s"; + try { + Method method = client.getClass().getMethod("alter_partition", String.class, String.class, Partition.class); + method.invoke(client, databaseName, tableName, partition); + } catch (InvocationTargetException ite) { + Throwable targetEx = ite.getTargetException(); + if (targetEx instanceof TException) { + throw (TException) targetEx; + } else { + throw new CatalogException(String.format(errorMsg, tableName, databaseName), targetEx); + } + } catch (NoSuchMethodException | IllegalAccessException e) { + throw new CatalogException(String.format(errorMsg, tableName, databaseName), e); + } + } + + @Override + public SimpleGenericUDAFParameterInfo createUDAFParameterInfo(ObjectInspector[] params, boolean isWindowing, boolean distinct, boolean allColumns) { + try { + Constructor constructor = SimpleGenericUDAFParameterInfo.class.getConstructor(ObjectInspector[].class, + boolean.class, boolean.class); + return (SimpleGenericUDAFParameterInfo) constructor.newInstance(params, distinct, allColumns); + } catch (NoSuchMethodException | IllegalAccessException | InstantiationException | InvocationTargetException e) { + throw new CatalogException("Failed to create SimpleGenericUDAFParameterInfo", e); + } + } + + @Override + public Class getMetaStoreUtilsClass() { + try { + return Class.forName("org.apache.hadoop.hive.metastore.MetaStoreUtils"); + } catch (ClassNotFoundException e) { + throw new CatalogException("Failed to find class MetaStoreUtils", e); + } + } + + @Override + public Class getHiveMetaStoreUtilsClass() { + return getMetaStoreUtilsClass(); + } + + @Override + public Class getDateDataTypeClass() { + return java.sql.Date.class; + } + + @Override + public Class getTimestampDataTypeClass() { + return java.sql.Timestamp.class; + } + + @Override + public FileStatus[] getFileStatusRecurse(Path path, int level, FileSystem fs) throws IOException { + try { + Method method = HiveStatsUtils.class.getMethod("getFileStatusRecurse", Path.class, Integer.TYPE, FileSystem.class); + // getFileStatusRecurse is a static method + return (FileStatus[]) method.invoke(null, path, level, fs); + } catch (Exception ex) { + throw new CatalogException("Failed to invoke HiveStatsUtils.getFileStatusRecurse()", ex); + } + } + + @Override + public void makeSpecFromName(Map partSpec, Path currPath) { + try { + Method method = Warehouse.class.getMethod("makeSpecFromName", Map.class, Path.class); + // makeSpecFromName is a static method + method.invoke(null, partSpec, currPath); + } catch (Exception ex) { + throw new CatalogException("Failed to invoke Warehouse.makeSpecFromName()", ex); + } + } + + @Override + public ObjectInspector getObjectInspectorForConstant(PrimitiveTypeInfo primitiveTypeInfo, Object value) { + String className; + value = HiveInspectors.hivePrimitiveToWritable(value); + // Java constant object inspectors are not available until 1.2.0 -- https://issues.apache.org/jira/browse/HIVE-9766 + // So we have to use writable constant object inspectors for 1.1.x + switch (primitiveTypeInfo.getPrimitiveCategory()) { + case BOOLEAN: + className = WritableConstantBooleanObjectInspector.class.getName(); + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case BYTE: + className = WritableConstantByteObjectInspector.class.getName(); + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case SHORT: + className = WritableConstantShortObjectInspector.class.getName(); + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case INT: + className = WritableConstantIntObjectInspector.class.getName(); + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case LONG: + className = WritableConstantLongObjectInspector.class.getName(); + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case FLOAT: + className = WritableConstantFloatObjectInspector.class.getName(); + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case DOUBLE: + className = WritableConstantDoubleObjectInspector.class.getName(); + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case STRING: + className = WritableConstantStringObjectInspector.class.getName(); + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case CHAR: + className = WritableConstantHiveCharObjectInspector.class.getName(); + try { + return (ObjectInspector) Class.forName(className).getDeclaredConstructor( + CharTypeInfo.class, value.getClass()).newInstance(primitiveTypeInfo, value); + } catch (Exception e) { + throw new FlinkHiveUDFException("Failed to create writable constant object inspector", e); + } + case VARCHAR: + className = WritableConstantHiveVarcharObjectInspector.class.getName(); + try { + return (ObjectInspector) Class.forName(className).getDeclaredConstructor( + VarcharTypeInfo.class, value.getClass()).newInstance(primitiveTypeInfo, value); + } catch (Exception e) { + throw new FlinkHiveUDFException("Failed to create writable constant object inspector", e); + } + case DATE: + className = WritableConstantDateObjectInspector.class.getName(); + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case TIMESTAMP: + className = WritableConstantTimestampObjectInspector.class.getName(); + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case DECIMAL: + className = WritableConstantHiveDecimalObjectInspector.class.getName(); + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case BINARY: + className = WritableConstantBinaryObjectInspector.class.getName(); + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case UNKNOWN: + case VOID: + // If type is null, we use the Constant String to replace + className = WritableConstantStringObjectInspector.class.getName(); + return HiveReflectionUtils.createConstantObjectInspector(className, value.toString()); + default: + throw new FlinkHiveUDFException( + String.format("Cannot find ConstantObjectInspector for %s", primitiveTypeInfo)); + } + } + + @Override + public ColumnStatisticsData toHiveDateColStats(CatalogColumnStatisticsDataDate flinkDateColStats) { + throw new UnsupportedOperationException("DATE column stats are not supported until Hive 1.2.0"); + } + + @Override + public boolean isDateStats(ColumnStatisticsData colStatsData) { + return false; + } + + @Override + public CatalogColumnStatisticsDataDate toFlinkDateColStats(ColumnStatisticsData hiveDateColStats) { + throw new UnsupportedOperationException("DATE column stats are not supported until Hive 1.2.0"); + } +} diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV111.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV111.java new file mode 100644 index 000000000000..bafa50a949bc --- /dev/null +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV111.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.catalog.hive.client; + +/** + * Shim for Hive version 1.1.1. + */ +public class HiveShimV111 extends HiveShimV110 { + +} diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV120.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV120.java index d1ed4440f119..fc929476fd9e 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV120.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV120.java @@ -19,41 +19,30 @@ package org.apache.flink.table.catalog.hive.client; import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.hive.util.HiveReflectionUtils; +import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataDate; +import org.apache.flink.table.catalog.stats.Date; +import org.apache.flink.table.functions.hive.FlinkHiveUDFException; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.common.FileUtils; -import org.apache.hadoop.hive.common.HiveStatsUtils; -import org.apache.hadoop.hive.common.StatsSetupConst; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.metastore.RetryingMetaStoreClient; -import org.apache.hadoop.hive.metastore.Warehouse; -import org.apache.hadoop.hive.metastore.api.Function; +import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData; import org.apache.hadoop.hive.metastore.api.InvalidOperationException; import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; -import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.api.UnknownDBException; -import org.apache.hadoop.hive.ql.udf.generic.SimpleGenericUDAFParameterInfo; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; import org.apache.thrift.TException; -import java.io.IOException; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; /** * Shim for Hive version 1.2.0. */ -public class HiveShimV120 implements HiveShim { +public class HiveShimV120 extends HiveShimV111 { @Override public IMetaStoreClient getHiveMetastoreClient(HiveConf hiveConf) { @@ -66,132 +55,117 @@ public IMetaStoreClient getHiveMetastoreClient(HiveConf hiveConf) { } } - @Override - // 1.x client doesn't support filtering tables by type, so here we need to get all tables and filter by ourselves - public List getViews(IMetaStoreClient client, String databaseName) throws UnknownDBException, TException { - // We don't have to use reflection here because client.getAllTables(String) is supposed to be there for - // all versions. - List tableNames = client.getAllTables(databaseName); - List views = new ArrayList<>(); - for (String name : tableNames) { - Table table = client.getTable(databaseName, name); - String viewDef = table.getViewOriginalText(); - if (viewDef != null && !viewDef.isEmpty()) { - views.add(table.getTableName()); - } - } - return views; - } - - @Override - public Function getFunction(IMetaStoreClient client, String dbName, String functionName) throws NoSuchObjectException, TException { - try { - // hive-1.x doesn't throw NoSuchObjectException if function doesn't exist, instead it throws a MetaException - return client.getFunction(dbName, functionName); - } catch (MetaException e) { - // need to check the cause and message of this MetaException to decide whether it should actually be a NoSuchObjectException - if (e.getCause() instanceof NoSuchObjectException) { - throw (NoSuchObjectException) e.getCause(); - } - if (e.getMessage().startsWith(NoSuchObjectException.class.getSimpleName())) { - throw new NoSuchObjectException(e.getMessage()); - } - throw e; - } - } - - @Override - public boolean moveToTrash(FileSystem fs, Path path, Configuration conf, boolean purge) throws IOException { - try { - Method method = FileUtils.class.getDeclaredMethod("moveToTrash", FileSystem.class, Path.class, Configuration.class); - return (boolean) method.invoke(null, fs, path, conf); - } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) { - throw new IOException("Failed to move " + path + " to trash", e); - } - } - @Override public void alterTable(IMetaStoreClient client, String databaseName, String tableName, Table table) throws InvalidOperationException, MetaException, TException { - // For Hive-1.2.1, we need to tell HMS not to update stats. Otherwise, the stats we put in the table + // For Hive-1.2.x, we need to tell HMS not to update stats. Otherwise, the stats we put in the table // parameters can be overridden. The extra config we add here will be removed by HMS after it's used. - table.getParameters().put(StatsSetupConst.DO_NOT_UPDATE_STATS, "true"); + // Don't use StatsSetupConst.DO_NOT_UPDATE_STATS because it wasn't defined in Hive 1.1.x. + table.getParameters().put("DO_NOT_UPDATE_STATS", "true"); client.alter_table(databaseName, tableName, table); } @Override - public void alterPartition(IMetaStoreClient client, String databaseName, String tableName, Partition partition) - throws InvalidOperationException, MetaException, TException { - String errorMsg = "Failed to alter partition for table %s in database %s"; - try { - Method method = client.getClass().getMethod("alter_partition", String.class, String.class, Partition.class); - method.invoke(client, databaseName, tableName, partition); - } catch (InvocationTargetException ite) { - Throwable targetEx = ite.getTargetException(); - if (targetEx instanceof TException) { - throw (TException) targetEx; - } else { - throw new CatalogException(String.format(errorMsg, tableName, databaseName), targetEx); - } - } catch (NoSuchMethodException | IllegalAccessException e) { - throw new CatalogException(String.format(errorMsg, tableName, databaseName), e); - } - } - - @Override - public SimpleGenericUDAFParameterInfo createUDAFParameterInfo(ObjectInspector[] params, boolean isWindowing, boolean distinct, boolean allColumns) { - try { - Constructor constructor = SimpleGenericUDAFParameterInfo.class.getConstructor(ObjectInspector[].class, - boolean.class, boolean.class); - return (SimpleGenericUDAFParameterInfo) constructor.newInstance(params, distinct, allColumns); - } catch (NoSuchMethodException | IllegalAccessException | InstantiationException | InvocationTargetException e) { - throw new CatalogException("Failed to create SimpleGenericUDAFParameterInfo", e); + public ObjectInspector getObjectInspectorForConstant(PrimitiveTypeInfo primitiveTypeInfo, Object value) { + String className; + switch (primitiveTypeInfo.getPrimitiveCategory()) { + case BOOLEAN: + className = "org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantBooleanObjectInspector"; + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case BYTE: + className = "org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantByteObjectInspector"; + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case SHORT: + className = "org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantShortObjectInspector"; + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case INT: + className = "org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantIntObjectInspector"; + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case LONG: + className = "org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantLongObjectInspector"; + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case FLOAT: + className = "org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantFloatObjectInspector"; + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case DOUBLE: + className = "org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantDoubleObjectInspector"; + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case STRING: + className = "org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantStringObjectInspector"; + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case CHAR: + className = "org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantHiveCharObjectInspector"; + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case VARCHAR: + className = "org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantHiveVarcharObjectInspector"; + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case DATE: + className = "org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantDateObjectInspector"; + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case TIMESTAMP: + className = "org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantTimestampObjectInspector"; + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case DECIMAL: + className = "org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantHiveDecimalObjectInspector"; + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case BINARY: + className = "org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantBinaryObjectInspector"; + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case UNKNOWN: + case VOID: + // If type is null, we use the Java Constant String to replace + className = "org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantStringObjectInspector"; + return HiveReflectionUtils.createConstantObjectInspector(className, value.toString()); + default: + throw new FlinkHiveUDFException( + String.format("Cannot find ConstantObjectInspector for %s", primitiveTypeInfo)); } } @Override - public Class getMetaStoreUtilsClass() { + public ColumnStatisticsData toHiveDateColStats(CatalogColumnStatisticsDataDate flinkDateColStats) { try { - return Class.forName("org.apache.hadoop.hive.metastore.MetaStoreUtils"); - } catch (ClassNotFoundException e) { - throw new CatalogException("Failed to find class MetaStoreUtils", e); + Class dateStatsClz = Class.forName("org.apache.hadoop.hive.metastore.api.DateColumnStatsData"); + Object dateStats = dateStatsClz.getDeclaredConstructor(long.class, long.class) + .newInstance(flinkDateColStats.getNullCount(), flinkDateColStats.getNdv()); + Class hmsDateClz = Class.forName("org.apache.hadoop.hive.metastore.api.Date"); + Method setHigh = dateStatsClz.getDeclaredMethod("setHighValue", hmsDateClz); + Method setLow = dateStatsClz.getDeclaredMethod("setLowValue", hmsDateClz); + Constructor hmsDateConstructor = hmsDateClz.getConstructor(long.class); + setHigh.invoke(dateStats, hmsDateConstructor.newInstance(flinkDateColStats.getMax().getDaysSinceEpoch())); + setLow.invoke(dateStats, hmsDateConstructor.newInstance(flinkDateColStats.getMin().getDaysSinceEpoch())); + Class colStatsClz = ColumnStatisticsData.class; + return (ColumnStatisticsData) colStatsClz.getDeclaredMethod("dateStats", dateStatsClz).invoke(null, dateStats); + } catch (ClassNotFoundException | NoSuchMethodException | InstantiationException | IllegalAccessException | InvocationTargetException e) { + throw new CatalogException("Failed to create Hive statistics for date column", e); } } @Override - public Class getHiveMetaStoreUtilsClass() { - return getMetaStoreUtilsClass(); - } - - @Override - public Class getDateDataTypeClass() { - return java.sql.Date.class; - } - - @Override - public Class getTimestampDataTypeClass() { - return java.sql.Timestamp.class; - } - - @Override - public FileStatus[] getFileStatusRecurse(Path path, int level, FileSystem fs) throws IOException { + public boolean isDateStats(ColumnStatisticsData colStatsData) { try { - Method method = HiveStatsUtils.class.getMethod("getFileStatusRecurse", Path.class, Integer.TYPE, FileSystem.class); - // getFileStatusRecurse is a static method - return (FileStatus[]) method.invoke(null, path, level, fs); - } catch (Exception ex) { - throw new CatalogException("Failed to invoke HiveStatsUtils.getFileStatusRecurse()", ex); + Method method = ColumnStatisticsData.class.getDeclaredMethod("isSetDateStats"); + return (boolean) method.invoke(colStatsData); + } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) { + throw new CatalogException("Failed to decide whether ColumnStatisticsData is for DATE column", e); } } @Override - public void makeSpecFromName(Map partSpec, Path currPath) { + public CatalogColumnStatisticsDataDate toFlinkDateColStats(ColumnStatisticsData hiveDateColStats) { try { - Method method = Warehouse.class.getMethod("makeSpecFromName", Map.class, Path.class); - // makeSpecFromName is a static method - method.invoke(null, partSpec, currPath); - } catch (Exception ex) { - throw new CatalogException("Failed to invoke Warehouse.makeSpecFromName()", ex); + Object dateStats = ColumnStatisticsData.class.getDeclaredMethod("getDateStats").invoke(hiveDateColStats); + Class dateStatsClz = dateStats.getClass(); + long numDV = (long) dateStatsClz.getMethod("getNumDVs").invoke(dateStats); + long numNull = (long) dateStatsClz.getMethod("getNumNulls").invoke(dateStats); + Object hmsHighDate = dateStatsClz.getMethod("getHighValue").invoke(dateStats); + Object hmsLowDate = dateStatsClz.getMethod("getLowValue").invoke(dateStats); + Class hmsDateClz = hmsHighDate.getClass(); + Method hmsDateDays = hmsDateClz.getMethod("getDaysSinceEpoch"); + long highDateDays = (long) hmsDateDays.invoke(hmsHighDate); + long lowDateDays = (long) hmsDateDays.invoke(hmsLowDate); + return new CatalogColumnStatisticsDataDate(new Date(lowDateDays), new Date(highDateDays), numDV, numNull); + } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) { + throw new CatalogException("Failed to create Flink statistics for date column", e); } } - } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveReflectionUtils.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveReflectionUtils.java index 1941096c9b72..b03eb66ff896 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveReflectionUtils.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveReflectionUtils.java @@ -28,8 +28,7 @@ import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.serde2.Deserializer; import org.apache.hadoop.hive.serde2.SerDeException; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantDateObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantTimestampObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; @@ -88,25 +87,14 @@ public static List getPvals(HiveShim hiveShim, List partCol } } - public static JavaConstantDateObjectInspector createJavaConstantDateObjectInspector(HiveShim hiveShim, Object value) { - Constructor meth = null; + public static ObjectInspector createConstantObjectInspector(String className, Object value) { try { - meth = JavaConstantDateObjectInspector.class.getDeclaredConstructor(hiveShim.getDateDataTypeClass()); - meth.setAccessible(true); - return (JavaConstantDateObjectInspector) meth.newInstance(value); - } catch (NoSuchMethodException | InstantiationException | IllegalAccessException | InvocationTargetException e) { - throw new FlinkHiveUDFException("Failed to instantiate JavaConstantDateObjectInspector"); - } - } - - public static JavaConstantTimestampObjectInspector createJavaConstantTimestampObjectInspector(HiveShim hiveShim, Object value) { - Constructor meth = null; - try { - meth = JavaConstantTimestampObjectInspector.class.getDeclaredConstructor(hiveShim.getDateDataTypeClass()); - meth.setAccessible(true); - return (JavaConstantTimestampObjectInspector) meth.newInstance(value); - } catch (NoSuchMethodException | InstantiationException | IllegalAccessException | InvocationTargetException e) { - throw new FlinkHiveUDFException("Failed to instantiate JavaConstantTimestampObjectInspector"); + Constructor method = Class.forName(className).getDeclaredConstructor(value.getClass()); + method.setAccessible(true); + return (ObjectInspector) method.newInstance(value); + } catch (ClassNotFoundException | NoSuchMethodException | InstantiationException | IllegalAccessException + | InvocationTargetException e) { + throw new FlinkHiveUDFException("Failed to instantiate JavaConstantDateObjectInspector", e); } } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveStatsUtil.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveStatsUtil.java index 232b5f6e2712..7f0551f25900 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveStatsUtil.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveStatsUtil.java @@ -19,6 +19,8 @@ package org.apache.flink.table.catalog.hive.util; import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.hive.client.HiveShim; +import org.apache.flink.table.catalog.hive.client.HiveShimLoader; import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataBase; import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataBinary; import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataBoolean; @@ -35,7 +37,6 @@ import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj; -import org.apache.hadoop.hive.metastore.api.DateColumnStatsData; import org.apache.hadoop.hive.metastore.api.DoubleColumnStatsData; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.LongColumnStatsData; @@ -69,13 +70,13 @@ private HiveStatsUtil() {} /** * Create a map of Flink column stats from the given Hive column stats. */ - public static Map createCatalogColumnStats(@Nonnull List hiveColStats) { + public static Map createCatalogColumnStats(@Nonnull List hiveColStats, String hiveVersion) { checkNotNull(hiveColStats, "hiveColStats can not be null"); Map colStats = new HashMap<>(); for (ColumnStatisticsObj colStatsObj : hiveColStats) { CatalogColumnStatisticsDataBase columnStats = createTableColumnStats( HiveTypeUtil.toFlinkType(TypeInfoUtils.getTypeInfoFromTypeString(colStatsObj.getColType())), - colStatsObj.getStatsData()); + colStatsObj.getStatsData(), hiveVersion); colStats.put(colStatsObj.getColName(), columnStats); } @@ -87,9 +88,10 @@ public static Map createCatalogColumnSt */ public static ColumnStatistics createTableColumnStats( Table hiveTable, - Map colStats) { + Map colStats, + String hiveVersion) { ColumnStatisticsDesc desc = new ColumnStatisticsDesc(true, hiveTable.getDbName(), hiveTable.getTableName()); - return createHiveColumnStatistics(colStats, hiveTable.getSd(), desc); + return createHiveColumnStatistics(colStats, hiveTable.getSd(), desc, hiveVersion); } /** @@ -98,16 +100,18 @@ public static ColumnStatistics createTableColumnStats( public static ColumnStatistics createPartitionColumnStats( Partition hivePartition, String partName, - Map colStats) { + Map colStats, + String hiveVersion) { ColumnStatisticsDesc desc = new ColumnStatisticsDesc(false, hivePartition.getDbName(), hivePartition.getTableName()); desc.setPartName(partName); - return createHiveColumnStatistics(colStats, hivePartition.getSd(), desc); + return createHiveColumnStatistics(colStats, hivePartition.getSd(), desc, hiveVersion); } private static ColumnStatistics createHiveColumnStatistics( Map colStats, StorageDescriptor sd, - ColumnStatisticsDesc desc) { + ColumnStatisticsDesc desc, + String hiveVersion) { List colStatsList = new ArrayList<>(); for (FieldSchema field : sd.getCols()) { @@ -115,8 +119,10 @@ private static ColumnStatistics createHiveColumnStatistics( String hiveColType = field.getType(); CatalogColumnStatisticsDataBase flinkColStat = colStats.get(field.getName()); if (null != flinkColStat) { - ColumnStatisticsData statsData = - getColumnStatisticsData(HiveTypeUtil.toFlinkType(TypeInfoUtils.getTypeInfoFromTypeString(hiveColType)), flinkColStat); + ColumnStatisticsData statsData = getColumnStatisticsData( + HiveTypeUtil.toFlinkType(TypeInfoUtils.getTypeInfoFromTypeString(hiveColType)), + flinkColStat, + hiveVersion); ColumnStatisticsObj columnStatisticsObj = new ColumnStatisticsObj(hiveColName, hiveColType, statsData); colStatsList.add(columnStatisticsObj); } @@ -128,7 +134,8 @@ private static ColumnStatistics createHiveColumnStatistics( /** * Create Flink ColumnStats from Hive ColumnStatisticsData. */ - private static CatalogColumnStatisticsDataBase createTableColumnStats(DataType colType, ColumnStatisticsData stats) { + private static CatalogColumnStatisticsDataBase createTableColumnStats(DataType colType, ColumnStatisticsData stats, String hiveVersion) { + HiveShim hiveShim = HiveShimLoader.loadHiveShim(hiveVersion); if (stats.isSetBinaryStats()) { BinaryColumnStatsData binaryStats = stats.getBinaryStats(); return new CatalogColumnStatisticsDataBinary( @@ -141,13 +148,8 @@ private static CatalogColumnStatisticsDataBase createTableColumnStats(DataType c booleanStats.getNumTrues(), booleanStats.getNumFalses(), booleanStats.getNumNulls()); - } else if (stats.isSetDateStats()) { - DateColumnStatsData dateStats = stats.getDateStats(); - return new CatalogColumnStatisticsDataDate( - new org.apache.flink.table.catalog.stats.Date(dateStats.getLowValue().getDaysSinceEpoch()), - new org.apache.flink.table.catalog.stats.Date(dateStats.getHighValue().getDaysSinceEpoch()), - dateStats.getNumDVs(), - dateStats.getNumNulls()); + } else if (hiveShim.isDateStats(stats)) { + return hiveShim.toFlinkDateColStats(stats); } else if (stats.isSetDoubleStats()) { DoubleColumnStatsData doubleStats = stats.getDoubleStats(); return new CatalogColumnStatisticsDataDouble( @@ -180,7 +182,8 @@ private static CatalogColumnStatisticsDataBase createTableColumnStats(DataType c * Note we currently assume that, in Flink, the max and min of ColumnStats will be same type as the Flink column type. * For example, for SHORT and Long columns, the max and min of their ColumnStats should be of type SHORT and LONG. */ - private static ColumnStatisticsData getColumnStatisticsData(DataType colType, CatalogColumnStatisticsDataBase colStat) { + private static ColumnStatisticsData getColumnStatisticsData(DataType colType, CatalogColumnStatisticsDataBase colStat, + String hiveVersion) { LogicalTypeRoot type = colType.getLogicalType().getTypeRoot(); if (type.equals(LogicalTypeRoot.CHAR) || type.equals(LogicalTypeRoot.VARCHAR)) { @@ -222,11 +225,8 @@ private static ColumnStatisticsData getColumnStatisticsData(DataType colType, Ca } } else if (type.equals(LogicalTypeRoot.DATE)) { if (colStat instanceof CatalogColumnStatisticsDataDate) { - CatalogColumnStatisticsDataDate dateColumnStatsData = (CatalogColumnStatisticsDataDate) colStat; - DateColumnStatsData dateStats = new DateColumnStatsData(dateColumnStatsData.getNullCount(), dateColumnStatsData.getNdv()); - dateStats.setHighValue(new org.apache.hadoop.hive.metastore.api.Date(dateColumnStatsData.getMax().getDaysSinceEpoch())); - dateStats.setLowValue(new org.apache.hadoop.hive.metastore.api.Date(dateColumnStatsData.getMin().getDaysSinceEpoch())); - return ColumnStatisticsData.dateStats(dateStats); + HiveShim hiveShim = HiveShimLoader.loadHiveShim(hiveVersion); + return hiveShim.toHiveDateColStats((CatalogColumnStatisticsDataDate) colStat); } } else if (type.equals(LogicalTypeRoot.VARBINARY) || type.equals(LogicalTypeRoot.BINARY)) { diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveTypeUtil.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveTypeUtil.java index 68c3ede22427..b5f7dacec64c 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveTypeUtil.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveTypeUtil.java @@ -323,4 +323,13 @@ protected TypeInfo defaultMethod(LogicalType logicalType) { String.format("Flink doesn't support converting type %s to Hive type yet.", dataType.toString())); } } + + /** + * INTERVAL are not available in older versions. So better to have our own enum for primitive categories. + */ + public enum HivePrimitiveCategory { + VOID, BOOLEAN, BYTE, SHORT, INT, LONG, FLOAT, DOUBLE, STRING, + DATE, TIMESTAMP, BINARY, DECIMAL, VARCHAR, CHAR, INTERVAL_YEAR_MONTH, INTERVAL_DAY_TIME, + UNKNOWN + } } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveGenericUDF.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveGenericUDF.java index e970ace597f8..5b1ebdaa02a4 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveGenericUDF.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveGenericUDF.java @@ -41,7 +41,7 @@ public class HiveGenericUDF extends HiveScalarFunction { private static final Logger LOG = LoggerFactory.getLogger(HiveGenericUDF.class); private transient GenericUDF.DeferredObject[] deferredObjects; - private transient HiveShim hiveShim; + private HiveShim hiveShim; public HiveGenericUDF(HiveFunctionWrapper hiveFunctionWrapper, HiveShim hiveShim) { super(hiveFunctionWrapper); @@ -56,9 +56,10 @@ public void openInternal() { function = hiveFunctionWrapper.createFunction(); + ObjectInspector[] argInspectors = HiveInspectors.toInspectors(hiveShim, constantArguments, argTypes); + try { - returnInspector = function.initializeAndFoldConstants( - HiveInspectors.toInspectors(hiveShim, constantArguments, argTypes)); + returnInspector = function.initializeAndFoldConstants(argInspectors); } catch (UDFArgumentException e) { throw new FlinkHiveUDFException(e); } @@ -67,8 +68,7 @@ public void openInternal() { for (int i = 0; i < deferredObjects.length; i++) { deferredObjects[i] = new DeferredObjectAdapter( - TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo( - HiveTypeUtil.toHiveTypeInfo(argTypes[i])), + argInspectors[i], argTypes[i].getLogicalType() ); } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveGenericUDTF.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveGenericUDTF.java index a7dbc8b9a57f..3578a83a7164 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveGenericUDTF.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveGenericUDTF.java @@ -64,7 +64,7 @@ public class HiveGenericUDTF extends TableFunction implements HiveFunction private transient boolean allIdentityConverter; private transient HiveObjectConversion[] conversions; - private transient HiveShim hiveShim; + private HiveShim hiveShim; public HiveGenericUDTF(HiveFunctionWrapper hiveFunctionWrapper, HiveShim hiveShim) { this.hiveFunctionWrapper = hiveFunctionWrapper; diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/conversion/HiveInspectors.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/conversion/HiveInspectors.java index e7e88b2fe0ed..377e1e981e68 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/conversion/HiveInspectors.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/conversion/HiveInspectors.java @@ -22,7 +22,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.table.catalog.exceptions.CatalogException; import org.apache.flink.table.catalog.hive.client.HiveShim; -import org.apache.flink.table.catalog.hive.util.HiveReflectionUtils; import org.apache.flink.table.catalog.hive.util.HiveTypeUtil; import org.apache.flink.table.functions.hive.FlinkHiveUDFException; import org.apache.flink.table.types.DataType; @@ -45,7 +44,6 @@ import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable; import org.apache.hadoop.hive.serde2.io.ShortWritable; import org.apache.hadoop.hive.serde2.io.TimestampWritable; -import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; @@ -64,18 +62,6 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveVarcharObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantBinaryObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantBooleanObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantByteObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantDoubleObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantFloatObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantHiveCharObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantHiveDecimalObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantHiveVarcharObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantIntObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantLongObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantShortObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaConstantStringObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; import org.apache.hadoop.hive.serde2.objectinspector.primitive.ShortObjectInspector; @@ -95,10 +81,12 @@ import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.Writable; import java.lang.reflect.Array; import java.math.BigDecimal; import java.sql.Date; +import java.sql.Timestamp; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -130,64 +118,20 @@ public static ObjectInspector[] toInspectors(HiveShim hiveShim, Object[] args, D TypeInfoUtils.getStandardJavaObjectInspectorFromTypeInfo( HiveTypeUtil.toHiveTypeInfo(argTypes[i])); } else { - argumentInspectors[i] = - HiveInspectors.getPrimitiveJavaConstantObjectInspector( - hiveShim, - (PrimitiveTypeInfo) HiveTypeUtil.toHiveTypeInfo(argTypes[i]), - constant - ); + PrimitiveTypeInfo primitiveTypeInfo = (PrimitiveTypeInfo) HiveTypeUtil.toHiveTypeInfo(argTypes[i]); + argumentInspectors[i] = hiveShim.getObjectInspectorForConstant(primitiveTypeInfo, constant); } } return argumentInspectors; } - private static ConstantObjectInspector getPrimitiveJavaConstantObjectInspector(HiveShim hiveShim, - PrimitiveTypeInfo typeInfo, Object value) { - switch (typeInfo.getPrimitiveCategory()) { - case BOOLEAN: - return new JavaConstantBooleanObjectInspector((Boolean) value); - case BYTE: - return new JavaConstantByteObjectInspector((Byte) value); - case SHORT: - return new JavaConstantShortObjectInspector((Short) value); - case INT: - return new JavaConstantIntObjectInspector((Integer) value); - case LONG: - return new JavaConstantLongObjectInspector((Long) value); - case FLOAT: - return new JavaConstantFloatObjectInspector((Float) value); - case DOUBLE: - return new JavaConstantDoubleObjectInspector((Double) value); - case STRING: - return new JavaConstantStringObjectInspector((String) value); - case CHAR: - return new JavaConstantHiveCharObjectInspector((HiveChar) value); - case VARCHAR: - return new JavaConstantHiveVarcharObjectInspector((HiveVarchar) value); - case DATE: - return HiveReflectionUtils.createJavaConstantDateObjectInspector(hiveShim, value); - case TIMESTAMP: - return HiveReflectionUtils.createJavaConstantTimestampObjectInspector(hiveShim, value); - case DECIMAL: - return new JavaConstantHiveDecimalObjectInspector((HiveDecimal) value); - case BINARY: - return new JavaConstantBinaryObjectInspector((byte[]) value); - case UNKNOWN: - case VOID: - // If type is null, we use the Java Constant String to replace - return new JavaConstantStringObjectInspector((String) value); - default: - throw new FlinkHiveUDFException( - String.format("Cannot find ConstantObjectInspector for %s", typeInfo)); - } - } - /** * Get conversion for converting Flink object to Hive object from an ObjectInspector and the corresponding Flink DataType. */ public static HiveObjectConversion getConversion(ObjectInspector inspector, LogicalType dataType) { if (inspector instanceof PrimitiveObjectInspector) { + HiveObjectConversion conversion; if (inspector instanceof BooleanObjectInspector || inspector instanceof StringObjectInspector || inspector instanceof ByteObjectInspector || @@ -199,14 +143,20 @@ public static HiveObjectConversion getConversion(ObjectInspector inspector, Logi inspector instanceof DateObjectInspector || inspector instanceof TimestampObjectInspector || inspector instanceof BinaryObjectInspector) { - return IdentityConversion.INSTANCE; + conversion = IdentityConversion.INSTANCE; } else if (inspector instanceof HiveCharObjectInspector) { - return o -> new HiveChar((String) o, ((CharType) dataType).getLength()); + conversion = o -> new HiveChar((String) o, ((CharType) dataType).getLength()); } else if (inspector instanceof HiveVarcharObjectInspector) { - return o -> new HiveVarchar((String) o, ((VarCharType) dataType).getLength()); + conversion = o -> new HiveVarchar((String) o, ((VarCharType) dataType).getLength()); } else if (inspector instanceof HiveDecimalObjectInspector) { - return o -> o == null ? null : HiveDecimal.create((BigDecimal) o); + conversion = o -> o == null ? null : HiveDecimal.create((BigDecimal) o); + } else { + throw new FlinkHiveUDFException("Unsupported primitive object inspector " + inspector.getClass().getName()); + } + if (((PrimitiveObjectInspector) inspector).preferWritable()) { + conversion = new WritableHiveObjectConversion(conversion); } + return conversion; } if (inspector instanceof ListObjectInspector) { @@ -460,7 +410,7 @@ private static Class getClassFromObjectInspector(ObjectInspector inspector) { switch (inspector.getCategory()) { case PRIMITIVE: { PrimitiveObjectInspector primitiveOI = (PrimitiveObjectInspector) inspector; - switch (primitiveOI.getPrimitiveCategory()) { + switch (HiveTypeUtil.HivePrimitiveCategory.valueOf(primitiveOI.getPrimitiveCategory().name())) { case STRING: case CHAR: case VARCHAR: @@ -506,4 +456,47 @@ private static Class getClassFromObjectInspector(ObjectInspector inspector) { throw new IllegalArgumentException("Unsupported type " + inspector.getCategory().name()); } } + + /** + * Converts a Hive primitive java object to corresponding Writable object. + */ + public static Writable hivePrimitiveToWritable(Object value) { + Writable writable; + // in case value is already a Writable + if (value instanceof Writable) { + writable = (Writable) value; + } else if (value instanceof Boolean) { + writable = new BooleanWritable((Boolean) value); + } else if (value instanceof Byte) { + writable = new ByteWritable((Byte) value); + } else if (value instanceof Short) { + writable = new ShortWritable((Short) value); + } else if (value instanceof Integer) { + writable = new IntWritable((Integer) value); + } else if (value instanceof Long) { + writable = new LongWritable((Long) value); + } else if (value instanceof Float) { + writable = new FloatWritable((Float) value); + } else if (value instanceof Double) { + writable = new DoubleWritable((Double) value); + } else if (value instanceof String) { + writable = new Text((String) value); + } else if (value instanceof HiveChar) { + writable = new HiveCharWritable((HiveChar) value); + } else if (value instanceof HiveVarchar) { + writable = new HiveVarcharWritable((HiveVarchar) value); + } else if (value instanceof Date) { + writable = new DateWritable((Date) value); + } else if (value instanceof Timestamp) { + writable = new TimestampWritable((Timestamp) value); + } else if (value instanceof BigDecimal) { + HiveDecimal hiveDecimal = HiveDecimal.create((BigDecimal) value); + writable = new HiveDecimalWritable(hiveDecimal); + } else if (value instanceof byte[]) { + writable = new BytesWritable((byte[]) value); + } else { + throw new CatalogException("Unsupported primitive java value of class " + value.getClass().getName()); + } + return writable; + } } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/conversion/WritableHiveObjectConversion.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/conversion/WritableHiveObjectConversion.java new file mode 100644 index 000000000000..c659569965df --- /dev/null +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/conversion/WritableHiveObjectConversion.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.hive.conversion; + +/** + * A HiveObjectConversion that converts Flink objects to Hive Writable objects. + */ +public class WritableHiveObjectConversion implements HiveObjectConversion { + + private final HiveObjectConversion flinkToJavaConversion; + + WritableHiveObjectConversion(HiveObjectConversion flinkToJavaConversion) { + this.flinkToJavaConversion = flinkToJavaConversion; + } + + @Override + public Object toHiveObject(Object o) { + return HiveInspectors.hivePrimitiveToWritable(flinkToJavaConversion.toHiveObject(o)); + } +} diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/FlinkStandaloneHiveServerContext.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/FlinkStandaloneHiveServerContext.java index 236f253445f6..6c90da86bf44 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/FlinkStandaloneHiveServerContext.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/FlinkStandaloneHiveServerContext.java @@ -107,6 +107,9 @@ private void configureMiscHiveSettings(HiveConf hiveConf) { hiveConf.setBoolVar(HIVE_SERVER2_LOGGING_OPERATION_ENABLED, false); hiveConf.setVar(HADOOPBIN, "NO_BIN!"); + + // To avoid https://issues.apache.org/jira/browse/HIVE-13185 when loading data into tables + hiveConf.setBoolVar(HiveConf.ConfVars.HIVECHECKFILEFORMAT, false); } private void overrideHiveConf(HiveConf hiveConf) { diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerShimLoader.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerShimLoader.java index ba2ff57208f8..2a344d1b8805 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerShimLoader.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerShimLoader.java @@ -37,6 +37,8 @@ public static HiveRunnerShim load() { String hiveVersion = HiveShimLoader.getHiveVersion(); return hiveRunnerShims.computeIfAbsent(hiveVersion, v -> { switch (v) { + case HiveShimLoader.HIVE_VERSION_V1_1_0: + case HiveShimLoader.HIVE_VERSION_V1_1_1: case HiveShimLoader.HIVE_VERSION_V1_2_0: case HiveShimLoader.HIVE_VERSION_V1_2_1: case HiveShimLoader.HIVE_VERSION_V1_2_2: 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 878cd686f79d..8a8aa0f7af6f 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 @@ -84,12 +84,11 @@ public void testReadNonPartitionedTable() throws Exception { final String dbName = "source_db"; final String tblName = "test"; hiveShell.execute("CREATE TABLE source_db.test ( a INT, b INT, c STRING, d BIGINT, e DOUBLE)"); - hiveShell.insertInto(dbName, tblName) - .withAllColumns() - .addRow(1, 1, "a", 1000L, 1.11) - .addRow(2, 2, "b", 2000L, 2.22) - .addRow(3, 3, "c", 3000L, 3.33) - .addRow(4, 4, "d", 4000L, 4.44) + HiveTestUtils.createTextTableInserter(hiveShell, dbName, tblName) + .addRow(new Object[]{1, 1, "a", 1000L, 1.11}) + .addRow(new Object[]{2, 2, "b", 2000L, 2.22}) + .addRow(new Object[]{3, 3, "c", 3000L, 3.33}) + .addRow(new Object[]{4, 4, "d", 4000L, 4.44}) .commit(); TableEnvironment tEnv = HiveTestUtils.createTableEnv(); @@ -116,9 +115,8 @@ public void testReadComplexDataType() throws Exception { map.put(1, "a"); map.put(2, "b"); Object[] struct = new Object[]{3, 3L}; - hiveShell.insertInto(dbName, tblName) - .withAllColumns() - .addRow(array, map, struct) + HiveTestUtils.createTextTableInserter(hiveShell, dbName, tblName) + .addRow(new Object[]{array, map, struct}) .commit(); TableEnvironment tEnv = HiveTestUtils.createTableEnv(); tEnv.registerCatalog(catalogName, hiveCatalog); @@ -141,13 +139,14 @@ public void testReadPartitionTable() throws Exception { final String tblName = "test_table_pt"; hiveShell.execute("CREATE TABLE source_db.test_table_pt " + "(year STRING, value INT) partitioned by (pt int);"); - hiveShell.insertInto(dbName, tblName) - .withColumns("year", "value", "pt") - .addRow("2014", 3, 0) - .addRow("2014", 4, 0) - .addRow("2015", 2, 1) - .addRow("2015", 5, 1) - .commit(); + HiveTestUtils.createTextTableInserter(hiveShell, dbName, tblName) + .addRow(new Object[]{"2014", 3}) + .addRow(new Object[]{"2014", 4}) + .commit("pt=0"); + HiveTestUtils.createTextTableInserter(hiveShell, dbName, tblName) + .addRow(new Object[]{"2015", 2}) + .addRow(new Object[]{"2015", 5}) + .commit("pt=1"); TableEnvironment tEnv = HiveTestUtils.createTableEnv(); tEnv.registerCatalog(catalogName, hiveCatalog); Table src = tEnv.sqlQuery("select * from hive.source_db.test_table_pt"); @@ -165,13 +164,14 @@ public void testPartitionPrunning() throws Exception { final String tblName = "test_table_pt_1"; hiveShell.execute("CREATE TABLE source_db.test_table_pt_1 " + "(year STRING, value INT) partitioned by (pt int);"); - hiveShell.insertInto(dbName, tblName) - .withColumns("year", "value", "pt") - .addRow("2014", 3, 0) - .addRow("2014", 4, 0) - .addRow("2015", 2, 1) - .addRow("2015", 5, 1) - .commit(); + HiveTestUtils.createTextTableInserter(hiveShell, dbName, tblName) + .addRow(new Object[]{"2014", 3}) + .addRow(new Object[]{"2014", 4}) + .commit("pt=0"); + HiveTestUtils.createTextTableInserter(hiveShell, dbName, tblName) + .addRow(new Object[]{"2015", 2}) + .addRow(new Object[]{"2015", 5}) + .commit("pt=1"); TableEnvironment tEnv = HiveTestUtils.createTableEnv(); tEnv.registerCatalog(catalogName, hiveCatalog); Table src = tEnv.sqlQuery("select * from hive.source_db.test_table_pt_1 where pt = 0"); @@ -196,11 +196,13 @@ 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(); + HiveTestUtils.createTextTableInserter(hiveShell, "default", "src") + .addRow(new Object[]{1, "a"}) + .addRow(new Object[]{2, "b"}) + .commit("p1=2013, p2='2013'"); + HiveTestUtils.createTextTableInserter(hiveShell, "default", "src") + .addRow(new Object[]{3, "c"}) + .commit("p1=2014, p2='2014'"); TableEnvironment tableEnv = HiveTestUtils.createTableEnv(); tableEnv.registerCatalog(catalogName, hiveCatalog); Table table = tableEnv.sqlQuery("select p1, count(y) from hive.`default`.src group by p1"); diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java index 9484e550b9d1..2c50bef67884 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java @@ -77,7 +77,7 @@ public void testDefaultPartitionName() throws Exception { hiveShell.execute("create database db1"); hiveShell.execute("create table db1.src (x int, y int)"); hiveShell.execute("create table db1.part (x int) partitioned by (y int)"); - hiveShell.insertInto("db1", "src").addRow(1, 1).addRow(2, null).commit(); + HiveTestUtils.createTextTableInserter(hiveShell, "db1", "src").addRow(new Object[]{1, 1}).addRow(new Object[]{2, null}).commit(); TableEnvironment tableEnv = getTableEnvWithHiveCatalog(); @@ -186,7 +186,7 @@ public void testInsertOverwrite() throws Exception { try { // non-partitioned hiveShell.execute("create table db1.dest (x int, y string)"); - hiveShell.insertInto("db1", "dest").addRow(1, "a").addRow(2, "b").commit(); + HiveTestUtils.createTextTableInserter(hiveShell, "db1", "dest").addRow(new Object[]{1, "a"}).addRow(new Object[]{2, "b"}).commit(); verifyHiveQueryResult("select * from db1.dest", Arrays.asList("1\ta", "2\tb")); TableEnvironment tableEnv = getTableEnvWithHiveCatalog(); tableEnv.sqlUpdate("insert overwrite db1.dest values (3,'c')"); @@ -195,7 +195,8 @@ public void testInsertOverwrite() throws Exception { // static partition hiveShell.execute("create table db1.part(x int) partitioned by (y int)"); - hiveShell.insertInto("db1", "part").addRow(1, 1).addRow(2, 2).commit(); + HiveTestUtils.createTextTableInserter(hiveShell, "db1", "part").addRow(new Object[]{1}).commit("y=1"); + HiveTestUtils.createTextTableInserter(hiveShell, "db1", "part").addRow(new Object[]{2}).commit("y=2"); tableEnv = getTableEnvWithHiveCatalog(); tableEnv.sqlUpdate("insert overwrite db1.part partition (y=1) select 100"); tableEnv.execute("insert overwrite static partition"); @@ -217,7 +218,7 @@ public void testStaticPartition() throws Exception { hiveShell.execute("create database db1"); try { hiveShell.execute("create table db1.src (x int)"); - hiveShell.insertInto("db1", "src").addRow(1).addRow(2).commit(); + HiveTestUtils.createTextTableInserter(hiveShell, "db1", "src").addRow(new Object[]{1}).addRow(new Object[]{2}).commit(); hiveShell.execute("create table db1.dest (x int) partitioned by (p1 string, p2 double)"); TableEnvironment tableEnv = getTableEnvWithHiveCatalog(); tableEnv.sqlUpdate("insert into db1.dest partition (p1='1''1', p2=1.1) select x from db1.src"); @@ -234,7 +235,11 @@ public void testDynamicPartition() throws Exception { hiveShell.execute("create database db1"); try { hiveShell.execute("create table db1.src (x int, y string, z double)"); - hiveShell.insertInto("db1", "src").addRow(1, "a", 1.1).addRow(2, "a", 2.2).addRow(3, "b", 3.3).commit(); + HiveTestUtils.createTextTableInserter(hiveShell, "db1", "src") + .addRow(new Object[]{1, "a", 1.1}) + .addRow(new Object[]{2, "a", 2.2}) + .addRow(new Object[]{3, "b", 3.3}) + .commit(); hiveShell.execute("create table db1.dest (x int) partitioned by (p1 string, p2 double)"); TableEnvironment tableEnv = getTableEnvWithHiveCatalog(); tableEnv.sqlUpdate("insert into db1.dest select * from db1.src"); @@ -251,7 +256,7 @@ public void testPartialDynamicPartition() throws Exception { hiveShell.execute("create database db1"); try { hiveShell.execute("create table db1.src (x int, y string)"); - hiveShell.insertInto("db1", "src").addRow(1, "a").addRow(2, "b").commit(); + HiveTestUtils.createTextTableInserter(hiveShell, "db1", "src").addRow(new Object[]{1, "a"}).addRow(new Object[]{2, "b"}).commit(); hiveShell.execute("create table db1.dest (x int) partitioned by (p1 double, p2 string)"); TableEnvironment tableEnv = getTableEnvWithHiveCatalog(); tableEnv.sqlUpdate("insert into db1.dest partition (p1=1.1) select x,y from db1.src"); diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogHiveMetadataTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogHiveMetadataTest.java index 0fe10d7d3ae8..fb5c04a04ae5 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogHiveMetadataTest.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogHiveMetadataTest.java @@ -23,6 +23,7 @@ import org.apache.flink.table.catalog.CatalogPartitionSpec; import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.CatalogTableImpl; +import org.apache.flink.table.catalog.hive.client.HiveShimLoader; import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataBase; import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataBinary; @@ -79,26 +80,32 @@ public void testCreateTable_StorageFormatSet() throws Exception { // ------ table and column stats ------ @Test public void testAlterTableColumnStatistics() throws Exception { + String hiveVersion = ((HiveCatalog) catalog).getHiveVersion(); + boolean supportDateStats = hiveVersion.compareTo(HiveShimLoader.HIVE_VERSION_V1_2_0) >= 0; catalog.createDatabase(db1, createDb(), false); - TableSchema tableSchema = TableSchema.builder() - .field("first", DataTypes.STRING()) - .field("second", DataTypes.INT()) - .field("third", DataTypes.BOOLEAN()) - .field("fourth", DataTypes.DATE()) - .field("fifth", DataTypes.DOUBLE()) - .field("sixth", DataTypes.BIGINT()) - .field("seventh", DataTypes.BYTES()) - .build(); + TableSchema.Builder builder = TableSchema.builder() + .field("first", DataTypes.STRING()) + .field("second", DataTypes.INT()) + .field("third", DataTypes.BOOLEAN()) + .field("fourth", DataTypes.DOUBLE()) + .field("fifth", DataTypes.BIGINT()) + .field("sixth", DataTypes.BYTES()); + if (supportDateStats) { + builder.field("seventh", DataTypes.DATE()); + } + TableSchema tableSchema = builder.build(); CatalogTable catalogTable = new CatalogTableImpl(tableSchema, getBatchTableProperties(), TEST_COMMENT); catalog.createTable(path1, catalogTable, false); Map columnStatisticsDataBaseMap = new HashMap<>(); columnStatisticsDataBaseMap.put("first", new CatalogColumnStatisticsDataString(10, 5.2, 3, 100)); columnStatisticsDataBaseMap.put("second", new CatalogColumnStatisticsDataLong(0, 1000, 3, 0)); columnStatisticsDataBaseMap.put("third", new CatalogColumnStatisticsDataBoolean(15, 20, 3)); - columnStatisticsDataBaseMap.put("fourth", new CatalogColumnStatisticsDataDate(new Date(71L), new Date(17923L), 1321, 0L)); - columnStatisticsDataBaseMap.put("fifth", new CatalogColumnStatisticsDataDouble(15.02, 20.01, 3, 10)); - columnStatisticsDataBaseMap.put("sixth", new CatalogColumnStatisticsDataLong(0, 20, 3, 2)); - columnStatisticsDataBaseMap.put("seventh", new CatalogColumnStatisticsDataBinary(150, 20, 3)); + columnStatisticsDataBaseMap.put("fourth", new CatalogColumnStatisticsDataDouble(15.02, 20.01, 3, 10)); + columnStatisticsDataBaseMap.put("fifth", new CatalogColumnStatisticsDataLong(0, 20, 3, 2)); + columnStatisticsDataBaseMap.put("sixth", new CatalogColumnStatisticsDataBinary(150, 20, 3)); + if (supportDateStats) { + columnStatisticsDataBaseMap.put("seventh", new CatalogColumnStatisticsDataDate(new Date(71L), new Date(17923L), 1321, 0L)); + } CatalogColumnStatistics catalogColumnStatistics = new CatalogColumnStatistics(columnStatisticsDataBaseMap); catalog.alterTableColumnStatistics(path1, catalogColumnStatistics, false); diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogTestBase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogTestBase.java index 232c250595ac..88895a3c9ea0 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogTestBase.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogTestBase.java @@ -21,6 +21,7 @@ import org.apache.flink.table.catalog.CatalogTestBase; import org.apache.flink.table.catalog.ObjectPath; +import org.junit.Assume; import org.junit.Test; /** @@ -28,6 +29,16 @@ */ public abstract class HiveCatalogTestBase extends CatalogTestBase { + // ------ table and column stats ------ + + @Override + @Test + public void testAlterTableStats() throws Exception { + String hiveVersion = ((HiveCatalog) catalog).getHiveVersion(); + Assume.assumeTrue(hiveVersion.compareTo("1.2.1") >= 0); + super.testAlterTableStats(); + } + // ------ functions ------ @Test diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java index 2e06d519288c..ec142cb9bfe4 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java @@ -37,15 +37,20 @@ import org.apache.flink.util.AbstractID; import org.apache.flink.util.StringUtils; +import com.klarna.hiverunner.HiveShell; import org.apache.hadoop.hive.conf.HiveConf; import org.junit.rules.TemporaryFolder; +import java.io.BufferedWriter; +import java.io.File; +import java.io.FileWriter; import java.io.IOException; import java.net.BindException; import java.net.InetSocketAddress; import java.net.ServerSocket; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; @@ -63,6 +68,8 @@ public class HiveTestUtils { private static final int MIN_EPH_PORT = 49152; private static final int MAX_EPH_PORT = 61000; + private static final byte[] SEPARATORS = new byte[]{(byte) 1, (byte) 2, (byte) 3}; + /** * Create a HiveCatalog with an embedded Hive Metastore. */ @@ -138,4 +145,106 @@ public static List collectTable(TableEnvironment tableEnv, Table table) thr ArrayList data = result.getAccumulatorResult(id); return SerializedListAccumulator.deserializeList(data, serializer); } + + // Insert into a single partition of a text table. + public static TextTableInserter createTextTableInserter(HiveShell hiveShell, String dbName, String tableName) { + return new TextTableInserter(hiveShell, dbName, tableName); + } + + /** + * insert table operation. + */ + public static class TextTableInserter { + + private final HiveShell hiveShell; + private final String dbName; + private final String tableName; + private final List rows; + + public TextTableInserter(HiveShell hiveShell, String dbName, String tableName) { + this.hiveShell = hiveShell; + this.dbName = dbName; + this.tableName = tableName; + rows = new ArrayList<>(); + } + + public TextTableInserter addRow(Object[] row) { + rows.add(row); + return this; + } + + public void commit() { + commit(null); + } + + public void commit(String partitionSpec) { + try { + File file = File.createTempFile("table_data_", null); + try (BufferedWriter writer = new BufferedWriter(new FileWriter(file))) { + for (int i = 0; i < rows.size(); i++) { + if (i > 0) { + writer.newLine(); + } + writer.write(toText(rows.get(i))); + } + } + String load = String.format("load data local inpath '%s' into table %s.%s", file.getAbsolutePath(), dbName, tableName); + if (partitionSpec != null) { + load += String.format(" partition (%s)", partitionSpec); + } + hiveShell.execute(load); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private String toText(Object[] row) { + StringBuilder builder = new StringBuilder(); + for (Object col : row) { + if (builder.length() > 0) { + builder.appendCodePoint(SEPARATORS[0]); + } + String colStr = toText(col); + if (colStr != null) { + builder.append(toText(col)); + } + } + return builder.toString(); + } + + private String toText(Object obj) { + if (obj == null) { + return null; + } + StringBuilder builder = new StringBuilder(); + if (obj instanceof Map) { + for (Object key : ((Map) obj).keySet()) { + if (builder.length() > 0) { + builder.appendCodePoint(SEPARATORS[1]); + } + builder.append(toText(key)); + builder.appendCodePoint(SEPARATORS[2]); + builder.append(toText(((Map) obj).get(key))); + } + } else if (obj instanceof Object[]) { + Object[] array = (Object[]) obj; + for (Object element : array) { + if (builder.length() > 0) { + builder.appendCodePoint(SEPARATORS[1]); + } + builder.append(toText(element)); + } + } else if (obj instanceof List) { + for (Object element : (List) obj) { + if (builder.length() > 0) { + builder.appendCodePoint(SEPARATORS[1]); + } + builder.append(toText(element)); + } + } else { + builder.append(obj); + } + return builder.toString(); + } + } } diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/functions/hive/HiveGenericUDFTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/functions/hive/HiveGenericUDFTest.java index 938f794dfa84..f7c74dcd58b3 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/functions/hive/HiveGenericUDFTest.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/functions/hive/HiveGenericUDFTest.java @@ -34,11 +34,11 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFCeil; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFCoalesce; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFDateDiff; -import org.apache.hadoop.hive.ql.udf.generic.GenericUDFDateFormat; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFDecode; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFMapKeys; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFStringToMap; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFStruct; +import org.junit.Assume; import org.junit.Test; import java.lang.reflect.InvocationTargetException; @@ -52,6 +52,7 @@ */ public class HiveGenericUDFTest { private static HiveShim hiveShim = HiveShimLoader.loadHiveShim(HiveShimLoader.getHiveVersion()); + private static final boolean HIVE_120_OR_LATER = HiveShimLoader.getHiveVersion().compareTo(HiveShimLoader.HIVE_VERSION_V1_2_0) >= 0; @Test public void testAbs() { @@ -111,12 +112,13 @@ public void testAddMonths() { } @Test - public void testDateFormat() throws NoSuchMethodException, IllegalAccessException, InvocationTargetException { + public void testDateFormat() throws Exception { + Assume.assumeTrue(HIVE_120_OR_LATER); String constYear = "y"; String constMonth = "M"; HiveGenericUDF udf = init( - GenericUDFDateFormat.class, + Class.forName("org.apache.hadoop.hive.ql.udf.generic.GenericUDFDateFormat"), new Object[] { null, constYear @@ -130,7 +132,7 @@ public void testDateFormat() throws NoSuchMethodException, IllegalAccessExceptio assertEquals("2009", udf.eval("2009-08-31", constYear)); udf = init( - GenericUDFDateFormat.class, + Class.forName("org.apache.hadoop.hive.ql.udf.generic.GenericUDFDateFormat"), new Object[] { null, constMonth diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/functions/hive/HiveGenericUDTFTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/functions/hive/HiveGenericUDTFTest.java index 12cb08031cff..755f56fe3dfa 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/functions/hive/HiveGenericUDTFTest.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/functions/hive/HiveGenericUDTFTest.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDTFPosExplode; import org.apache.hadoop.hive.ql.udf.generic.GenericUDTFStack; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; @@ -232,8 +233,15 @@ public void collect(Object o) { * Test over sum int udtf. */ public static class TestOverSumIntUDTF extends GenericUDTF { + + ObjectInspectorConverters.Converter[] converters; + @Override public StructObjectInspector initialize(ObjectInspector[] argOIs) throws UDFArgumentException { + converters = new ObjectInspectorConverters.Converter[argOIs.length]; + for (int i = 0; i < converters.length; i++) { + converters[i] = ObjectInspectorConverters.getConverter(argOIs[i], PrimitiveObjectInspectorFactory.javaIntObjectInspector); + } return ObjectInspectorFactory.getStandardStructObjectInspector( Collections.singletonList("col1"), Collections.singletonList(PrimitiveObjectInspectorFactory.javaIntObjectInspector)); @@ -242,8 +250,8 @@ public StructObjectInspector initialize(ObjectInspector[] argOIs) throws UDFArgu @Override public void process(Object[] args) throws HiveException { int total = 0; - for (Object arg : args) { - total += (int) arg; + for (int i = 0; i < args.length; i++) { + total += (int) converters[i].convert(args[i]); } for (Object ignored : args) { forward(total); From 31e89c7e12b6ba3cae2daa09627b1abb083fc470 Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Fri, 25 Oct 2019 14:36:49 -0700 Subject: [PATCH 348/746] [FLINK-14218][table] support precise function reference in FunctionCatalog Enable referencing functions with fully qualified name in FunctionCatalog. This closes #9962. --- .../pyflink/table/tests/test_aggregate.py | 4 +- flink-python/pyflink/table/tests/test_calc.py | 14 +- .../table/tests/test_column_operation.py | 8 +- .../pyflink/table/tests/test_correlate.py | 2 +- flink-python/pyflink/table/tests/test_join.py | 8 +- flink-python/pyflink/table/tests/test_sort.py | 2 +- .../flink/table/catalog/FunctionCatalog.java | 125 +++++++++++------ .../table/catalog/FunctionCatalogUtil.java | 31 ---- .../flink/table/catalog/FunctionLookup.java | 17 +-- .../table/catalog/GenericInMemoryCatalog.java | 39 ++++-- .../expressions/UnresolvedCallExpression.java | 34 ++--- .../resolver/ExpressionResolver.java | 8 +- .../resolver/LookupCallResolver.java | 5 +- .../rules/QualifyBuiltInFunctionsRule.java | 4 +- .../rules/ResolveCallByArgumentsRule.java | 6 +- .../table/catalog/FunctionCatalogTest.java | 71 +++++++++- .../apache/flink/table/catalog/Catalog.java | 7 +- .../table/expressions/CallExpression.java | 23 +-- .../table/functions/FunctionIdentifier.java | 132 ++++++++++++++++++ .../catalog/FunctionCatalogOperatorTable.java | 4 +- .../planner/plan/QueryOperationConverter.java | 2 +- .../planner/plan/utils/RexNodeExtractor.scala | 6 +- .../catalog/FunctionCatalogOperatorTable.java | 4 +- .../table/plan/util/RexProgramExtractor.scala | 3 +- .../table/api/stream/sql/AggregateTest.scala | 6 +- 25 files changed, 400 insertions(+), 165 deletions(-) delete mode 100644 flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalogUtil.java create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/FunctionIdentifier.java diff --git a/flink-python/pyflink/table/tests/test_aggregate.py b/flink-python/pyflink/table/tests/test_aggregate.py index 9c78fded5325..d7700c790c70 100644 --- a/flink-python/pyflink/table/tests/test_aggregate.py +++ b/flink-python/pyflink/table/tests/test_aggregate.py @@ -26,8 +26,8 @@ def test_group_by(self): result = t.group_by("c").select("a.sum, c as b") query_operation = result._j_table.getQueryOperation().getChildren().get(0) self.assertEqual("[c]", query_operation.getGroupingExpressions().toString()) - self.assertEqual("[`default_catalog`.`default_database`.`as`(" - "`default_catalog`.`default_database`.`sum`(a), 'EXPR$0')]", + self.assertEqual("[`as`(" + "`sum`(a), 'EXPR$0')]", query_operation.getAggregateExpressions().toString()) diff --git a/flink-python/pyflink/table/tests/test_calc.py b/flink-python/pyflink/table/tests/test_calc.py index 6699efd40302..2b788831e524 100644 --- a/flink-python/pyflink/table/tests/test_calc.py +++ b/flink-python/pyflink/table/tests/test_calc.py @@ -33,7 +33,7 @@ def test_select(self): t = self.t_env.from_elements([(1, 'hi', 'hello')], ['a', 'b', 'c']) result = t.select("a + 1, b, c") query_operation = result._j_table.getQueryOperation() - self.assertEqual('[`default_catalog`.`default_database`.`plus`(a, 1), b, c]', + self.assertEqual('[`plus`(a, 1), b, c]', query_operation.getProjectList().toString()) def test_alias(self): @@ -47,18 +47,18 @@ def test_where(self): t = t_env.from_elements([(1, 'Hi', 'Hello')], ['a', 'b', 'c']) result = t.where("a > 1 && b = 'Hello'") query_operation = result._j_table.getQueryOperation() - self.assertEqual("`default_catalog`.`default_database`.`and`(" - "`default_catalog`.`default_database`.`greaterThan`(a, 1), " - "`default_catalog`.`default_database`.`equals`(b, 'Hello'))", + self.assertEqual("`and`(" + "`greaterthan`(a, 1), " + "`equals`(b, 'Hello'))", query_operation.getCondition().toString()) def test_filter(self): t = self.t_env.from_elements([(1, 'Hi', 'Hello')], ['a', 'b', 'c']) result = t.filter("a > 1 && b = 'Hello'") query_operation = result._j_table.getQueryOperation() - self.assertEqual("`default_catalog`.`default_database`.`and`(" - "`default_catalog`.`default_database`.`greaterThan`(a, 1), " - "`default_catalog`.`default_database`.`equals`(b, 'Hello'))", + self.assertEqual("`and`(" + "`greaterthan`(a, 1), " + "`equals`(b, 'Hello'))", query_operation.getCondition().toString()) def test_from_element(self): diff --git a/flink-python/pyflink/table/tests/test_column_operation.py b/flink-python/pyflink/table/tests/test_column_operation.py index e9a49e6a8b10..d88cf8c7be6f 100644 --- a/flink-python/pyflink/table/tests/test_column_operation.py +++ b/flink-python/pyflink/table/tests/test_column_operation.py @@ -25,16 +25,16 @@ def test_add_columns(self): t = self.t_env.from_elements([(1, 'Hi', 'Hello')], ['a', 'b', 'c']) result = t.select("a").add_columns("a + 1 as b, a + 2 as c") query_operation = result._j_table.getQueryOperation() - self.assertEqual('[a, `default_catalog`.`default_database`.`plus`(a, 1), ' - '`default_catalog`.`default_database`.`plus`(a, 2)]', + self.assertEqual('[a, `plus`(a, 1), ' + '`plus`(a, 2)]', query_operation.getProjectList().toString()) def test_add_or_replace_columns(self): t = self.t_env.from_elements([(1, 'Hi', 'Hello')], ['a', 'b', 'c']) result = t.select("a").add_or_replace_columns("a + 1 as b, a + 2 as a") query_operation = result._j_table.getQueryOperation() - self.assertEqual('[`default_catalog`.`default_database`.`plus`(a, 2), ' - '`default_catalog`.`default_database`.`plus`(a, 1)]', + self.assertEqual('[`plus`(a, 2), ' + '`plus`(a, 1)]', query_operation.getProjectList().toString()) def test_rename_columns(self): diff --git a/flink-python/pyflink/table/tests/test_correlate.py b/flink-python/pyflink/table/tests/test_correlate.py index bfd70d535da2..3a9b57f32a52 100644 --- a/flink-python/pyflink/table/tests/test_correlate.py +++ b/flink-python/pyflink/table/tests/test_correlate.py @@ -42,7 +42,7 @@ def test_join_lateral_with_join_predicate(self): query_operation = result._j_table.getQueryOperation() self.assertEqual('INNER', query_operation.getJoinType().toString()) self.assertTrue(query_operation.isCorrelated()) - self.assertEqual('`default_catalog`.`default_database`.`equals`(id, word)', + self.assertEqual('`equals`(id, word)', query_operation.getCondition().toString()) def test_left_outer_join_lateral(self): diff --git a/flink-python/pyflink/table/tests/test_join.py b/flink-python/pyflink/table/tests/test_join.py index f36d379f4b54..d1fc2cc75896 100644 --- a/flink-python/pyflink/table/tests/test_join.py +++ b/flink-python/pyflink/table/tests/test_join.py @@ -29,7 +29,7 @@ def test_join_without_where(self): query_operation = result._j_table.getQueryOperation() self.assertEqual('INNER', query_operation.getJoinType().toString()) - self.assertEqual('`default_catalog`.`default_database`.`equals`(a, d)', + self.assertEqual('`equals`(a, d)', query_operation.getCondition().toString()) self.assertFalse(query_operation.isCorrelated()) @@ -52,7 +52,7 @@ def test_left_outer_join_without_where(self): query_operation = result._j_table.getQueryOperation() self.assertEqual('LEFT_OUTER', query_operation.getJoinType().toString()) - self.assertEqual('`default_catalog`.`default_database`.`equals`(a, d)', + self.assertEqual('`equals`(a, d)', query_operation.getCondition().toString()) self.assertFalse(query_operation.isCorrelated()) @@ -75,7 +75,7 @@ def test_right_outer_join(self): query_operation = result._j_table.getQueryOperation() self.assertEqual('RIGHT_OUTER', query_operation.getJoinType().toString()) - self.assertEqual('`default_catalog`.`default_database`.`equals`(a, d)', + self.assertEqual('`equals`(a, d)', query_operation.getCondition().toString()) self.assertFalse(query_operation.isCorrelated()) @@ -87,7 +87,7 @@ def test_full_outer_join(self): result = t1.full_outer_join(t2, "a = d") query_operation = result._j_table.getQueryOperation() self.assertEqual('FULL_OUTER', query_operation.getJoinType().toString()) - self.assertEqual('`default_catalog`.`default_database`.`equals`(a, d)', + self.assertEqual('`equals`(a, d)', query_operation.getCondition().toString()) self.assertFalse(query_operation.isCorrelated()) diff --git a/flink-python/pyflink/table/tests/test_sort.py b/flink-python/pyflink/table/tests/test_sort.py index d03b42a50db6..f41f43e558f2 100644 --- a/flink-python/pyflink/table/tests/test_sort.py +++ b/flink-python/pyflink/table/tests/test_sort.py @@ -28,7 +28,7 @@ def test_order_by_offset_fetch(self): query_operation = result._j_table.getQueryOperation() self.assertEqual(2, query_operation.getOffset()) self.assertEqual(2, query_operation.getFetch()) - self.assertEqual('[`default_catalog`.`default_database`.`desc`(a)]', + self.assertEqual('[`desc`(a)]', query_operation.getOrder().toString()) 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 09313bffed5b..6527e9fdd8c4 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 @@ -19,7 +19,6 @@ package org.apache.flink.table.catalog; import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.table.api.TableException; import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; @@ -29,6 +28,7 @@ import org.apache.flink.table.functions.AggregateFunctionDefinition; import org.apache.flink.table.functions.FunctionDefinition; import org.apache.flink.table.functions.FunctionDefinitionUtil; +import org.apache.flink.table.functions.FunctionIdentifier; import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.functions.ScalarFunctionDefinition; import org.apache.flink.table.functions.TableAggregateFunction; @@ -227,49 +227,100 @@ private Set getUserDefinedFunctionNames() { } @Override - public Optional lookupFunction(String name) { - String normalizedName = FunctionCatalogUtil.normalizeName(name); + public Optional lookupFunction(FunctionIdentifier identifier) { - FunctionDefinition userCandidate; - - Catalog catalog = catalogManager.getCatalog(catalogManager.getCurrentCatalog()).get(); + // precise function reference + if (identifier.getIdentifier().isPresent()) { + return resolvePreciseFunctionReference(identifier.getIdentifier().get()); + } else { + // ambiguous function reference + + String functionName = identifier.getSimpleName().get(); + + FunctionDefinition userCandidate; + + Catalog catalog = catalogManager.getCatalog(catalogManager.getCurrentCatalog()).get(); + try { + CatalogFunction catalogFunction = catalog.getFunction( + new ObjectPath(catalogManager.getCurrentDatabase(), functionName) + ); + + if (catalog.getFunctionDefinitionFactory().isPresent()) { + userCandidate = catalog.getFunctionDefinitionFactory().get().createFunctionDefinition(functionName, catalogFunction); + } else { + userCandidate = FunctionDefinitionUtil.createFunctionDefinition(functionName, catalogFunction); + } + + return Optional.of( + new FunctionLookup.Result( + FunctionIdentifier.of( + ObjectIdentifier.of( + catalogManager.getCurrentCatalog(), + catalogManager.getCurrentDatabase(), + functionName)), + userCandidate) + ); + + } catch (FunctionNotExistException e) { + // ignore + } - try { - CatalogFunction catalogFunction = catalog.getFunction( - new ObjectPath(catalogManager.getCurrentDatabase(), normalizedName)); + // If no corresponding function is found in catalog, check in-memory functions + userCandidate = tempSystemFunctions.get(functionName); - if (catalog.getFunctionDefinitionFactory().isPresent()) { - userCandidate = catalog.getFunctionDefinitionFactory().get().createFunctionDefinition(normalizedName, catalogFunction); + final Optional foundDefinition; + if (userCandidate != null) { + foundDefinition = Optional.of(userCandidate); } else { - userCandidate = FunctionDefinitionUtil.createFunctionDefinition(normalizedName, catalogFunction); + foundDefinition = moduleManager.getFunctionDefinition(functionName); } + return foundDefinition.map(d -> new FunctionLookup.Result( + FunctionIdentifier.of(identifier.getSimpleName().get()), + d) + ); + } + } + + private Optional resolvePreciseFunctionReference(ObjectIdentifier oi) { + // resolve order: + // 1. Temporary functions + // 2. Catalog functions + + FunctionDefinition potentialResult = tempCatalogFunctions.get(oi); + + if (potentialResult != null) { return Optional.of( new FunctionLookup.Result( - ObjectIdentifier.of(catalogManager.getCurrentCatalog(), catalogManager.getCurrentDatabase(), name), - userCandidate) + FunctionIdentifier.of(oi), + potentialResult + ) ); - } catch (FunctionNotExistException e) { - // Ignore } - // If no corresponding function is found in catalog, check in-memory functions - userCandidate = tempSystemFunctions.get(normalizedName); - - final Optional foundDefinition; - if (userCandidate != null) { - foundDefinition = Optional.of(userCandidate); - } else { - foundDefinition = moduleManager.getFunctionDefinition(normalizedName); + Catalog catalog = catalogManager.getCatalog(oi.getCatalogName()).get(); + + if (catalog != null) { + try { + CatalogFunction catalogFunction = catalog.getFunction( + new ObjectPath(oi.getDatabaseName(), oi.getObjectName())); + + FunctionDefinition fd; + if (catalog.getFunctionDefinitionFactory().isPresent()) { + fd = catalog.getFunctionDefinitionFactory().get() + .createFunctionDefinition(oi.getObjectName(), catalogFunction); + } else { + fd = FunctionDefinitionUtil.createFunctionDefinition(oi.getObjectName(), catalogFunction); + } + + return Optional.of( + new FunctionLookup.Result(FunctionIdentifier.of(oi), fd)); + } catch (FunctionNotExistException e) { + // Ignore + } } - return foundDefinition.map(definition -> new FunctionLookup.Result( - ObjectIdentifier.of( - catalogManager.getBuiltInCatalogName(), - catalogManager.getBuiltInDatabaseName(), - name), - definition) - ); + return Optional.empty(); } @Override @@ -281,18 +332,10 @@ public PlannerTypeInferenceUtil getPlannerTypeInferenceUtil() { } private void registerTempSystemFunction(String name, FunctionDefinition functionDefinition) { - tempSystemFunctions.put(FunctionCatalogUtil.normalizeName(name), functionDefinition); + tempSystemFunctions.put(FunctionIdentifier.normalizeName(name), functionDefinition); } private void registerTempCatalogFunction(ObjectIdentifier oi, FunctionDefinition functionDefinition) { - tempCatalogFunctions.put(normalizeObjectIdentifier(oi), functionDefinition); - } - - @VisibleForTesting - static ObjectIdentifier normalizeObjectIdentifier(ObjectIdentifier oi) { - return ObjectIdentifier.of( - oi.getCatalogName(), - oi.getDatabaseName(), - FunctionCatalogUtil.normalizeName(oi.getObjectName())); + tempCatalogFunctions.put(FunctionIdentifier.normalizeObjectIdentifier(oi), functionDefinition); } } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalogUtil.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalogUtil.java deleted file mode 100644 index a22c7d40bf91..000000000000 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalogUtil.java +++ /dev/null @@ -1,31 +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.catalog; - -/** - * Utils for FunctionCatalog. - */ -public class FunctionCatalogUtil { - /** - * Normalize a function name. - */ - public static String normalizeName(String name) { - return name.toLowerCase(); - } -} diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionLookup.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionLookup.java index 347f105fc422..686b0337a88a 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionLookup.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionLookup.java @@ -23,6 +23,7 @@ import org.apache.flink.table.delegation.PlannerTypeInferenceUtil; import org.apache.flink.table.functions.BuiltInFunctionDefinition; import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.functions.FunctionIdentifier; import java.util.Optional; @@ -33,15 +34,15 @@ public interface FunctionLookup { /** - * Lookup a function by name. The lookup is case insensitive. + * Lookup a function by function identifier. The lookup is case insensitive. */ - Optional lookupFunction(String name); + Optional lookupFunction(FunctionIdentifier identifier); /** * Helper method for looking up a built-in function. */ default Result lookupBuiltInFunction(BuiltInFunctionDefinition definition) { - return lookupFunction(definition.getName()) + return lookupFunction(FunctionIdentifier.of(definition.getName())) .orElseThrow(() -> new TableException( String.format( "Required built-in function [%s] could not be found in any catalog.", @@ -60,17 +61,17 @@ default Result lookupBuiltInFunction(BuiltInFunctionDefinition definition) { */ class Result { - private final ObjectIdentifier objectIdentifier; + private final FunctionIdentifier functionIdentifier; private final FunctionDefinition functionDefinition; - public Result(ObjectIdentifier objectIdentifier, FunctionDefinition functionDefinition) { - this.objectIdentifier = objectIdentifier; + public Result(FunctionIdentifier functionIdentifier, FunctionDefinition functionDefinition) { + this.functionIdentifier = functionIdentifier; this.functionDefinition = functionDefinition; } - public ObjectIdentifier getObjectIdentifier() { - return objectIdentifier; + public FunctionIdentifier getFunctionIdentifier() { + return functionIdentifier; } public FunctionDefinition getFunctionDefinition() { diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/GenericInMemoryCatalog.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/GenericInMemoryCatalog.java index c0924ca52623..82cd997c5c13 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/GenericInMemoryCatalog.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/GenericInMemoryCatalog.java @@ -32,6 +32,7 @@ import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; import org.apache.flink.table.catalog.stats.CatalogTableStatistics; +import org.apache.flink.table.functions.FunctionIdentifier; import org.apache.flink.util.StringUtils; import java.util.ArrayList; @@ -340,11 +341,13 @@ private void ensureTableExists(ObjectPath tablePath) throws TableNotExistExcepti // ------ functions ------ @Override - public void createFunction(ObjectPath functionPath, CatalogFunction function, boolean ignoreIfExists) + public void createFunction(ObjectPath path, CatalogFunction function, boolean ignoreIfExists) throws FunctionAlreadyExistException, DatabaseNotExistException { - checkNotNull(functionPath); + checkNotNull(path); checkNotNull(function); + ObjectPath functionPath = normalize(path); + if (!databaseExists(functionPath.getDatabaseName())) { throw new DatabaseNotExistException(getName(), functionPath.getDatabaseName()); } @@ -359,11 +362,13 @@ public void createFunction(ObjectPath functionPath, CatalogFunction function, bo } @Override - public void alterFunction(ObjectPath functionPath, CatalogFunction newFunction, boolean ignoreIfNotExists) + public void alterFunction(ObjectPath path, CatalogFunction newFunction, boolean ignoreIfNotExists) throws FunctionNotExistException { - checkNotNull(functionPath); + checkNotNull(path); checkNotNull(newFunction); + ObjectPath functionPath = normalize(path); + CatalogFunction existingFunction = functions.get(functionPath); if (existingFunction != null) { @@ -381,8 +386,10 @@ public void alterFunction(ObjectPath functionPath, CatalogFunction newFunction, } @Override - public void dropFunction(ObjectPath functionPath, boolean ignoreIfNotExists) throws FunctionNotExistException { - checkNotNull(functionPath); + public void dropFunction(ObjectPath path, boolean ignoreIfNotExists) throws FunctionNotExistException { + checkNotNull(path); + + ObjectPath functionPath = normalize(path); if (functionExists(functionPath)) { functions.remove(functionPath); @@ -400,13 +407,16 @@ public List listFunctions(String databaseName) throws DatabaseNotExistEx } return functions.keySet().stream() - .filter(k -> k.getDatabaseName().equals(databaseName)).map(k -> k.getObjectName()) + .filter(k -> k.getDatabaseName().equals(databaseName)) + .map(k -> k.getObjectName()) .collect(Collectors.toList()); } @Override - public CatalogFunction getFunction(ObjectPath functionPath) throws FunctionNotExistException { - checkNotNull(functionPath); + public CatalogFunction getFunction(ObjectPath path) throws FunctionNotExistException { + checkNotNull(path); + + ObjectPath functionPath = normalize(path); if (!functionExists(functionPath)) { throw new FunctionNotExistException(getName(), functionPath); @@ -416,11 +426,18 @@ public CatalogFunction getFunction(ObjectPath functionPath) throws FunctionNotEx } @Override - public boolean functionExists(ObjectPath functionPath) { - checkNotNull(functionPath); + public boolean functionExists(ObjectPath path) { + checkNotNull(path); + + ObjectPath functionPath = normalize(path); + return databaseExists(functionPath.getDatabaseName()) && functions.containsKey(functionPath); } + private ObjectPath normalize(ObjectPath path) { + return new ObjectPath(path.getDatabaseName(), FunctionIdentifier.normalizeName(path.getObjectName())); + } + // ------ partitions ------ @Override diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/UnresolvedCallExpression.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/UnresolvedCallExpression.java index afd23ba203af..b42c88d78fd5 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/UnresolvedCallExpression.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/UnresolvedCallExpression.java @@ -19,8 +19,8 @@ package org.apache.flink.table.expressions; import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.functions.FunctionIdentifier; import org.apache.flink.table.types.DataType; import org.apache.flink.util.Preconditions; @@ -41,24 +41,24 @@ *

    A unresolved call contains: *

      *
    • a {@link FunctionDefinition} that identifies the function to be called
    • - *
    • an optional {@link ObjectIdentifier} that tracks the origin of a function
    • + *
    • an optional {@link FunctionIdentifier} that tracks the origin of a function
    • *
    */ @PublicEvolving public final class UnresolvedCallExpression implements Expression { - private final @Nullable ObjectIdentifier objectIdentifier; + private final @Nullable FunctionIdentifier functionIdentifier; private final FunctionDefinition functionDefinition; private final List args; public UnresolvedCallExpression( - ObjectIdentifier objectIdentifier, + FunctionIdentifier functionIdentifier, FunctionDefinition functionDefinition, List args) { - this.objectIdentifier = - Preconditions.checkNotNull(objectIdentifier, "Object identifier must not be null."); + this.functionIdentifier = + Preconditions.checkNotNull(functionIdentifier, "Function identifier must not be null."); this.functionDefinition = Preconditions.checkNotNull(functionDefinition, "Function definition must not be null."); this.args = Collections.unmodifiableList( @@ -68,15 +68,15 @@ public UnresolvedCallExpression( public UnresolvedCallExpression( FunctionDefinition functionDefinition, List args) { - this.objectIdentifier = null; + this.functionIdentifier = null; this.functionDefinition = Preconditions.checkNotNull(functionDefinition, "Function definition must not be null."); this.args = Collections.unmodifiableList( new ArrayList<>(Preconditions.checkNotNull(args, "Arguments must not be null."))); } - public Optional getObjectIdentifier() { - return Optional.ofNullable(objectIdentifier); + public Optional getFunctionIdentifier() { + return Optional.ofNullable(functionIdentifier); } public FunctionDefinition getFunctionDefinition() { @@ -84,26 +84,26 @@ public FunctionDefinition getFunctionDefinition() { } public UnresolvedCallExpression replaceArgs(List args) { - if (objectIdentifier == null) { + if (functionIdentifier == null) { return new UnresolvedCallExpression( functionDefinition, args); } return new UnresolvedCallExpression( - objectIdentifier, + functionIdentifier, functionDefinition, args); } public CallExpression resolve(List args, DataType dataType) { - if (objectIdentifier == null) { + if (functionIdentifier == null) { return new CallExpression( functionDefinition, args, dataType); } return new CallExpression( - objectIdentifier, + functionIdentifier, functionDefinition, args, dataType); @@ -112,10 +112,10 @@ public CallExpression resolve(List args, DataType dataType) @Override public String asSummaryString() { final String functionName; - if (objectIdentifier == null) { + if (functionIdentifier == null) { functionName = functionDefinition.toString(); } else { - functionName = objectIdentifier.asSerializableString(); + functionName = functionIdentifier.asSerializableString(); } final String argList = args.stream() @@ -144,14 +144,14 @@ public boolean equals(Object o) { return false; } UnresolvedCallExpression that = (UnresolvedCallExpression) o; - return Objects.equals(objectIdentifier, that.objectIdentifier) && + return Objects.equals(functionIdentifier, that.functionIdentifier) && functionDefinition.equals(that.functionDefinition) && args.equals(that.args); } @Override public int hashCode() { - return Objects.hash(objectIdentifier, functionDefinition, args); + return Objects.hash(functionIdentifier, functionDefinition, args); } @Override diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/ExpressionResolver.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/ExpressionResolver.java index e4c07fa183f9..1763404de59e 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/ExpressionResolver.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/ExpressionResolver.java @@ -296,7 +296,7 @@ public CallExpression as(ResolvedExpression expression, String alias) { .lookupBuiltInFunction(BuiltInFunctionDefinitions.AS); return new CallExpression( - lookupOfAs.getObjectIdentifier(), + lookupOfAs.getFunctionIdentifier(), lookupOfAs.getFunctionDefinition(), Arrays.asList(expression, valueLiteral(alias)), expression.getOutputDataType()); @@ -307,7 +307,7 @@ public CallExpression cast(ResolvedExpression expression, DataType dataType) { .lookupBuiltInFunction(BuiltInFunctionDefinitions.CAST); return new CallExpression( - lookupOfCast.getObjectIdentifier(), + lookupOfCast.getFunctionIdentifier(), lookupOfCast.getFunctionDefinition(), Arrays.asList(expression, typeLiteral(dataType)), dataType); @@ -318,7 +318,7 @@ public CallExpression wrappingCall(BuiltInFunctionDefinition definition, Resolve .lookupBuiltInFunction(definition); return new CallExpression( - lookupOfDefinition.getObjectIdentifier(), + lookupOfDefinition.getFunctionIdentifier(), lookupOfDefinition.getFunctionDefinition(), Collections.singletonList(expression), expression.getOutputDataType()); // the output type is equal to the input type @@ -329,7 +329,7 @@ public CallExpression get(ResolvedExpression composite, ValueLiteralExpression k .lookupBuiltInFunction(BuiltInFunctionDefinitions.GET); return new CallExpression( - lookupOfGet.getObjectIdentifier(), + lookupOfGet.getFunctionIdentifier(), lookupOfGet.getFunctionDefinition(), Arrays.asList(composite, key), dataType); diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/LookupCallResolver.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/LookupCallResolver.java index 1ad34e324a31..ef4508fc92e3 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/LookupCallResolver.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/LookupCallResolver.java @@ -25,6 +25,7 @@ import org.apache.flink.table.expressions.LookupCallExpression; import org.apache.flink.table.expressions.UnresolvedCallExpression; import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor; +import org.apache.flink.table.functions.FunctionIdentifier; import java.util.List; import java.util.stream.Collectors; @@ -42,11 +43,11 @@ public LookupCallResolver(FunctionLookup functionLookup) { } public Expression visit(LookupCallExpression lookupCall) { - final FunctionLookup.Result result = functionLookup.lookupFunction(lookupCall.getUnresolvedName()) + final FunctionLookup.Result result = functionLookup.lookupFunction(FunctionIdentifier.of(lookupCall.getUnresolvedName())) .orElseThrow(() -> new ValidationException("Undefined function: " + lookupCall.getUnresolvedName())); return new UnresolvedCallExpression( - result.getObjectIdentifier(), + result.getFunctionIdentifier(), result.getFunctionDefinition(), resolveChildren(lookupCall.getChildren())); } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/QualifyBuiltInFunctionsRule.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/QualifyBuiltInFunctionsRule.java index 70b5e78e6ccf..cf3b9bad8b22 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/QualifyBuiltInFunctionsRule.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/QualifyBuiltInFunctionsRule.java @@ -49,14 +49,14 @@ private class QualifyBuiltInFunctionVisitor extends RuleExpressionVisitor c.accept(this)) diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/ResolveCallByArgumentsRule.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/ResolveCallByArgumentsRule.java index d8deba780ae7..29035047e310 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/ResolveCallByArgumentsRule.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/rules/ResolveCallByArgumentsRule.java @@ -23,7 +23,6 @@ import org.apache.flink.api.common.typeutils.CompositeType; import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.delegation.PlannerTypeInferenceUtil; import org.apache.flink.table.expressions.CallExpression; import org.apache.flink.table.expressions.Expression; @@ -33,6 +32,7 @@ import org.apache.flink.table.functions.BuiltInFunctionDefinition; import org.apache.flink.table.functions.BuiltInFunctionDefinitions; import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.functions.FunctionIdentifier; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.inference.CallContext; import org.apache.flink.table.types.inference.TypeInference; @@ -145,8 +145,8 @@ private ResolvedExpression runTypeInference( TypeInference inference, List resolvedArgs) { - final String name = unresolvedCall.getObjectIdentifier() - .map(ObjectIdentifier::toString) + final String name = unresolvedCall.getFunctionIdentifier() + .map(FunctionIdentifier::toString) .orElseGet(() -> unresolvedCall.getFunctionDefinition().toString()); final TypeInferenceUtil.Result inferenceResult = TypeInferenceUtil.runTypeInference( diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java index 852d691b43e8..2461cdc79625 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java @@ -18,27 +18,43 @@ package org.apache.flink.table.catalog; +import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.FunctionAlreadyExistException; +import org.apache.flink.table.functions.FunctionIdentifier; +import org.apache.flink.table.functions.ScalarFunction; +import org.apache.flink.table.functions.ScalarFunctionDefinition; import org.apache.flink.table.module.ModuleManager; +import org.junit.Before; import org.junit.Test; import java.util.Collections; import java.util.HashSet; import java.util.Set; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; /** * Test for {@link FunctionCatalog}. */ public class FunctionCatalogTest { + private FunctionCatalog functionCatalog; + private Catalog catalog; + + @Before + public void init() throws DatabaseAlreadyExistException { + catalog = new GenericInMemoryCatalog("test"); + catalog.createDatabase("test", new CatalogDatabaseImpl(Collections.EMPTY_MAP, null), false); + functionCatalog = new FunctionCatalog( + new CatalogManager("test", catalog), new ModuleManager()); + } @Test public void testGetBuiltInFunctions() { - FunctionCatalog functionCatalog = new FunctionCatalog( - new CatalogManager("test", new GenericInMemoryCatalog("test")), - new ModuleManager()); - Set actual = new HashSet<>(); Collections.addAll(actual, functionCatalog.getFunctions()); @@ -46,4 +62,51 @@ public void testGetBuiltInFunctions() { assertTrue(actual.containsAll(expected)); } + + @Test + public void testPreciseFunctionReference() throws FunctionAlreadyExistException, DatabaseNotExistException { + ObjectIdentifier oi = ObjectIdentifier.of("test", "test", "test_function"); + + // test no function is found + assertFalse(functionCatalog.lookupFunction(FunctionIdentifier.of(oi)).isPresent()); + + // test catalog function is found + catalog.createFunction( + oi.toObjectPath(), + new CatalogFunctionImpl(TestFunction1.class.getName(), Collections.emptyMap()), false); + + FunctionLookup.Result result = functionCatalog.lookupFunction(FunctionIdentifier.of(oi)).get(); + + assertFalse(result.getFunctionIdentifier().getSimpleName().isPresent()); + assertEquals(oi, result.getFunctionIdentifier().getIdentifier().get()); + assertNotNull(result.getFunctionDefinition()); + assertTrue(((ScalarFunctionDefinition) result.getFunctionDefinition()).getScalarFunction() instanceof TestFunction1); + + // test temp catalog function is found + functionCatalog.registerTempCatalogScalarFunction( + oi, + new TestFunction2() + ); + + result = functionCatalog.lookupFunction(FunctionIdentifier.of(oi)).get(); + + assertFalse(result.getFunctionIdentifier().getSimpleName().isPresent()); + assertEquals(oi, result.getFunctionIdentifier().getIdentifier().get()); + assertNotNull(result.getFunctionDefinition()); + assertTrue(((ScalarFunctionDefinition) result.getFunctionDefinition()).getScalarFunction() instanceof TestFunction2); + } + + /** + * Testing function. + */ + public static class TestFunction1 extends ScalarFunction { + + } + + /** + * Testing function. + */ + public static class TestFunction2 extends ScalarFunction { + + } } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/Catalog.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/Catalog.java index 8f210b9b0b99..a43d09105b21 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/Catalog.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/Catalog.java @@ -375,6 +375,7 @@ void alterPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec, Ca /** * Get the function. + * Function name should be handled in a case insensitive way. * * @param functionPath path of the function * @return the requested function @@ -385,6 +386,7 @@ void alterPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec, Ca /** * Check whether a function exists or not. + * Function name should be handled in a case insensitive way. * * @param functionPath path of the function * @return true if the function exists in the catalog @@ -395,6 +397,7 @@ void alterPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec, Ca /** * Create a function. + * Function name should be handled in a case insensitive way. * * @param functionPath path of the function * @param function the function to be created @@ -410,6 +413,7 @@ void createFunction(ObjectPath functionPath, CatalogFunction function, boolean i /** * Modify an existing function. + * Function name should be handled in a case insensitive way. * * @param functionPath path of the function * @param newFunction the function to be modified @@ -424,6 +428,7 @@ void alterFunction(ObjectPath functionPath, CatalogFunction newFunction, boolean /** * Drop a function. + * Function name should be handled in a case insensitive way. * * @param functionPath path of the function to be dropped * @param ignoreIfNotExists plag to specify behavior if the function does not exist: @@ -437,8 +442,6 @@ void dropFunction(ObjectPath functionPath, boolean ignoreIfNotExists) // ------ statistics ------ - // ------ statistics ------ - /** * Get the statistics of a table. * diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/CallExpression.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/CallExpression.java index 1b7082cc447f..aedaa2bba0b9 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/CallExpression.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/CallExpression.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.functions.FunctionIdentifier; import org.apache.flink.table.types.DataType; import org.apache.flink.util.Preconditions; @@ -46,7 +47,7 @@ @PublicEvolving public final class CallExpression implements ResolvedExpression { - private final @Nullable ObjectIdentifier objectIdentifier; + private final @Nullable FunctionIdentifier functionIdentifier; private final FunctionDefinition functionDefinition; @@ -55,12 +56,12 @@ public final class CallExpression implements ResolvedExpression { private final DataType dataType; public CallExpression( - ObjectIdentifier objectIdentifier, + FunctionIdentifier functionIdentifier, FunctionDefinition functionDefinition, List args, DataType dataType) { - this.objectIdentifier = - Preconditions.checkNotNull(objectIdentifier, "Object identifier must not be null."); + this.functionIdentifier = + Preconditions.checkNotNull(functionIdentifier, "Object identifier must not be null."); this.functionDefinition = Preconditions.checkNotNull(functionDefinition, "Function definition must not be null."); this.args = new ArrayList<>(Preconditions.checkNotNull(args, "Arguments must not be null.")); @@ -71,14 +72,14 @@ public CallExpression( FunctionDefinition functionDefinition, List args, DataType dataType) { - this.objectIdentifier = null; + this.functionIdentifier = null; this.functionDefinition = Preconditions.checkNotNull(functionDefinition, "Function definition must not be null."); this.args = new ArrayList<>(Preconditions.checkNotNull(args, "Arguments must not be null.")); this.dataType = Preconditions.checkNotNull(dataType, "Data type must not be null."); } - public Optional getObjectIdentifier() { - return Optional.ofNullable(objectIdentifier); + public Optional getFunctionIdentifier() { + return Optional.ofNullable(functionIdentifier); } public FunctionDefinition getFunctionDefinition() { @@ -98,10 +99,10 @@ public List getResolvedChildren() { @Override public String asSummaryString() { final String functionName; - if (objectIdentifier == null) { + if (functionIdentifier == null) { functionName = functionDefinition.toString(); } else { - functionName = objectIdentifier.asSerializableString(); + functionName = functionIdentifier.asSerializableString(); } final String argList = args.stream() @@ -130,7 +131,7 @@ public boolean equals(Object o) { return false; } CallExpression that = (CallExpression) o; - return Objects.equals(objectIdentifier, that.objectIdentifier) && + return Objects.equals(functionIdentifier, that.functionIdentifier) && functionDefinition.equals(that.functionDefinition) && args.equals(that.args) && dataType.equals(that.dataType); @@ -138,7 +139,7 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(objectIdentifier, functionDefinition, args, dataType); + return Objects.hash(functionIdentifier, functionDefinition, args, dataType); } @Override diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/FunctionIdentifier.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/FunctionIdentifier.java new file mode 100644 index 000000000000..dba2454ef9b3 --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/FunctionIdentifier.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.functions; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.util.StringUtils; + +import java.io.Serializable; +import java.util.Objects; +import java.util.Optional; + +import static org.apache.flink.table.utils.EncodingUtils.escapeIdentifier; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Identifies a system function with function name or a catalog function with a fully qualified identifier. + * Function catalog is responsible for resolving an identifier to a function. + */ +@PublicEvolving +public final class FunctionIdentifier implements Serializable { + + private final ObjectIdentifier objectIdentifier; + + private final String functionName; + + public static FunctionIdentifier of(ObjectIdentifier oi){ + return new FunctionIdentifier(oi); + } + + public static FunctionIdentifier of(String functionName){ + return new FunctionIdentifier(functionName); + } + + private FunctionIdentifier(ObjectIdentifier objectIdentifier){ + checkNotNull(objectIdentifier, "Object identifier cannot be null"); + this.objectIdentifier = normalizeObjectIdentifier(objectIdentifier); + this.functionName = null; + } + + private FunctionIdentifier(String functionName){ + checkArgument(!StringUtils.isNullOrWhitespaceOnly(functionName), + "function name cannot be null or empty string"); + this.functionName = normalizeName(functionName); + this.objectIdentifier = null; + } + + /** + * Normalize a function name. + */ + public static String normalizeName(String name) { + return name.toLowerCase(); + } + + /** + * Normalize an object identifier by only normalizing the function name. + */ + public static ObjectIdentifier normalizeObjectIdentifier(ObjectIdentifier oi) { + return ObjectIdentifier.of( + oi.getCatalogName(), + oi.getDatabaseName(), + normalizeName(oi.getObjectName())); + } + + public Optional getIdentifier(){ + return Optional.ofNullable(objectIdentifier); + } + + public Optional getSimpleName(){ + return Optional.ofNullable(functionName); + } + + /** + * Returns a string that fully serializes this instance. The serialized string can be used for + * transmitting or persisting an object identifier. + */ + public String asSerializableString() { + if (objectIdentifier != null) { + return String.format( + "%s.%s.%s", + escapeIdentifier(objectIdentifier.getCatalogName()), + escapeIdentifier(objectIdentifier.getDatabaseName()), + escapeIdentifier(objectIdentifier.getObjectName())); + } else { + return String.format("%s", escapeIdentifier(functionName)); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + FunctionIdentifier that = (FunctionIdentifier) o; + + if (getIdentifier() != null && getIdentifier().equals(that.getIdentifier())) { + return true; + } else { + return functionName.equals(that.functionName); + } + } + + @Override + public int hashCode() { + return Objects.hash(objectIdentifier, functionName); + } + + @Override + public String toString() { + return asSerializableString(); + } +} diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/FunctionCatalogOperatorTable.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/FunctionCatalogOperatorTable.java index 3c875ddfef1b..b9e0a165ad59 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/FunctionCatalogOperatorTable.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/FunctionCatalogOperatorTable.java @@ -24,6 +24,7 @@ import org.apache.flink.table.catalog.FunctionLookup; import org.apache.flink.table.functions.AggregateFunctionDefinition; import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.functions.FunctionIdentifier; import org.apache.flink.table.functions.ScalarFunctionDefinition; import org.apache.flink.table.functions.TableFunctionDefinition; import org.apache.flink.table.planner.calcite.FlinkTypeFactory; @@ -84,7 +85,8 @@ public void lookupOperatorOverloads( } String name = opName.getSimple(); - Optional candidateFunction = functionCatalog.lookupFunction(name); + Optional candidateFunction = functionCatalog.lookupFunction( + FunctionIdentifier.of(name)); candidateFunction.flatMap(lookupResult -> convertToSqlFunction(category, name, lookupResult.getFunctionDefinition()) diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java index ac0d3b68957e..b49bd28b49b4 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java @@ -473,7 +473,7 @@ public RexNode visit(CallExpression callExpression) { }).collect(Collectors.toList()); CallExpression newCall = new CallExpression( - callExpression.getObjectIdentifier().get(), callExpression.getFunctionDefinition(), newChildren, + callExpression.getFunctionIdentifier().get(), callExpression.getFunctionDefinition(), newChildren, callExpression.getOutputDataType()); return convertExprToRexNode(newCall); } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala index b4535bf1b479..99fcda4c1ab8 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala @@ -31,15 +31,15 @@ import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromLog import org.apache.flink.table.types.DataType import org.apache.flink.table.types.logical.LogicalTypeRoot._ import org.apache.flink.util.Preconditions - import org.apache.calcite.plan.RelOptUtil import org.apache.calcite.rex._ import org.apache.calcite.sql.fun.{SqlStdOperatorTable, SqlTrimFunction} import org.apache.calcite.sql.{SqlFunction, SqlPostfixOperator} import org.apache.calcite.util.Util - import java.util.{TimeZone, List => JList} +import org.apache.flink.table.functions.FunctionIdentifier + import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ import scala.collection.mutable @@ -433,7 +433,7 @@ class RexNodeToExpressionConverter( name: String, operands: Seq[ResolvedExpression], outputType: DataType): Option[ResolvedExpression] = { - Try(functionCatalog.lookupFunction(name)) match { + Try(functionCatalog.lookupFunction(FunctionIdentifier.of(name))) match { case Success(f: java.util.Optional[FunctionLookup.Result]) => if (f.isPresent) { Some(new CallExpression(f.get().getFunctionDefinition, operands, outputType)) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/FunctionCatalogOperatorTable.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/FunctionCatalogOperatorTable.java index 1b6f5a80456b..b56abf00c2f0 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/FunctionCatalogOperatorTable.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/FunctionCatalogOperatorTable.java @@ -22,6 +22,7 @@ import org.apache.flink.table.calcite.FlinkTypeFactory; import org.apache.flink.table.functions.AggregateFunctionDefinition; import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.functions.FunctionIdentifier; import org.apache.flink.table.functions.ScalarFunctionDefinition; import org.apache.flink.table.functions.TableFunctionDefinition; import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils; @@ -71,7 +72,8 @@ public void lookupOperatorOverloads( } String name = opName.getSimple(); - Optional candidateFunction = functionCatalog.lookupFunction(name); + Optional candidateFunction = functionCatalog.lookupFunction( + FunctionIdentifier.of(name)); candidateFunction.flatMap(lookupResult -> convertToSqlFunction(category, name, lookupResult.getFunctionDefinition()) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala index 998a731e6f8a..7e754b015674 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala @@ -31,6 +31,7 @@ import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.catalog.FunctionCatalog import org.apache.flink.table.expressions.utils.ApiExpressionUtils.unresolvedCall import org.apache.flink.table.expressions._ +import org.apache.flink.table.functions.FunctionIdentifier import org.apache.flink.table.util.JavaScalaConversionUtil import org.apache.flink.util.Preconditions import org.slf4j.{Logger, LoggerFactory} @@ -279,7 +280,7 @@ class RexNodeToExpressionConverter( val expressionBridge = new ExpressionBridge[PlannerExpression]( functionCatalog, PlannerExpressionConverter.INSTANCE) - JavaScalaConversionUtil.toScala(functionCatalog.lookupFunction(name)) + JavaScalaConversionUtil.toScala(functionCatalog.lookupFunction(FunctionIdentifier.of(name))) .flatMap(result => Try(expressionBridge.bridge( unresolvedCall(result.getFunctionDefinition, operands: _*))).toOption diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/AggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/AggregateTest.scala index 1bd89f1cbcc7..ae016b09c058 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/AggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/AggregateTest.scala @@ -28,7 +28,7 @@ import org.apache.flink.table.api.scala.internal.StreamTableEnvironmentImpl import org.apache.flink.table.api.{TableConfig, Types} import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog, GenericInMemoryCatalog} import org.apache.flink.table.delegation.{Executor, Planner} -import org.apache.flink.table.functions.{AggregateFunction, AggregateFunctionDefinition} +import org.apache.flink.table.functions.{AggregateFunction, AggregateFunctionDefinition, FunctionIdentifier} import org.apache.flink.table.module.ModuleManager import org.apache.flink.table.utils.TableTestUtil.{streamTableNode, term, unaryNode} import org.apache.flink.table.utils.{StreamTableTestUtil, TableTestBase} @@ -85,7 +85,7 @@ class AggregateTest extends TableTestBase { tablEnv.registerFunction("udag", new MyAgg) val aggFunctionDefinition = functionCatalog - .lookupFunction("udag").get() + .lookupFunction(FunctionIdentifier.of("udag")).get() .getFunctionDefinition .asInstanceOf[AggregateFunctionDefinition] @@ -98,7 +98,7 @@ class AggregateTest extends TableTestBase { tablEnv.registerFunction("udag2", new MyAgg2) val aggFunctionDefinition2 = functionCatalog - .lookupFunction("udag2").get() + .lookupFunction(FunctionIdentifier.of("udag2")).get() .getFunctionDefinition .asInstanceOf[AggregateFunctionDefinition] From 4c87259c1b931318a917eba665a219b63788a735 Mon Sep 17 00:00:00 2001 From: Hao Dang Date: Tue, 29 Oct 2019 20:14:24 -0700 Subject: [PATCH 349/746] [FLINK-14398][table-planner] Further split input unboxing code into separate methods (#10000) --- .../flink/table/codegen/CodeGenerator.scala | 56 ++++++++++++++----- .../codegen/CollectorCodeGenerator.scala | 22 ++------ .../table/codegen/FunctionCodeGenerator.scala | 26 ++++++--- .../table/codegen/MatchCodeGenerator.scala | 33 +++++++---- .../table/runtime/stream/sql/SqlITCase.scala | 40 +++++++++++++ 5 files changed, 129 insertions(+), 48 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala index c6f7d762650b..17906e588439 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala @@ -1052,23 +1052,51 @@ abstract class CodeGenerator( // ---------------------------------------------------------------------------------------------- // generator helping methods // ---------------------------------------------------------------------------------------------- + protected def makeReusableInSplits(expr: GeneratedExpression): GeneratedExpression = { + // prepare declaration in class + val resultTypeTerm = primitiveTypeTermForTypeInfo(expr.resultType) + if (nullCheck && !expr.nullTerm.equals(NEVER_NULL) && !expr.nullTerm.equals(ALWAYS_NULL)) { + reusableMemberStatements.add(s"private boolean ${expr.nullTerm};") + } + reusableMemberStatements.add(s"private $resultTypeTerm ${expr.resultTerm};") - protected def makeReusableInSplits(exprs: Iterable[GeneratedExpression]): Unit = { - // add results of expressions to member area such that all split functions can access it - exprs.foreach { expr => - - // declaration - val resultTypeTerm = primitiveTypeTermForTypeInfo(expr.resultType) - if (nullCheck && !expr.nullTerm.equals(NEVER_NULL) && !expr.nullTerm.equals(ALWAYS_NULL)) { - reusableMemberStatements.add(s"private boolean ${expr.nullTerm};") - } - reusableMemberStatements.add(s"private $resultTypeTerm ${expr.resultTerm};") - - // assignment + // when expr has no code, no need to split it into a method, but still need to assign + if (expr.code.isEmpty) { if (nullCheck && !expr.nullTerm.equals(NEVER_NULL) && !expr.nullTerm.equals(ALWAYS_NULL)) { reusablePerRecordStatements.add(s"this.${expr.nullTerm} = ${expr.nullTerm};") } reusablePerRecordStatements.add(s"this.${expr.resultTerm} = ${expr.resultTerm};") + expr + } else { + // create a method for the unboxing block + val methodName = newName(s"inputUnboxingSplit") + val method = + if (nullCheck && !expr.nullTerm.equals(NEVER_NULL) && !expr.nullTerm.equals(ALWAYS_NULL)) { + s""" + |private final void $methodName() throws Exception { + | ${expr.code} + | this.${expr.nullTerm} = ${expr.nullTerm}; + | this.${expr.resultTerm} = ${expr.resultTerm}; + |} + """.stripMargin + } else { + s""" + |private final void $methodName() throws Exception { + | ${expr.code} + | this.${expr.resultTerm} = ${expr.resultTerm}; + |} + """.stripMargin + } + + // add this method to reusable section for later generation + reusableMemberStatements.add(method) + + // create method call + GeneratedExpression( + expr.resultTerm, + expr.nullTerm, + s"$methodName();", + expr.resultType) } } @@ -1081,7 +1109,9 @@ abstract class CodeGenerator( hasCodeSplits = true // add input unboxing to member area such that all split functions can access it - makeReusableInSplits(reusableInputUnboxingExprs.values) + reusableInputUnboxingExprs.keys.foreach( + key => + reusableInputUnboxingExprs(key) = makeReusableInSplits(reusableInputUnboxingExprs(key))) // add split methods to the member area and return the code necessary to call those methods val methodCalls = splits.map { split => diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/CollectorCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/CollectorCodeGenerator.scala index ddc13c5b45b2..c437156d7171 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/CollectorCodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/CollectorCodeGenerator.scala @@ -73,20 +73,6 @@ class CollectorCodeGenerator( val input1TypeClass = boxedTypeTermForTypeInfo(input1) val input2TypeClass = boxedTypeTermForTypeInfo(collectedType) - // declaration in case of code splits - val recordMember = if (hasCodeSplits) { - s"private $input2TypeClass $input2Term;" - } else { - "" - } - - // assignment in case of code splits - val recordAssignment = if (hasCodeSplits) { - s"$input2Term" // use member - } else { - s"$input2TypeClass $input2Term" // local variable - } - reusableMemberStatements ++= filterGenerator.reusableMemberStatements reusableInitStatements ++= filterGenerator.reusableInitStatements reusablePerRecordStatements ++= filterGenerator.reusablePerRecordStatements @@ -94,7 +80,9 @@ class CollectorCodeGenerator( val funcCode = j""" |public class $className extends ${classOf[TableFunctionCollector[_]].getCanonicalName} { | - | $recordMember + | private $input1TypeClass $input1Term; + | private $input2TypeClass $input2Term; + | | ${reuseMemberCode()} | | public $className() throws Exception { @@ -109,8 +97,8 @@ class CollectorCodeGenerator( | @Override | public void collect(Object record) throws Exception { | super.collect(record); - | $input1TypeClass $input1Term = ($input1TypeClass) getInput(); - | $recordAssignment = ($input2TypeClass) record; + | $input1Term = ($input1TypeClass) getInput(); + | $input2Term = ($input2TypeClass) record; | ${reuseInputUnboxingCode()} | ${reusePerRecordCode()} | $bodyCode diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/FunctionCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/FunctionCodeGenerator.scala index 8ac18cdda46e..527e6b059d8f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/FunctionCodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/FunctionCodeGenerator.scala @@ -101,18 +101,22 @@ class FunctionCodeGenerator( if (clazz == classOf[FlatMapFunction[_, _]]) { val baseClass = classOf[RichFlatMapFunction[_, _]] val inputTypeTerm = boxedTypeTermForTypeInfo(input1) + // declaration: make variable accessible for separated method + reusableMemberStatements.add(s"private $inputTypeTerm $input1Term;") (baseClass, s"void flatMap(Object _in1, $collectorTypeTerm $collectorTerm)", - List(s"$inputTypeTerm $input1Term = ($inputTypeTerm) _in1;")) + List(s"$input1Term = ($inputTypeTerm) _in1;")) } // MapFunction else if (clazz == classOf[MapFunction[_, _]]) { val baseClass = classOf[RichMapFunction[_, _]] val inputTypeTerm = boxedTypeTermForTypeInfo(input1) + // declaration: make variable accessible for separated method + reusableMemberStatements.add(s"private $inputTypeTerm $input1Term;") (baseClass, "Object map(Object _in1)", - List(s"$inputTypeTerm $input1Term = ($inputTypeTerm) _in1;")) + List(s"$input1Term = ($inputTypeTerm) _in1;")) } // FlatJoinFunction @@ -121,10 +125,13 @@ class FunctionCodeGenerator( val inputTypeTerm1 = boxedTypeTermForTypeInfo(input1) val inputTypeTerm2 = boxedTypeTermForTypeInfo(input2.getOrElse( throw new CodeGenException("Input 2 for FlatJoinFunction should not be null"))) + // declaration: make variables accessible for separated methods + reusableMemberStatements.add(s"private $inputTypeTerm1 $input1Term;") + reusableMemberStatements.add(s"private $inputTypeTerm2 $input2Term;") (baseClass, s"void join(Object _in1, Object _in2, $collectorTypeTerm $collectorTerm)", - List(s"$inputTypeTerm1 $input1Term = ($inputTypeTerm1) _in1;", - s"$inputTypeTerm2 $input2Term = ($inputTypeTerm2) _in2;")) + List(s"$input1Term = ($inputTypeTerm1) _in1;", + s"$input2Term = ($inputTypeTerm2) _in2;")) } // JoinFunction @@ -133,10 +140,13 @@ class FunctionCodeGenerator( val inputTypeTerm1 = boxedTypeTermForTypeInfo(input1) val inputTypeTerm2 = boxedTypeTermForTypeInfo(input2.getOrElse( throw new CodeGenException("Input 2 for JoinFunction should not be null"))) + // declaration: make variables accessible for separated methods + reusableMemberStatements.add(s"private $inputTypeTerm1 $input1Term;") + reusableMemberStatements.add(s"private $inputTypeTerm2 $input2Term;") (baseClass, s"Object join(Object _in1, Object _in2)", - List(s"$inputTypeTerm1 $input1Term = ($inputTypeTerm1) _in1;", - s"$inputTypeTerm2 $input2Term = ($inputTypeTerm2) _in2;")) + List(s"$input1Term = ($inputTypeTerm1) _in1;", + s"$input2Term = ($inputTypeTerm2) _in2;")) } // ProcessFunction @@ -155,10 +165,12 @@ class FunctionCodeGenerator( Nil } + // declaration: make variable accessible for separated method + reusableMemberStatements.add(s"private $inputTypeTerm $input1Term;") (baseClass, s"void processElement(Object _in1, $contextTypeTerm $contextTerm, " + s"$collectorTypeTerm $collectorTerm)", - List(s"$inputTypeTerm $input1Term = ($inputTypeTerm) _in1;") ++ globalContext) + List(s"$input1Term = ($inputTypeTerm) _in1;") ++ globalContext) } else { // TODO more functions diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala index 903521ff8dff..f2481883415a 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala @@ -316,20 +316,22 @@ class MatchCodeGenerator( val baseClass = classOf[RichIterativeCondition[_]] val inputTypeTerm = boxedTypeTermForTypeInfo(input) val contextType = classOf[IterativeCondition.Context[_]].getCanonicalName - + // declaration: make variable accessible for separated methods + reusableMemberStatements.add(s"private $inputTypeTerm $input1Term;") (baseClass, s"boolean filter(Object _in1, $contextType $contextTerm)", - List(s"$inputTypeTerm $input1Term = ($inputTypeTerm) _in1;")) + List(s"$input1Term = ($inputTypeTerm) _in1;")) } else if (clazz == classOf[PatternProcessFunction[_, _]]) { val baseClass = classOf[PatternProcessFunction[_, _]] val inputTypeTerm = s"java.util.Map>" val contextTypeTerm = classOf[PatternProcessFunction.Context].getCanonicalName - + // declaration: make variable accessible for separated method + reusableMemberStatements.add(s"private $inputTypeTerm $input1Term;") (baseClass, - s"void processMatch($inputTypeTerm $input1Term, $contextTypeTerm $contextTerm, " + + s"void processMatch($inputTypeTerm _in1, $contextTypeTerm $contextTerm, " + s"$collectorTypeTerm $collectorTerm)", - List()) + List(s"this.$input1Term = ($inputTypeTerm) _in1;")) } else { throw new CodeGenException("Unsupported Function.") } @@ -434,7 +436,7 @@ class MatchCodeGenerator( returnType.fieldNames) aggregatesPerVariable.values.foreach(_.generateAggFunction()) if (hasCodeSplits) { - makeReusableInSplits(reusableAggregationExpr.values) + makeReusableInSplits() } exp @@ -444,12 +446,18 @@ class MatchCodeGenerator( val exp = call.accept(this) aggregatesPerVariable.values.foreach(_.generateAggFunction()) if (hasCodeSplits) { - makeReusableInSplits(reusableAggregationExpr.values) + makeReusableInSplits() } exp } + private def makeReusableInSplits(): Unit = { + reusableAggregationExpr.keys.foreach( + key => + reusableAggregationExpr(key) = makeReusableInSplits(reusableAggregationExpr(key))) + } + override def visitCall(call: RexCall): GeneratedExpression = { call.getOperator match { case PREV | NEXT => @@ -539,11 +547,13 @@ class MatchCodeGenerator( } else { "" } + + reusableMemberStatements.add(s"java.util.List $listName = new java.util.ArrayList();") val listCode = if (patternName == ALL_PATTERN_VARIABLE) { addReusablePatternNames() val patternTerm = newName("pattern") j""" - |java.util.List $listName = new java.util.ArrayList(); + |$listName = new java.util.ArrayList(); |for (String $patternTerm : $patternNamesTerm) { | for ($eventTypeTerm $eventNameTerm : | $contextTerm.getEventsForPattern($patternTerm)) { @@ -554,7 +564,7 @@ class MatchCodeGenerator( } else { val escapedPatternName = EncodingUtils.escapeJava(patternName) j""" - |java.util.List $listName = new java.util.ArrayList(); + |$listName = new java.util.ArrayList(); |for ($eventTypeTerm $eventNameTerm : | $contextTerm.getEventsForPattern("$escapedPatternName")) { | $listName.add($eventNameTerm); @@ -574,13 +584,14 @@ class MatchCodeGenerator( private def generateMeasurePatternVariableExp(patternName: String): GeneratedPatternList = { val listName = newName("patternEvents") + reusableMemberStatements.add(s"java.util.List $listName = new java.util.ArrayList();") val code = if (patternName == ALL_PATTERN_VARIABLE) { addReusablePatternNames() val patternTerm = newName("pattern") j""" - |java.util.List $listName = new java.util.ArrayList(); + |$listName = new java.util.ArrayList(); |for (String $patternTerm : $patternNamesTerm) { | java.util.List rows = (java.util.List) $input1Term.get($patternTerm); | if (rows != null) { @@ -591,7 +602,7 @@ class MatchCodeGenerator( } else { val escapedPatternName = EncodingUtils.escapeJava(patternName) j""" - |java.util.List $listName = (java.util.List) $input1Term.get("$escapedPatternName"); + |$listName = (java.util.List) $input1Term.get("$escapedPatternName"); |if ($listName == null) { | $listName = java.util.Collections.emptyList(); |} diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala index 3d14abcc1377..ff25a0afbe60 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala @@ -39,6 +39,7 @@ import org.junit.Assert._ import org.junit._ import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer class SqlITCase extends StreamingWithStateTestBase { @@ -870,6 +871,45 @@ class SqlITCase extends StreamingWithStateTestBase { assertEquals(List(expected.toString()), StreamITCase.testResults.sorted) } + + @Test + def testProjectionWithManyColumns(): Unit = { + + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = StreamTableEnvironment.create(env) + StreamITCase.clear + + // force code split + tEnv.getConfig.setMaxGeneratedCodeLength(1) + + val length = 1000 + val rowData = List.range(0, length) + val row: Row = new Row(length) + val fieldTypes = new ArrayBuffer[TypeInformation[_]]() + val fieldNames = new ArrayBuffer[String]() + rowData.foreach { i => + row.setField(i, i) + fieldTypes += Types.INT() + fieldNames += s"f$i" + } + + val data = new mutable.MutableList[Row] + data.+=(row) + val t = env.fromCollection(data)(new RowTypeInfo(fieldTypes.toArray: _*)).toTable(tEnv) + tEnv.registerTable("MyTable", t) + + val expected = List(rowData.reverse.mkString(",")) + val sql = + s""" + |SELECT ${fieldNames.reverse.mkString(", ")} FROM MyTable + """.stripMargin + + val result = tEnv.sqlQuery(sql).toAppendStream[Row] + result.addSink(new StreamITCase.StringSink[Row]) + env.execute() + + assertEquals(expected, StreamITCase.testResults) + } } object SqlITCase { From 785b7b28b37e29b00381fb71cac4872ef9575cac Mon Sep 17 00:00:00 2001 From: lining Date: Wed, 30 Oct 2019 17:38:49 +0800 Subject: [PATCH 350/746] [FLINK-11405][rest] Add maxExceptions query parameter --- .../generated/rest_v1_dispatcher.html | 10 ++ .../src/test/resources/rest_api_v1.snapshot | 5 +- .../handler/job/JobExceptionsHandler.java | 19 ++- .../rest/messages/JobExceptionsHeaders.java | 7 +- .../rest/messages/JobExceptionsInfo.java | 21 +++ .../job/JobExceptionsMessageParameters.java | 41 +++++ .../job/UpperLimitExceptionParameter.java | 50 ++++++ .../handler/job/JobExceptionsHandlerTest.java | 144 ++++++++++++++++++ 8 files changed, 285 insertions(+), 12 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobExceptionsMessageParameters.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/UpperLimitExceptionParameter.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobExceptionsHandlerTest.java diff --git a/docs/_includes/generated/rest_v1_dispatcher.html b/docs/_includes/generated/rest_v1_dispatcher.html index a3b91ee2e83d..dfe4fa58c572 100644 --- a/docs/_includes/generated/rest_v1_dispatcher.html +++ b/docs/_includes/generated/rest_v1_dispatcher.html @@ -1893,6 +1893,16 @@ + + Query parameters + + + +
      +
    • maxExceptions (optional): Comma-separated list of integer values that specifies the upper limit of exceptions to return.
    • +
    + + 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 65345ba20ded..5ed248995d49 100644 --- a/flink-runtime-web/src/test/resources/rest_api_v1.snapshot +++ b/flink-runtime-web/src/test/resources/rest_api_v1.snapshot @@ -1302,7 +1302,10 @@ } ] }, "query-parameters" : { - "queryParameters" : [ ] + "queryParameters" : [ { + "key" : "maxExceptions", + "mandatory" : false + } ] }, "request" : { "type" : "any" diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobExceptionsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobExceptionsHandler.java index d5f67254f08c..a0ac819873e9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobExceptionsHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobExceptionsHandler.java @@ -28,9 +28,10 @@ import org.apache.flink.runtime.rest.messages.EmptyRequestBody; import org.apache.flink.runtime.rest.messages.JobExceptionsInfo; import org.apache.flink.runtime.rest.messages.JobIDPathParameter; -import org.apache.flink.runtime.rest.messages.JobMessageParameters; import org.apache.flink.runtime.rest.messages.MessageHeaders; import org.apache.flink.runtime.rest.messages.ResponseBody; +import org.apache.flink.runtime.rest.messages.job.JobExceptionsMessageParameters; +import org.apache.flink.runtime.rest.messages.job.UpperLimitExceptionParameter; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.webmonitor.RestfulGateway; import org.apache.flink.runtime.webmonitor.history.ArchivedJson; @@ -49,7 +50,7 @@ /** * Handler serving the job exceptions. */ -public class JobExceptionsHandler extends AbstractExecutionGraphHandler implements JsonArchivist { +public class JobExceptionsHandler extends AbstractExecutionGraphHandler implements JsonArchivist { static final int MAX_NUMBER_EXCEPTION_TO_REPORT = 20; @@ -57,7 +58,7 @@ public JobExceptionsHandler( GatewayRetriever leaderRetriever, Time timeout, Map responseHeaders, - MessageHeaders messageHeaders, + MessageHeaders messageHeaders, ExecutionGraphCache executionGraphCache, Executor executor) { @@ -71,19 +72,21 @@ public JobExceptionsHandler( } @Override - protected JobExceptionsInfo handleRequest(HandlerRequest request, AccessExecutionGraph executionGraph) { - return createJobExceptionsInfo(executionGraph); + protected JobExceptionsInfo handleRequest(HandlerRequest request, AccessExecutionGraph executionGraph) { + List exceptionToReportMaxSizes = request.getQueryParameter(UpperLimitExceptionParameter.class); + final int exceptionToReportMaxSize = exceptionToReportMaxSizes.size() > 0 ? exceptionToReportMaxSizes.get(0) : MAX_NUMBER_EXCEPTION_TO_REPORT; + return createJobExceptionsInfo(executionGraph, exceptionToReportMaxSize); } @Override public Collection archiveJsonWithPath(AccessExecutionGraph graph) throws IOException { - ResponseBody json = createJobExceptionsInfo(graph); + ResponseBody json = createJobExceptionsInfo(graph, MAX_NUMBER_EXCEPTION_TO_REPORT); String path = getMessageHeaders().getTargetRestEndpointURL() .replace(':' + JobIDPathParameter.KEY, graph.getJobID().toString()); return Collections.singletonList(new ArchivedJson(path, json)); } - private static JobExceptionsInfo createJobExceptionsInfo(AccessExecutionGraph executionGraph) { + private static JobExceptionsInfo createJobExceptionsInfo(AccessExecutionGraph executionGraph, int exceptionToReportMaxSize) { ErrorInfo rootException = executionGraph.getFailureInfo(); String rootExceptionMessage = null; Long rootTimestamp = null; @@ -97,7 +100,7 @@ private static JobExceptionsInfo createJobExceptionsInfo(AccessExecutionGraph ex for (AccessExecutionVertex task : executionGraph.getAllExecutionVertices()) { String t = task.getFailureCauseAsString(); if (t != null && !t.equals(ExceptionUtils.STRINGIFIED_NULL_EXCEPTION)) { - if (taskExceptionList.size() >= MAX_NUMBER_EXCEPTION_TO_REPORT) { + if (taskExceptionList.size() >= exceptionToReportMaxSize) { truncated = true; break; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsHeaders.java index fb0e5a13b8fe..19966c120839 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsHeaders.java @@ -20,13 +20,14 @@ import org.apache.flink.runtime.rest.HttpMethodWrapper; import org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler; +import org.apache.flink.runtime.rest.messages.job.JobExceptionsMessageParameters; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; /** * Message headers for the {@link JobExceptionsHandler}. */ -public class JobExceptionsHeaders implements MessageHeaders { +public class JobExceptionsHeaders implements MessageHeaders { private static final JobExceptionsHeaders INSTANCE = new JobExceptionsHeaders(); @@ -50,8 +51,8 @@ public HttpResponseStatus getResponseStatusCode() { } @Override - public JobMessageParameters getUnresolvedMessageParameters() { - return new JobMessageParameters(); + public JobExceptionsMessageParameters getUnresolvedMessageParameters() { + return new JobExceptionsMessageParameters(); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsInfo.java index 37b1fb901dcf..4de4a3a2d27e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobExceptionsInfo.java @@ -22,6 +22,7 @@ import org.apache.flink.util.Preconditions; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; @@ -81,6 +82,26 @@ public int hashCode() { return Objects.hash(rootException, rootTimestamp, allExceptions, truncated); } + @JsonIgnore + public String getRootException() { + return rootException; + } + + @JsonIgnore + public Long getRootTimestamp() { + return rootTimestamp; + } + + @JsonIgnore + public List getAllExceptions() { + return allExceptions; + } + + @JsonIgnore + public boolean isTruncated() { + return truncated; + } + //--------------------------------------------------------------------------------- // Static helper classes //--------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobExceptionsMessageParameters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobExceptionsMessageParameters.java new file mode 100644 index 000000000000..7ca755025e1f --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobExceptionsMessageParameters.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.runtime.rest.messages.job; + +import org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler; +import org.apache.flink.runtime.rest.messages.JobMessageParameters; +import org.apache.flink.runtime.rest.messages.MessageParameters; +import org.apache.flink.runtime.rest.messages.MessageQueryParameter; + +import java.util.Collection; +import java.util.Collections; + +/** + * {@link MessageParameters} for {@link JobExceptionsHandler}. + */ +public class JobExceptionsMessageParameters extends JobMessageParameters { + + private final UpperLimitExceptionParameter upperLimitExceptionParameter = new UpperLimitExceptionParameter(); + + @Override + public Collection> getQueryParameters() { + return Collections.singletonList(upperLimitExceptionParameter); + } + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/UpperLimitExceptionParameter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/UpperLimitExceptionParameter.java new file mode 100644 index 000000000000..c79c2f3130db --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/UpperLimitExceptionParameter.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.rest.messages.job; + +import org.apache.flink.runtime.rest.messages.MessageParameter; +import org.apache.flink.runtime.rest.messages.MessageQueryParameter; + +/** + * Specifies the upper limit of exceptions to return for JobExceptionsHandler. + * @see org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler + */ +public class UpperLimitExceptionParameter extends MessageQueryParameter { + + public static final String KEY = "maxExceptions"; + + public UpperLimitExceptionParameter() { + super(KEY, MessageParameter.MessageParameterRequisiteness.OPTIONAL); + } + + @Override + public Integer convertStringToValue(String value) { + return Integer.valueOf(value); + } + + @Override + public String convertValueToString(Integer value) { + return value.toString(); + } + + @Override + public String getDescription() { + return "Comma-separated list of integer values that specifies the upper limit of exceptions to return."; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobExceptionsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobExceptionsHandlerTest.java new file mode 100644 index 000000000000..abbd3fce2baf --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobExceptionsHandlerTest.java @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.rest.handler.job; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; +import org.apache.flink.runtime.executiongraph.ArchivedExecution; +import org.apache.flink.runtime.executiongraph.ArchivedExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.ArchivedExecutionVertex; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.rest.handler.HandlerRequest; +import org.apache.flink.runtime.rest.handler.HandlerRequestException; +import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphCache; +import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedExecutionGraphBuilder; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.JobExceptionsHeaders; +import org.apache.flink.runtime.rest.messages.JobExceptionsInfo; +import org.apache.flink.runtime.rest.messages.JobIDPathParameter; +import org.apache.flink.runtime.rest.messages.job.JobExceptionsMessageParameters; +import org.apache.flink.runtime.rest.messages.job.UpperLimitExceptionParameter; +import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; +import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.runtime.util.EvictingBoundedList; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +/** + * Test for the {@link JobExceptionsHandler}. + */ +public class JobExceptionsHandlerTest extends TestLogger { + + @Test + public void testGetJobExceptionsInfo() throws HandlerRequestException { + final JobExceptionsHandler jobExceptionsHandler = new JobExceptionsHandler( + () -> null, + TestingUtils.TIMEOUT(), + Collections.emptyMap(), + JobExceptionsHeaders.getInstance(), + new ExecutionGraphCache(TestingUtils.TIMEOUT(), TestingUtils.TIMEOUT()), + TestingUtils.defaultExecutor()); + final int numExceptions = 20; + final AccessExecutionGraph archivedExecutionGraph = createAccessExecutionGraph(numExceptions); + checkExceptionLimit(jobExceptionsHandler, archivedExecutionGraph, numExceptions, 10); + checkExceptionLimit(jobExceptionsHandler, archivedExecutionGraph, numExceptions, numExceptions); + checkExceptionLimit(jobExceptionsHandler, archivedExecutionGraph, numExceptions, 30); + } + + private static void checkExceptionLimit(JobExceptionsHandler jobExceptionsHandler, AccessExecutionGraph graph, int maxNumExceptions, int numExpectedException) throws HandlerRequestException { + final HandlerRequest handlerRequest = createRequest(graph.getJobID(), numExpectedException); + final JobExceptionsInfo jobExceptionsInfo = jobExceptionsHandler.handleRequest(handlerRequest, graph); + final int numReportedException = maxNumExceptions >= numExpectedException ? numExpectedException : maxNumExceptions; + assertEquals(jobExceptionsInfo.getAllExceptions().size(), numReportedException); + } + + private static AccessExecutionGraph createAccessExecutionGraph(int numTasks) { + Map tasks = new HashMap<>(); + for (int i = 0; i < numTasks; i++) { + final JobVertexID jobVertexId = new JobVertexID(); + tasks.put(jobVertexId, createArchivedExecutionJobVertex(jobVertexId)); + } + return new ArchivedExecutionGraphBuilder() + .setTasks(tasks) + .build(); + } + + private static ArchivedExecutionJobVertex createArchivedExecutionJobVertex(JobVertexID jobVertexID) { + final StringifiedAccumulatorResult[] emptyAccumulators = new StringifiedAccumulatorResult[0]; + final long[] timestamps = new long[ExecutionState.values().length]; + final ExecutionState expectedState = ExecutionState.RUNNING; + + final LocalTaskManagerLocation assignedResourceLocation = new LocalTaskManagerLocation(); + final AllocationID allocationID = new AllocationID(); + + final int subtaskIndex = 1; + final int attempt = 2; + return new ArchivedExecutionJobVertex( + new ArchivedExecutionVertex[]{ + new ArchivedExecutionVertex( + subtaskIndex, + "test task", + new ArchivedExecution( + new StringifiedAccumulatorResult[0], + null, + new ExecutionAttemptID(), + attempt, + expectedState, + "error", + assignedResourceLocation, + allocationID, + subtaskIndex, + timestamps), + new EvictingBoundedList<>(0) + ) + }, + jobVertexID, + jobVertexID.toString(), + 1, + 1, + ResourceProfile.UNKNOWN, + emptyAccumulators); + } + + private static HandlerRequest createRequest(JobID jobId, int size) throws HandlerRequestException { + final Map pathParameters = new HashMap<>(); + pathParameters.put(JobIDPathParameter.KEY, jobId.toString()); + final Map> queryParameters = new HashMap<>(); + queryParameters.put(UpperLimitExceptionParameter.KEY, Collections.singletonList("" + size)); + + return new HandlerRequest<>( + EmptyRequestBody.getInstance(), + new JobExceptionsMessageParameters(), + pathParameters, + queryParameters); + } +} From 16a815a756b35b1693241ef74a78e1b99748a6d4 Mon Sep 17 00:00:00 2001 From: Piotr Przybylski Date: Wed, 30 Oct 2019 10:41:57 +0100 Subject: [PATCH 351/746] [FLINK-12147][metrics][influxdb] Bump influxdb-java to 2.16 Solves issue with sending metrics with infinity values (e.g. from Kafka) which are unsupported by InfluxDB. New driver version silently drops unsupported value types (NaN, -Inf, +Inf). --- flink-metrics/flink-metrics-influxdb/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-metrics/flink-metrics-influxdb/pom.xml b/flink-metrics/flink-metrics-influxdb/pom.xml index 02713613ab88..354c4591297b 100644 --- a/flink-metrics/flink-metrics-influxdb/pom.xml +++ b/flink-metrics/flink-metrics-influxdb/pom.xml @@ -57,7 +57,7 @@ under the License. org.influxdb influxdb-java - 2.14 + 2.16 From 0518cddce435476e098ad45fe4ad20d020bed888 Mon Sep 17 00:00:00 2001 From: David Anderson Date: Wed, 30 Oct 2019 10:47:12 +0100 Subject: [PATCH 352/746] [hotfix][docs] Fix REST label rotation in SSL figure --- docs/fig/ssl_internal_external.svg | 541 ++++++++++++++--------------- 1 file changed, 270 insertions(+), 271 deletions(-) diff --git a/docs/fig/ssl_internal_external.svg b/docs/fig/ssl_internal_external.svg index 04262d29cbc2..42d845ecda07 100755 --- a/docs/fig/ssl_internal_external.svg +++ b/docs/fig/ssl_internal_external.svg @@ -29,7 +29,8 @@ under the License. height="364.59875" id="svg2" version="1.1" - inkscape:version="0.48.5 r10040"> + inkscape:version="1.0beta1 (d565813, 2019-09-28)" + sodipodi:docname="ssl_internal_external.svg"> + inkscape:window-width="1678" + inkscape:window-height="907" + inkscape:window-x="0" + inkscape:window-y="23" + inkscape:window-maximized="0" + inkscape:document-rotation="0" /> @@ -62,7 +64,7 @@ under the License. image/svg+xml - + @@ -71,266 +73,263 @@ under the License. inkscape:groupmode="layer" id="layer1" transform="translate(139.76354,-243.79437)"> - - - Task - Manager - - Task - Manager - - Task - Manager - - Resource - Manager - - Job - Manager - - Master Process - - - - - - - - - - - RPC / BLOB - Data Plane - Data Plane - - - - REST - Internal - External - - Browser - - CLI - - Tools - - - HTTP - - Dispatcher - + + Task + Manager + + Task + Manager + + Task + Manager + + Resource + Manager + + Job + Manager + + Master Process + + + + + + + + + + + RPC / BLOB + Data Plane + Data Plane + + + + REST + Internal + External + + Browser + + CLI + + Tools + + + HTTP + + Dispatcher From a862daddcea9ed83ae028778538bdbd83ee546a7 Mon Sep 17 00:00:00 2001 From: Yun Tang Date: Wed, 30 Oct 2019 18:27:44 +0800 Subject: [PATCH 353/746] [FLINK-14403][coordination] Remove legacy NotifyCheckpointComplete and TriggerCheckpoint --- .../checkpoint/NotifyCheckpointComplete.java | 76 --------------- .../checkpoint/TriggerCheckpoint.java | 94 ------------------- .../messages/CheckpointMessagesTest.java | 19 ---- 3 files changed, 189 deletions(-) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/messages/checkpoint/NotifyCheckpointComplete.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/messages/checkpoint/TriggerCheckpoint.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/messages/checkpoint/NotifyCheckpointComplete.java b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/checkpoint/NotifyCheckpointComplete.java deleted file mode 100644 index 8afd3734eab4..000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/messages/checkpoint/NotifyCheckpointComplete.java +++ /dev/null @@ -1,76 +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.messages.checkpoint; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; - -/** - * This message is sent from the {@link org.apache.flink.runtime.jobmaster.JobMaster} to the - * {@link org.apache.flink.runtime.taskexecutor.TaskExecutor} to tell a task that the checkpoint - * has been confirmed and that the task can commit the checkpoint to the outside world. - */ -public class NotifyCheckpointComplete extends AbstractCheckpointMessage implements java.io.Serializable { - - private static final long serialVersionUID = 2094094662279578953L; - - /** The timestamp associated with the checkpoint. */ - private final long timestamp; - - public NotifyCheckpointComplete(JobID job, ExecutionAttemptID taskExecutionId, long checkpointId, long timestamp) { - super(job, taskExecutionId, checkpointId); - this.timestamp = timestamp; - } - - // -------------------------------------------------------------------------------------------- - - public long getTimestamp() { - return timestamp; - } - - // -------------------------------------------------------------------------------------------- - - @Override - public int hashCode() { - return super.hashCode() + (int) (timestamp ^ (timestamp >>> 32)); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - else if (o instanceof NotifyCheckpointComplete) { - NotifyCheckpointComplete that = (NotifyCheckpointComplete) o; - return this.timestamp == that.timestamp && super.equals(o); - } - else { - return false; - } - } - - @Override - public String toString() { - return String.format( - "ConfirmCheckpoint %d for (%s/%s)", - getCheckpointId(), - getJob(), - getTaskExecutionId()); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/messages/checkpoint/TriggerCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/checkpoint/TriggerCheckpoint.java deleted file mode 100644 index 23325cd8328f..000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/messages/checkpoint/TriggerCheckpoint.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.messages.checkpoint; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.runtime.checkpoint.CheckpointOptions; -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * This message is sent from the {@link org.apache.flink.runtime.jobmaster.JobMaster} to the - * {@link org.apache.flink.runtime.taskexecutor.TaskExecutor} to tell a certain task to trigger its - * checkpoint. - */ -public class TriggerCheckpoint extends AbstractCheckpointMessage { - - private static final long serialVersionUID = 2094094662279578953L; - - /** The timestamp associated with the checkpoint. */ - private final long timestamp; - - /** Options for how to perform the checkpoint. */ - private final CheckpointOptions checkpointOptions; - - public TriggerCheckpoint( - JobID job, - ExecutionAttemptID taskExecutionId, - long checkpointId, - long timestamp, - CheckpointOptions checkpointOptions) { - - super(job, taskExecutionId, checkpointId); - this.timestamp = timestamp; - this.checkpointOptions = checkNotNull(checkpointOptions); - } - - // -------------------------------------------------------------------------------------------- - - public long getTimestamp() { - return timestamp; - } - - public CheckpointOptions getCheckpointOptions() { - return checkpointOptions; - } - - // -------------------------------------------------------------------------------------------- - - @Override - public int hashCode() { - return super.hashCode() + (int) (timestamp ^ (timestamp >>> 32)); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - else if (o instanceof TriggerCheckpoint) { - TriggerCheckpoint that = (TriggerCheckpoint) o; - return this.timestamp == that.timestamp && super.equals(o); - } - else { - return false; - } - } - - @Override - public String toString() { - return String.format( - "Trigger Checkpoint %d@%d for (%s/%s)", - getCheckpointId(), - getTimestamp(), - getJob(), - getTaskExecutionId()); - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/messages/CheckpointMessagesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/messages/CheckpointMessagesTest.java index 0fb46e0c230c..15216cfeb54b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/messages/CheckpointMessagesTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/messages/CheckpointMessagesTest.java @@ -23,15 +23,12 @@ import org.apache.flink.core.testutils.CommonTestUtils; import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; -import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; -import org.apache.flink.runtime.messages.checkpoint.NotifyCheckpointComplete; -import org.apache.flink.runtime.messages.checkpoint.TriggerCheckpoint; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.StreamStateHandle; @@ -50,22 +47,6 @@ */ public class CheckpointMessagesTest { - @Test - public void testTriggerAndConfirmCheckpoint() { - try { - NotifyCheckpointComplete cc = new NotifyCheckpointComplete(new JobID(), new ExecutionAttemptID(), 45287698767345L, 467L); - testSerializabilityEqualsHashCode(cc); - - TriggerCheckpoint tc = new TriggerCheckpoint(new JobID(), new ExecutionAttemptID(), 347652734L, 7576752L, CheckpointOptions.forCheckpointWithDefaultLocation()); - testSerializabilityEqualsHashCode(tc); - - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - @Test public void testConfirmTaskCheckpointed() { try { From 11eb7c413bc71e05a0f15b7aa1cf06a7e8e41c59 Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Wed, 23 Oct 2019 16:53:10 +0800 Subject: [PATCH 354/746] [FLINK-14535][table-planner-blink] Fix cast exception when count distinct on decimal fields The conversion class of DecimalType of distinct key should be Decimal instead of BigDecimal. This closes #10001 --- .../planner/plan/utils/AggregateUtil.scala | 14 +- .../runtime/stream/sql/AggregateITCase.scala | 191 ++++++++---------- .../stream/sql/SplitAggregateITCase.scala | 103 +++++++++- .../planner/utils/DateTimeTestUtil.scala | 18 +- 4 files changed, 207 insertions(+), 119 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala index ebd2863f0754..92a5d3d8b6ad 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala @@ -20,7 +20,7 @@ package org.apache.flink.table.planner.plan.utils import org.apache.flink.api.common.typeinfo.Types import org.apache.flink.table.api.config.ExecutionConfigOptions import org.apache.flink.table.api.{DataTypes, TableConfig, TableException} -import org.apache.flink.table.dataformat.BaseRow +import org.apache.flink.table.dataformat.{BaseRow, BinaryString, Decimal} import org.apache.flink.table.dataview.MapViewTypeInfo import org.apache.flink.table.expressions.ExpressionUtils.extractValue import org.apache.flink.table.expressions._ @@ -39,21 +39,18 @@ import org.apache.flink.table.planner.plan.`trait`.RelModifiedMonotonicity import org.apache.flink.table.runtime.operators.bundle.trigger.CountBundleTrigger import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.{fromDataTypeToLogicalType, fromLogicalTypeToDataType} import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter.fromDataTypeToTypeInfo -import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo import org.apache.flink.table.types.DataType import org.apache.flink.table.types.logical.LogicalTypeRoot._ import org.apache.flink.table.types.logical.utils.LogicalTypeChecks import org.apache.flink.table.types.logical.utils.LogicalTypeChecks.hasRoot import org.apache.flink.table.types.logical.{LogicalTypeRoot, _} import org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType - import org.apache.calcite.rel.`type`._ import org.apache.calcite.rel.core.{Aggregate, AggregateCall} import org.apache.calcite.sql.fun._ import org.apache.calcite.sql.validate.SqlMonotonicity import org.apache.calcite.sql.{SqlKind, SqlRankFunction} import org.apache.calcite.tools.RelBuilder - import java.time.Duration import java.util @@ -502,10 +499,15 @@ object AggregateUtil extends Enumeration { case INTERVAL_YEAR_MONTH => DataTypes.INT case INTERVAL_DAY_TIME => DataTypes.BIGINT - case VARCHAR | CHAR => fromLegacyInfoToDataType(BinaryStringTypeInfo.INSTANCE) + case VARCHAR => + val dt = argTypes(0).asInstanceOf[VarCharType] + DataTypes.VARCHAR(dt.getLength).bridgedTo(classOf[BinaryString]) + case CHAR => + val dt = argTypes(0).asInstanceOf[CharType] + DataTypes.CHAR(dt.getLength).bridgedTo(classOf[BinaryString]) case DECIMAL => val dt = argTypes(0).asInstanceOf[DecimalType] - DataTypes.DECIMAL(dt.getPrecision, dt.getScale) + DataTypes.DECIMAL(dt.getPrecision, dt.getScale).bridgedTo(classOf[Decimal]) case t => throw new TableException(s"Distinct aggregate function does not support type: $t.\n" + s"Please re-check the data type.") diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateITCase.scala index aa6f15b3504e..88e7aad7d447 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateITCase.scala @@ -42,6 +42,7 @@ import org.junit.runner.RunWith import org.junit.runners.Parameterized import java.lang.{Integer => JInt, Long => JLong} +import java.math.{BigDecimal => JBigDecimal} import scala.collection.{Seq, mutable} import scala.util.Random @@ -181,35 +182,94 @@ class AggregateITCase( @Test def testCountDistinct(): Unit = { + val ids = List( + 1, + 2, 2, + 3, 3, 3, + 4, 4, 4, 4, + 5, 5, 5, 5, 5) + + val dateTimes = List( + "1970-01-01 00:00:01", + "1970-01-01 00:00:02", null, + "1970-01-01 00:00:04", "1970-01-01 00:00:05", "1970-01-01 00:00:06", + "1970-01-01 00:00:07", null, null, "1970-01-01 00:00:10", + + "1970-01-01 00:00:11", "1970-01-01 00:00:11", "1970-01-01 00:00:13", + "1970-01-01 00:00:14", "1970-01-01 00:00:15") + + val dates = List( + "1970-01-01", + "1970-01-02", null, + "1970-01-04", "1970-01-05", "1970-01-06", + "1970-01-07", null, null, "1970-01-10", + "1970-01-11", "1970-01-11", "1970-01-13", "1970-01-14", "1970-01-15") + + val times = List( + "00:00:01", + "00:00:02", null, + "00:00:04", "00:00:05", "00:00:06", + "00:00:07", null, null, "00:00:10", + "00:00:11", "00:00:11", "00:00:13", "00:00:14", "00:00:15") + + val integers = List( + "1", + "2", null, + "4", "5", "6", + "7", null, null, "10", + "11", "11", "13", "14", "15") + + val chars = List( + "A", + "B", null, + "D", "E", "F", + "H", null, null, "K", + "L", "L", "N", "O", "P") + val data = new mutable.MutableList[Row] - data.+=(Row.of(JInt.valueOf(1), JLong.valueOf(1L), "A")) - data.+=(Row.of(JInt.valueOf(2), JLong.valueOf(2L), "B")) - data.+=(Row.of(null, JLong.valueOf(2L), "B")) - data.+=(Row.of(JInt.valueOf(3), JLong.valueOf(2L), "B")) - data.+=(Row.of(JInt.valueOf(4), JLong.valueOf(3L), "C")) - data.+=(Row.of(JInt.valueOf(5), JLong.valueOf(3L), "C")) - data.+=(Row.of(JInt.valueOf(5), JLong.valueOf(3L), null)) - data.+=(Row.of(JInt.valueOf(6), JLong.valueOf(3L), "C")) - data.+=(Row.of(JInt.valueOf(7), JLong.valueOf(4L), "B")) - data.+=(Row.of(JInt.valueOf(8), JLong.valueOf(4L), "A")) - data.+=(Row.of(JInt.valueOf(9), JLong.valueOf(4L), "D")) - data.+=(Row.of(null, JLong.valueOf(4L), null)) - data.+=(Row.of(JInt.valueOf(10), JLong.valueOf(4L), "E")) - data.+=(Row.of(JInt.valueOf(11), JLong.valueOf(5L), "A")) - data.+=(Row.of(JInt.valueOf(12), JLong.valueOf(5L), "B")) - - val rowType: RowTypeInfo = new RowTypeInfo(Types.INT, Types.LONG, Types.STRING) - - val t = failingDataSource(data)(rowType).toTable(tEnv, 'a, 'b, 'c) - tEnv.registerTable("T", t) + + for (i <- ids.indices) { + val v = integers(i) + val decimal = if (v == null) null else new JBigDecimal(v) + val int = if (v == null) null else JInt.valueOf(v) + val long = if (v == null) null else JLong.valueOf(v) + data.+=(Row.of( + Int.box(ids(i)), localDateTime(dateTimes(i)), localDate(dates(i)), + mLocalTime(times(i)), decimal, int, long, chars(i))) + } + + val inputs = util.Random.shuffle(data) + + val rowType = new RowTypeInfo( + Types.INT, Types.LOCAL_DATE_TIME, Types.LOCAL_DATE, Types.LOCAL_TIME, + Types.DECIMAL, Types.INT, Types.LONG, Types.STRING) + + val t = failingDataSource(inputs)(rowType).toTable(tEnv, 'id, 'a, 'b, 'c, 'd, 'e, 'f, 'g) + tEnv.createTemporaryView("T", t) val t1 = tEnv.sqlQuery( - "SELECT b, count(*), count(distinct c), count(distinct a) FROM T GROUP BY b") + s""" + |SELECT + | id, + | count(distinct a), + | count(distinct b), + | count(distinct c), + | count(distinct d), + | count(distinct e), + | count(distinct f), + | count(distinct g) + |FROM T GROUP BY id + """.stripMargin) val sink = new TestingRetractSink t1.toRetractStream[Row].addSink(sink) env.execute() - val expected = List("1,1,1,1", "2,3,1,2", "3,4,1,3", "4,5,4,4", "5,2,2,2") + val expected = List( + "1,1,1,1,1,1,1,1", + "2,1,1,1,1,1,1,1", + "3,3,3,3,3,3,3,3", + "4,2,2,2,2,2,2,2", + "5,4,4,4,4,4,4,4") assertEquals(expected.sorted, sink.getRetractResults.sorted) } @@ -1043,93 +1103,6 @@ class AggregateITCase( assertEquals(expected.sorted, sink.getRetractResults.sorted) } - @Test - def testTimestampDistinct(): Unit = { - val data = new mutable.MutableList[Row] - data.+=(Row.of(localDateTime("1970-01-01 00:00:01"), Long.box(1L), "A")) - data.+=(Row.of(localDateTime("1970-01-01 00:00:02"), Long.box(2L), "B")) - data.+=(Row.of(localDateTime("1970-01-01 00:00:03"), Long.box(2L), "B")) - data.+=(Row.of(localDateTime("1970-01-01 00:00:04"), Long.box(3L), "C")) - data.+=(Row.of(localDateTime("1970-01-01 00:00:05"), Long.box(3L), "C")) - data.+=(Row.of(localDateTime("1970-01-01 00:00:06"), Long.box(3L), "C")) - data.+=(Row.of(localDateTime("1970-01-01 00:00:07"), Long.box(4L), "B")) - data.+=(Row.of(localDateTime("1970-01-01 00:00:08"), Long.box(4L), "A")) - data.+=(Row.of(localDateTime("1970-01-01 00:00:09"), Long.box(4L), "D")) - data.+=(Row.of(localDateTime("1970-01-01 00:00:10"), Long.box(4L), "E")) - data.+=(Row.of(localDateTime("1970-01-01 00:00:11"), Long.box(5L), "A")) - data.+=(Row.of(localDateTime("1970-01-01 00:00:12"), Long.box(5L), "B")) - - val t = failingDataSource(data)(new RowTypeInfo( - Types.LOCAL_DATE_TIME, Types.LONG, Types.STRING)).toTable(tEnv, 'a, 'b, 'c) - tEnv.registerTable("T", t) - val t1 = tEnv.sqlQuery("SELECT b, count(distinct c), count(distinct a) FROM T GROUP BY b") - - val sink = new TestingRetractSink - t1.toRetractStream[Row].addSink(sink) - env.execute() - - val expected = List("1,1,1", "2,1,2", "3,1,3", "4,4,4", "5,2,2") - assertEquals(expected.sorted, sink.getRetractResults.sorted) - } - - @Test - def testDateDistinct(): Unit = { - val data = new mutable.MutableList[Row] - data.+=(Row.of(localDate("1970-01-01"), Long.box(1L), "A")) - data.+=(Row.of(localDate("1970-01-02"), Long.box(2L), "B")) - data.+=(Row.of(localDate("1970-01-03"), Long.box(2L), "B")) - data.+=(Row.of(localDate("1970-01-04"), Long.box(3L), "C")) - data.+=(Row.of(localDate("1970-01-05"), Long.box(3L), "C")) - data.+=(Row.of(localDate("1970-01-06"), Long.box(3L), "C")) - data.+=(Row.of(localDate("1970-01-07"), Long.box(4L), "B")) - data.+=(Row.of(localDate("1970-01-08"), Long.box(4L), "A")) - data.+=(Row.of(localDate("1970-01-09"), Long.box(4L), "D")) - data.+=(Row.of(localDate("1970-01-10"), Long.box(4L), "E")) - data.+=(Row.of(localDate("1970-01-11"), Long.box(5L), "A")) - data.+=(Row.of(localDate("1970-01-12"), Long.box(5L), "B")) - - val t = failingDataSource(data)(new RowTypeInfo( - Types.LOCAL_DATE, Types.LONG, Types.STRING)).toTable(tEnv, 'a, 'b, 'c) - tEnv.registerTable("T", t) - val t1 = tEnv.sqlQuery("SELECT b, count(distinct c), count(distinct a) FROM T GROUP BY b") - - val sink = new TestingRetractSink - t1.toRetractStream[Row].addSink(sink) - env.execute() - - val expected = List("1,1,1", "2,1,2", "3,1,3", "4,4,4", "5,2,2") - assertEquals(expected.sorted, sink.getRetractResults.sorted) - } - - @Test - def testTimeDistinct(): Unit = { - val data = new mutable.MutableList[Row] - data.+=(Row.of(mLocalTime("00:00:01"), Long.box(1L), "A")) - data.+=(Row.of(mLocalTime("00:00:02"), Long.box(2L), "B")) - data.+=(Row.of(mLocalTime("00:00:03"), Long.box(2L), "B")) - data.+=(Row.of(mLocalTime("00:00:04"), Long.box(3L), "C")) - data.+=(Row.of(mLocalTime("00:00:05"), Long.box(3L), "C")) - data.+=(Row.of(mLocalTime("00:00:06"), Long.box(3L), "C")) - data.+=(Row.of(mLocalTime("00:00:07"), Long.box(4L), "B")) - data.+=(Row.of(mLocalTime("00:00:08"), Long.box(4L), "A")) - data.+=(Row.of(mLocalTime("00:00:09"), Long.box(4L), "D")) - data.+=(Row.of(mLocalTime("00:00:10"), Long.box(4L), "E")) - data.+=(Row.of(mLocalTime("00:00:11"), Long.box(5L), "A")) - data.+=(Row.of(mLocalTime("00:00:12"), Long.box(5L), "B")) - - val t = failingDataSource(data)(new RowTypeInfo( - Types.LOCAL_TIME, Types.LONG, Types.STRING)).toTable(tEnv, 'a, 'b, 'c) - tEnv.registerTable("T", t) - val t1 = tEnv.sqlQuery("SELECT b, count(distinct c), count(distinct a) FROM T GROUP BY b") - - val sink = new TestingRetractSink - t1.toRetractStream[Row].addSink(sink) - env.execute() - - val expected = List("1,1,1", "2,1,2", "3,1,3", "4,4,4", "5,2,2") - assertEquals(expected.sorted, sink.getRetractResults.sorted) - } - @Test def testCountDistinctWithBinaryRowSource(): Unit = { // this case is failed before, because of object reuse problem diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/SplitAggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/SplitAggregateITCase.scala index a5f35f51bd61..3581e9a2cffa 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/SplitAggregateITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/SplitAggregateITCase.scala @@ -18,14 +18,17 @@ package org.apache.flink.table.planner.runtime.stream.sql +import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.api.scala._ import org.apache.flink.table.api.config.OptimizerConfigOptions import org.apache.flink.table.api.scala._ +import org.apache.flink.table.api.Types import org.apache.flink.table.planner.runtime.stream.sql.SplitAggregateITCase.PartialAggMode import org.apache.flink.table.planner.runtime.utils.StreamingWithAggTestBase.{AggMode, LocalGlobalOff, LocalGlobalOn} import org.apache.flink.table.planner.runtime.utils.StreamingWithMiniBatchTestBase.MiniBatchOn import org.apache.flink.table.planner.runtime.utils.StreamingWithStateTestBase.{HEAP_BACKEND, ROCKSDB_BACKEND, StateBackendMode} import org.apache.flink.table.planner.runtime.utils.{StreamingWithAggTestBase, TestingRetractSink} +import org.apache.flink.table.planner.utils.DateTimeTestUtil.{localDate, localDateTime, localTime => mLocalTime} import org.apache.flink.types.Row import org.junit.Assert.assertEquals @@ -33,10 +36,13 @@ import org.junit.runner.RunWith import org.junit.runners.Parameterized import org.junit.{Before, Ignore, Test} +import java.lang.{Integer => JInt, Long => JLong} +import java.math.{BigDecimal => JBigDecimal} import java.util import scala.collection.JavaConversions._ -import scala.collection.Seq +import scala.collection.{Seq, mutable} +import scala.util.Random @RunWith(classOf[Parameterized]) class SplitAggregateITCase( @@ -79,6 +85,101 @@ class SplitAggregateITCase( tEnv.registerTable("T", t) } + @Test + def testCountDistinct(): Unit = { + val ids = List( + 1, + 2, 2, + 3, 3, 3, + 4, 4, 4, 4, + 5, 5, 5, 5, 5) + + val dateTimes = List( + "1970-01-01 00:00:01", + "1970-01-01 00:00:02", null, + "1970-01-01 00:00:04", "1970-01-01 00:00:05", "1970-01-01 00:00:06", + "1970-01-01 00:00:07", null, null, "1970-01-01 00:00:10", + + "1970-01-01 00:00:11", "1970-01-01 00:00:11", "1970-01-01 00:00:13", + "1970-01-01 00:00:14", "1970-01-01 00:00:15") + + val dates = List( + "1970-01-01", + "1970-01-02", null, + "1970-01-04", "1970-01-05", "1970-01-06", + "1970-01-07", null, null, "1970-01-10", + "1970-01-11", "1970-01-11", "1970-01-13", "1970-01-14", "1970-01-15") + + val times = List( + "00:00:01", + "00:00:02", null, + "00:00:04", "00:00:05", "00:00:06", + "00:00:07", null, null, "00:00:10", + "00:00:11", "00:00:11", "00:00:13", "00:00:14", "00:00:15") + + val integers = List( + "1", + "2", null, + "4", "5", "6", + "7", null, null, "10", + "11", "11", "13", "14", "15") + + val chars = List( + "A", + "B", null, + "D", "E", "F", + "H", null, null, "K", + "L", "L", "N", "O", "P") + + val data = new mutable.MutableList[Row] + + for (i <- ids.indices) { + val v = integers(i) + val decimal = if (v == null) null else new JBigDecimal(v) + val int = if (v == null) null else JInt.valueOf(v) + val long = if (v == null) null else JLong.valueOf(v) + data.+=(Row.of( + Int.box(ids(i)), localDateTime(dateTimes(i)), localDate(dates(i)), + mLocalTime(times(i)), decimal, int, long, chars(i))) + } + + val inputs = Random.shuffle(data) + + val rowType = new RowTypeInfo( + Types.INT, Types.LOCAL_DATE_TIME, Types.LOCAL_DATE, Types.LOCAL_TIME, + Types.DECIMAL, Types.INT, Types.LONG, Types.STRING) + + val t = failingDataSource(inputs)(rowType).toTable(tEnv, 'id, 'a, 'b, 'c, 'd, 'e, 'f, 'g) + tEnv.createTemporaryView("MyTable", t) + val t1 = tEnv.sqlQuery( + s""" + |SELECT + | id, + | count(distinct a), + | count(distinct b), + | count(distinct c), + | count(distinct d), + | count(distinct e), + | count(distinct f), + | count(distinct g) + |FROM MyTable + |GROUP BY id + """.stripMargin) + + val sink = new TestingRetractSink + t1.toRetractStream[Row].addSink(sink) + env.execute() + + val expected = List( + "1,1,1,1,1,1,1,1", + "2,1,1,1,1,1,1,1", + "3,3,3,3,3,3,3,3", + "4,2,2,2,2,2,2,2", + "5,4,4,4,4,4,4,4") + assertEquals(expected.sorted, sink.getRetractResults.sorted) + } + + @Test def testSingleDistinctAgg(): Unit = { val t1 = tEnv.sqlQuery("SELECT COUNT(DISTINCT c) FROM T") diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/DateTimeTestUtil.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/DateTimeTestUtil.scala index d80fcea5720e..f4e9578c8622 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/DateTimeTestUtil.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/DateTimeTestUtil.scala @@ -28,14 +28,26 @@ import java.time.{LocalDate, LocalDateTime, LocalTime} object DateTimeTestUtil { def localDate(s: String): LocalDate = { - LocalDateConverter.INSTANCE.toExternal(dateStringToUnixDate(s)) + if (s == null) { + null + } else { + LocalDateConverter.INSTANCE.toExternal(dateStringToUnixDate(s)) + } } def localTime(s: String): LocalTime = { - LocalTimeConverter.INSTANCE.toExternal(DateTimeUtils.timeStringToUnixDate(s)) + if (s == null) { + null + } else { + LocalTimeConverter.INSTANCE.toExternal(DateTimeUtils.timeStringToUnixDate(s)) + } } def localDateTime(s: String): LocalDateTime = { - LocalDateTimeConverter.INSTANCE.toExternal(DateTimeUtils.timestampStringToUnixDate(s)) + if (s == null) { + null + } else { + LocalDateTimeConverter.INSTANCE.toExternal(DateTimeUtils.timestampStringToUnixDate(s)) + } } } From 1da3055338e5f23a3ec4a54152dbf6b4e35e3862 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Wed, 30 Oct 2019 20:11:52 +0800 Subject: [PATCH 355/746] [FLINK-13869][table-planner-blink][hive] Fix Hive functions can not work in blink planner streaming mode (#10013) --- .../hive/HiveCatalogUseBlinkITCase.java | 163 +++++++++++------- .../functions/utils/AggSqlFunction.scala | 6 +- .../functions/utils/ScalarSqlFunction.scala | 4 + .../functions/utils/TableSqlFunction.scala | 3 +- .../planner/plan/utils/AggregateUtil.scala | 2 +- 5 files changed, 116 insertions(+), 62 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogUseBlinkITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogUseBlinkITCase.java index d61012579f91..343c8c95c7f8 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogUseBlinkITCase.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogUseBlinkITCase.java @@ -18,12 +18,17 @@ package org.apache.flink.table.catalog.hive; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.connectors.hive.FlinkStandaloneHiveRunner; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.Types; +import org.apache.flink.table.api.java.StreamTableEnvironment; import org.apache.flink.table.catalog.CatalogFunctionImpl; import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.CatalogTableBuilder; @@ -35,7 +40,10 @@ import org.apache.flink.table.functions.hive.util.TestHiveSimpleUDF; import org.apache.flink.table.functions.hive.util.TestHiveUDTF; import org.apache.flink.table.planner.runtime.utils.BatchTestBase; +import org.apache.flink.table.planner.runtime.utils.TestingRetractSink; +import org.apache.flink.table.util.JavaScalaConversionUtil; import org.apache.flink.test.util.AbstractTestBase; +import org.apache.flink.types.Row; import org.apache.flink.util.FileUtils; import com.klarna.hiverunner.HiveShell; @@ -54,6 +62,7 @@ import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; +import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.List; @@ -96,14 +105,6 @@ public static void closeCatalog() { @Test public void testBlinkUdf() throws Exception { - TableEnvironment tEnv = TableEnvironment.create( - EnvironmentSettings.newInstance().useBlinkPlanner().inBatchMode().build()); - - BatchTestBase.configForMiniCluster(tEnv.getConfig()); - - tEnv.registerCatalog("myhive", hiveCatalog); - tEnv.useCatalog("myhive"); - TableSchema schema = TableSchema.builder() .field("name", DataTypes.STRING()) .field("age", DataTypes.INT()) @@ -122,41 +123,12 @@ public void testBlinkUdf() throws Exception { .withComment("Comment.") .build(); - Path p = Paths.get(tempFolder.newFolder().getAbsolutePath(), "test.csv"); - - TableSchema sinkSchema = TableSchema.builder() - .field("name1", Types.STRING()) - .field("name2", Types.STRING()) - .field("sum1", Types.INT()) - .field("sum2", Types.LONG()) - .build(); - - FormatDescriptor sinkFormat = new OldCsv() - .field("name1", Types.STRING()) - .field("name2", Types.STRING()) - .field("sum1", Types.INT()) - .field("sum2", Types.LONG()); - CatalogTable sink = - new CatalogTableBuilder( - new FileSystem().path(p.toAbsolutePath().toString()), - sinkSchema) - .withFormat(sinkFormat) - .inAppendMode() - .withComment("Comment.") - .build(); - hiveCatalog.createTable( new ObjectPath(HiveCatalog.DEFAULT_DB, sourceTableName), source, false ); - hiveCatalog.createTable( - new ObjectPath(HiveCatalog.DEFAULT_DB, sinkTableName), - sink, - false - ); - hiveCatalog.createFunction( new ObjectPath(HiveCatalog.DEFAULT_DB, "myudf"), new CatalogFunctionImpl(TestHiveSimpleUDF.class.getCanonicalName(), new HashMap<>()), @@ -174,34 +146,107 @@ public void testBlinkUdf() throws Exception { new CatalogFunctionImpl(GenericUDAFSum.class.getCanonicalName(), new HashMap<>()), false); + testUdf(true); + testUdf(false); + } + + private void testUdf(boolean batch) throws Exception { + TableEnvironment tEnv; + EnvironmentSettings.Builder envBuilder = EnvironmentSettings.newInstance().useBlinkPlanner(); + if (batch) { + envBuilder.inBatchMode(); + } else { + envBuilder.inStreamingMode(); + } + if (batch) { + tEnv = TableEnvironment.create(envBuilder.build()); + } else { + tEnv = StreamTableEnvironment.create( + StreamExecutionEnvironment.getExecutionEnvironment(), envBuilder.build()); + } + + BatchTestBase.configForMiniCluster(tEnv.getConfig()); + + tEnv.registerCatalog("myhive", hiveCatalog); + tEnv.useCatalog("myhive"); + String innerSql = format("select mygenericudf(myudf(name), 1) as a, mygenericudf(myudf(age), 1) as b," + " s from %s, lateral table(myudtf(name, 1)) as T(s)", sourceTableName); - tEnv.sqlUpdate( - format("insert into %s select a, s, sum(b), myudaf(b) from (%s) group by a, s", - sinkTableName, - innerSql)); - tEnv.execute("myjob"); - - // assert written result - StringBuilder builder = new StringBuilder(); - try (Stream paths = Files.walk(Paths.get(p.toAbsolutePath().toString()))) { - paths.filter(Files::isRegularFile).forEach(path -> { - try { - String content = FileUtils.readFileUtf8(path.toFile()); - if (content.isEmpty()) { - return; + String selectSql = format("select a, s, sum(b), myudaf(b) from (%s) group by a, s", innerSql); + + List results; + if (batch) { + Path p = Paths.get(tempFolder.newFolder().getAbsolutePath(), "test.csv"); + + TableSchema sinkSchema = TableSchema.builder() + .field("name1", Types.STRING()) + .field("name2", Types.STRING()) + .field("sum1", Types.INT()) + .field("sum2", Types.LONG()) + .build(); + + FormatDescriptor sinkFormat = new OldCsv() + .field("name1", Types.STRING()) + .field("name2", Types.STRING()) + .field("sum1", Types.INT()) + .field("sum2", Types.LONG()); + CatalogTable sink = + new CatalogTableBuilder( + new FileSystem().path(p.toAbsolutePath().toString()), + sinkSchema) + .withFormat(sinkFormat) + .inAppendMode() + .withComment("Comment.") + .build(); + + hiveCatalog.createTable( + new ObjectPath(HiveCatalog.DEFAULT_DB, sinkTableName), + sink, + false + ); + + tEnv.sqlUpdate(format("insert into %s " + selectSql, sinkTableName)); + tEnv.execute("myjob"); + + // assert written result + StringBuilder builder = new StringBuilder(); + try (Stream paths = Files.walk(Paths.get(p.toAbsolutePath().toString()))) { + paths.filter(Files::isRegularFile).forEach(path -> { + try { + String content = FileUtils.readFileUtf8(path.toFile()); + if (content.isEmpty()) { + return; + } + builder.append(content); + } catch (IOException e) { + throw new RuntimeException(e); } - builder.append(content); - } catch (IOException e) { - throw new RuntimeException(e); - } - }); + }); + } + results = Arrays.stream(builder.toString().split("\n")) + .filter(s -> !s.isEmpty()) + .collect(Collectors.toList()); + } else { + StreamTableEnvironment streamTEnv = (StreamTableEnvironment) tEnv; + TestingRetractSink sink = new TestingRetractSink(); + streamTEnv.toRetractStream(tEnv.sqlQuery(selectSql), Row.class) + .map(new JavaToScala()) + .addSink((SinkFunction) sink); + streamTEnv.execute(""); + results = JavaScalaConversionUtil.toJava(sink.getRetractResults()); } - List results = Arrays.stream(builder.toString().split("\n")) - .filter(s -> !s.isEmpty()) - .collect(Collectors.toList()); + + results = new ArrayList<>(results); results.sort(String::compareTo); Assert.assertEquals(Arrays.asList("1,1,2,2", "2,2,4,4", "3,3,6,6"), results); } + + private static class JavaToScala implements MapFunction, scala.Tuple2> { + + @Override + public scala.Tuple2 map(Tuple2 value) throws Exception { + return new scala.Tuple2<>(value.f0, value.f1); + } + } } 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 7effbba7e501..5213929ecb55 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 @@ -51,7 +51,7 @@ import java.util class AggSqlFunction( name: String, displayName: String, - aggregateFunction: UserDefinedAggregateFunction[_, _], + val aggregateFunction: UserDefinedAggregateFunction[_, _], val externalResultType: DataType, val externalAccType: DataType, typeFactory: FlinkTypeFactory, @@ -72,6 +72,10 @@ class AggSqlFunction( typeFactory ) { + /** + * This is temporary solution for hive udf and should be removed once FLIP-65 is finished, + * please pass the non-null input arguments. + */ def makeFunction( constants: Array[AnyRef], argTypes: Array[LogicalType]): UserDefinedAggregateFunction[_, _] = aggregateFunction 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 8e5609cacabc..f036cbea952f 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 @@ -58,6 +58,10 @@ class ScalarSqlFunction( null, SqlFunctionCategory.USER_DEFINED_FUNCTION) { + /** + * This is temporary solution for hive udf and should be removed once FLIP-65 is finished, + * please pass the non-null input arguments. + */ def makeFunction(constants: Array[AnyRef], argTypes: Array[LogicalType]): ScalarFunction = scalarFunction diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/TableSqlFunction.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/TableSqlFunction.scala index c3f5ac32c6fb..4cb1d53c7660 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/TableSqlFunction.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/TableSqlFunction.scala @@ -67,7 +67,8 @@ class TableSqlFunction( functionImpl) { /** - * Get the user-defined table function. + * This is temporary solution for hive udf and should be removed once FLIP-65 is finished, + * please pass the non-null input arguments. */ def makeFunction(constants: Array[AnyRef], argTypes: Array[LogicalType]): TableFunction[_] = udtf diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala index 92a5d3d8b6ad..973a6dab82fa 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala @@ -749,7 +749,7 @@ object AggregateUtil extends Enumeration { private[flink] def isTableAggregate(aggCalls: util.List[AggregateCall]): Boolean = { aggCalls .filter(e => e.getAggregation.isInstanceOf[AggSqlFunction]) - .map(e => e.getAggregation.asInstanceOf[AggSqlFunction].makeFunction(null, null)) + .map(e => e.getAggregation.asInstanceOf[AggSqlFunction].aggregateFunction) .exists(_.isInstanceOf[TableAggregateFunction[_, _]]) } } From ef0dc509a42a36d22286c64751ea760d538404ec Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Wed, 30 Oct 2019 21:07:53 +0800 Subject: [PATCH 356/746] [FLINK-14253][table-planner-blink] Add hash distribution and sort grouping only when dynamic partition insert This closes #9796 --- .../calcite/RelTimeIndicatorConverter.scala | 3 +- .../planner/delegation/PlannerBase.scala | 14 +-- .../plan/nodes/calcite/LogicalSink.scala | 10 +- .../plan/nodes/logical/FlinkLogicalSink.scala | 13 ++- .../physical/batch/BatchExecSinkRule.scala | 25 +++-- .../physical/stream/StreamExecSinkRule.scala | 25 +++-- .../plan/batch/sql/PartitionableSinkTest.xml | 100 ++++++++++++++++++ .../plan/stream/sql/PartitionableSinkTest.xml | 98 +++++++++++++++++ .../batch/sql/PartitionableSinkTest.scala | 70 ++++++++++++ .../stream/sql/PartitionableSinkTest.scala | 70 ++++++++++++ .../batch/sql/PartitionableSinkITCase.scala | 67 +++++++----- .../table/planner/utils/TableTestBase.scala | 40 +++++-- 12 files changed, 461 insertions(+), 74 deletions(-) create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/PartitionableSinkTest.xml create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/PartitionableSinkTest.xml create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/PartitionableSinkTest.scala create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/PartitionableSinkTest.scala diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.scala index 9354543ddc8b..6866de59642c 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.scala @@ -185,7 +185,8 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle { newInput, sink.sink, sink.sinkName, - sink.catalogTable) + sink.catalogTable, + sink.staticPartitions) case _ => throw new TableException(s"Unsupported logical operator: ${other.getClass.getSimpleName}") diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala index 691910b47692..0e9436dc6b17 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala @@ -40,7 +40,7 @@ import org.apache.flink.table.planner.plan.reuse.SubplanReuser import org.apache.flink.table.planner.plan.utils.SameRelObjectShuttle import org.apache.flink.table.planner.sinks.{DataStreamTableSink, TableSinkUtils} import org.apache.flink.table.planner.utils.JavaScalaConversionUtil -import org.apache.flink.table.sinks.{OverwritableTableSink, PartitionableTableSink, TableSink} +import org.apache.flink.table.sinks.{OverwritableTableSink, TableSink} import org.apache.flink.table.types.utils.LegacyTypeInfoDataTypeConverter import org.apache.calcite.jdbc.CalciteSchemaBuilder.asRootSchema @@ -175,11 +175,6 @@ abstract class PlannerBase( val identifier = catalogSink.getTableIdentifier getTableSink(identifier).map { case (table, sink) => TableSinkUtils.validateSink(catalogSink, identifier, sink, table.getPartitionKeys) - sink match { - case partitionableSink: PartitionableTableSink => - partitionableSink.setStaticPartition(catalogSink.getStaticPartitions) - case _ => - } sink match { case overwritableTableSink: OverwritableTableSink => overwritableTableSink.setOverwrite(catalogSink.isOverwrite) @@ -188,7 +183,12 @@ abstract class PlannerBase( s"${classOf[OverwritableTableSink].getSimpleName} but actually got " + sink.getClass.getName) } - LogicalSink.create(input, sink, identifier.toString, table) + LogicalSink.create( + input, + sink, + identifier.toString, + table, + catalogSink.getStaticPartitions.toMap) } match { case Some(sinkRel) => sinkRel case None => diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalSink.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalSink.scala index cbd64ba991eb..96afefb964ae 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalSink.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalSink.scala @@ -39,12 +39,13 @@ final class LogicalSink( input: RelNode, sink: TableSink[_], sinkName: String, - val catalogTable: CatalogTable) + val catalogTable: CatalogTable, + val staticPartitions: Map[String, String]) extends Sink(cluster, traitSet, input, sink, sinkName) { override def copy(traitSet: RelTraitSet, inputs: util.List[RelNode]): RelNode = { new LogicalSink( - cluster, traitSet, inputs.head, sink, sinkName, catalogTable) + cluster, traitSet, inputs.head, sink, sinkName, catalogTable, staticPartitions) } } @@ -54,9 +55,10 @@ object LogicalSink { def create(input: RelNode, sink: TableSink[_], sinkName: String, - catalogTable: CatalogTable = null): LogicalSink = { + catalogTable: CatalogTable = null, + staticPartitions: Map[String, String] = Map()): LogicalSink = { val traits = input.getCluster.traitSetOf(Convention.NONE) new LogicalSink( - input.getCluster, traits, input, sink, sinkName, catalogTable) + input.getCluster, traits, input, sink, sinkName, catalogTable, staticPartitions) } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalSink.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalSink.scala index a1c96b7673c1..e12428468fc1 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalSink.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalSink.scala @@ -41,13 +41,14 @@ class FlinkLogicalSink( input: RelNode, sink: TableSink[_], sinkName: String, - val catalogTable: CatalogTable) + val catalogTable: CatalogTable, + val staticPartitions: Map[String, String]) extends Sink(cluster, traitSet, input, sink, sinkName) with FlinkLogicalRel { override def copy(traitSet: RelTraitSet, inputs: util.List[RelNode]): RelNode = { new FlinkLogicalSink( - cluster, traitSet, inputs.head, sink, sinkName, catalogTable) + cluster, traitSet, inputs.head, sink, sinkName, catalogTable, staticPartitions) } } @@ -66,7 +67,8 @@ private class FlinkLogicalSinkConverter newInput, sink.sink, sink.sinkName, - sink.catalogTable) + sink.catalogTable, + sink.staticPartitions) } } @@ -77,10 +79,11 @@ object FlinkLogicalSink { input: RelNode, sink: TableSink[_], sinkName: String, - catalogTable: CatalogTable = null): FlinkLogicalSink = { + catalogTable: CatalogTable = null, + staticPartitions: Map[String, String] = Map()): FlinkLogicalSink = { val cluster = input.getCluster val traitSet = cluster.traitSetOf(FlinkConventions.LOGICAL).simplify() new FlinkLogicalSink( - cluster, traitSet, input, sink, sinkName, catalogTable) + cluster, traitSet, input, sink, sinkName, catalogTable, staticPartitions) } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSinkRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSinkRule.scala index beb52a6e60ab..959dc03a7b23 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSinkRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSinkRule.scala @@ -45,18 +45,23 @@ class BatchExecSinkRule extends ConverterRule( if (sinkNode.catalogTable != null && sinkNode.catalogTable.isPartitioned) { sinkNode.sink match { case partitionSink: PartitionableTableSink => - val partKeys = sinkNode.catalogTable.getPartitionKeys - val partitionIndices = - partKeys.map(partitionSink.getTableSchema.getFieldNames.indexOf(_)) + partitionSink.setStaticPartition(sinkNode.staticPartitions) + val dynamicPartFields = sinkNode.catalogTable.getPartitionKeys + .filter(!sinkNode.staticPartitions.contains(_)) - requiredTraitSet = requiredTraitSet.plus( - FlinkRelDistribution.hash(partitionIndices - .map(Integer.valueOf), requireStrict = false)) + if (dynamicPartFields.nonEmpty) { + val dynamicPartIndices = + dynamicPartFields.map(partitionSink.getTableSchema.getFieldNames.indexOf(_)) - if (partitionSink.configurePartitionGrouping(true)) { - // default to asc. - val fieldCollations = partitionIndices.map(FlinkRelOptUtil.ofRelFieldCollation) - requiredTraitSet = requiredTraitSet.plus(RelCollations.of(fieldCollations: _*)) + requiredTraitSet = requiredTraitSet.plus( + FlinkRelDistribution.hash(dynamicPartIndices + .map(Integer.valueOf), requireStrict = false)) + + if (partitionSink.configurePartitionGrouping(true)) { + // default to asc. + val fieldCollations = dynamicPartIndices.map(FlinkRelOptUtil.ofRelFieldCollation) + requiredTraitSet = requiredTraitSet.plus(RelCollations.of(fieldCollations: _*)) + } } case _ => throw new TableException("We need PartitionableTableSink to write data to" + s" partitioned table: ${sinkNode.sinkName}") diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecSinkRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecSinkRule.scala index 7280e4822375..c66774cdf6ff 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecSinkRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecSinkRule.scala @@ -45,18 +45,23 @@ class StreamExecSinkRule extends ConverterRule( if (sinkNode.catalogTable != null && sinkNode.catalogTable.isPartitioned) { sinkNode.sink match { case partitionSink: PartitionableTableSink => - val partKeys = sinkNode.catalogTable.getPartitionKeys - val partitionIndices = partKeys - .map(partitionSink.getTableSchema.getFieldNames.indexOf(_)) + partitionSink.setStaticPartition(sinkNode.staticPartitions) + val dynamicPartFields = sinkNode.catalogTable.getPartitionKeys + .filter(!sinkNode.staticPartitions.contains(_)) - if (partitionSink.configurePartitionGrouping(false)) { - throw new TableException("Partition grouping in stream mode is not supported yet!") - } + if (dynamicPartFields.nonEmpty) { + val dynamicPartIndices = + dynamicPartFields.map(partitionSink.getTableSchema.getFieldNames.indexOf(_)) + + if (partitionSink.configurePartitionGrouping(false)) { + throw new TableException("Partition grouping in stream mode is not supported yet!") + } - if (!partitionSink.isInstanceOf[DataStreamTableSink[_]]) { - requiredTraitSet = requiredTraitSet.plus( - FlinkRelDistribution.hash(partitionIndices - .map(Integer.valueOf), requireStrict = false)) + if (!partitionSink.isInstanceOf[DataStreamTableSink[_]]) { + requiredTraitSet = requiredTraitSet.plus( + FlinkRelDistribution.hash(dynamicPartIndices + .map(Integer.valueOf), requireStrict = false)) + } } case _ => throw new TableException("We need PartitionableTableSink to write data to" + s" partitioned table: ${sinkNode.sinkName}") diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/PartitionableSinkTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/PartitionableSinkTest.xml new file mode 100644 index 000000000000..ff2a13bfde6e --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/PartitionableSinkTest.xml @@ -0,0 +1,100 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/PartitionableSinkTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/PartitionableSinkTest.xml new file mode 100644 index 000000000000..0df21a8b3f2b --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/PartitionableSinkTest.xml @@ -0,0 +1,98 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/PartitionableSinkTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/PartitionableSinkTest.scala new file mode 100644 index 000000000000..0fd5626debe0 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/PartitionableSinkTest.scala @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.batch.sql + +import org.apache.flink.api.common.typeinfo.{TypeInformation, Types} +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.scala._ +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.api.{SqlDialect, TableConfig, ValidationException} +import org.apache.flink.table.planner.runtime.batch.sql.PartitionableSinkITCase +import org.apache.flink.table.planner.utils.TableTestBase + +import org.junit.Test + +class PartitionableSinkTest extends TableTestBase { + + val conf = new TableConfig + conf.setSqlDialect(SqlDialect.HIVE) + private val util = batchTestUtil(conf) + util.addTableSource[(Long, Long, Long)]("MyTable", 'a, 'b, 'c) + PartitionableSinkITCase.registerTableSink( + util.tableEnv, + "sink", + new RowTypeInfo( + Array[TypeInformation[_]](Types.LONG, Types.LONG, Types.LONG), + Array("a", "b", "c")), + grouping = true, + Array("b", "c")) + + @Test + def testStatic(): Unit = { + util.verifySqlUpdate("INSERT INTO sink PARTITION (b=1, c=1) SELECT a FROM MyTable") + } + + @Test + def testDynamic(): Unit = { + util.verifySqlUpdate("INSERT INTO sink SELECT a, b, c FROM MyTable") + } + + @Test + def testPartial(): Unit = { + util.verifySqlUpdate("INSERT INTO sink PARTITION (b=1) SELECT a, c FROM MyTable") + } + + @Test(expected = classOf[ValidationException]) + def testWrongStatic(): Unit = { + util.verifySqlUpdate("INSERT INTO sink PARTITION (a=1) SELECT b, c FROM MyTable") + } + + @Test(expected = classOf[ValidationException]) + def testWrongFields(): Unit = { + util.verifySqlUpdate("INSERT INTO sink PARTITION (b=1) SELECT a, b, c FROM MyTable") + } +} diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/PartitionableSinkTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/PartitionableSinkTest.scala new file mode 100644 index 000000000000..b1e084156e83 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/PartitionableSinkTest.scala @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.stream.sql + +import org.apache.flink.api.common.typeinfo.{TypeInformation, Types} +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.scala._ +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.api.{SqlDialect, TableConfig, ValidationException} +import org.apache.flink.table.planner.runtime.batch.sql.PartitionableSinkITCase +import org.apache.flink.table.planner.utils.TableTestBase + +import org.junit.Test + +class PartitionableSinkTest extends TableTestBase { + + val conf = new TableConfig + conf.setSqlDialect(SqlDialect.HIVE) + private val util = streamTestUtil(conf) + util.addTableSource[(Long, Long, Long)]("MyTable", 'a, 'b, 'c) + PartitionableSinkITCase.registerTableSink( + util.tableEnv, + "sink", + new RowTypeInfo( + Array[TypeInformation[_]](Types.LONG, Types.LONG, Types.LONG), + Array("a", "b", "c")), + grouping = false, + Array("b", "c")) + + @Test + def testStatic(): Unit = { + util.verifySqlUpdate("INSERT INTO sink PARTITION (b=1, c=1) SELECT a FROM MyTable") + } + + @Test + def testDynamic(): Unit = { + util.verifySqlUpdate("INSERT INTO sink SELECT a, b, c FROM MyTable") + } + + @Test + def testPartial(): Unit = { + util.verifySqlUpdate("INSERT INTO sink PARTITION (b=1) SELECT a, c FROM MyTable") + } + + @Test(expected = classOf[ValidationException]) + def testWrongStatic(): Unit = { + util.verifySqlUpdate("INSERT INTO sink PARTITION (a=1) SELECT b, c FROM MyTable") + } + + @Test(expected = classOf[ValidationException]) + def testWrongFields(): Unit = { + util.verifySqlUpdate("INSERT INTO sink PARTITION (b=1) SELECT a, b, c FROM MyTable") + } +} diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala index dacc0fd7f38c..86d3940b668a 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala @@ -26,7 +26,7 @@ import org.apache.flink.streaming.api.datastream.{DataStream, DataStreamSink} import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import org.apache.flink.streaming.api.functions.sink.RichSinkFunction import org.apache.flink.table.api.config.ExecutionConfigOptions -import org.apache.flink.table.api.{SqlDialect, TableException, TableSchema, ValidationException} +import org.apache.flink.table.api.{SqlDialect, TableEnvironment, TableException, TableSchema, ValidationException} import org.apache.flink.table.catalog.{CatalogTableImpl, ObjectPath} import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE import org.apache.flink.table.descriptors.DescriptorProperties @@ -148,19 +148,19 @@ class PartitionableSinkITCase extends BatchTestBase { registerTableSink(partitionColumns = Array("a", "b")) tEnv.sqlUpdate("insert into sinkTable partition(a=1) select b, c from sortTable") tEnv.execute("testJob") - assertEquals(List("1,3,I'm fine, thank", - "1,3,I'm fine, thank you", - "1,3,I'm fine, thank you, and you?"), - RESULT1.toList) - assertEquals(List("1,2,Hi", - "1,2,Hello"), - RESULT2.toList) - assertEquals(List("1,1,Hello world", - "1,1,Hello world, how are you?", + assertEquals(List("1,1,Hello world", "1,1,Hello world, how are you?"), RESULT1.toList) + assertEquals(List( "1,4,你好,陌生人", "1,4,你好,陌生人,我是", "1,4,你好,陌生人,我是中国人", "1,4,你好,陌生人,我是中国人,你来自哪里?"), + RESULT2.toList) + assertEquals(List( + "1,2,Hi", + "1,2,Hello", + "1,3,I'm fine, thank", + "1,3,I'm fine, thank you", + "1,3,I'm fine, thank you, and you?"), RESULT3.toList) } @@ -186,21 +186,7 @@ class PartitionableSinkITCase extends BatchTestBase { rowType: RowTypeInfo = type3, grouping: Boolean = true, partitionColumns: Array[String] = Array[String]("a")): Unit = { - val properties = new DescriptorProperties() - properties.putString("supports-grouping", grouping.toString) - properties.putString(CONNECTOR_TYPE, "TestPartitionableSink") - partitionColumns.zipWithIndex.foreach { case (part, i) => - properties.putString("partition-column." + i, part) - } - - val table = new CatalogTableImpl( - new TableSchema(Array("a", "b", "c"), rowType.getFieldTypes), - util.Arrays.asList[String](partitionColumns: _*), - properties.asMap(), - "" - ) - tEnv.getCatalog(tEnv.getCurrentCatalog).get() - .createTable(new ObjectPath(tEnv.getCurrentDatabase, tableName), table, false) + PartitionableSinkITCase.registerTableSink(tEnv, tableName, rowType, grouping, partitionColumns) } } @@ -272,6 +258,29 @@ object PartitionableSinkITCase { row(4, 4L, "你好,陌生人,我是中国人"), row(4, 4L, "你好,陌生人,我是中国人,你来自哪里?") ) + + def registerTableSink( + tEnv: TableEnvironment, + tableName: String, + rowType: RowTypeInfo, + grouping: Boolean, + partitionColumns: Array[String]): Unit = { + val properties = new DescriptorProperties() + properties.putString("supports-grouping", grouping.toString) + properties.putString(CONNECTOR_TYPE, "TestPartitionableSink") + partitionColumns.zipWithIndex.foreach { case (part, i) => + properties.putString("partition-column." + i, part) + } + + val table = new CatalogTableImpl( + new TableSchema(Array("a", "b", "c"), rowType.getFieldTypes), + util.Arrays.asList[String](partitionColumns: _*), + properties.asMap(), + "" + ) + tEnv.getCatalog(tEnv.getCurrentCatalog).get() + .createTable(new ObjectPath(tEnv.getCurrentDatabase, tableName), table, false) + } } private class TestSink( @@ -293,18 +302,18 @@ private class TestSink( } override def getTableSchema: TableSchema = { - new TableSchema(Array("a", "b", "c"), type3.getFieldTypes) + new TableSchema(Array("a", "b", "c"), rowType.getFieldTypes) } - override def getOutputType: RowTypeInfo = type3 + override def getOutputType: RowTypeInfo = rowType override def emitDataStream(dataStream: DataStream[Row]): Unit = { - dataStream.addSink(new UnsafeMemorySinkFunction(type3)) + dataStream.addSink(new UnsafeMemorySinkFunction(rowType)) .setParallelism(dataStream.getParallelism) } override def consumeDataStream(dataStream: DataStream[Row]): DataStreamSink[_] = { - dataStream.addSink(new UnsafeMemorySinkFunction(type3)) + dataStream.addSink(new UnsafeMemorySinkFunction(rowType)) .setParallelism(dataStream.getParallelism) } 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 c55cd32bb725..ec2b8896e821 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 @@ -87,13 +87,15 @@ abstract class TableTestBase { @Rule def name: TestName = testName - def streamTestUtil(): StreamTableTestUtil = StreamTableTestUtil(this) + def streamTestUtil(conf: TableConfig = new TableConfig): StreamTableTestUtil = + StreamTableTestUtil(this, conf = conf) def scalaStreamTestUtil(): ScalaStreamTableTestUtil = ScalaStreamTableTestUtil(this) def javaStreamTestUtil(): JavaStreamTableTestUtil = JavaStreamTableTestUtil(this) - def batchTestUtil(): BatchTableTestUtil = BatchTableTestUtil(this) + def batchTestUtil(conf: TableConfig = new TableConfig): BatchTableTestUtil = + BatchTableTestUtil(this, conf = conf) def scalaBatchTestUtil(): ScalaBatchTableTestUtil = ScalaBatchTableTestUtil(this) @@ -478,9 +480,9 @@ abstract class TableTestUtil( test: TableTestBase, // determines if the table environment should work in a batch or streaming mode isStreamingMode: Boolean, - catalogManager: Option[CatalogManager] = None) + catalogManager: Option[CatalogManager] = None, + val tableConfig: TableConfig) extends TableTestUtilBase(test, isStreamingMode) { - protected val tableConfig: TableConfig = new TableConfig protected val testingTableEnv: TestingTableEnvironment = TestingTableEnvironment.create(setting, catalogManager, tableConfig) val tableEnv: TableEnvironment = testingTableEnv @@ -558,6 +560,15 @@ abstract class TableTestUtil( testingTableEnv.registerFunction(name, function) } + def verifySqlUpdate(sql: String): Unit = { + doVerifySqlUpdate( + sql, + SqlExplainLevel.DIGEST_ATTRIBUTES, + withRowType = false, + withRetractTraits = false, + printPlanBefore = true) + } + def verifyPlan(): Unit = { doVerifyPlan( SqlExplainLevel.DIGEST_ATTRIBUTES, @@ -566,6 +577,17 @@ abstract class TableTestUtil( printPlanBefore = true) } + def doVerifySqlUpdate( + sql: String, + explainLevel: SqlExplainLevel, + withRowType: Boolean, + withRetractTraits: Boolean, + printPlanBefore: Boolean): Unit = { + tableEnv.sqlUpdate(sql) + assertEqualsOrExpand("sql", sql) + doVerifyPlan(explainLevel, withRowType, withRetractTraits, printPlanBefore) + } + def doVerifyPlan( explainLevel: SqlExplainLevel, withRowType: Boolean, @@ -675,8 +697,9 @@ abstract class JavaTableTestUtil( */ case class StreamTableTestUtil( test: TableTestBase, - catalogManager: Option[CatalogManager] = None) - extends TableTestUtil(test, isStreamingMode = true, catalogManager) { + catalogManager: Option[CatalogManager] = None, + conf: TableConfig = new TableConfig) + extends TableTestUtil(test, isStreamingMode = true, catalogManager, conf) { /** * Register a table with specific row time field and offset. @@ -814,8 +837,9 @@ case class JavaStreamTableTestUtil(test: TableTestBase) extends JavaTableTestUti */ case class BatchTableTestUtil( test: TableTestBase, - catalogManager: Option[CatalogManager] = None) - extends TableTestUtil(test, isStreamingMode = false, catalogManager) { + catalogManager: Option[CatalogManager] = None, + conf: TableConfig = new TableConfig) + extends TableTestUtil(test, isStreamingMode = false, catalogManager, conf) { def buildBatchProgram(firstProgramNameToRemove: String): Unit = { val program = FlinkBatchProgram.buildProgram(tableEnv.getConfig.getConfiguration) From 2352586a4786d9f67b48f068088abb7749a18a5f Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 25 Oct 2019 14:07:07 +0200 Subject: [PATCH 357/746] [FLINK-14532][coordination] Rename PartitionTracker --- .../flink/runtime/executiongraph/Execution.java | 6 +++--- .../runtime/executiongraph/ExecutionGraph.java | 12 ++++++------ .../executiongraph/ExecutionGraphBuilder.java | 6 +++--- ...nGraphResultPartitionAvailabilityChecker.java | 6 +++--- ...acker.java => JobMasterPartitionTracker.java} | 2 +- ...l.java => JobMasterPartitionTrackerImpl.java} | 4 ++-- .../partition/PartitionTrackerFactory.java | 4 ++-- .../flink/runtime/jobmaster/JobMaster.java | 4 ++-- .../DefaultJobMasterServiceFactory.java | 4 ++-- .../runtime/scheduler/DefaultScheduler.java | 4 ++-- .../scheduler/DefaultSchedulerFactory.java | 4 ++-- .../flink/runtime/scheduler/LegacyScheduler.java | 4 ++-- .../scheduler/LegacySchedulerFactory.java | 4 ++-- .../flink/runtime/scheduler/SchedulerBase.java | 8 ++++---- .../runtime/scheduler/SchedulerNGFactory.java | 4 ++-- .../CheckpointSettingsSerializableTest.java | 4 ++-- ...edRegionStrategyNGConcurrentFailoverTest.java | 6 +++--- ...artPipelinedRegionStrategyNGFailoverTest.java | 6 +++--- .../ExecutionGraphDeploymentTest.java | 4 ++-- .../ExecutionGraphPartitionReleaseTest.java | 10 +++++----- .../ExecutionGraphRescalingTest.java | 10 +++++----- ...phResultPartitionAvailabilityCheckerTest.java | 4 ++-- .../ExecutionGraphSchedulingTest.java | 4 ++-- .../executiongraph/ExecutionGraphTestUtils.java | 8 ++++---- .../ExecutionPartitionLifecycleTest.java | 12 ++++++------ .../ExecutionVertexLocalityTest.java | 4 ++-- .../executiongraph/ExecutionVertexTest.java | 4 ++-- ...va => JobMasterPartitionTrackerImplTest.java} | 14 +++++++------- ...r.java => NoOpJobMasterPartitionTracker.java} | 4 ++-- ...ava => TestingJobMasterPartitionTracker.java} | 4 ++-- .../flink/runtime/jobmaster/JobMasterTest.java | 16 ++++++++-------- .../runtime/scheduler/DefaultSchedulerTest.java | 4 ++-- .../LegacySchedulerBatchSchedulingTest.java | 4 ++-- 33 files changed, 99 insertions(+), 99 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/{PartitionTracker.java => JobMasterPartitionTracker.java} (98%) rename flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/{PartitionTrackerImpl.java => JobMasterPartitionTrackerImpl.java} (98%) rename flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/{PartitionTrackerImplTest.java => JobMasterPartitionTrackerImplTest.java} (95%) rename flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/{NoOpPartitionTracker.java => NoOpJobMasterPartitionTracker.java} (93%) rename flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/{TestingPartitionTracker.java => TestingJobMasterPartitionTracker.java} (96%) 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 595fb2ce9d7b..d99d6a3c1d1b 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 @@ -39,7 +39,7 @@ import org.apache.flink.runtime.deployment.TaskDeploymentDescriptorFactory; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.instance.SlotSharingGroupId; -import org.apache.flink.runtime.io.network.partition.PartitionTracker; +import org.apache.flink.runtime.io.network.partition.JobMasterPartitionTracker; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; @@ -1343,7 +1343,7 @@ private void sendCancelRpcCall(int numberRetries) { } private void startTrackingPartitions(final ResourceID taskExecutorId, final Collection partitions) { - PartitionTracker partitionTracker = vertex.getExecutionGraph().getPartitionTracker(); + JobMasterPartitionTracker partitionTracker = vertex.getExecutionGraph().getPartitionTracker(); for (ResultPartitionDeploymentDescriptor partition : partitions) { partitionTracker.startTrackingPartition( taskExecutorId, @@ -1357,7 +1357,7 @@ void handlePartitionCleanup(boolean releasePipelinedPartitions, boolean releaseB } final Collection partitionIds = getPartitionIds(); - final PartitionTracker partitionTracker = getVertex().getExecutionGraph().getPartitionTracker(); + final JobMasterPartitionTracker partitionTracker = getVertex().getExecutionGraph().getPartitionTracker(); if (!partitionIds.isEmpty()) { if (releaseBlockingPartitions) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index a3b637485ab9..19102ddbbfdc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -57,8 +57,8 @@ import org.apache.flink.runtime.executiongraph.restart.ExecutionGraphRestartCallback; import org.apache.flink.runtime.executiongraph.restart.RestartCallback; import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; -import org.apache.flink.runtime.io.network.partition.PartitionTracker; -import org.apache.flink.runtime.io.network.partition.PartitionTrackerImpl; +import org.apache.flink.runtime.io.network.partition.JobMasterPartitionTracker; +import org.apache.flink.runtime.io.network.partition.JobMasterPartitionTrackerImpl; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; @@ -309,7 +309,7 @@ public class ExecutionGraph implements AccessExecutionGraph { * strong reference to any user-defined classes.*/ private volatile ErrorInfo failureInfo; - private final PartitionTracker partitionTracker; + private final JobMasterPartitionTracker partitionTracker; private final ResultPartitionAvailabilityChecker resultPartitionAvailabilityChecker; @@ -442,7 +442,7 @@ public ExecutionGraph( allocationTimeout, new NotReleasingPartitionReleaseStrategy.Factory(), NettyShuffleMaster.INSTANCE, - new PartitionTrackerImpl( + new JobMasterPartitionTrackerImpl( jobInformation.getJobId(), NettyShuffleMaster.INSTANCE, ignored -> Optional.empty()), @@ -464,7 +464,7 @@ public ExecutionGraph( Time allocationTimeout, PartitionReleaseStrategy.Factory partitionReleaseStrategyFactory, ShuffleMaster shuffleMaster, - PartitionTracker partitionTracker, + JobMasterPartitionTracker partitionTracker, ScheduleMode scheduleMode, boolean allowQueuedScheduling) throws IOException { @@ -1858,7 +1858,7 @@ ShuffleMaster getShuffleMaster() { return shuffleMaster; } - public PartitionTracker getPartitionTracker() { + public JobMasterPartitionTracker getPartitionTracker() { return partitionTracker; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java index adbd5fdea755..090b5b412be3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java @@ -45,7 +45,7 @@ import org.apache.flink.runtime.executiongraph.metrics.RestartTimeGauge; import org.apache.flink.runtime.executiongraph.metrics.UpTimeGauge; import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; -import org.apache.flink.runtime.io.network.partition.PartitionTracker; +import org.apache.flink.runtime.io.network.partition.JobMasterPartitionTracker; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; @@ -98,7 +98,7 @@ public static ExecutionGraph buildGraph( Time allocationTimeout, Logger log, ShuffleMaster shuffleMaster, - PartitionTracker partitionTracker) throws JobExecutionException, JobException { + JobMasterPartitionTracker partitionTracker) throws JobExecutionException, JobException { final FailoverStrategy.Factory failoverStrategy = FailoverStrategyLoader.loadFailoverStrategy(jobManagerConfig, log); @@ -139,7 +139,7 @@ public static ExecutionGraph buildGraph( Time allocationTimeout, Logger log, ShuffleMaster shuffleMaster, - PartitionTracker partitionTracker, + JobMasterPartitionTracker partitionTracker, FailoverStrategy.Factory failoverStrategyFactory) throws JobExecutionException, JobException { checkNotNull(jobGraph, "job graph cannot be null"); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphResultPartitionAvailabilityChecker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphResultPartitionAvailabilityChecker.java index 2ec0a34aaca7..da5155ba1c0f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphResultPartitionAvailabilityChecker.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphResultPartitionAvailabilityChecker.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.executiongraph; import org.apache.flink.runtime.executiongraph.failover.flip1.ResultPartitionAvailabilityChecker; -import org.apache.flink.runtime.io.network.partition.PartitionTracker; +import org.apache.flink.runtime.io.network.partition.JobMasterPartitionTracker; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; @@ -36,11 +36,11 @@ public class ExecutionGraphResultPartitionAvailabilityChecker implements ResultP private final Function partitionIDMapper; /** The tracker that tracks all available result partitions. */ - private final PartitionTracker partitionTracker; + private final JobMasterPartitionTracker partitionTracker; ExecutionGraphResultPartitionAvailabilityChecker( final Function partitionIDMapper, - final PartitionTracker partitionTracker) { + final JobMasterPartitionTracker partitionTracker) { this.partitionIDMapper = checkNotNull(partitionIDMapper); this.partitionTracker = checkNotNull(partitionTracker); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionTracker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTracker.java similarity index 98% rename from flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionTracker.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTracker.java index 268f4f97ab7a..6cc9be28e33d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionTracker.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTracker.java @@ -25,7 +25,7 @@ /** * Utility for tracking partitions and issuing release calls to task executors and shuffle masters. */ -public interface PartitionTracker { +public interface JobMasterPartitionTracker { /** * Starts the tracking of the given partition for the given task executor ID. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionTrackerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImpl.java similarity index 98% rename from flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionTrackerImpl.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImpl.java index 092047b317b5..d7397608f214 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionTrackerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImpl.java @@ -39,7 +39,7 @@ /** * Utility for tracking partitions and issuing release calls to task executors and shuffle masters. */ -public class PartitionTrackerImpl implements PartitionTracker { +public class JobMasterPartitionTrackerImpl implements JobMasterPartitionTracker { private final JobID jobId; @@ -50,7 +50,7 @@ public class PartitionTrackerImpl implements PartitionTracker { private final PartitionTrackerFactory.TaskExecutorGatewayLookup taskExecutorGatewayLookup; - public PartitionTrackerImpl( + public JobMasterPartitionTrackerImpl( JobID jobId, ShuffleMaster shuffleMaster, PartitionTrackerFactory.TaskExecutorGatewayLookup taskExecutorGatewayLookup) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionTrackerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionTrackerFactory.java index a1c3416dfe0e..4fc235950ba1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionTrackerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionTrackerFactory.java @@ -23,7 +23,7 @@ import java.util.Optional; /** - * Factory for {@link PartitionTracker}. + * Factory for {@link JobMasterPartitionTracker}. */ @FunctionalInterface public interface PartitionTrackerFactory { @@ -34,7 +34,7 @@ public interface PartitionTrackerFactory { * @param taskExecutorGatewayLookup lookup function to access task executor gateways * @return created PartitionTracker */ - PartitionTracker create(TaskExecutorGatewayLookup taskExecutorGatewayLookup); + JobMasterPartitionTracker create(TaskExecutorGatewayLookup taskExecutorGatewayLookup); /** * Lookup function for {@link TaskExecutorGateway}. 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 665c4aa479d0..8261459c4501 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 @@ -40,7 +40,7 @@ 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.JobMasterPartitionTracker; import org.apache.flink.runtime.io.network.partition.PartitionTrackerFactory; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; @@ -198,7 +198,7 @@ public class JobMaster extends FencedRpcEndpoint implements JobMast private Map accumulators; - private final PartitionTracker partitionTracker; + private final JobMasterPartitionTracker partitionTracker; // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/DefaultJobMasterServiceFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/DefaultJobMasterServiceFactory.java index eee2d7cdcd2b..3fa31cf9ce08 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/DefaultJobMasterServiceFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/DefaultJobMasterServiceFactory.java @@ -21,7 +21,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.io.network.partition.PartitionTrackerImpl; +import org.apache.flink.runtime.io.network.partition.JobMasterPartitionTrackerImpl; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.OnCompletionActions; import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; @@ -108,7 +108,7 @@ public JobMaster createJobMasterService( userCodeClassloader, schedulerNGFactory, shuffleMaster, - lookup -> new PartitionTrackerImpl( + lookup -> new JobMasterPartitionTrackerImpl( jobGraph.getJobID(), shuffleMaster, lookup 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 d95a05a147a3..a8e2f05ac380 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 @@ -32,7 +32,7 @@ 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.JobMasterPartitionTracker; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException; @@ -106,7 +106,7 @@ public DefaultScheduler( final JobManagerJobMetricGroup jobManagerJobMetricGroup, final Time slotRequestTimeout, final ShuffleMaster shuffleMaster, - final PartitionTracker partitionTracker, + final JobMasterPartitionTracker partitionTracker, final SchedulingStrategyFactory schedulingStrategyFactory, final FailoverStrategy.Factory failoverStrategyFactory, final RestartBackoffTimeStrategy restartBackoffTimeStrategy, 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 003035fad78f..5de8286cb3ae 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 @@ -28,7 +28,7 @@ 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.io.network.partition.JobMasterPartitionTracker; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.ScheduleMode; import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; @@ -65,7 +65,7 @@ public SchedulerNG createInstance( final JobManagerJobMetricGroup jobManagerJobMetricGroup, final Time slotRequestTimeout, final ShuffleMaster shuffleMaster, - final PartitionTracker partitionTracker) throws Exception { + final JobMasterPartitionTracker partitionTracker) throws Exception { final SchedulingStrategyFactory schedulingStrategyFactory = createSchedulingStrategyFactory(jobGraph.getScheduleMode()); final RestartBackoffTimeStrategy restartBackoffTimeStrategy = RestartBackoffTimeStrategyFactoryLoader 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 7be0d9d24787..e227dba1f58c 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 @@ -25,7 +25,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.executiongraph.ExecutionGraph; import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory; -import org.apache.flink.runtime.io.network.partition.PartitionTracker; +import org.apache.flink.runtime.io.network.partition.JobMasterPartitionTracker; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; import org.apache.flink.runtime.metrics.groups.JobManagerJobMetricGroup; @@ -60,7 +60,7 @@ public LegacyScheduler( final JobManagerJobMetricGroup jobManagerJobMetricGroup, final Time slotRequestTimeout, final ShuffleMaster shuffleMaster, - final PartitionTracker partitionTracker) throws Exception { + final JobMasterPartitionTracker partitionTracker) throws Exception { super( log, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/LegacySchedulerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/LegacySchedulerFactory.java index 245869732188..6147c06524cc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/LegacySchedulerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/LegacySchedulerFactory.java @@ -24,7 +24,7 @@ import org.apache.flink.runtime.blob.BlobWriter; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory; -import org.apache.flink.runtime.io.network.partition.PartitionTracker; +import org.apache.flink.runtime.io.network.partition.JobMasterPartitionTracker; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; import org.apache.flink.runtime.metrics.groups.JobManagerJobMetricGroup; @@ -65,7 +65,7 @@ public SchedulerNG createInstance( final JobManagerJobMetricGroup jobManagerJobMetricGroup, final Time slotRequestTimeout, final ShuffleMaster shuffleMaster, - final PartitionTracker partitionTracker) throws Exception { + final JobMasterPartitionTracker partitionTracker) throws Exception { return new LegacyScheduler( log, 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 1a231daac482..7161bbab9e89 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 @@ -52,7 +52,7 @@ 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.JobMasterPartitionTracker; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -156,7 +156,7 @@ public SchedulerBase( final JobManagerJobMetricGroup jobManagerJobMetricGroup, final Time slotRequestTimeout, final ShuffleMaster shuffleMaster, - final PartitionTracker partitionTracker) throws Exception { + final JobMasterPartitionTracker partitionTracker) throws Exception { this.log = checkNotNull(log); this.jobGraph = checkNotNull(jobGraph); @@ -193,7 +193,7 @@ public SchedulerBase( private ExecutionGraph createAndRestoreExecutionGraph( JobManagerJobMetricGroup currentJobManagerJobMetricGroup, ShuffleMaster shuffleMaster, - PartitionTracker partitionTracker) throws Exception { + JobMasterPartitionTracker partitionTracker) throws Exception { ExecutionGraph newExecutionGraph = createExecutionGraph(currentJobManagerJobMetricGroup, shuffleMaster, partitionTracker); @@ -217,7 +217,7 @@ private ExecutionGraph createAndRestoreExecutionGraph( private ExecutionGraph createExecutionGraph( JobManagerJobMetricGroup currentJobManagerJobMetricGroup, ShuffleMaster shuffleMaster, - final PartitionTracker partitionTracker) throws JobExecutionException, JobException { + final JobMasterPartitionTracker partitionTracker) throws JobExecutionException, JobException { return ExecutionGraphBuilder.buildGraph( null, jobGraph, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerNGFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerNGFactory.java index 3d2e096f6e5e..7d92cea9be71 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerNGFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerNGFactory.java @@ -23,7 +23,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.io.network.partition.PartitionTracker; +import org.apache.flink.runtime.io.network.partition.JobMasterPartitionTracker; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; import org.apache.flink.runtime.metrics.groups.JobManagerJobMetricGroup; @@ -55,6 +55,6 @@ SchedulerNG createInstance( JobManagerJobMetricGroup jobManagerJobMetricGroup, Time slotRequestTimeout, ShuffleMaster shuffleMaster, - PartitionTracker partitionTracker) throws Exception; + JobMasterPartitionTracker partitionTracker) throws Exception; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointSettingsSerializableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointSettingsSerializableTest.java index 5fbd7f724778..409117231060 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointSettingsSerializableTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointSettingsSerializableTest.java @@ -31,7 +31,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionGraph; import org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder; import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; -import org.apache.flink.runtime.io.network.partition.NoOpPartitionTracker; +import org.apache.flink.runtime.io.network.partition.NoOpJobMasterPartitionTracker; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; @@ -122,7 +122,7 @@ public void testDeserializationOfUserCodeWithUserClassLoader() throws Exception timeout, log, NettyShuffleMaster.INSTANCE, - NoOpPartitionTracker.INSTANCE); + NoOpJobMasterPartitionTracker.INSTANCE); assertEquals(1, eg.getCheckpointCoordinator().getNumberOfRegisteredMasterHooks()); assertTrue(jobGraph.getCheckpointingSettings().getDefaultStateBackend().deserializeValue(classLoader) instanceof CustomStateBackend); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGConcurrentFailoverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGConcurrentFailoverTest.java index b23ec268962a..2162ab077ab4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGConcurrentFailoverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGConcurrentFailoverTest.java @@ -27,8 +27,8 @@ import org.apache.flink.runtime.executiongraph.failover.AdaptedRestartPipelinedRegionStrategyNG; import org.apache.flink.runtime.executiongraph.failover.FailoverStrategy; import org.apache.flink.runtime.executiongraph.utils.SimpleSlotProvider; -import org.apache.flink.runtime.io.network.partition.PartitionTracker; -import org.apache.flink.runtime.io.network.partition.PartitionTrackerImpl; +import org.apache.flink.runtime.io.network.partition.JobMasterPartitionTracker; +import org.apache.flink.runtime.io.network.partition.JobMasterPartitionTrackerImpl; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -265,7 +265,7 @@ private ExecutionGraph createExecutionGraph() throws Exception { final SimpleSlotProvider slotProvider = new SimpleSlotProvider(TEST_JOB_ID, DEFAULT_PARALLELISM); - final PartitionTracker partitionTracker = new PartitionTrackerImpl( + final JobMasterPartitionTracker partitionTracker = new JobMasterPartitionTrackerImpl( jg.getJobID(), NettyShuffleMaster.INSTANCE, ignored -> Optional.empty()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGFailoverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGFailoverTest.java index f966d1693aca..65126dcc483f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGFailoverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGFailoverTest.java @@ -30,8 +30,8 @@ import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.executiongraph.restart.RestartCallback; import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; -import org.apache.flink.runtime.io.network.partition.PartitionTracker; -import org.apache.flink.runtime.io.network.partition.PartitionTrackerImpl; +import org.apache.flink.runtime.io.network.partition.JobMasterPartitionTracker; +import org.apache.flink.runtime.io.network.partition.JobMasterPartitionTrackerImpl; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.io.network.partition.consumer.PartitionConnectionException; @@ -385,7 +385,7 @@ private ExecutionGraph createExecutionGraph( final JobGraph jobGraph, final RestartStrategy restartStrategy) throws Exception { - final PartitionTracker partitionTracker = new PartitionTrackerImpl( + final JobMasterPartitionTracker partitionTracker = new JobMasterPartitionTrackerImpl( jobGraph.getJobID(), NettyShuffleMaster.INSTANCE, ignored -> Optional.empty()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java index ebef29e9fecc..78fa2aba777e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java @@ -42,7 +42,7 @@ import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; -import org.apache.flink.runtime.io.network.partition.NoOpPartitionTracker; +import org.apache.flink.runtime.io.network.partition.NoOpJobMasterPartitionTracker; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -791,7 +791,7 @@ private ExecutionGraph createExecutionGraph(Configuration configuration) throws timeout, LoggerFactory.getLogger(getClass()), NettyShuffleMaster.INSTANCE, - NoOpPartitionTracker.INSTANCE); + NoOpJobMasterPartitionTracker.INSTANCE); } private static final class ExecutionStageMatcher extends TypeSafeMatcher> { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphPartitionReleaseTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphPartitionReleaseTest.java index 5c48b5683c6d..ea45ff3a1335 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphPartitionReleaseTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphPartitionReleaseTest.java @@ -27,10 +27,10 @@ import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.failover.flip1.partitionrelease.PartitionReleaseStrategy; import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; -import org.apache.flink.runtime.io.network.partition.PartitionTracker; +import org.apache.flink.runtime.io.network.partition.JobMasterPartitionTracker; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; -import org.apache.flink.runtime.io.network.partition.TestingPartitionTracker; +import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker; import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -74,7 +74,7 @@ public void testStrategyNotifiedOfFinishedVerticesAndResultsRespected() throws E sinkVertex.connectNewDataSetAsInput(operatorVertex, DistributionPattern.POINTWISE, ResultPartitionType.BLOCKING); // setup partition tracker to intercept partition release calls - final TestingPartitionTracker partitionTracker = new TestingPartitionTracker(); + final TestingJobMasterPartitionTracker partitionTracker = new TestingJobMasterPartitionTracker(); final Queue releasedPartitions = new ArrayDeque<>(); partitionTracker.setStopTrackingAndReleasePartitionsConsumer( partitionIds -> releasedPartitions.add(partitionIds.iterator().next())); @@ -125,7 +125,7 @@ public void testStrategyNotifiedOfUnFinishedVertices() throws Exception { operator3Vertex.connectNewDataSetAsInput(operator1Vertex, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); // setup partition tracker to intercept partition release calls - final TestingPartitionTracker partitionTracker = new TestingPartitionTracker(); + final TestingJobMasterPartitionTracker partitionTracker = new TestingJobMasterPartitionTracker(); final Queue releasedPartitions = new ArrayDeque<>(); partitionTracker.setStopTrackingAndReleasePartitionsConsumer( partitionIds -> releasedPartitions.add(partitionIds.iterator().next())); @@ -176,7 +176,7 @@ private static Execution getCurrentExecution(final JobVertex jobVertex, final Ex return executionGraph.getJobVertex(jobVertex.getID()).getTaskVertices()[0].getCurrentExecutionAttempt(); } - private ExecutionGraph createExecutionGraph(final PartitionTracker partitionTracker, final JobVertex... vertices) throws Exception { + private ExecutionGraph createExecutionGraph(final JobMasterPartitionTracker partitionTracker, final JobVertex... vertices) throws Exception { final ExecutionGraph executionGraph = ExecutionGraphBuilder.buildGraph( null, new JobGraph(new JobID(), "test job", vertices), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRescalingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRescalingTest.java index 8e69df9b516a..39ebeeb2857c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRescalingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRescalingTest.java @@ -25,7 +25,7 @@ import org.apache.flink.runtime.blob.VoidBlobWriter; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; -import org.apache.flink.runtime.io.network.partition.NoOpPartitionTracker; +import org.apache.flink.runtime.io.network.partition.NoOpJobMasterPartitionTracker; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -81,7 +81,7 @@ public void testExecutionGraphArbitraryDopConstructionTest() throws Exception { AkkaUtils.getDefaultTimeout(), TEST_LOGGER, NettyShuffleMaster.INSTANCE, - NoOpPartitionTracker.INSTANCE); + NoOpJobMasterPartitionTracker.INSTANCE); for (JobVertex jv : jobVertices) { assertThat(jv.getParallelism(), is(initialParallelism)); @@ -112,7 +112,7 @@ public void testExecutionGraphArbitraryDopConstructionTest() throws Exception { AkkaUtils.getDefaultTimeout(), TEST_LOGGER, NettyShuffleMaster.INSTANCE, - NoOpPartitionTracker.INSTANCE); + NoOpJobMasterPartitionTracker.INSTANCE); for (JobVertex jv : jobVertices) { assertThat(jv.getParallelism(), is(1)); @@ -143,7 +143,7 @@ public void testExecutionGraphArbitraryDopConstructionTest() throws Exception { AkkaUtils.getDefaultTimeout(), TEST_LOGGER, NettyShuffleMaster.INSTANCE, - NoOpPartitionTracker.INSTANCE); + NoOpJobMasterPartitionTracker.INSTANCE); for (JobVertex jv : jobVertices) { assertThat(jv.getParallelism(), is(scaleUpParallelism)); @@ -187,7 +187,7 @@ public void testExecutionGraphConstructionFailsRescaleDopExceedMaxParallelism() AkkaUtils.getDefaultTimeout(), TEST_LOGGER, NettyShuffleMaster.INSTANCE, - NoOpPartitionTracker.INSTANCE); + NoOpJobMasterPartitionTracker.INSTANCE); fail("Building the ExecutionGraph with a parallelism higher than the max parallelism should fail."); } catch (JobException e) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphResultPartitionAvailabilityCheckerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphResultPartitionAvailabilityCheckerTest.java index ccbd439205f1..4c11aed3b797 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphResultPartitionAvailabilityCheckerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphResultPartitionAvailabilityCheckerTest.java @@ -20,7 +20,7 @@ import org.apache.flink.runtime.executiongraph.failover.flip1.ResultPartitionAvailabilityChecker; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; -import org.apache.flink.runtime.io.network.partition.TestingPartitionTracker; +import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.util.TestLogger; @@ -54,7 +54,7 @@ public void testPartitionAvailabilityCheck() { }}; // let the partition tracker respect the expected availability result - final TestingPartitionTracker partitionTracker = new TestingPartitionTracker(); + final TestingJobMasterPartitionTracker partitionTracker = new TestingJobMasterPartitionTracker(); partitionTracker.setIsPartitionTrackedFunction(rpID -> expectedAvailability.get(rpID.getPartitionId())); // the execution attempt ID should make no difference in this case diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java index 802dff07f12c..0aaf85799092 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java @@ -31,7 +31,7 @@ import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; import org.apache.flink.runtime.instance.SimpleSlotContext; -import org.apache.flink.runtime.io.network.partition.NoOpPartitionTracker; +import org.apache.flink.runtime.io.network.partition.NoOpJobMasterPartitionTracker; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -594,7 +594,7 @@ private ExecutionGraph createExecutionGraph(JobGraph jobGraph, SlotProvider slot timeout, log, NettyShuffleMaster.INSTANCE, - NoOpPartitionTracker.INSTANCE); + NoOpJobMasterPartitionTracker.INSTANCE); } @Nonnull diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java index dbe2a0f29c09..288f5f12835f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java @@ -39,8 +39,8 @@ import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; 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.PartitionTracker; +import org.apache.flink.runtime.io.network.partition.NoOpJobMasterPartitionTracker; +import org.apache.flink.runtime.io.network.partition.JobMasterPartitionTracker; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobVertex; @@ -539,7 +539,7 @@ public static class TestingExecutionGraphBuilder { private ScheduledExecutorService futureExecutor = TestingUtils.defaultExecutor(); private Configuration jobMasterConfig = new Configuration(); private JobGraph jobGraph; - private PartitionTracker partitionTracker = NoOpPartitionTracker.INSTANCE; + private JobMasterPartitionTracker partitionTracker = NoOpJobMasterPartitionTracker.INSTANCE; private FailoverStrategy.Factory failoverStrategyFactory = new RestartAllStrategy.Factory(); public TestingExecutionGraphBuilder(final JobVertex ... jobVertices) { @@ -622,7 +622,7 @@ public TestingExecutionGraphBuilder setShuffleMaster(final ShuffleMaster shuf return this; } - public TestingExecutionGraphBuilder setPartitionTracker(final PartitionTracker partitionTracker) { + public TestingExecutionGraphBuilder setPartitionTracker(final JobMasterPartitionTracker partitionTracker) { this.partitionTracker = partitionTracker; return this; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionPartitionLifecycleTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionPartitionLifecycleTest.java index 058857c36c29..0e1d3ddc22e1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionPartitionLifecycleTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionPartitionLifecycleTest.java @@ -34,11 +34,11 @@ import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; import org.apache.flink.runtime.instance.SlotSharingGroupId; -import org.apache.flink.runtime.io.network.partition.NoOpPartitionTracker; -import org.apache.flink.runtime.io.network.partition.PartitionTracker; +import org.apache.flink.runtime.io.network.partition.NoOpJobMasterPartitionTracker; +import org.apache.flink.runtime.io.network.partition.JobMasterPartitionTracker; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; -import org.apache.flink.runtime.io.network.partition.TestingPartitionTracker; +import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker; import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -120,7 +120,7 @@ private void testPartitionReleaseOnStateTransitionsAfterRunning(Consumer s CompletableFuture> partitionStartTrackingFuture = new CompletableFuture<>(); CompletableFuture> partitionStopTrackingFuture = new CompletableFuture<>(); CompletableFuture> partitionStopTrackingAndReleaseFuture = new CompletableFuture<>(); - final TestingPartitionTracker partitionTracker = new TestingPartitionTracker(); + final TestingJobMasterPartitionTracker partitionTracker = new TestingJobMasterPartitionTracker(); partitionTracker.setStartTrackingPartitionsConsumer( (resourceID, resultPartitionDeploymentDescriptor) -> partitionStartTrackingFuture.complete(Tuple2.of(resourceID, resultPartitionDeploymentDescriptor)) @@ -224,7 +224,7 @@ private void testPartitionTrackingForStateTransition(final Consumer s } } - private void setupExecutionGraphAndStartRunningJob(ResultPartitionType resultPartitionType, PartitionTracker partitionTracker, TaskManagerGateway taskManagerGateway, ShuffleMaster shuffleMaster) throws JobException, JobExecutionException { + private void setupExecutionGraphAndStartRunningJob(ResultPartitionType resultPartitionType, JobMasterPartitionTracker partitionTracker, TaskManagerGateway taskManagerGateway, ShuffleMaster shuffleMaster) throws JobException, JobExecutionException { final JobVertex producerVertex = createNoOpJobVertex(); final JobVertex consumerVertex = createNoOpJobVertex(); consumerVertex.connectNewDataSetAsInput(producerVertex, DistributionPattern.ALL_TO_ALL, resultPartitionType); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexLocalityTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexLocalityTest.java index 37e5183dfc9d..195b5189d438 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexLocalityTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexLocalityTest.java @@ -30,7 +30,7 @@ import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.restart.FixedDelayRestartStrategy; import org.apache.flink.runtime.instance.SimpleSlotContext; -import org.apache.flink.runtime.io.network.partition.NoOpPartitionTracker; +import org.apache.flink.runtime.io.network.partition.NoOpJobMasterPartitionTracker; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -231,7 +231,7 @@ private ExecutionGraph createTestGraph(int parallelism, boolean allToAll) throws timeout, log, NettyShuffleMaster.INSTANCE, - NoOpPartitionTracker.INSTANCE); + NoOpJobMasterPartitionTracker.INSTANCE); } private void initializeLocation(ExecutionVertex vertex, TaskManagerLocation location) throws Exception { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexTest.java index c9f3ef0c13b4..4909101228e1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexTest.java @@ -20,7 +20,7 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; -import org.apache.flink.runtime.io.network.partition.TestingPartitionTracker; +import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker; import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.jobgraph.JobVertex; @@ -48,7 +48,7 @@ public void testResetForNewExecutionReleasesPartitions() throws Exception { consumerJobVertex.connectNewDataSetAsInput(producerJobVertex, DistributionPattern.POINTWISE, ResultPartitionType.BLOCKING); final CompletableFuture> releasePartitionsFuture = new CompletableFuture<>(); - final TestingPartitionTracker partitionTracker = new TestingPartitionTracker(); + final TestingJobMasterPartitionTracker partitionTracker = new TestingJobMasterPartitionTracker(); partitionTracker.setStopTrackingAndReleasePartitionsConsumer(releasePartitionsFuture::complete); final ExecutionGraph executionGraph = new ExecutionGraphTestUtils.TestingExecutionGraphBuilder(producerJobVertex, consumerJobVertex) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTrackerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImplTest.java similarity index 95% rename from flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTrackerImplTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImplTest.java index 37a359590cbf..8f39956a91c8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTrackerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImplTest.java @@ -45,9 +45,9 @@ import static org.junit.Assert.assertEquals; /** - * Tests for the {@link PartitionTrackerImpl}. + * Tests for the {@link JobMasterPartitionTrackerImpl}. */ -public class PartitionTrackerImplTest extends TestLogger { +public class JobMasterPartitionTrackerImplTest extends TestLogger { @Test public void testPipelinedPartitionIsNotTracked() { @@ -60,7 +60,7 @@ public void testBlockingPartitionIsTracked() { } private void testReleaseOnConsumptionHandling(ResultPartitionType resultPartitionType) { - final PartitionTracker partitionTracker = new PartitionTrackerImpl( + final JobMasterPartitionTracker partitionTracker = new JobMasterPartitionTrackerImpl( new JobID(), new TestingShuffleMaster(), ignored -> Optional.empty() @@ -81,7 +81,7 @@ private void testReleaseOnConsumptionHandling(ResultPartitionType resultPartitio @Test public void testStartStopTracking() { final Queue>> taskExecutorReleaseCalls = new ArrayBlockingQueue<>(4); - final PartitionTracker partitionTracker = new PartitionTrackerImpl( + final JobMasterPartitionTracker partitionTracker = new JobMasterPartitionTrackerImpl( new JobID(), new TestingShuffleMaster(), resourceId -> Optional.of(createTaskExecutorGateway(resourceId, taskExecutorReleaseCalls)) @@ -112,7 +112,7 @@ public void testReleaseCallsWithLocalResources() { final JobID jobId = new JobID(); final Queue>> taskExecutorReleaseCalls = new ArrayBlockingQueue<>(4); - final PartitionTracker partitionTracker = new PartitionTrackerImpl( + final JobMasterPartitionTracker partitionTracker = new JobMasterPartitionTrackerImpl( jobId, shuffleMaster, resourceId -> Optional.of(createTaskExecutorGateway(resourceId, taskExecutorReleaseCalls)) @@ -168,7 +168,7 @@ public void testReleaseCallsWithoutLocalResources() { final TestingShuffleMaster shuffleMaster = new TestingShuffleMaster(); final Queue>> taskExecutorReleaseCalls = new ArrayBlockingQueue<>(4); - final PartitionTracker partitionTracker = new PartitionTrackerImpl( + final JobMasterPartitionTracker partitionTracker = new JobMasterPartitionTrackerImpl( new JobID(), shuffleMaster, resourceId -> Optional.of(createTaskExecutorGateway(resourceId, taskExecutorReleaseCalls)) @@ -214,7 +214,7 @@ public void testStopTrackingIssuesNoReleaseCalls() { final TestingShuffleMaster shuffleMaster = new TestingShuffleMaster(); final Queue>> taskExecutorReleaseCalls = new ArrayBlockingQueue<>(4); - final PartitionTracker partitionTracker = new PartitionTrackerImpl( + final JobMasterPartitionTracker partitionTracker = new JobMasterPartitionTrackerImpl( new JobID(), shuffleMaster, resourceId -> Optional.of(createTaskExecutorGateway(resourceId, taskExecutorReleaseCalls)) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/NoOpPartitionTracker.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/NoOpJobMasterPartitionTracker.java similarity index 93% rename from flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/NoOpPartitionTracker.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/NoOpJobMasterPartitionTracker.java index 2888b31f1d18..f0bf2527e1bd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/NoOpPartitionTracker.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/NoOpJobMasterPartitionTracker.java @@ -23,9 +23,9 @@ import java.util.Collection; /** - * No-op implementation of {@link PartitionTracker}. + * No-op implementation of {@link JobMasterPartitionTracker}. */ -public enum NoOpPartitionTracker implements PartitionTracker { +public enum NoOpJobMasterPartitionTracker implements JobMasterPartitionTracker { INSTANCE; public static final PartitionTrackerFactory FACTORY = lookup -> INSTANCE; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingPartitionTracker.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingJobMasterPartitionTracker.java similarity index 96% rename from flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingPartitionTracker.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingJobMasterPartitionTracker.java index 6ba333d836d5..9a996ecf3504 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingPartitionTracker.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingJobMasterPartitionTracker.java @@ -26,9 +26,9 @@ import java.util.function.Function; /** - * Test {@link PartitionTracker} implementation. + * Test {@link JobMasterPartitionTracker} implementation. */ -public class TestingPartitionTracker implements PartitionTracker { +public class TestingJobMasterPartitionTracker implements JobMasterPartitionTracker { private Function isTrackingPartitionsForFunction = ignored -> false; private Function isPartitionTrackedFunction = ignored -> false; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java index 4bcfd2cb6965..e7f061ee7c69 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java @@ -66,11 +66,11 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.instance.SimpleSlotContext; -import org.apache.flink.runtime.io.network.partition.NoOpPartitionTracker; +import org.apache.flink.runtime.io.network.partition.NoOpJobMasterPartitionTracker; import org.apache.flink.runtime.io.network.partition.PartitionTrackerFactory; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; -import org.apache.flink.runtime.io.network.partition.TestingPartitionTracker; +import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker; import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -316,7 +316,7 @@ public void testDeclineCheckpointInvocationWithUserException() throws Exception JobMasterTest.class.getClassLoader(), schedulerNGFactory, NettyShuffleMaster.INSTANCE, - NoOpPartitionTracker.FACTORY) { + NoOpJobMasterPartitionTracker.FACTORY) { @Override public void declineCheckpoint(DeclineCheckpoint declineCheckpoint) { declineCheckpointMessageFuture.complete(declineCheckpoint.getReason()); @@ -1639,7 +1639,7 @@ public void testTriggerSavepointTimeout() throws Exception { JobMasterTest.class.getClassLoader(), schedulerNGFactory, NettyShuffleMaster.INSTANCE, - NoOpPartitionTracker.FACTORY) { + NoOpJobMasterPartitionTracker.FACTORY) { @Override public CompletableFuture triggerSavepoint( @@ -1730,7 +1730,7 @@ public void testTaskExecutorNotReleasedOnFailedAllocationIfPartitionIsAllocated( final LocalTaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); final AtomicBoolean isTrackingPartitions = new AtomicBoolean(true); - final TestingPartitionTracker partitionTracker = new TestingPartitionTracker(); + final TestingJobMasterPartitionTracker partitionTracker = new TestingJobMasterPartitionTracker(); partitionTracker.setIsTrackingPartitionsForFunction(ignored -> isTrackingPartitions.get()); final JobMaster jobMaster = new JobMasterBuilder() @@ -1865,7 +1865,7 @@ public void testPartitionTableCleanupOnDisconnect() throws Exception { final JobGraph jobGraph = createSingleVertexJobGraph(); final CompletableFuture partitionCleanupTaskExecutorId = new CompletableFuture<>(); - final TestingPartitionTracker partitionTracker = new TestingPartitionTracker(); + final TestingJobMasterPartitionTracker partitionTracker = new TestingJobMasterPartitionTracker(); partitionTracker.setStopTrackingAllPartitionsConsumer(partitionCleanupTaskExecutorId::complete); final JobMaster jobMaster = new JobMasterBuilder() @@ -1907,7 +1907,7 @@ public void testPartitionReleaseOnJobTermination() throws Exception { final JobGraph jobGraph = createSingleVertexJobGraph(); final CompletableFuture partitionCleanupTaskExecutorId = new CompletableFuture<>(); - final TestingPartitionTracker partitionTracker = new TestingPartitionTracker(); + final TestingJobMasterPartitionTracker partitionTracker = new TestingJobMasterPartitionTracker(); partitionTracker.setStopTrackingAndReleaseAllPartitionsConsumer(partitionCleanupTaskExecutorId::complete); final JobMaster jobMaster = new JobMasterBuilder() @@ -2259,7 +2259,7 @@ private final class JobMasterBuilder { private ShuffleMaster shuffleMaster = NettyShuffleMaster.INSTANCE; - private PartitionTrackerFactory partitionTrackerFactory = NoOpPartitionTracker.FACTORY; + private PartitionTrackerFactory partitionTrackerFactory = NoOpJobMasterPartitionTracker.FACTORY; private JobMasterBuilder withConfiguration(Configuration configuration) { this.configuration = configuration; 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 66c449981fcd..3fa2ad95f5da 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 @@ -34,7 +34,7 @@ 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.io.network.partition.NoOpJobMasterPartitionTracker; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -464,7 +464,7 @@ private DefaultScheduler createScheduler( UnregisteredMetricGroups.createUnregisteredJobManagerJobMetricGroup(), Time.seconds(300), NettyShuffleMaster.INSTANCE, - NoOpPartitionTracker.INSTANCE, + NoOpJobMasterPartitionTracker.INSTANCE, schedulingStrategyFactory, new RestartPipelinedRegionStrategy.Factory(), testRestartBackoffTimeStrategy, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/LegacySchedulerBatchSchedulingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/LegacySchedulerBatchSchedulingTest.java index 00b6c18167a6..03df7130daab 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/LegacySchedulerBatchSchedulingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/LegacySchedulerBatchSchedulingTest.java @@ -30,7 +30,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.JobStatusListener; import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy; -import org.apache.flink.runtime.io.network.partition.NoOpPartitionTracker; +import org.apache.flink.runtime.io.network.partition.NoOpJobMasterPartitionTracker; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobVertex; @@ -207,7 +207,7 @@ private LegacyScheduler createLegacyScheduler(JobGraph jobGraph, SlotPool slotPo UnregisteredMetricGroups.createUnregisteredJobManagerJobMetricGroup(), slotRequestTimeout, NettyShuffleMaster.INSTANCE, - NoOpPartitionTracker.INSTANCE); + NoOpJobMasterPartitionTracker.INSTANCE); legacyScheduler.setMainThreadExecutor(mainThreadExecutor); From 4bc56a48c717ccc63b000c32495064fbb1190929 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 25 Oct 2019 14:02:06 +0200 Subject: [PATCH 358/746] [FLINK-14532][coordination] Split PartitionTracker --- .../partition/AbstractPartitionTracker.java | 117 ++++++++++++++++++ .../partition/JobMasterPartitionTracker.java | 22 +--- .../JobMasterPartitionTrackerImpl.java | 93 ++++---------- .../network/partition/PartitionTracker.java | 50 ++++++++ .../partition/PartitionTrackerEntry.java | 46 +++++++ .../AbstractPartitionTrackerTest.java | 101 +++++++++++++++ .../JobMasterPartitionTrackerImplTest.java | 79 ++---------- .../NoOpJobMasterPartitionTracker.java | 7 +- .../TestingJobMasterPartitionTracker.java | 7 +- 9 files changed, 353 insertions(+), 169 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/AbstractPartitionTracker.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionTracker.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionTrackerEntry.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/AbstractPartitionTrackerTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/AbstractPartitionTracker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/AbstractPartitionTracker.java new file mode 100644 index 000000000000..c23a6f1a1fdf --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/AbstractPartitionTracker.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.partition; + +import org.apache.flink.runtime.taskexecutor.partition.PartitionTable; +import org.apache.flink.util.Preconditions; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Base partition tracker implementation, providing underlying data-structures for storing partitions, their associated + * keys and meta-information. + */ +public abstract class AbstractPartitionTracker implements PartitionTracker { + + private final PartitionTable partitionTable = new PartitionTable<>(); + private final Map> partitionInfos = new HashMap<>(); + + void startTrackingPartition(K key, ResultPartitionID resultPartitionId, M metaInfo) { + partitionInfos.put(resultPartitionId, new PartitionInfo<>(key, metaInfo)); + partitionTable.startTrackingPartitions(key, Collections.singletonList(resultPartitionId)); + } + + @Override + public Collection> stopTrackingPartitionsFor(K key) { + Preconditions.checkNotNull(key); + + // this is a bit icky since we make 2 calls to pT#stopTrackingPartitions + final Collection resultPartitionIds = partitionTable.stopTrackingPartitions(key); + + return stopTrackingPartitions(resultPartitionIds); + } + + @Override + public Collection> stopTrackingPartitions(Collection resultPartitionIds) { + Preconditions.checkNotNull(resultPartitionIds); + + return resultPartitionIds.stream() + .map(this::internalStopTrackingPartition) + .flatMap(AbstractPartitionTracker::asStream) + .collect(Collectors.toList()); + } + + @Override + public boolean isTrackingPartitionsFor(K key) { + Preconditions.checkNotNull(key); + + return partitionTable.hasTrackedPartitions(key); + } + + @Override + public boolean isPartitionTracked(final ResultPartitionID resultPartitionID) { + Preconditions.checkNotNull(resultPartitionID); + + return partitionInfos.containsKey(resultPartitionID); + } + + private Optional> internalStopTrackingPartition(ResultPartitionID resultPartitionId) { + Preconditions.checkNotNull(resultPartitionId); + + final PartitionInfo partitionInfo = partitionInfos.remove(resultPartitionId); + if (partitionInfo == null) { + return Optional.empty(); + } + partitionTable.stopTrackingPartitions(partitionInfo.getKey(), Collections.singletonList(resultPartitionId)); + + return Optional.of(new PartitionTrackerEntry<>(resultPartitionId, partitionInfo.key, partitionInfo.getMetaInfo())); + } + + private static class PartitionInfo { + + private final K key; + private final M metaInfo; + + PartitionInfo(K key, M metaInfo) { + this.key = key; + this.metaInfo = metaInfo; + } + + public K getKey() { + return key; + } + + public M getMetaInfo() { + return metaInfo; + } + } + + private static Stream asStream(Optional optional) { + if (optional.isPresent()) { + return Stream.of(optional.get()); + } else { + return Stream.empty(); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTracker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTracker.java index 6cc9be28e33d..8f0f33857f76 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTracker.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTracker.java @@ -25,7 +25,7 @@ /** * Utility for tracking partitions and issuing release calls to task executors and shuffle masters. */ -public interface JobMasterPartitionTracker { +public interface JobMasterPartitionTracker extends PartitionTracker { /** * Starts the tracking of the given partition for the given task executor ID. @@ -35,33 +35,13 @@ public interface JobMasterPartitionTracker { */ void startTrackingPartition(ResourceID producingTaskExecutorId, ResultPartitionDeploymentDescriptor resultPartitionDeploymentDescriptor); - /** - * Stops the tracking of all partitions for the given task executor ID, without issuing any release calls. - */ - void stopTrackingPartitionsFor(ResourceID producingTaskExecutorId); - /** * Releases the given partitions and stop the tracking of partitions that were released. */ void stopTrackingAndReleasePartitions(Collection resultPartitionIds); - /** - * Stops the tracking of the given partitions. - */ - void stopTrackingPartitions(Collection resultPartitionIds); - /** * Releases all partitions for the given task executor ID, and stop the tracking of partitions that were released. */ void stopTrackingAndReleasePartitionsFor(ResourceID producingTaskExecutorId); - - /** - * Returns whether any partition is being tracked for the given task executor ID. - */ - boolean isTrackingPartitionsFor(ResourceID producingTaskExecutorId); - - /** - * Returns whether the given partition is being tracked. - */ - boolean isPartitionTracked(ResultPartitionID resultPartitionID); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImpl.java index d7397608f214..8159e08dd666 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImpl.java @@ -22,16 +22,14 @@ import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.shuffle.ShuffleMaster; -import org.apache.flink.runtime.taskexecutor.partition.PartitionTable; import org.apache.flink.util.Preconditions; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Set; +import java.util.function.Function; import java.util.stream.Collectors; import static java.util.stream.Collectors.toList; @@ -39,13 +37,12 @@ /** * Utility for tracking partitions and issuing release calls to task executors and shuffle masters. */ -public class JobMasterPartitionTrackerImpl implements JobMasterPartitionTracker { +public class JobMasterPartitionTrackerImpl + extends AbstractPartitionTracker + implements JobMasterPartitionTracker { private final JobID jobId; - private final PartitionTable partitionTable = new PartitionTable<>(); - private final Map partitionInfos = new HashMap<>(); - private final ShuffleMaster shuffleMaster; private final PartitionTrackerFactory.TaskExecutorGatewayLookup taskExecutorGatewayLookup; @@ -72,20 +69,7 @@ public void startTrackingPartition(ResourceID producingTaskExecutorId, ResultPar final ResultPartitionID resultPartitionId = resultPartitionDeploymentDescriptor.getShuffleDescriptor().getResultPartitionID(); - partitionInfos.put(resultPartitionId, new PartitionInfo(producingTaskExecutorId, resultPartitionDeploymentDescriptor)); - partitionTable.startTrackingPartitions(producingTaskExecutorId, Collections.singletonList(resultPartitionId)); - } - - @Override - public void stopTrackingPartitionsFor(ResourceID producingTaskExecutorId) { - Preconditions.checkNotNull(producingTaskExecutorId); - - // this is a bit icky since we make 2 calls to pT#stopTrackingPartitions - final Collection resultPartitionIds = partitionTable.stopTrackingPartitions(producingTaskExecutorId); - - for (ResultPartitionID resultPartitionId : resultPartitionIds) { - internalStopTrackingPartition(resultPartitionId); - } + startTrackingPartition(producingTaskExecutorId, resultPartitionId, resultPartitionDeploymentDescriptor); } @Override @@ -93,58 +77,26 @@ public void stopTrackingAndReleasePartitions(Collection resul Preconditions.checkNotNull(resultPartitionIds); // stop tracking partitions to be released and group them by task executor ID - Map> partitionsToReleaseByResourceId = resultPartitionIds.stream() - .map(this::internalStopTrackingPartition) - .filter(Optional::isPresent) - .map(Optional::get) - .collect(Collectors.groupingBy( - partitionMetaData -> partitionMetaData.producingTaskExecutorResourceId, - Collectors.mapping( - partitionMetaData -> partitionMetaData.resultPartitionDeploymentDescriptor, - toList()))); + Map> partitionsToReleaseByResourceId = + stopTrackingPartitions(resultPartitionIds) + .stream() + .collect(Collectors.groupingBy( + PartitionTrackerEntry::getKey, + Collectors.mapping( + PartitionTrackerEntry::getMetaInfo, + toList()))); partitionsToReleaseByResourceId.forEach(this::internalReleasePartitions); } - @Override - public void stopTrackingPartitions(Collection resultPartitionIds) { - Preconditions.checkNotNull(resultPartitionIds); - - resultPartitionIds.forEach(this::internalStopTrackingPartition); - } - @Override public void stopTrackingAndReleasePartitionsFor(ResourceID producingTaskExecutorId) { Preconditions.checkNotNull(producingTaskExecutorId); - // this is a bit icky since we make 2 calls to pT#stopTrackingPartitions - Collection resultPartitionIds = partitionTable.stopTrackingPartitions(producingTaskExecutorId); - - stopTrackingAndReleasePartitions(resultPartitionIds); - } - - @Override - public boolean isTrackingPartitionsFor(ResourceID producingTaskExecutorId) { - Preconditions.checkNotNull(producingTaskExecutorId); - - return partitionTable.hasTrackedPartitions(producingTaskExecutorId); - } - - @Override - public boolean isPartitionTracked(final ResultPartitionID resultPartitionID) { - Preconditions.checkNotNull(resultPartitionID); + Collection resultPartitionIds = + project(stopTrackingPartitionsFor(producingTaskExecutorId), PartitionTrackerEntry::getMetaInfo); - return partitionInfos.containsKey(resultPartitionID); - } - - private Optional internalStopTrackingPartition(ResultPartitionID resultPartitionId) { - final PartitionInfo partitionInfo = partitionInfos.remove(resultPartitionId); - if (partitionInfo == null) { - return Optional.empty(); - } - partitionTable.stopTrackingPartitions(partitionInfo.producingTaskExecutorResourceId, Collections.singletonList(resultPartitionId)); - - return Optional.of(partitionInfo); + internalReleasePartitions(producingTaskExecutorId, resultPartitionIds); } private void internalReleasePartitions( @@ -179,13 +131,10 @@ private void internalReleasePartitionsOnShuffleMaster(Collection Collection project(Collection collection, Function projector) { + return collection + .stream() + .map(projector) + .collect(toList()); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionTracker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionTracker.java new file mode 100644 index 000000000000..b612c4942a8d --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionTracker.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.io.network.partition; + +import java.util.Collection; + +/** + * Utility for tracking partitions. + * + *

    This interface deliberately does not have a method to start tracking partitions, so that implementation are + * flexible in their definitions for this method (otherwise one would end up with multiple methods, with one part likely + * being unused). + */ +public interface PartitionTracker { + + /** + * Stops the tracking of all partitions for the given key. + */ + Collection> stopTrackingPartitionsFor(K key); + + /** + * Stops the tracking of the given partitions. + */ + Collection> stopTrackingPartitions(Collection resultPartitionIds); + + /** + * Returns whether any partition is being tracked for the given key. + */ + boolean isTrackingPartitionsFor(K key); + + /** + * Returns whether the given partition is being tracked. + */ + boolean isPartitionTracked(ResultPartitionID resultPartitionID); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionTrackerEntry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionTrackerEntry.java new file mode 100644 index 000000000000..ea7a150f9030 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PartitionTrackerEntry.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.partition; + +/** + * Encapsulates all information that a {@link PartitionTracker} keeps for a partition. + */ +public class PartitionTrackerEntry { + + private final ResultPartitionID resultPartitionId; + private final K key; + private final M metaInfo; + + PartitionTrackerEntry(ResultPartitionID resultPartitionId, K key, M metaInfo) { + this.resultPartitionId = resultPartitionId; + this.key = key; + this.metaInfo = metaInfo; + } + + public ResultPartitionID getResultPartitionId() { + return resultPartitionId; + } + + public K getKey() { + return key; + } + + public M getMetaInfo() { + return metaInfo; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/AbstractPartitionTrackerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/AbstractPartitionTrackerTest.java new file mode 100644 index 000000000000..3afabcd3a2af --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/AbstractPartitionTrackerTest.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.partition; + +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.shuffle.PartitionDescriptor; +import org.apache.flink.runtime.shuffle.ShuffleDescriptor; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import java.util.Optional; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; + +/** + * Tests for the {@link AbstractPartitionTracker}. + */ +public class AbstractPartitionTrackerTest extends TestLogger { + + @Test + public void testStartStopTracking() { + final TestPartitionTracker partitionTracker = new TestPartitionTracker(); + + final ResourceID executorWithTrackedPartition = new ResourceID("tracked"); + final ResourceID executorWithoutTrackedPartition = new ResourceID("untracked"); + + assertThat(partitionTracker.isTrackingPartitionsFor(executorWithTrackedPartition), is(false)); + assertThat(partitionTracker.isTrackingPartitionsFor(executorWithoutTrackedPartition), is(false)); + + partitionTracker.startTrackingPartition(executorWithTrackedPartition, new ResultPartitionID()); + + assertThat(partitionTracker.isTrackingPartitionsFor(executorWithTrackedPartition), is(true)); + assertThat(partitionTracker.isTrackingPartitionsFor(executorWithoutTrackedPartition), is(false)); + + partitionTracker.stopTrackingPartitionsFor(executorWithTrackedPartition); + + assertThat(partitionTracker.isTrackingPartitionsFor(executorWithTrackedPartition), is(false)); + assertThat(partitionTracker.isTrackingPartitionsFor(executorWithoutTrackedPartition), is(false)); + } + + static ResultPartitionDeploymentDescriptor createResultPartitionDeploymentDescriptor( + ResultPartitionID resultPartitionId, + boolean hasLocalResources) { + return createResultPartitionDeploymentDescriptor(resultPartitionId, ResultPartitionType.BLOCKING, hasLocalResources); + } + + static ResultPartitionDeploymentDescriptor createResultPartitionDeploymentDescriptor( + ResultPartitionID resultPartitionId, + ResultPartitionType type, + boolean hasLocalResources) { + + return new ResultPartitionDeploymentDescriptor( + new PartitionDescriptor( + new IntermediateDataSetID(), + resultPartitionId.getPartitionId(), + type, + 1, + 0), + new ShuffleDescriptor() { + @Override + public ResultPartitionID getResultPartitionID() { + return resultPartitionId; + } + + @Override + public Optional storesLocalResourcesOn() { + return hasLocalResources + ? Optional.of(ResourceID.generate()) + : Optional.empty(); + } + }, + 1, + true); + } + + private static class TestPartitionTracker extends AbstractPartitionTracker { + + public void startTrackingPartition(ResourceID key, ResultPartitionID resultPartitionID) { + startTrackingPartition(key, resultPartitionID, 0); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImplTest.java index 8f39956a91c8..bfaad3db6c6b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImplTest.java @@ -20,8 +20,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; -import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.shuffle.PartitionDescriptor; import org.apache.flink.runtime.shuffle.ProducerDescriptor; import org.apache.flink.runtime.shuffle.ShuffleDescriptor; @@ -59,7 +57,7 @@ public void testBlockingPartitionIsTracked() { testReleaseOnConsumptionHandling(ResultPartitionType.BLOCKING); } - private void testReleaseOnConsumptionHandling(ResultPartitionType resultPartitionType) { + private static void testReleaseOnConsumptionHandling(ResultPartitionType resultPartitionType) { final JobMasterPartitionTracker partitionTracker = new JobMasterPartitionTrackerImpl( new JobID(), new TestingShuffleMaster(), @@ -70,7 +68,7 @@ private void testReleaseOnConsumptionHandling(ResultPartitionType resultPartitio final ResultPartitionID resultPartitionId = new ResultPartitionID(); partitionTracker.startTrackingPartition( resourceId, - createResultPartitionDeploymentDescriptor( + AbstractPartitionTrackerTest.createResultPartitionDeploymentDescriptor( resultPartitionId, resultPartitionType, false)); @@ -78,34 +76,6 @@ private void testReleaseOnConsumptionHandling(ResultPartitionType resultPartitio assertThat(partitionTracker.isTrackingPartitionsFor(resourceId), is(resultPartitionType.isBlocking())); } - @Test - public void testStartStopTracking() { - final Queue>> taskExecutorReleaseCalls = new ArrayBlockingQueue<>(4); - final JobMasterPartitionTracker partitionTracker = new JobMasterPartitionTrackerImpl( - new JobID(), - new TestingShuffleMaster(), - resourceId -> Optional.of(createTaskExecutorGateway(resourceId, taskExecutorReleaseCalls)) - ); - - final ResourceID executorWithTrackedPartition = new ResourceID("tracked"); - final ResourceID executorWithoutTrackedPartition = new ResourceID("untracked"); - - assertThat(partitionTracker.isTrackingPartitionsFor(executorWithTrackedPartition), is(false)); - assertThat(partitionTracker.isTrackingPartitionsFor(executorWithoutTrackedPartition), is(false)); - - partitionTracker.startTrackingPartition( - executorWithTrackedPartition, - createResultPartitionDeploymentDescriptor(new ResultPartitionID(), true)); - - assertThat(partitionTracker.isTrackingPartitionsFor(executorWithTrackedPartition), is(true)); - assertThat(partitionTracker.isTrackingPartitionsFor(executorWithoutTrackedPartition), is(false)); - - partitionTracker.stopTrackingPartitionsFor(executorWithTrackedPartition); - - assertThat(partitionTracker.isTrackingPartitionsFor(executorWithTrackedPartition), is(false)); - assertThat(partitionTracker.isTrackingPartitionsFor(executorWithoutTrackedPartition), is(false)); - } - @Test public void testReleaseCallsWithLocalResources() { final TestingShuffleMaster shuffleMaster = new TestingShuffleMaster(); @@ -125,10 +95,10 @@ public void testReleaseCallsWithLocalResources() { partitionTracker.startTrackingPartition( taskExecutorId1, - createResultPartitionDeploymentDescriptor(resultPartitionId1, true)); + AbstractPartitionTrackerTest.createResultPartitionDeploymentDescriptor(resultPartitionId1, true)); partitionTracker.startTrackingPartition( taskExecutorId2, - createResultPartitionDeploymentDescriptor(resultPartitionId2, true)); + AbstractPartitionTrackerTest.createResultPartitionDeploymentDescriptor(resultPartitionId2, true)); { partitionTracker.stopTrackingAndReleasePartitionsFor(taskExecutorId1); @@ -181,10 +151,10 @@ public void testReleaseCallsWithoutLocalResources() { partitionTracker.startTrackingPartition( taskExecutorId1, - createResultPartitionDeploymentDescriptor(resultPartitionId1, false)); + AbstractPartitionTrackerTest.createResultPartitionDeploymentDescriptor(resultPartitionId1, false)); partitionTracker.startTrackingPartition( taskExecutorId2, - createResultPartitionDeploymentDescriptor(resultPartitionId2, false)); + AbstractPartitionTrackerTest.createResultPartitionDeploymentDescriptor(resultPartitionId2, false)); { partitionTracker.stopTrackingAndReleasePartitionsFor(taskExecutorId1); @@ -225,7 +195,7 @@ public void testStopTrackingIssuesNoReleaseCalls() { partitionTracker.startTrackingPartition( taskExecutorId1, - createResultPartitionDeploymentDescriptor(resultPartitionId1, true)); + AbstractPartitionTrackerTest.createResultPartitionDeploymentDescriptor(resultPartitionId1, true)); partitionTracker.stopTrackingPartitionsFor(taskExecutorId1); @@ -233,41 +203,6 @@ public void testStopTrackingIssuesNoReleaseCalls() { assertEquals(0, shuffleMaster.externallyReleasedPartitions.size()); } - private static ResultPartitionDeploymentDescriptor createResultPartitionDeploymentDescriptor( - ResultPartitionID resultPartitionId, - boolean hasLocalResources) { - return createResultPartitionDeploymentDescriptor(resultPartitionId, ResultPartitionType.BLOCKING, hasLocalResources); - } - - private static ResultPartitionDeploymentDescriptor createResultPartitionDeploymentDescriptor( - ResultPartitionID resultPartitionId, - ResultPartitionType type, - boolean hasLocalResources) { - - return new ResultPartitionDeploymentDescriptor( - new PartitionDescriptor( - new IntermediateDataSetID(), - resultPartitionId.getPartitionId(), - type, - 1, - 0), - new ShuffleDescriptor() { - @Override - public ResultPartitionID getResultPartitionID() { - return resultPartitionId; - } - - @Override - public Optional storesLocalResourcesOn() { - return hasLocalResources - ? Optional.of(ResourceID.generate()) - : Optional.empty(); - } - }, - 1, - true); - } - private static TaskExecutorGateway createTaskExecutorGateway(ResourceID taskExecutorId, Collection>> releaseCalls) { return new TestingTaskExecutorGatewayBuilder() .setReleaseOrPromotePartitionsConsumer((jobId, partitionToRelease, partitionsToPromote) -> releaseCalls.add(Tuple3.of(taskExecutorId, jobId, partitionToRelease))) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/NoOpJobMasterPartitionTracker.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/NoOpJobMasterPartitionTracker.java index f0bf2527e1bd..f099ba6b8dfb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/NoOpJobMasterPartitionTracker.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/NoOpJobMasterPartitionTracker.java @@ -21,6 +21,7 @@ import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import java.util.Collection; +import java.util.Collections; /** * No-op implementation of {@link JobMasterPartitionTracker}. @@ -35,7 +36,8 @@ public void startTrackingPartition(ResourceID producingTaskExecutorId, ResultPar } @Override - public void stopTrackingPartitionsFor(ResourceID producingTaskExecutorId) { + public Collection> stopTrackingPartitionsFor(ResourceID key) { + return Collections.emptyList(); } @Override @@ -43,7 +45,8 @@ public void stopTrackingAndReleasePartitions(Collection resul } @Override - public void stopTrackingPartitions(Collection resultPartitionIds) { + public Collection> stopTrackingPartitions(Collection resultPartitionIds) { + return Collections.emptyList(); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingJobMasterPartitionTracker.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingJobMasterPartitionTracker.java index 9a996ecf3504..0920dae24ea8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingJobMasterPartitionTracker.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/TestingJobMasterPartitionTracker.java @@ -21,6 +21,7 @@ import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import java.util.Collection; +import java.util.Collections; import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.function.Function; @@ -72,8 +73,9 @@ public void startTrackingPartition(ResourceID producingTaskExecutorId, ResultPar } @Override - public void stopTrackingPartitionsFor(ResourceID producingTaskExecutorId) { + public Collection> stopTrackingPartitionsFor(ResourceID producingTaskExecutorId) { stopTrackingAllPartitionsConsumer.accept(producingTaskExecutorId); + return Collections.emptyList(); } @Override @@ -82,8 +84,9 @@ public void stopTrackingAndReleasePartitions(Collection resul } @Override - public void stopTrackingPartitions(Collection resultPartitionIds) { + public Collection> stopTrackingPartitions(Collection resultPartitionIds) { stopTrackingPartitionsConsumer.accept(resultPartitionIds); + return Collections.emptyList(); } @Override From 470fd2f66fe1da0b640ac5154fa6562740aa25f8 Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Fri, 25 Oct 2019 14:53:31 -0700 Subject: [PATCH 359/746] [FLINK-14534][table] FunctionCatalog.getUserDefinedFunctions() should include temp functions FunctionCatalog.getUserDefinedFunctions() should include temp functions. This closes #9998. --- .../gateway/local/ExecutionContextTest.java | 4 +-- .../gateway/local/LocalExecutorITCase.java | 2 +- .../flink/table/catalog/FunctionCatalog.java | 36 +++++++++++++------ 3 files changed, 29 insertions(+), 13 deletions(-) diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/ExecutionContextTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/ExecutionContextTest.java index a3a9ce9ce37b..e8f114541352 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/ExecutionContextTest.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/ExecutionContextTest.java @@ -158,7 +158,7 @@ public void testDatabases() throws Exception { public void testFunctions() throws Exception { final ExecutionContext context = createDefaultExecutionContext(); final TableEnvironment tableEnv = context.createEnvironmentInstance().getTableEnvironment(); - final String[] expected = new String[]{"scalarUDF", "tableUDF", "aggregateUDF"}; + final String[] expected = new String[]{"scalarudf", "tableudf", "aggregateudf"}; final String[] actual = tableEnv.listUserDefinedFunctions(); Arrays.sort(expected); Arrays.sort(actual); @@ -225,7 +225,7 @@ public void testTemporalTables() throws Exception { tableEnv.listTables()); assertArrayEquals( - new String[]{"SourceTemporalTable", "ViewTemporalTable"}, + new String[]{"sourcetemporaltable", "viewtemporaltable"}, tableEnv.listUserDefinedFunctions()); assertArrayEquals( 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 f4c80aa6c2aa..923d14149ac8 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 @@ -223,7 +223,7 @@ public void testListUserDefinedFunctions() throws Exception { final List actualTables = executor.listUserDefinedFunctions(session); - final List expectedTables = Arrays.asList("aggregateUDF", "tableUDF", "scalarUDF"); + final List expectedTables = Arrays.asList("aggregateudf", "tableudf", "scalarudf"); assertEquals(expectedTables, actualTables); } 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 6527e9fdd8c4..2b3d6dc9ec57 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 @@ -193,14 +193,22 @@ public void registerTempCatalogAggregateFunction( ); } + /** + * Get names of all user defined functions, including temp system functions, temp catalog functions and catalog functions + * in the current catalog and current database. + */ public String[] getUserDefinedFunctions() { return getUserDefinedFunctionNames().toArray(new String[0]); } + /** + * Get names of all functions, including temp system functions, system functions, temp catalog functions and catalog functions + * in the current catalog and current database. + */ public String[] getFunctions() { Set result = getUserDefinedFunctionNames(); - // Get built-in functions + // add system functions result.addAll(moduleManager.listFunctions()); return result.toArray(new String[0]); @@ -209,20 +217,28 @@ public String[] getFunctions() { private Set getUserDefinedFunctionNames() { Set result = new HashSet<>(); - // Get functions in catalog - Catalog catalog = catalogManager.getCatalog(catalogManager.getCurrentCatalog()).get(); + // add temp system functions + result.addAll(tempSystemFunctions.keySet()); + + String currentCatalog = catalogManager.getCurrentCatalog(); + String currentDatabase = catalogManager.getCurrentDatabase(); + + // add temp catalog functions + result.addAll(tempCatalogFunctions.keySet().stream() + .filter(oi -> oi.getCatalogName().equals(currentCatalog) + && oi.getDatabaseName().equals(currentDatabase)) + .map(oi -> oi.getObjectName()) + .collect(Collectors.toSet()) + ); + + // add catalog functions + Catalog catalog = catalogManager.getCatalog(currentCatalog).get(); try { - result.addAll(catalog.listFunctions(catalogManager.getCurrentDatabase())); + result.addAll(catalog.listFunctions(currentDatabase)); } catch (DatabaseNotExistException e) { // Ignore since there will always be a current database of the current catalog } - // Get functions registered in memory - result.addAll( - tempSystemFunctions.values().stream() - .map(FunctionDefinition::toString) - .collect(Collectors.toSet())); - return result; } From 9029b2547553afa63695c18516ca69f2a0c86f02 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Tue, 29 Oct 2019 00:40:00 +0800 Subject: [PATCH 360/746] [hotfix][runtime] Fix checkstyle violations in ExecutionVertex --- .../apache/flink/runtime/executiongraph/ExecutionVertex.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 500691b0496b..e01470c4eb25 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 @@ -822,7 +822,7 @@ boolean checkInputDependencyConstraints() { * Get whether an input of the vertex is consumable. * An input is consumable when when any partition in it is consumable. * - * Note that a BLOCKING result partition is only consumable when all partitions in the result are FINISHED. + *

    Note that a BLOCKING result partition is only consumable when all partitions in the result are FINISHED. * * @return whether the input is consumable */ From 2377de590059cc20e0ea4af12e104362d8d66a16 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Tue, 29 Oct 2019 22:21:50 +0800 Subject: [PATCH 361/746] [hotfix][runtime] Add overloaded method CheckpointCoordinator#restoreLatestCheckpointedState - New method expects a task Set instead of a Map. This is possible because the Map's key was not used. - Deprecate the old method. --- .../checkpoint/CheckpointCoordinator.java | 35 ++++++++++++++++++- .../checkpoint/StateAssignmentOperation.java | 14 ++++---- .../runtime/scheduler/SchedulerBase.java | 3 +- .../CheckpointCoordinatorMasterHooksTest.java | 10 +++--- .../CheckpointCoordinatorRestoringTest.java | 34 +++++++++--------- .../checkpoint/CheckpointCoordinatorTest.java | 8 +++-- .../CheckpointStateRestoreTest.java | 18 +++++----- 7 files changed, 79 insertions(+), 43 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index 6d34d8c1ccb8..8cabf675ea9b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -56,10 +56,12 @@ import java.io.IOException; import java.util.ArrayDeque; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.LinkedHashMap; 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; @@ -1049,11 +1051,42 @@ private void triggerQueuedRequests() { * that restores non-partitioned state from this * checkpoint. */ + @Deprecated public boolean restoreLatestCheckpointedState( Map tasks, boolean errorIfNoCheckpoint, boolean allowNonRestoredState) throws Exception { + return restoreLatestCheckpointedState(new HashSet<>(tasks.values()), errorIfNoCheckpoint, allowNonRestoredState); + } + + /** + * Restores the latest checkpointed state. + * + * @param tasks Set of job vertices to restore. State for these vertices is + * restored via {@link Execution#setInitialState(JobManagerTaskRestore)}. + * @param errorIfNoCheckpoint Fail if no completed checkpoint is available to + * restore from. + * @param allowNonRestoredState Allow checkpoint state that cannot be mapped + * to any job vertex in tasks. + * @return true if state was restored, false otherwise. + * @throws IllegalStateException If the CheckpointCoordinator is shut down. + * @throws IllegalStateException If no completed checkpoint is available and + * the failIfNoCheckpoint flag has been set. + * @throws IllegalStateException If the checkpoint contains state that cannot be + * mapped to any job vertex in tasks and the + * allowNonRestoredState flag has not been set. + * @throws IllegalStateException If the max parallelism changed for an operator + * that restores state from this checkpoint. + * @throws IllegalStateException If the parallelism changed for an operator + * that restores non-partitioned state from this + * checkpoint. + */ + public boolean restoreLatestCheckpointedState( + final Set tasks, + final boolean errorIfNoCheckpoint, + final boolean allowNonRestoredState) throws Exception { + synchronized (lock) { if (shutdown) { throw new IllegalStateException("CheckpointCoordinator is shut down"); @@ -1162,7 +1195,7 @@ public boolean restoreSavepoint( LOG.info("Reset the checkpoint ID of job {} to {}.", job, nextCheckpointId); - return restoreLatestCheckpointedState(tasks, true, allowNonRestored); + return restoreLatestCheckpointedState(new HashSet<>(tasks.values()), true, allowNonRestored); } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java index fdb62ebf0c97..ac7a9d8b06fb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java @@ -22,7 +22,6 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.executiongraph.Execution; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; -import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.OperatorInstanceID; import org.apache.flink.runtime.state.KeyGroupRange; @@ -55,7 +54,7 @@ public class StateAssignmentOperation { private static final Logger LOG = LoggerFactory.getLogger(StateAssignmentOperation.class); - private final Map tasks; + private final Set tasks; private final Map operatorStates; private final long restoreCheckpointId; @@ -63,7 +62,7 @@ public class StateAssignmentOperation { public StateAssignmentOperation( long restoreCheckpointId, - Map tasks, + Set tasks, Map operatorStates, boolean allowNonRestoredState) { @@ -78,8 +77,7 @@ public void assignStates() { checkStateMappingCompleteness(allowNonRestoredState, operatorStates, tasks); - for (Map.Entry task : this.tasks.entrySet()) { - final ExecutionJobVertex executionJobVertex = task.getValue(); + for (ExecutionJobVertex executionJobVertex : this.tasks) { // find the states of all operators belonging to this task List operatorIDs = executionJobVertex.getOperatorIDs(); @@ -106,7 +104,7 @@ public void assignStates() { continue; } - assignAttemptState(task.getValue(), operatorStates); + assignAttemptState(executionJobVertex, operatorStates); } } @@ -550,10 +548,10 @@ private static void checkParallelismPreconditions(OperatorState operatorState, E private static void checkStateMappingCompleteness( boolean allowNonRestoredState, Map operatorStates, - Map tasks) { + Set tasks) { Set allOperatorIDs = new HashSet<>(); - for (ExecutionJobVertex executionJobVertex : tasks.values()) { + for (ExecutionJobVertex executionJobVertex : tasks) { allOperatorIDs.addAll(executionJobVertex.getOperatorIDs()); } for (Map.Entry operatorGroupStateEntry : operatorStates.entrySet()) { 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 7161bbab9e89..37a604e4b152 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 @@ -88,6 +88,7 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.util.Collection; +import java.util.HashSet; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; @@ -202,7 +203,7 @@ private ExecutionGraph createAndRestoreExecutionGraph( if (checkpointCoordinator != null) { // check whether we find a valid checkpoint if (!checkpointCoordinator.restoreLatestCheckpointedState( - newExecutionGraph.getAllVertices(), + new HashSet<>(newExecutionGraph.getAllVertices().values()), false, false)) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java index 7c9a3578aedc..51c8b9844be3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java @@ -24,10 +24,8 @@ import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor; import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; -import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.executiongraph.ExecutionVertex; import org.apache.flink.runtime.jobgraph.JobStatus; -import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; @@ -144,7 +142,7 @@ public void testHookReset() throws Exception { // initialize the hooks cc.restoreLatestCheckpointedState( - Collections.emptyMap(), + Collections.emptySet(), false, false); verify(hook1, times(1)).reset(); @@ -284,7 +282,7 @@ public void testHooksAreCalledOnRestore() throws Exception { cc.getCheckpointStore().addCheckpoint(checkpoint); cc.restoreLatestCheckpointedState( - Collections.emptyMap(), + Collections.emptySet(), true, false); @@ -339,7 +337,7 @@ public void checkUnMatchedStateOnRestore() throws Exception { // since we have unmatched state, this should fail try { cc.restoreLatestCheckpointedState( - Collections.emptyMap(), + Collections.emptySet(), true, false); fail("exception expected"); @@ -348,7 +346,7 @@ public void checkUnMatchedStateOnRestore() throws Exception { // permitting unmatched state should succeed cc.restoreLatestCheckpointedState( - Collections.emptyMap(), + Collections.emptySet(), true, true); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java index e0be7eecc481..404e2a6f588d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java @@ -60,9 +60,11 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.concurrent.CompletableFuture; import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.compareKeyedState; @@ -212,10 +214,10 @@ public void testRestoreLatestCheckpointedState() throws Exception { store.shutdown(JobStatus.SUSPENDED); // restore the store - Map tasks = new HashMap<>(); + Set tasks = new HashSet<>(); - tasks.put(jobVertexID1, jobVertex1); - tasks.put(jobVertexID2, jobVertex2); + tasks.add(jobVertex1); + tasks.add(jobVertex2); coord.restoreLatestCheckpointedState(tasks, true, false); @@ -273,9 +275,9 @@ private void testRestoreLatestCheckpointIsPreferSavepoint(boolean isPreferCheckp ExecutionJobVertex stateless = mockExecutionJobVertex(statelessId, new ExecutionVertex[] { stateless1 }); - Map map = new HashMap(); - map.put(statefulId, stateful); - map.put(statelessId, stateless); + Set tasks = new HashSet<>(); + tasks.add(stateful); + tasks.add(stateless); CompletedCheckpointStore store = new RecoverableCompletedCheckpointStore(2); @@ -357,7 +359,7 @@ private void testRestoreLatestCheckpointIsPreferSavepoint(boolean isPreferCheckp assertNotNull(savepointFuture.get()); //restore and jump the latest savepoint - coord.restoreLatestCheckpointedState(map, true, false); + coord.restoreLatestCheckpointedState(tasks, true, false); //compare and see if it used the checkpoint's subtaskStates BaseMatcher matcher = new BaseMatcher() { @@ -516,7 +518,7 @@ private void testRestoreLatestCheckpointedStateWithChangingParallelism(boolean s assertEquals(1, completedCheckpoints.size()); - Map tasks = new HashMap<>(); + Set tasks = new HashSet<>(); List newKeyGroupPartitions2 = StateAssignmentOperation.createKeyGroupPartitions(maxParallelism2, newParallelism2); @@ -532,8 +534,8 @@ private void testRestoreLatestCheckpointedStateWithChangingParallelism(boolean s newParallelism2, maxParallelism2); - tasks.put(jobVertexID1, newJobVertex1); - tasks.put(jobVertexID2, newJobVertex2); + tasks.add(newJobVertex1); + tasks.add(newJobVertex2); coord.restoreLatestCheckpointedState(tasks, true, false); // verify the restored state @@ -679,7 +681,7 @@ public void testRestoreLatestCheckpointFailureWhenMaxParallelismChanges() throws assertEquals(1, completedCheckpoints.size()); - Map tasks = new HashMap<>(); + Set tasks = new HashSet<>(); int newMaxParallelism1 = 20; int newMaxParallelism2 = 42; @@ -694,8 +696,8 @@ public void testRestoreLatestCheckpointFailureWhenMaxParallelismChanges() throws parallelism2, newMaxParallelism2); - tasks.put(jobVertexID1, newJobVertex1); - tasks.put(jobVertexID2, newJobVertex2); + tasks.add(newJobVertex1); + tasks.add(newJobVertex2); coord.restoreLatestCheckpointedState(tasks, true, false); @@ -846,10 +848,10 @@ public void testStateRecoveryWithTopologyChange(int scaleType) throws Exception newParallelism2, maxParallelism2); - Map tasks = new HashMap<>(); + Set tasks = new HashSet<>(); - tasks.put(id5.f0, newJobVertex1); - tasks.put(id3.f0, newJobVertex2); + tasks.add(newJobVertex1); + tasks.add(newJobVertex2); JobID jobID = new JobID(); StandaloneCompletedCheckpointStore standaloneCompletedCheckpointStore = diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java index 22d59bcc00ab..b3894aa33852 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java @@ -68,10 +68,12 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Random; +import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -2252,7 +2254,7 @@ public void testCheckpointStatsTrackerRestoreCallback() throws Exception { CheckpointStatsTracker tracker = mock(CheckpointStatsTracker.class); coord.setCheckpointStatsTracker(tracker); - assertTrue(coord.restoreLatestCheckpointedState(Collections.emptyMap(), false, true)); + assertTrue(coord.restoreLatestCheckpointedState(Collections.emptySet(), false, true)); verify(tracker, times(1)) .reportRestoredCheckpoint(any(RestoredCheckpointStats.class)); @@ -2380,8 +2382,8 @@ public void testSharedStateRegistrationOnRestore() throws Exception { store.shutdown(JobStatus.SUSPENDED); // restore the store - Map tasks = new HashMap<>(); - tasks.put(jobVertexID1, jobVertex1); + Set tasks = new HashSet<>(); + tasks.add(jobVertex1); coord.restoreLatestCheckpointedState(tasks, true, false); // validate that all shared states are registered again after the recovery. diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java index ca5a42e28fc4..32b9583fa640 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java @@ -46,9 +46,11 @@ import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; @@ -108,9 +110,9 @@ public void testSetState() { ExecutionJobVertex stateless = mockExecutionJobVertex(statelessId, new ExecutionVertex[] { stateless1, stateless2 }); - Map map = new HashMap(); - map.put(statefulId, stateful); - map.put(statelessId, stateless); + Set tasks = new HashSet<>(); + tasks.add(stateful); + tasks.add(stateless); CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( 200000L, @@ -163,7 +165,7 @@ public void testSetState() { assertEquals(0, coord.getNumberOfPendingCheckpoints()); // let the coordinator inject the state - coord.restoreLatestCheckpointedState(map, true, false); + coord.restoreLatestCheckpointedState(tasks, true, false); // verify that each stateful vertex got the state @@ -222,7 +224,7 @@ public void testNoCheckpointAvailable() { failureManager); try { - coord.restoreLatestCheckpointedState(new HashMap(), true, false); + coord.restoreLatestCheckpointedState(Collections.emptySet(), true, false); fail("this should throw an exception"); } catch (IllegalStateException e) { @@ -259,9 +261,9 @@ public void testNonRestoredState() throws Exception { ExecutionJobVertex jobVertex1 = mockExecutionJobVertex(jobVertexId1, new ExecutionVertex[] { vertex11, vertex12, vertex13 }); ExecutionJobVertex jobVertex2 = mockExecutionJobVertex(jobVertexId2, new ExecutionVertex[] { vertex21, vertex22 }); - Map tasks = new HashMap<>(); - tasks.put(jobVertexId1, jobVertex1); - tasks.put(jobVertexId2, jobVertex2); + Set tasks = new HashSet<>(); + tasks.add(jobVertex1); + tasks.add(jobVertex2); CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( Integer.MAX_VALUE, From 5faa81a7fd427fa8c049177b65c4b43153da96e7 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Thu, 17 Oct 2019 10:56:28 +0800 Subject: [PATCH 362/746] [hotfix][tests] Introduce TestMasterHook --- .../CheckpointCoordinatorMasterHooksTest.java | 5 +- .../checkpoint/hooks/TestMasterHook.java | 91 +++++++++++++++++++ 2 files changed, 95 insertions(+), 1 deletion(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/hooks/TestMasterHook.java diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java index 51c8b9844be3..953ec0dba4ca 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java @@ -479,7 +479,10 @@ private static T mockGeneric(Class clazz) { // ------------------------------------------------------------------------ - private static final class StringSerializer implements SimpleVersionedSerializer { + /** + * A test implementation of {@link SimpleVersionedSerializer} for String type. + */ + public static final class StringSerializer implements SimpleVersionedSerializer { static final int VERSION = 77; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/hooks/TestMasterHook.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/hooks/TestMasterHook.java new file mode 100644 index 000000000000..bc651e45d39a --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/hooks/TestMasterHook.java @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.checkpoint.hooks; + +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorMasterHooksTest; +import org.apache.flink.runtime.checkpoint.MasterTriggerRestoreHook; + +import javax.annotation.Nullable; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link MasterTriggerRestoreHook} implementation for testing. + */ +public class TestMasterHook implements MasterTriggerRestoreHook { + + private static final String DEFAULT_STATE = "default"; + + private final String id; + + private int restoreCount = 0; + + private boolean failOnRestore = false; + + private TestMasterHook(String id) { + this.id = checkNotNull(id); + } + + public static TestMasterHook fromId(String id) { + return new TestMasterHook(id); + } + + @Override + public String getIdentifier() { + return id; + } + + @Override + public CompletableFuture triggerCheckpoint( + final long checkpointId, + final long timestamp, + final Executor executor) { + + return CompletableFuture.completedFuture(DEFAULT_STATE); + } + + @Override + public void restoreCheckpoint(final long checkpointId, @Nullable final String checkpointData) throws Exception { + restoreCount++; + if (failOnRestore) { + throw new Exception("Failing mast hook state restore on purpose."); + } + } + + @Override + public SimpleVersionedSerializer createCheckpointDataSerializer() { + return new CheckpointCoordinatorMasterHooksTest.StringSerializer(); + } + + public int getRestoreCount() { + return restoreCount; + } + + public void enableFailOnRestore() { + this.failOnRestore = true; + } + + public void disableFailOnRestore() { + this.failOnRestore = false; + } +} From 9a389464cdf6425be74a56fc898b82c750e03e73 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Tue, 29 Oct 2019 22:27:24 +0800 Subject: [PATCH 363/746] [FLINK-14389][runtime] Restore task state before restarting tasks in DefaultScheduler This closes #9920. --- .../executiongraph/ExecutionVertex.java | 10 +- .../runtime/scheduler/DefaultScheduler.java | 9 +- .../runtime/scheduler/SchedulerBase.java | 36 ++++- .../scheduler/DefaultSchedulerTest.java | 142 ++++++++++++++++++ 4 files changed, 185 insertions(+), 12 deletions(-) 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 e01470c4eb25..cd94e49bc056 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 @@ -616,14 +616,8 @@ public Execution resetForNewExecution(final long timestamp, final long originati } } - public void resetForNewExecutionIfInTerminalState() { - if (isExecutionInTerminalState()) { - resetForNewExecutionInternal(System.currentTimeMillis(), getExecutionGraph().getGlobalModVersion()); - } - } - - private boolean isExecutionInTerminalState() { - return currentExecution.getState().isTerminal(); + public void resetForNewExecution() { + resetForNewExecutionInternal(System.currentTimeMillis(), getExecutionGraph().getGlobalModVersion()); } private Execution resetForNewExecutionInternal(final long timestamp, final long originatingGlobalModVersion) { 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 a8e2f05ac380..b46eb7ff2610 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 @@ -210,7 +210,14 @@ private Runnable restartTasks(final Set executionVertexV return () -> { final Set verticesToRestart = executionVertexVersioner.getUnmodifiedExecutionVertices(executionVertexVersions); - resetForNewExecutionIfInTerminalState(verticesToRestart); + resetForNewExecutions(verticesToRestart); + + try { + restoreState(verticesToRestart); + } catch (Throwable t) { + handleGlobalFailure(t); + return; + } schedulingStrategy.restartTasks(verticesToRestart); }; 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 37a604e4b152..27c2856fe3d3 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 @@ -30,6 +30,8 @@ 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.CheckpointException; +import org.apache.flink.runtime.checkpoint.CheckpointFailureReason; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; @@ -90,6 +92,7 @@ import java.util.Collection; import java.util.HashSet; import java.util.Optional; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.Executor; @@ -278,9 +281,36 @@ private void tryRestoreExecutionGraphFromSavepoint(ExecutionGraph executionGraph } } - protected void resetForNewExecutionIfInTerminalState(final Collection verticesToDeploy) { - verticesToDeploy.forEach(executionVertexId -> getExecutionVertex(executionVertexId) - .resetForNewExecutionIfInTerminalState()); + protected void resetForNewExecutions(final Collection vertices) { + vertices.forEach(executionVertexId -> getExecutionVertex(executionVertexId) + .resetForNewExecution()); + } + + protected void restoreState(final Set vertices) throws Exception { + // if there is checkpointed state, reload it into the executions + if (executionGraph.getCheckpointCoordinator() != null) { + // abort pending checkpoints to + // i) enable new checkpoint triggering without waiting for last checkpoint expired. + // ii) ensure the EXACTLY_ONCE semantics if needed. + executionGraph.getCheckpointCoordinator().abortPendingCheckpoints( + new CheckpointException(CheckpointFailureReason.JOB_FAILOVER_REGION)); + + executionGraph.getCheckpointCoordinator().restoreLatestCheckpointedState( + getInvolvedExecutionJobVertices(vertices), + false, + true); + } + } + + private Set getInvolvedExecutionJobVertices( + final Set executionVertices) { + + final Set tasks = new HashSet<>(); + for (ExecutionVertexID executionVertexID : executionVertices) { + final ExecutionVertex executionVertex = getExecutionVertex(executionVertexID); + tasks.add(executionVertex.getJobVertex()); + } + return tasks; } protected void transitionToScheduled(final Collection verticesToDeploy) { 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 3fa2ad95f5da..ac60b79610bf 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 @@ -24,7 +24,10 @@ 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.CheckpointCoordinator; +import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; +import org.apache.flink.runtime.checkpoint.hooks.TestMasterHook; import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor; import org.apache.flink.runtime.execution.ExecutionState; @@ -40,8 +43,12 @@ 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.jobgraph.tasks.CheckpointCoordinatorConfiguration; +import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException; +import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; 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; @@ -65,6 +72,7 @@ import org.junit.Before; import org.junit.Test; +import java.util.ArrayList; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -388,6 +396,140 @@ public void vertexIsNotAffectedByOutdatedDeployment() { assertThat(sv1.getState(), is(equalTo(ExecutionState.SCHEDULED))); } + @Test + public void abortPendingCheckpointsWhenRestartingTasks() { + final JobGraph jobGraph = singleNonParallelJobVertexJobGraph(); + enableCheckpointing(jobGraph); + + 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.RUNNING)); + + final CheckpointCoordinator checkpointCoordinator = getCheckpointCoordinator(scheduler); + + checkpointCoordinator.triggerCheckpoint(System.currentTimeMillis(), false); + assertThat(checkpointCoordinator.getNumberOfPendingCheckpoints(), is(equalTo(1))); + + scheduler.updateTaskExecutionState(new TaskExecutionState(jobGraph.getJobID(), attemptId, ExecutionState.FAILED)); + taskRestartExecutor.triggerScheduledTasks(); + assertThat(checkpointCoordinator.getNumberOfPendingCheckpoints(), is(equalTo(0))); + } + + @Test + public void restoreStateWhenRestartingTasks() throws Exception { + final JobGraph jobGraph = singleNonParallelJobVertexJobGraph(); + enableCheckpointing(jobGraph); + + 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.RUNNING)); + + final CheckpointCoordinator checkpointCoordinator = getCheckpointCoordinator(scheduler); + + // register a stateful master hook to help verify state restore + final TestMasterHook masterHook = TestMasterHook.fromId("testHook"); + checkpointCoordinator.addMasterHook(masterHook); + + // complete one checkpoint for state restore + checkpointCoordinator.triggerCheckpoint(System.currentTimeMillis(), false); + final long checkpointId = checkpointCoordinator.getPendingCheckpoints().keySet().iterator().next(); + acknowledgePendingCheckpoint(scheduler, checkpointId); + + scheduler.updateTaskExecutionState(new TaskExecutionState(jobGraph.getJobID(), attemptId, ExecutionState.FAILED)); + taskRestartExecutor.triggerScheduledTasks(); + assertThat(masterHook.getRestoreCount(), is(equalTo(1))); + } + + @Test + public void failGlobalWhenRestoringStateFails() throws Exception { + final JobGraph jobGraph = singleNonParallelJobVertexJobGraph(); + final JobVertex onlyJobVertex = getOnlyJobVertex(jobGraph); + enableCheckpointing(jobGraph); + + 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.RUNNING)); + + final CheckpointCoordinator checkpointCoordinator = getCheckpointCoordinator(scheduler); + + // register a master hook to fail state restore + final TestMasterHook masterHook = TestMasterHook.fromId("testHook"); + masterHook.enableFailOnRestore(); + checkpointCoordinator.addMasterHook(masterHook); + + // complete one checkpoint for state restore + checkpointCoordinator.triggerCheckpoint(System.currentTimeMillis(), false); + final long checkpointId = checkpointCoordinator.getPendingCheckpoints().keySet().iterator().next(); + acknowledgePendingCheckpoint(scheduler, checkpointId); + + scheduler.updateTaskExecutionState(new TaskExecutionState(jobGraph.getJobID(), attemptId, ExecutionState.FAILED)); + taskRestartExecutor.triggerScheduledTasks(); + final List deployedExecutionVertices = testExecutionVertexOperations.getDeployedVertices(); + + // the first task failover should be skipped on state restore failure + final ExecutionVertexID executionVertexId = new ExecutionVertexID(onlyJobVertex.getID(), 0); + assertThat(deployedExecutionVertices, contains(executionVertexId)); + + // a global failure should be triggered on state restore failure + masterHook.disableFailOnRestore(); + taskRestartExecutor.triggerScheduledTasks(); + assertThat(deployedExecutionVertices, contains(executionVertexId, executionVertexId)); + } + + private void acknowledgePendingCheckpoint(final SchedulerBase scheduler, final long checkpointId) throws Exception { + final CheckpointCoordinator checkpointCoordinator = getCheckpointCoordinator(scheduler); + + for (ArchivedExecutionVertex executionVertex : scheduler.requestJob().getAllExecutionVertices()) { + final ExecutionAttemptID attemptId = executionVertex.getCurrentExecutionAttempt().getAttemptId(); + final AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint( + scheduler.getJobGraph().getJobID(), + attemptId, + checkpointId); + checkpointCoordinator.receiveAcknowledgeMessage(acknowledgeCheckpoint, "Unknown location"); + } + } + + private void enableCheckpointing(final JobGraph jobGraph) { + final List triggerVertices = new ArrayList<>(); + final List ackVertices = new ArrayList<>(); + final List commitVertices = new ArrayList<>(); + + for (JobVertex vertex : jobGraph.getVertices()) { + if (vertex.isInputVertex()) { + triggerVertices.add(vertex.getID()); + } + commitVertices.add(vertex.getID()); + ackVertices.add(vertex.getID()); + } + + jobGraph.setSnapshotSettings( + new JobCheckpointingSettings( + triggerVertices, + ackVertices, + commitVertices, + new CheckpointCoordinatorConfiguration( + Long.MAX_VALUE, // disable periodical checkpointing + 10 * 60 * 1000, + 0, + 1, + CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION, + false, + false, + 0), + null)); + } + + private CheckpointCoordinator getCheckpointCoordinator(final SchedulerBase scheduler) { + // TODO: get CheckpointCoordinator from the scheduler directly after it is factored out from ExecutionGraph + return scheduler.getExecutionGraph().getCheckpointCoordinator(); + } + private void waitForTermination(final DefaultScheduler scheduler) throws Exception { scheduler.getTerminationFuture().get(TIMEOUT_MS, TimeUnit.MILLISECONDS); } From ffe3a61a7ed2aa856288ecd293550bdead70119d Mon Sep 17 00:00:00 2001 From: yanghua Date: Mon, 28 Oct 2019 15:25:27 +0800 Subject: [PATCH 364/746] [FLINK-12526][runtime] Remove STATE_UPDATER in ExecutionGraph This closes #10014. --- .../flink/runtime/executiongraph/ExecutionGraph.java | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 19102ddbbfdc..25a4a801c3a1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -119,7 +119,6 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLongFieldUpdater; -import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import java.util.stream.Collectors; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -171,11 +170,6 @@ */ public class ExecutionGraph implements AccessExecutionGraph { - /** In place updater for the execution graph's current state. Avoids having to use an - * AtomicReference and thus makes the frequent read access a bit faster. */ - private static final AtomicReferenceFieldUpdater STATE_UPDATER = - AtomicReferenceFieldUpdater.newUpdater(ExecutionGraph.class, JobStatus.class, "state"); - /** In place updater for the execution graph's current global recovery version. * Avoids having to use an AtomicLong and thus makes the frequent read access a bit faster */ private static final AtomicLongFieldUpdater GLOBAL_VERSION_UPDATER = @@ -1377,7 +1371,8 @@ private boolean transitionState(JobStatus current, JobStatus newState, Throwable } // now do the actual state transition - if (STATE_UPDATER.compareAndSet(this, current, newState)) { + if (state == current) { + state = newState; LOG.info("Job {} ({}) switched from state {} to {}.", getJobName(), getJobID(), current, newState, error); stateTimestamps[newState.ordinal()] = System.currentTimeMillis(); From 1fd1c18ce6ca73d7444645e2364c41867279f6c5 Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Tue, 29 Oct 2019 15:53:27 -0700 Subject: [PATCH 365/746] [FLINK-14219][table] support ambiguous function reference Support ambiguous function reference. This closes #10052. --- .../flink/table/catalog/FunctionCatalog.java | 72 +++++--------- .../table/catalog/FunctionCatalogTest.java | 98 +++++++++++++++++-- 2 files changed, 117 insertions(+), 53 deletions(-) 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 2b3d6dc9ec57..29208e6fec35 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 @@ -250,51 +250,7 @@ public Optional lookupFunction(FunctionIdentifier identif return resolvePreciseFunctionReference(identifier.getIdentifier().get()); } else { // ambiguous function reference - - String functionName = identifier.getSimpleName().get(); - - FunctionDefinition userCandidate; - - Catalog catalog = catalogManager.getCatalog(catalogManager.getCurrentCatalog()).get(); - try { - CatalogFunction catalogFunction = catalog.getFunction( - new ObjectPath(catalogManager.getCurrentDatabase(), functionName) - ); - - if (catalog.getFunctionDefinitionFactory().isPresent()) { - userCandidate = catalog.getFunctionDefinitionFactory().get().createFunctionDefinition(functionName, catalogFunction); - } else { - userCandidate = FunctionDefinitionUtil.createFunctionDefinition(functionName, catalogFunction); - } - - return Optional.of( - new FunctionLookup.Result( - FunctionIdentifier.of( - ObjectIdentifier.of( - catalogManager.getCurrentCatalog(), - catalogManager.getCurrentDatabase(), - functionName)), - userCandidate) - ); - - } catch (FunctionNotExistException e) { - // ignore - } - - // If no corresponding function is found in catalog, check in-memory functions - userCandidate = tempSystemFunctions.get(functionName); - - final Optional foundDefinition; - if (userCandidate != null) { - foundDefinition = Optional.of(userCandidate); - } else { - foundDefinition = moduleManager.getFunctionDefinition(functionName); - } - - return foundDefinition.map(d -> new FunctionLookup.Result( - FunctionIdentifier.of(identifier.getSimpleName().get()), - d) - ); + return resolveAmbiguousFunctionReference(identifier.getSimpleName().get()); } } @@ -339,7 +295,31 @@ private Optional resolvePreciseFunctionReference(ObjectId return Optional.empty(); } - @Override + private Optional resolveAmbiguousFunctionReference(String funcName) { + // resolve order: + // 1. Temporary system functions + // 2. System functions + // 3. Temporary catalog functions + // 4. Catalog functions + + if (tempSystemFunctions.containsKey(funcName)) { + return Optional.of( + new FunctionLookup.Result(FunctionIdentifier.of(funcName), tempSystemFunctions.get(funcName)) + ); + } + + Optional candidate = moduleManager.getFunctionDefinition(funcName); + if (candidate.isPresent()) { + return Optional.of( + new FunctionLookup.Result(FunctionIdentifier.of(funcName), candidate.get()) + ); + } + + return resolvePreciseFunctionReference( + ObjectIdentifier.of(catalogManager.getCurrentCatalog(), catalogManager.getCurrentDatabase(), funcName)); + } + + @Override public PlannerTypeInferenceUtil getPlannerTypeInferenceUtil() { Preconditions.checkNotNull( plannerTypeInferenceUtil, diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java index 2461cdc79625..ee94c00c807b 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java @@ -21,21 +21,24 @@ import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; import org.apache.flink.table.catalog.exceptions.FunctionAlreadyExistException; +import org.apache.flink.table.functions.FunctionDefinition; import org.apache.flink.table.functions.FunctionIdentifier; import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.functions.ScalarFunctionDefinition; +import org.apache.flink.table.module.Module; import org.apache.flink.table.module.ModuleManager; +import org.apache.flink.table.module.exceptions.ModuleAlreadyExistException; import org.junit.Before; import org.junit.Test; import java.util.Collections; import java.util.HashSet; +import java.util.Optional; import java.util.Set; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; /** @@ -44,13 +47,18 @@ public class FunctionCatalogTest { private FunctionCatalog functionCatalog; private Catalog catalog; + private ModuleManager moduleManager; + + private final String testCatalogName = "test"; + + private static final String TEST_FUNCTION_NAME = "test_function"; @Before public void init() throws DatabaseAlreadyExistException { - catalog = new GenericInMemoryCatalog("test"); - catalog.createDatabase("test", new CatalogDatabaseImpl(Collections.EMPTY_MAP, null), false); + catalog = new GenericInMemoryCatalog(testCatalogName); + moduleManager = new ModuleManager(); functionCatalog = new FunctionCatalog( - new CatalogManager("test", catalog), new ModuleManager()); + new CatalogManager(testCatalogName, catalog), moduleManager); } @Test @@ -65,7 +73,7 @@ public void testGetBuiltInFunctions() { @Test public void testPreciseFunctionReference() throws FunctionAlreadyExistException, DatabaseNotExistException { - ObjectIdentifier oi = ObjectIdentifier.of("test", "test", "test_function"); + ObjectIdentifier oi = ObjectIdentifier.of(testCatalogName, GenericInMemoryCatalog.DEFAULT_DB, TEST_FUNCTION_NAME); // test no function is found assertFalse(functionCatalog.lookupFunction(FunctionIdentifier.of(oi)).isPresent()); @@ -79,7 +87,6 @@ public void testPreciseFunctionReference() throws FunctionAlreadyExistException, assertFalse(result.getFunctionIdentifier().getSimpleName().isPresent()); assertEquals(oi, result.getFunctionIdentifier().getIdentifier().get()); - assertNotNull(result.getFunctionDefinition()); assertTrue(((ScalarFunctionDefinition) result.getFunctionDefinition()).getScalarFunction() instanceof TestFunction1); // test temp catalog function is found @@ -92,10 +99,73 @@ public void testPreciseFunctionReference() throws FunctionAlreadyExistException, assertFalse(result.getFunctionIdentifier().getSimpleName().isPresent()); assertEquals(oi, result.getFunctionIdentifier().getIdentifier().get()); - assertNotNull(result.getFunctionDefinition()); assertTrue(((ScalarFunctionDefinition) result.getFunctionDefinition()).getScalarFunction() instanceof TestFunction2); } + @Test + public void testAmbiguousFunctionReference() throws FunctionAlreadyExistException, DatabaseNotExistException, ModuleAlreadyExistException { + ObjectIdentifier oi = ObjectIdentifier.of( + testCatalogName, + GenericInMemoryCatalog.DEFAULT_DB, + TEST_FUNCTION_NAME); + + // test no function is found + assertFalse(functionCatalog.lookupFunction(FunctionIdentifier.of(TEST_FUNCTION_NAME)).isPresent()); + + // test catalog function is found + catalog.createFunction( + oi.toObjectPath(), + new CatalogFunctionImpl(TestFunction1.class.getName(), Collections.emptyMap()), false); + + FunctionLookup.Result result = functionCatalog.lookupFunction(FunctionIdentifier.of(TEST_FUNCTION_NAME)).get(); + + assertFalse(result.getFunctionIdentifier().getSimpleName().isPresent()); + assertEquals(oi, result.getFunctionIdentifier().getIdentifier().get()); + assertTrue(((ScalarFunctionDefinition) result.getFunctionDefinition()).getScalarFunction() instanceof TestFunction1); + + // test temp catalog function is found + functionCatalog.registerTempCatalogScalarFunction( + oi, + new TestFunction2() + ); + + result = functionCatalog.lookupFunction(FunctionIdentifier.of(TEST_FUNCTION_NAME)).get(); + + assertFalse(result.getFunctionIdentifier().getSimpleName().isPresent()); + assertEquals(oi, result.getFunctionIdentifier().getIdentifier().get()); + assertTrue(((ScalarFunctionDefinition) result.getFunctionDefinition()).getScalarFunction() instanceof TestFunction2); + + // test system function is found + moduleManager.loadModule("test_module", new TestModule()); + + result = functionCatalog.lookupFunction(FunctionIdentifier.of(TEST_FUNCTION_NAME)).get(); + + assertEquals(TEST_FUNCTION_NAME, result.getFunctionIdentifier().getSimpleName().get()); + assertTrue(((ScalarFunctionDefinition) result.getFunctionDefinition()).getScalarFunction() instanceof TestFunction3); + + // test temp system function is found + functionCatalog.registerTempSystemScalarFunction(TEST_FUNCTION_NAME, new TestFunction4()); + + result = functionCatalog.lookupFunction(FunctionIdentifier.of(TEST_FUNCTION_NAME)).get(); + + assertEquals(TEST_FUNCTION_NAME, result.getFunctionIdentifier().getSimpleName().get()); + assertTrue(((ScalarFunctionDefinition) result.getFunctionDefinition()).getScalarFunction() instanceof TestFunction4); + } + + private static class TestModule implements Module { + @Override + public Set listFunctions() { + return new HashSet() {{ + add(TEST_FUNCTION_NAME); + }}; + } + + @Override + public Optional getFunctionDefinition(String name) { + return Optional.of(new ScalarFunctionDefinition(TEST_FUNCTION_NAME, new TestFunction3())); + } + } + /** * Testing function. */ @@ -109,4 +179,18 @@ public static class TestFunction1 extends ScalarFunction { public static class TestFunction2 extends ScalarFunction { } + + /** + * Testing function. + */ + public static class TestFunction3 extends ScalarFunction { + + } + + /** + * Testing function. + */ + public static class TestFunction4 extends ScalarFunction { + + } } From b4dfb3d6120762c3103d69e25ff05c344825e9ea Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Mon, 28 Oct 2019 18:23:04 +0800 Subject: [PATCH 366/746] [FLINK-14547][table-planner-blink] Fix UDF cannot in the join condition in blink planner for Table API. Currently, UDF can not be used in the join condition for Table API in blink planner(SQL is ok). This closes #10016. --- .../planner/plan/QueryOperationConverter.java | 8 +++++++- .../planner/plan/batch/table/JoinTest.xml | 18 ++++++++++++++++++ .../planner/plan/batch/table/JoinTest.scala | 10 ++++++++++ 3 files changed, 35 insertions(+), 1 deletion(-) diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java index b49bd28b49b4..0e5b92b67f50 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java @@ -472,9 +472,15 @@ public RexNode visit(CallExpression callExpression) { return new RexNodeExpression(convertedNode, ((ResolvedExpression) expr).getOutputDataType()); }).collect(Collectors.toList()); - CallExpression newCall = new CallExpression( + CallExpression newCall; + if (callExpression.getFunctionIdentifier().isPresent()) { + newCall = new CallExpression( callExpression.getFunctionIdentifier().get(), callExpression.getFunctionDefinition(), newChildren, callExpression.getOutputDataType()); + } else { + newCall = new CallExpression( + callExpression.getFunctionDefinition(), newChildren, callExpression.getOutputDataType()); + } return convertExprToRexNode(newCall); } 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 22028912ea3f..e8e9c0528546 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 @@ -277,4 +277,22 @@ Calc(select=[b, y]) ]]> + + + + + + + + diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/JoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/JoinTest.scala index 5bfa82f8088e..5c129d0ada43 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/JoinTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/JoinTest.scala @@ -199,6 +199,16 @@ class JoinTest extends TableTestBase { .where('a < 'd) .select('c, 'g)) } + + @Test + def testUDFInJoinCondition(): Unit = { + val util = batchTestUtil() + val ds1 = util.addTableSource[(Int, Long, String)]("left",'a, 'b, 'c) + val ds2 = util.addTableSource[(Int, Long, String)]("right",'d, 'e, 'f) + + val joinT = ds1.join(ds2, 'b === 'e && Merger('a, 'd) === 10) + util.verifyPlan(joinT) + } } object JoinTest { From 6c6ada5b36577490fde74fc8d34b784a6e0dc25a Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Mon, 28 Oct 2019 20:56:02 +0800 Subject: [PATCH 367/746] [FLINK-14556][python] Correct the package structure of cloudpickle Correct the package structure of cloud pickle, otherwise "ImportError: No module named cloudpickle" error will be thrown when running in a standalone cluster. This closes #10046. --- flink-python/lib/cloudpickle-1.2.2-src.zip | Bin 23425 -> 23105 bytes 1 file changed, 0 insertions(+), 0 deletions(-) diff --git a/flink-python/lib/cloudpickle-1.2.2-src.zip b/flink-python/lib/cloudpickle-1.2.2-src.zip index 82bc04ec2740210a4fbd2c4cd1ba0947be478b07..4d73a881f5edc19df2ef56561dfa8868c1e305a9 100644 GIT binary patch delta 557 zcmZqN&UkPOV@iNGGYc032)N9T@CP%X1P_A@Lvl`jX-Yw6a&}IterO0M1M~Tb8)H)~ zdSXi}xEUB(1l1WBSjxQ^7(k|o0QGS&q%YF)-xiR;bkT}|Vee@M1}UH^lmFU?Prl12 z#tv1#`4{6|E0IE=%Frq99{oUluYgz_sB+>JEtsOotcgXVq5aG*n4XjXsRRN6hPRF&8r93J5HF*cG+8l9 zPDqLYqLC4(Wl5tOkOcd(DoV&4DO4DlIWB&WEb#>8Z2&K%saI6qBrMpkQDH!m~in#YKR4 E0H3OKrT_o{ delta 791 zcmX@Og|TrvV|{=(GYc032&g1j`GXlyLWn_zAvq_%G^HRjIXfp+*HF($&qzNsgp+}p z#iA!R6^KhKxEUB(zA`c}u!sP4AOHkim4(ih*J8X$A%z?B?8ykeS#Z$PV?) z#u<05CWqPxh#BHiEk3!yMjEDk^CBA#2~kOGYA5duK`6T&($6f25zv#XBV_8~;mZmM zUlfmk!j_Rqju{eDn4v2HjWYp;w~jtQHfFpbi2#EdXa^{$k?dd+VVKMpDW{^z05J;~ zrwmIPmtZvu80Km)pHE&FA*Z5)qG=jXFOqdok4?T8p$XTN4YbJ+MblXu9GXDk%r$vW zBrja^g-E$bNfga0q1ZJ0csjfK1-pWMn{>f52FbUO)XK^R3N|(%EM{V0_z?l(0RU(f BwHE*Y From e41c0feaea174642c7c05d3baef7d011b47401d8 Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Thu, 24 Oct 2019 16:34:24 +0200 Subject: [PATCH 368/746] [FLINK-14522] Introduce JavaGcCleanerWrapper to find Java GC Cleaner depending on JVM version --- .../core/memory/MemorySegmentFactory.java | 5 +- .../apache/flink/core/memory/MemoryUtils.java | 9 +- .../flink/util/JavaGcCleanerWrapper.java | 255 ++++++++++++++++++ .../flink/util/JavaGcCleanerWrapperTest.java | 42 +++ 4 files changed, 302 insertions(+), 9 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/util/JavaGcCleanerWrapper.java create mode 100644 flink-core/src/test/java/org/apache/flink/util/JavaGcCleanerWrapperTest.java diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java index 3a82f5cd5d69..c297a2648047 100644 --- a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java +++ b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java @@ -110,8 +110,9 @@ public static MemorySegment allocateUnpooledOffHeapMemory(int size, Object owner * @return A new memory segment, backed by off-heap unsafe memory. */ public static MemorySegment allocateOffHeapUnsafeMemory(int size, Object owner) { - // TODO: rollbacked to direct memory because of FLINK-13985 - return allocateUnpooledOffHeapMemory(size, owner); + long address = MemoryUtils.allocateUnsafe(size); + ByteBuffer offHeapBuffer = MemoryUtils.wrapUnsafeMemoryWithByteBuffer(address, size); + return new HybridMemorySegment(offHeapBuffer, owner, MemoryUtils.createMemoryGcCleaner(offHeapBuffer, address)); } /** diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java b/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java index ce10b06d2a46..a8fa077f9e4a 100644 --- a/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java +++ b/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.JavaGcCleanerWrapper; import java.lang.reflect.Constructor; import java.lang.reflect.Field; @@ -110,13 +111,7 @@ static long allocateUnsafe(long size) { */ @SuppressWarnings("UseOfSunClasses") static Runnable createMemoryGcCleaner(Object owner, long address) { - // TODO: rollbacked to direct memory because of FLINK-13985 - // The release call is wrapped with the sun.misc.Cleaner - // which will schedule it before GC is run for the owner object (not reachable in user code). - // but only if sun.misc.Cleaner::clean has not been already called explicitly by user before. - // If sun.misc.Cleaner::clean is called after GC it will not call the release. - // This way we guarantee that there will always be a release at some point but only once. - return null; // sun.misc.Cleaner.create(owner, () -> releaseUnsafe(address))::clean; + return JavaGcCleanerWrapper.create(owner, () -> releaseUnsafe(address)); } private static void releaseUnsafe(long address) { diff --git a/flink-core/src/main/java/org/apache/flink/util/JavaGcCleanerWrapper.java b/flink-core/src/main/java/org/apache/flink/util/JavaGcCleanerWrapper.java new file mode 100644 index 000000000000..3c1918f05e3f --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/util/JavaGcCleanerWrapper.java @@ -0,0 +1,255 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.Arrays; +import java.util.Collection; + +/** + * Java GC Cleaner wrapper. + * + *

    A clean operation can be wrapped with the Java GC Cleaner + * which will schedule this operation before GC is run for the given owner object (not reachable in user code). + * but only if the cleaner has not been already run explicitly by user before. + * If the cleaner is run after GC it will not run clean operation again. + * This way we guarantee that the clean operation will always run at some point but only once. + * + *

    The wrapper looks up the underlying Java GC Cleaner class in different packages + */ +public enum JavaGcCleanerWrapper { + ; + + private static final Logger LOG = LoggerFactory.getLogger(JavaGcCleanerWrapper.class); + + private static final Collection CLEANER_PROVIDERS = + Arrays.asList(LegacyCleanerProvider.INSTANCE, Java9CleanerProvider.INSTANCE); + private static final CleanerFactory CLEANER_FACTORY = findGcCleaner(); + + private static CleanerFactory findGcCleaner() { + CleanerFactory foundCleanerFactory = null; + Throwable t = null; + for (CleanerProvider cleanerProvider : CLEANER_PROVIDERS) { + try { + foundCleanerFactory = cleanerProvider.createCleanerFactory(); + break; + } catch (Throwable e) { + t = ExceptionUtils.firstOrSuppressed(e, t); + } + } + + if (foundCleanerFactory == null) { + String errorMessage = String.format("Failed to find GC Cleaner among available providers: %s", CLEANER_PROVIDERS); + throw new FlinkRuntimeException(errorMessage, t); + } + return foundCleanerFactory; + } + + public static Runnable create(Object owner, Runnable cleanOperation) { + return CLEANER_FACTORY.create(owner, cleanOperation); + } + + private interface CleanerProvider { + CleanerFactory createCleanerFactory() throws ClassNotFoundException; + } + + private interface CleanerFactory { + Runnable create(Object owner, Runnable cleanOperation); + } + + private enum LegacyCleanerProvider implements CleanerProvider { + INSTANCE; + + private static final String LEGACY_CLEANER_CLASS_NAME = "sun.misc.Cleaner"; + + @Override + public CleanerFactory createCleanerFactory() { + Class cleanerClass = findCleanerClass(); + Method cleanerCreateMethod = getCleanerCreateMethod(cleanerClass); + Method cleanerCleanMethod = getCleanerCleanMethod(cleanerClass); + return new LegacyCleanerFactory(cleanerCreateMethod, cleanerCleanMethod); + } + + private static Class findCleanerClass() { + try { + return Class.forName(LEGACY_CLEANER_CLASS_NAME); + } catch (ClassNotFoundException e) { + throw new FlinkRuntimeException("Failed to find Java legacy Cleaner class", e); + } + } + + private static Method getCleanerCreateMethod(Class cleanerClass) { + try { + return cleanerClass.getMethod("create", Object.class, Runnable.class); + } catch (NoSuchMethodException e) { + throw new FlinkRuntimeException("Failed to find Java legacy Cleaner#create method", e); + } + } + + private static Method getCleanerCleanMethod(Class cleanerClass) { + try { + return cleanerClass.getMethod("clean"); + } catch (NoSuchMethodException e) { + throw new FlinkRuntimeException("Failed to find Java legacy Cleaner#clean method", e); + } + } + + @Override + public String toString() { + return "Legacy cleaner provider before Java 9 using " + LEGACY_CLEANER_CLASS_NAME; + } + } + + private static final class LegacyCleanerFactory implements CleanerFactory { + private final Method cleanerCreateMethod; + private final Method cleanerCleanMethod; + + private LegacyCleanerFactory(Method cleanerCreateMethod, Method cleanerCleanMethod) { + this.cleanerCreateMethod = cleanerCreateMethod; + this.cleanerCleanMethod = cleanerCleanMethod; + } + + @Override + public Runnable create(Object owner, Runnable cleanupOperation) { + Object cleaner; + try { + cleaner = cleanerCreateMethod.invoke(null, owner, cleanupOperation); + } catch (IllegalAccessException | InvocationTargetException e) { + throw new FlinkRuntimeException("Failed to create a Java legacy Cleaner", e); + } + String ownerString = owner.toString(); // lambda should not capture the owner object + return () -> { + try { + cleanerCleanMethod.invoke(cleaner); + } catch (IllegalAccessException | InvocationTargetException e) { + String message = String.format("FATAL UNEXPECTED - Failed to invoke a Java legacy Cleaner for %s", ownerString); + LOG.error(message, e); + throw new FlinkRuntimeException(message, e); + } + }; + } + } + + /** New cleaner provider for Java 9+. */ + private enum Java9CleanerProvider implements CleanerProvider { + INSTANCE; + + private static final String JAVA9_CLEANER_CLASS_NAME = "java.lang.ref.Cleaner"; + + @Override + public CleanerFactory createCleanerFactory() { + Class cleanerClass = findCleanerClass(); + Method cleanerCreateMethod = getCleanerCreateMethod(cleanerClass); + Object cleaner = createCleaner(cleanerCreateMethod); + Method cleanerRegisterMethod = getCleanerRegisterMethod(cleanerClass); + Class cleanableClass = findCleanableClass(); + Method cleanMethod = getCleanMethod(cleanableClass); + return new Java9CleanerFactory(cleaner, cleanerRegisterMethod, cleanMethod); + } + + private static Class findCleanerClass() { + try { + return Class.forName(JAVA9_CLEANER_CLASS_NAME); + } catch (ClassNotFoundException e) { + throw new FlinkRuntimeException("Failed to find Java 9 Cleaner class", e); + } + } + + private static Method getCleanerCreateMethod(Class cleanerClass) { + try { + return cleanerClass.getMethod("create"); + } catch (NoSuchMethodException e) { + throw new FlinkRuntimeException("Failed to find Java 9 Cleaner#create method", e); + } + } + + private static Object createCleaner(Method cleanerCreateMethod) { + try { + return cleanerCreateMethod.invoke(null); + } catch (IllegalAccessException | InvocationTargetException e) { + throw new FlinkRuntimeException("Failed to create a Java 9 Cleaner", e); + } + } + + private static Method getCleanerRegisterMethod(Class cleanerClass) { + try { + return cleanerClass.getMethod("register", Object.class, Runnable.class); + } catch (NoSuchMethodException e) { + throw new FlinkRuntimeException("Failed to find Java 9 Cleaner#create method", e); + } + } + + private static Class findCleanableClass() { + try { + return Class.forName("java.lang.ref.Cleaner$Cleanable"); + } catch (ClassNotFoundException e) { + throw new FlinkRuntimeException("Failed to find Java 9 Cleaner#Cleanable class", e); + } + } + + private static Method getCleanMethod(Class cleanableClass) { + try { + return cleanableClass.getMethod("clean"); + } catch (NoSuchMethodException e) { + throw new FlinkRuntimeException("Failed to find Java 9 Cleaner$Cleanable#clean method", e); + } + } + + @Override + public String toString() { + return "New cleaner provider for Java 9+" + JAVA9_CLEANER_CLASS_NAME; + } + } + + private static final class Java9CleanerFactory implements CleanerFactory { + private final Object cleaner; + private final Method cleanerRegisterMethod; + private final Method cleanMethod; + + private Java9CleanerFactory(Object cleaner, Method cleanerRegisterMethod, Method cleanMethod) { + this.cleaner = cleaner; + this.cleanerRegisterMethod = cleanerRegisterMethod; + this.cleanMethod = cleanMethod; + } + + @Override + public Runnable create(Object owner, Runnable cleanupOperation) { + Object cleanable; + try { + cleanable = cleanerRegisterMethod.invoke(cleaner, owner, cleanupOperation); + } catch (IllegalAccessException | InvocationTargetException e) { + throw new FlinkRuntimeException("Failed to create a Java 9 Cleaner", e); + } + String ownerString = owner.toString(); // lambda should not capture the owner object + return () -> { + try { + cleanMethod.invoke(cleanable); + } catch (IllegalAccessException | InvocationTargetException e) { + String message = String.format("FATAL UNEXPECTED - Failed to invoke a Java 9 Cleaner$Cleanable for %s", ownerString); + LOG.error(message, e); + throw new FlinkRuntimeException(message, e); + } + }; + } + } +} diff --git a/flink-core/src/test/java/org/apache/flink/util/JavaGcCleanerWrapperTest.java b/flink-core/src/test/java/org/apache/flink/util/JavaGcCleanerWrapperTest.java new file mode 100644 index 000000000000..ead8fce5ff35 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/util/JavaGcCleanerWrapperTest.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.util; + +import org.junit.Test; + +import java.util.concurrent.atomic.AtomicInteger; + +import static org.hamcrest.CoreMatchers.is; +import static org.junit.Assert.assertThat; + +/** + * Tests for the {@link JavaGcCleanerWrapper}. + */ +public class JavaGcCleanerWrapperTest { + @Test + public void testCleanOperationRunsOnlyOnceEitherOnGcOrExplicitly() throws InterruptedException { + AtomicInteger callCounter = new AtomicInteger(); + Runnable cleaner = JavaGcCleanerWrapper.create(new Object(), callCounter::incrementAndGet); + System.gc(); // not guaranteed to be run always but should in practice + Thread.sleep(10); // more chance for GC to run + cleaner.run(); + cleaner.run(); + assertThat(callCounter.get(), is(1)); + } +} From d90e49905ed6f2f9a519ac274345f08c867a999d Mon Sep 17 00:00:00 2001 From: Tony Wei Date: Thu, 31 Oct 2019 17:58:43 +0800 Subject: [PATCH 369/746] [FLINK-14302] [kafka] FlinkKafkaInternalProducer should not send `ADD_PARTITIONS_TO_TXN` request if `newPartitionsInTransaction` is empty when enable EoS --- .../kafka/internal/FlinkKafkaProducer.java | 14 ++++- .../kafka/FlinkKafkaProducerITCase.java | 56 +++++++++++++++++++ .../internal/FlinkKafkaInternalProducer.java | 14 ++++- .../FlinkKafkaInternalProducerITCase.java | 56 +++++++++++++++++++ 4 files changed, 134 insertions(+), 6 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/FlinkKafkaProducer.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/FlinkKafkaProducer.java index 9f00606bf601..1e731399c053 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/FlinkKafkaProducer.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/FlinkKafkaProducer.java @@ -306,9 +306,17 @@ private void flushNewPartitions() { private TransactionalRequestResult enqueueNewPartitions() { Object transactionManager = getValue(kafkaProducer, "transactionManager"); synchronized (transactionManager) { - Object txnRequestHandler = invoke(transactionManager, "addPartitionsToTransactionHandler"); - invoke(transactionManager, "enqueueRequest", new Class[]{txnRequestHandler.getClass().getSuperclass()}, new Object[]{txnRequestHandler}); - TransactionalRequestResult result = (TransactionalRequestResult) getValue(txnRequestHandler, txnRequestHandler.getClass().getSuperclass(), "result"); + Object newPartitionsInTransaction = getValue(transactionManager, "newPartitionsInTransaction"); + Object newPartitionsInTransactionIsEmpty = invoke(newPartitionsInTransaction, "isEmpty"); + TransactionalRequestResult result; + if (newPartitionsInTransactionIsEmpty instanceof Boolean && !((Boolean) newPartitionsInTransactionIsEmpty)) { + Object txnRequestHandler = invoke(transactionManager, "addPartitionsToTransactionHandler"); + invoke(transactionManager, "enqueueRequest", new Class[]{txnRequestHandler.getClass().getSuperclass()}, new Object[]{txnRequestHandler}); + result = (TransactionalRequestResult) getValue(txnRequestHandler, txnRequestHandler.getClass().getSuperclass(), "result"); + } else { + result = new TransactionalRequestResult(); + result.done(); + } return result; } } diff --git a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java index 6e13cc6abcc3..b6de3c702839 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java @@ -22,12 +22,14 @@ import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables; +import kafka.server.KafkaServer; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Test; import java.io.IOException; @@ -45,6 +47,22 @@ public class FlinkKafkaProducerITCase extends KafkaTestBase { protected String transactionalId; protected Properties extraProperties; + @BeforeClass + public static void prepare() throws Exception { + LOG.info("-------------------------------------------------------------------------"); + LOG.info(" Starting KafkaTestBase "); + LOG.info("-------------------------------------------------------------------------"); + + Properties serverProperties = new Properties(); + serverProperties.put("transaction.state.log.num.partitions", Integer.toString(1)); + serverProperties.put("auto.leader.rebalance.enable", Boolean.toString(false)); + startClusters(KafkaTestEnvironment.createConfig() + .setKafkaServersNumber(NUMBER_OF_KAFKA_SERVERS) + .setSecureMode(false) + .setHideKafkaBehindProxy(true) + .setKafkaServerProperties(serverProperties)); + } + @Before public void before() { transactionalId = UUID.randomUUID().toString(); @@ -152,6 +170,20 @@ public void testFlushAfterClosed() { kafkaProducer.flush(); } + @Test(timeout = 30000L) + public void testProducerWhenCommitEmptyPartitionsToOutdatedTxnCoordinator() throws Exception { + String topic = "flink-kafka-producer-txn-coordinator-changed"; + createTestTopic(topic, 1, 2); + try (Producer kafkaProducer = new FlinkKafkaProducer<>(extraProperties)) { + kafkaProducer.initTransactions(); + kafkaProducer.beginTransaction(); + restartBroker(kafkaServer.getLeaderToShutDown("__transaction_state")); + kafkaProducer.flush(); + kafkaProducer.commitTransaction(); + } + deleteTestTopic(topic); + } + private FlinkKafkaProducer getClosedProducer(String topicName) { FlinkKafkaProducer kafkaProducer = new FlinkKafkaProducer<>(extraProperties); kafkaProducer.initTransactions(); @@ -171,4 +203,28 @@ private void assertRecord(String topicName, String expectedKey, String expectedV assertEquals(expectedValue, record.value()); } } + + private void restartBroker(int brokerId) { + KafkaServer toRestart = null; + for (KafkaServer server : kafkaServer.getBrokers()) { + if (kafkaServer.getBrokerId(server) == brokerId) { + toRestart = server; + } + } + + if (toRestart == null) { + StringBuilder listOfBrokers = new StringBuilder(); + for (KafkaServer server : kafkaServer.getBrokers()) { + listOfBrokers.append(kafkaServer.getBrokerId(server)); + listOfBrokers.append(" ; "); + } + + throw new IllegalArgumentException("Cannot find broker to restart: " + brokerId + + " ; available brokers: " + listOfBrokers.toString()); + } else { + toRestart.shutdown(); + toRestart.awaitShutdown(); + toRestart.startup(); + } + } } diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/FlinkKafkaInternalProducer.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/FlinkKafkaInternalProducer.java index 78bbb536fd8e..8c2246ed9ed1 100644 --- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/FlinkKafkaInternalProducer.java +++ b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/FlinkKafkaInternalProducer.java @@ -269,9 +269,17 @@ private void flushNewPartitions() { private TransactionalRequestResult enqueueNewPartitions() { Object transactionManager = getValue(kafkaProducer, "transactionManager"); synchronized (transactionManager) { - Object txnRequestHandler = invoke(transactionManager, "addPartitionsToTransactionHandler"); - invoke(transactionManager, "enqueueRequest", new Class[]{txnRequestHandler.getClass().getSuperclass()}, new Object[]{txnRequestHandler}); - TransactionalRequestResult result = (TransactionalRequestResult) getValue(txnRequestHandler, txnRequestHandler.getClass().getSuperclass(), "result"); + Object newPartitionsInTransaction = getValue(transactionManager, "newPartitionsInTransaction"); + Object newPartitionsInTransactionIsEmpty = invoke(newPartitionsInTransaction, "isEmpty"); + TransactionalRequestResult result; + if (newPartitionsInTransactionIsEmpty instanceof Boolean && !((Boolean) newPartitionsInTransactionIsEmpty)) { + Object txnRequestHandler = invoke(transactionManager, "addPartitionsToTransactionHandler"); + invoke(transactionManager, "enqueueRequest", new Class[]{txnRequestHandler.getClass().getSuperclass()}, new Object[]{txnRequestHandler}); + result = (TransactionalRequestResult) getValue(txnRequestHandler, txnRequestHandler.getClass().getSuperclass(), "result"); + } else { + result = new TransactionalRequestResult(); + result.done(); + } return result; } } diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaInternalProducerITCase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaInternalProducerITCase.java index 2d749ba8bee5..5062b70ab61b 100644 --- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaInternalProducerITCase.java +++ b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaInternalProducerITCase.java @@ -22,12 +22,14 @@ import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables; +import kafka.server.KafkaServer; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Test; import java.io.IOException; @@ -45,6 +47,22 @@ public class FlinkKafkaInternalProducerITCase extends KafkaTestBase { protected String transactionalId; protected Properties extraProperties; + @BeforeClass + public static void prepare() throws Exception { + LOG.info("-------------------------------------------------------------------------"); + LOG.info(" Starting KafkaTestBase "); + LOG.info("-------------------------------------------------------------------------"); + + Properties serverProperties = new Properties(); + serverProperties.put("transaction.state.log.num.partitions", Integer.toString(1)); + serverProperties.put("auto.leader.rebalance.enable", Boolean.toString(false)); + startClusters(KafkaTestEnvironment.createConfig() + .setKafkaServersNumber(NUMBER_OF_KAFKA_SERVERS) + .setSecureMode(false) + .setHideKafkaBehindProxy(true) + .setKafkaServerProperties(serverProperties)); + } + @Before public void before() { transactionalId = UUID.randomUUID().toString(); @@ -152,6 +170,20 @@ public void testFlushAfterClosed() { kafkaProducer.flush(); } + @Test(timeout = 30000L) + public void testProducerWhenCommitEmptyPartitionsToOutdatedTxnCoordinator() throws Exception { + String topic = "flink-kafka-producer-txn-coordinator-changed"; + createTestTopic(topic, 1, 2); + try (Producer kafkaProducer = new FlinkKafkaInternalProducer<>(extraProperties)) { + kafkaProducer.initTransactions(); + kafkaProducer.beginTransaction(); + restartBroker(kafkaServer.getLeaderToShutDown("__transaction_state")); + kafkaProducer.flush(); + kafkaProducer.commitTransaction(); + } + deleteTestTopic(topic); + } + private FlinkKafkaInternalProducer getClosedProducer(String topicName) { FlinkKafkaInternalProducer kafkaProducer = new FlinkKafkaInternalProducer<>(extraProperties); kafkaProducer.initTransactions(); @@ -171,4 +203,28 @@ private void assertRecord(String topicName, String expectedKey, String expectedV assertEquals(expectedValue, record.value()); } } + + private void restartBroker(int brokerId) { + KafkaServer toRestart = null; + for (KafkaServer server : kafkaServer.getBrokers()) { + if (kafkaServer.getBrokerId(server) == brokerId) { + toRestart = server; + } + } + + if (toRestart == null) { + StringBuilder listOfBrokers = new StringBuilder(); + for (KafkaServer server : kafkaServer.getBrokers()) { + listOfBrokers.append(kafkaServer.getBrokerId(server)); + listOfBrokers.append(" ; "); + } + + throw new IllegalArgumentException("Cannot find broker to restart: " + brokerId + + " ; available brokers: " + listOfBrokers.toString()); + } else { + toRestart.shutdown(); + toRestart.awaitShutdown(); + toRestart.startup(); + } + } } From 6ae3f25d0bbac8d28bfa481796656cfdd85baf22 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Thu, 31 Oct 2019 18:34:18 +0800 Subject: [PATCH 370/746] [FLINK-14059][core] Introduce option allVerticesInSameSlotSharingGroupByDefault in ExecutionConfig --- .../flink/api/common/ExecutionConfig.java | 35 +++++++++++++++++-- 1 file changed, 33 insertions(+), 2 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java index 7321fc4e2110..05d105667336 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java @@ -160,6 +160,9 @@ public class ExecutionConfig implements Serializable, Archiveable Date: Tue, 15 Oct 2019 10:21:13 +0800 Subject: [PATCH 371/746] [FLINK-14059][table] Blink batch executor disables allVerticesInSameSlotSharingGroupByDefault So that each logical pipelined region can be in a different slot sharing group by default. --- .../planner/delegation/BatchExecutor.java | 1 + .../planner/delegation/BatchExecutorTest.java | 71 +++++++++++++++++++ 2 files changed, 72 insertions(+) create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/delegation/BatchExecutorTest.java diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/BatchExecutor.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/BatchExecutor.java index 7bf4367cb48d..f9cc9265ab0d 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/BatchExecutor.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/BatchExecutor.java @@ -67,6 +67,7 @@ private void setBatchProperties(StreamExecutionEnvironment execEnv) { if (isShuffleModeAllBatch()) { executionConfig.setDefaultInputDependencyConstraint(InputDependencyConstraint.ALL); } + executionConfig.disableAllVerticesInSameSlotSharingGroupByDefault(); } @Override diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/delegation/BatchExecutorTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/delegation/BatchExecutorTest.java new file mode 100644 index 000000000000..eeca304f732a --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/delegation/BatchExecutorTest.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.table.planner.delegation; + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.streaming.api.environment.LocalStreamEnvironment; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.streaming.api.operators.StreamSource; +import org.apache.flink.streaming.api.transformations.SourceTransformation; +import org.apache.flink.table.api.TableConfig; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import java.util.Collections; + +import static org.junit.Assert.assertFalse; + +/** + * Test for {@link BatchExecutor}. + */ +public class BatchExecutorTest extends TestLogger { + + private final BatchExecutor batchExecutor; + + private final StreamGraph streamGraph; + + public BatchExecutorTest() { + batchExecutor = new BatchExecutor(LocalStreamEnvironment.getExecutionEnvironment()); + batchExecutor.setTableConfig(new TableConfig()); + + final Transformation testTransform = new SourceTransformation<>( + "MockTransform", + new StreamSource<>(new SourceFunction() { + @Override + public void run(SourceContext ctx) { + } + + @Override + public void cancel() { + } + }), + BasicTypeInfo.STRING_TYPE_INFO, + 1); + + streamGraph = batchExecutor.generateStreamGraph(Collections.singletonList(testTransform), "Test Job"); + } + + @Test + public void testAllVerticesInSameSlotSharingGroupByDefaultIsDisabled() { + assertFalse(streamGraph.getExecutionConfig().isAllVerticesInSameSlotSharingGroupByDefault()); + } +} From 782526eb6215962fc4caee6b818f0aade58518b9 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 18 Oct 2019 15:44:26 +0200 Subject: [PATCH 372/746] [FLINK-14476] Extend PartitionTracker to support promotions --- .../partition/JobMasterPartitionTracker.java | 5 + .../JobMasterPartitionTrackerImpl.java | 76 ++++++++++-- .../JobMasterPartitionTrackerImplTest.java | 110 +++++++++++++++--- .../NoOpJobMasterPartitionTracker.java | 4 + .../TestingJobMasterPartitionTracker.java | 10 ++ 5 files changed, 181 insertions(+), 24 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTracker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTracker.java index 8f0f33857f76..a2596fcd94ab 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTracker.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTracker.java @@ -44,4 +44,9 @@ public interface JobMasterPartitionTracker extends PartitionTracker resultPartitionIds = + project(stopTrackingPartitionsFor(producingTaskExecutorId), PartitionTrackerEntry::getMetaInfo); + + internalReleaseOrPromotePartitions(producingTaskExecutorId, resultPartitionIds); + } + private void internalReleasePartitions( ResourceID potentialPartitionLocation, Collection partitionDeploymentDescriptors) { internalReleasePartitionsOnTaskExecutor(potentialPartitionLocation, partitionDeploymentDescriptors); - internalReleasePartitionsOnShuffleMaster(partitionDeploymentDescriptors); + internalReleasePartitionsOnShuffleMaster(partitionDeploymentDescriptors.stream()); + } + + private void internalReleaseOrPromotePartitions( + ResourceID potentialPartitionLocation, + Collection partitionDeploymentDescriptors) { + + internalReleaseOrPromotePartitionsOnTaskExecutor(potentialPartitionLocation, partitionDeploymentDescriptors); + internalReleasePartitionsOnShuffleMaster(excludePersistentPartitions(partitionDeploymentDescriptors)); } private void internalReleasePartitionsOnTaskExecutor( @@ -112,21 +130,49 @@ private void internalReleasePartitionsOnTaskExecutor( Collection partitionDeploymentDescriptors) { final Set partitionsRequiringRpcReleaseCalls = partitionDeploymentDescriptors.stream() - .map(ResultPartitionDeploymentDescriptor::getShuffleDescriptor) - .filter(descriptor -> descriptor.storesLocalResourcesOn().isPresent()) - .map(ShuffleDescriptor::getResultPartitionID) + .filter(JobMasterPartitionTrackerImpl::isPartitionWithLocalResources) + .map(JobMasterPartitionTrackerImpl::getResultPartitionId) .collect(Collectors.toSet()); - if (!partitionsRequiringRpcReleaseCalls.isEmpty()) { + internalReleaseOrPromotePartitionsOnTaskExecutor( + potentialPartitionLocation, + partitionsRequiringRpcReleaseCalls, + Collections.emptySet() + ); + } + + private void internalReleaseOrPromotePartitionsOnTaskExecutor( + ResourceID potentialPartitionLocation, + Collection partitionDeploymentDescriptors) { + + Map> partitionsToReleaseByPersistence = partitionDeploymentDescriptors.stream() + .filter(JobMasterPartitionTrackerImpl::isPartitionWithLocalResources) + .collect(Collectors.partitioningBy( + resultPartitionDeploymentDescriptor -> resultPartitionDeploymentDescriptor.getPartitionType().isPersistent(), + Collectors.mapping(JobMasterPartitionTrackerImpl::getResultPartitionId, Collectors.toSet()))); + + internalReleaseOrPromotePartitionsOnTaskExecutor( + potentialPartitionLocation, + partitionsToReleaseByPersistence.get(false), + partitionsToReleaseByPersistence.get(true) + ); + } + + private void internalReleaseOrPromotePartitionsOnTaskExecutor( + ResourceID potentialPartitionLocation, + Set partitionsRequiringRpcReleaseCalls, + Set partitionsRequiringRpcPromoteCalls) { + + if (!partitionsRequiringRpcReleaseCalls.isEmpty() || !partitionsRequiringRpcPromoteCalls.isEmpty()) { taskExecutorGatewayLookup .lookup(potentialPartitionLocation) .ifPresent(taskExecutorGateway -> - taskExecutorGateway.releaseOrPromotePartitions(jobId, partitionsRequiringRpcReleaseCalls, Collections.emptySet())); + taskExecutorGateway.releaseOrPromotePartitions(jobId, partitionsRequiringRpcReleaseCalls, partitionsRequiringRpcPromoteCalls)); } } - private void internalReleasePartitionsOnShuffleMaster(Collection partitionDeploymentDescriptors) { - partitionDeploymentDescriptors.stream() + private void internalReleasePartitionsOnShuffleMaster(Stream partitionDeploymentDescriptors) { + partitionDeploymentDescriptors .map(ResultPartitionDeploymentDescriptor::getShuffleDescriptor) .forEach(shuffleMaster::releasePartitionExternally); } @@ -137,4 +183,16 @@ private static Collection project(Collection collection, Function excludePersistentPartitions(Collection partitionDeploymentDescriptors) { + return partitionDeploymentDescriptors.stream().filter(resultPartitionDeploymentDescriptor -> !resultPartitionDeploymentDescriptor.getPartitionType().isPersistent()); + } + + private static ResultPartitionID getResultPartitionId(ResultPartitionDeploymentDescriptor resultPartitionDeploymentDescriptor) { + return resultPartitionDeploymentDescriptor.getShuffleDescriptor().getResultPartitionID(); + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImplTest.java index bfaad3db6c6b..6765b79e2322 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImplTest.java @@ -18,8 +18,8 @@ package org.apache.flink.runtime.io.network.partition; import org.apache.flink.api.common.JobID; -import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.shuffle.PartitionDescriptor; import org.apache.flink.runtime.shuffle.ProducerDescriptor; import org.apache.flink.runtime.shuffle.ShuffleDescriptor; @@ -28,6 +28,8 @@ import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; import org.apache.flink.util.TestLogger; +import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables; + import org.junit.Test; import java.util.Collection; @@ -40,6 +42,7 @@ import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.empty; import static org.junit.Assert.assertEquals; /** @@ -81,7 +84,7 @@ public void testReleaseCallsWithLocalResources() { final TestingShuffleMaster shuffleMaster = new TestingShuffleMaster(); final JobID jobId = new JobID(); - final Queue>> taskExecutorReleaseCalls = new ArrayBlockingQueue<>(4); + final Queue taskExecutorReleaseCalls = new ArrayBlockingQueue<>(4); final JobMasterPartitionTracker partitionTracker = new JobMasterPartitionTrackerImpl( jobId, shuffleMaster, @@ -105,10 +108,11 @@ public void testReleaseCallsWithLocalResources() { assertEquals(1, taskExecutorReleaseCalls.size()); - Tuple3> taskExecutorReleaseCall = taskExecutorReleaseCalls.remove(); - assertEquals(taskExecutorId1, taskExecutorReleaseCall.f0); - assertEquals(jobId, taskExecutorReleaseCall.f1); - assertThat(taskExecutorReleaseCall.f2, contains(resultPartitionId1)); + ReleaseCall taskExecutorReleaseCall = taskExecutorReleaseCalls.remove(); + assertEquals(taskExecutorId1, taskExecutorReleaseCall.getTaskExecutorId()); + assertEquals(jobId, taskExecutorReleaseCall.getJobId()); + assertThat(taskExecutorReleaseCall.getReleasedPartitions(), contains(resultPartitionId1)); + assertThat(taskExecutorReleaseCall.getPromotedPartitions(), is(empty())); assertEquals(1, shuffleMaster.externallyReleasedPartitions.size()); assertEquals(resultPartitionId1, shuffleMaster.externallyReleasedPartitions.remove()); @@ -121,10 +125,11 @@ public void testReleaseCallsWithLocalResources() { assertEquals(1, taskExecutorReleaseCalls.size()); - Tuple3> releaseCall = taskExecutorReleaseCalls.remove(); - assertEquals(taskExecutorId2, releaseCall.f0); - assertEquals(jobId, releaseCall.f1); - assertThat(releaseCall.f2, contains(resultPartitionId2)); + ReleaseCall releaseCall = taskExecutorReleaseCalls.remove(); + assertEquals(taskExecutorId2, releaseCall.getTaskExecutorId()); + assertEquals(jobId, releaseCall.getJobId()); + assertThat(releaseCall.getReleasedPartitions(), contains(resultPartitionId2)); + assertThat(releaseCall.getPromotedPartitions(), is(empty())); assertEquals(1, shuffleMaster.externallyReleasedPartitions.size()); assertEquals(resultPartitionId2, shuffleMaster.externallyReleasedPartitions.remove()); @@ -137,7 +142,7 @@ public void testReleaseCallsWithLocalResources() { public void testReleaseCallsWithoutLocalResources() { final TestingShuffleMaster shuffleMaster = new TestingShuffleMaster(); - final Queue>> taskExecutorReleaseCalls = new ArrayBlockingQueue<>(4); + final Queue taskExecutorReleaseCalls = new ArrayBlockingQueue<>(4); final JobMasterPartitionTracker partitionTracker = new JobMasterPartitionTrackerImpl( new JobID(), shuffleMaster, @@ -183,8 +188,8 @@ public void testReleaseCallsWithoutLocalResources() { public void testStopTrackingIssuesNoReleaseCalls() { final TestingShuffleMaster shuffleMaster = new TestingShuffleMaster(); - final Queue>> taskExecutorReleaseCalls = new ArrayBlockingQueue<>(4); - final JobMasterPartitionTracker partitionTracker = new JobMasterPartitionTrackerImpl( + final Queue taskExecutorReleaseCalls = new ArrayBlockingQueue<>(4); + final JobMasterPartitionTrackerImpl partitionTracker = new JobMasterPartitionTrackerImpl( new JobID(), shuffleMaster, resourceId -> Optional.of(createTaskExecutorGateway(resourceId, taskExecutorReleaseCalls)) @@ -203,9 +208,54 @@ public void testStopTrackingIssuesNoReleaseCalls() { assertEquals(0, shuffleMaster.externallyReleasedPartitions.size()); } - private static TaskExecutorGateway createTaskExecutorGateway(ResourceID taskExecutorId, Collection>> releaseCalls) { + @Test + public void testReleaseOrPromote() { + final TestingShuffleMaster shuffleMaster = new TestingShuffleMaster(); + + final Queue taskExecutorReleaseCalls = new ArrayBlockingQueue<>(4); + final JobMasterPartitionTracker partitionTracker = new JobMasterPartitionTrackerImpl( + new JobID(), + shuffleMaster, + resourceId -> Optional.of(createTaskExecutorGateway(resourceId, taskExecutorReleaseCalls)) + ); + + final ResourceID taskExecutorId1 = ResourceID.generate(); + final ResultPartitionID jobPartitionId = new ResultPartitionID(); + final ResultPartitionID clusterPartitionId = new ResultPartitionID(); + + // any partition type that is not BLOCKING_PERSISTENT denotes a job partition + final ResultPartitionDeploymentDescriptor jobPartition = AbstractPartitionTrackerTest.createResultPartitionDeploymentDescriptor(jobPartitionId, ResultPartitionType.BLOCKING, true); + partitionTracker.startTrackingPartition( + taskExecutorId1, + jobPartition); + + // BLOCKING_PERSISTENT denotes a cluster partition + final ResultPartitionDeploymentDescriptor clusterPartition = AbstractPartitionTrackerTest.createResultPartitionDeploymentDescriptor(clusterPartitionId, ResultPartitionType.BLOCKING_PERSISTENT, true); + partitionTracker.startTrackingPartition( + taskExecutorId1, + clusterPartition); + + partitionTracker.stopTrackingAndReleaseOrPromotePartitionsFor(taskExecutorId1); + + // exactly one call should have been made to the hosting task executor + assertEquals(1, taskExecutorReleaseCalls.size()); + + // the job partition should have been released on the shuffle master + assertEquals(1, shuffleMaster.externallyReleasedPartitions.size()); + assertEquals(jobPartitionId, shuffleMaster.externallyReleasedPartitions.remove()); + + final ReleaseCall taskExecutorReleaseOrPromoteCall = taskExecutorReleaseCalls.remove(); + + // the job partition should be passed as a partition to release + assertEquals(jobPartitionId, Iterables.getOnlyElement(taskExecutorReleaseOrPromoteCall.getReleasedPartitions())); + + // the cluster partition should be passed as a partition to promote + assertEquals(clusterPartitionId, Iterables.getOnlyElement(taskExecutorReleaseOrPromoteCall.getPromotedPartitions())); + } + + private static TaskExecutorGateway createTaskExecutorGateway(ResourceID taskExecutorId, Collection releaseCalls) { return new TestingTaskExecutorGatewayBuilder() - .setReleaseOrPromotePartitionsConsumer((jobId, partitionToRelease, partitionsToPromote) -> releaseCalls.add(Tuple3.of(taskExecutorId, jobId, partitionToRelease))) + .setReleaseOrPromotePartitionsConsumer((jobId, partitionToRelease, partitionsToPromote) -> releaseCalls.add(new ReleaseCall(taskExecutorId, jobId, partitionToRelease, partitionsToPromote))) .createTestingTaskExecutorGateway(); } @@ -224,4 +274,34 @@ public void releasePartitionExternally(ShuffleDescriptor shuffleDescriptor) { } } + private static class ReleaseCall { + private final ResourceID taskExecutorId; + private final JobID jobId; + private final Collection releasedPartitions; + private final Collection promotedPartitions; + + private ReleaseCall(ResourceID taskExecutorId, JobID jobId, Collection releasedPartitions, Collection promotedPartitions) { + this.taskExecutorId = taskExecutorId; + this.jobId = jobId; + this.releasedPartitions = releasedPartitions; + this.promotedPartitions = promotedPartitions; + } + + public ResourceID getTaskExecutorId() { + return taskExecutorId; + } + + public JobID getJobId() { + return jobId; + } + + public Collection getReleasedPartitions() { + return releasedPartitions; + } + + public Collection getPromotedPartitions() { + return promotedPartitions; + } + } + } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/NoOpJobMasterPartitionTracker.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/NoOpJobMasterPartitionTracker.java index f099ba6b8dfb..a189b667221d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/NoOpJobMasterPartitionTracker.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/NoOpJobMasterPartitionTracker.java @@ -53,6 +53,10 @@ public Collection isPartitionTrackedFunction = ignored -> false; private Consumer stopTrackingAllPartitionsConsumer = ignored -> {}; private Consumer stopTrackingAndReleaseAllPartitionsConsumer = ignored -> {}; + private Consumer stopTrackingAndReleaseOrPromotePartitionsConsumer = ignored -> {}; private BiConsumer startTrackingPartitionsConsumer = (ignoredA, ignoredB) -> {}; private Consumer> stopTrackingAndReleasePartitionsConsumer = ignored -> {}; private Consumer> stopTrackingPartitionsConsumer = ignored -> {}; @@ -59,6 +60,10 @@ public void setStopTrackingAndReleaseAllPartitionsConsumer(Consumer this.stopTrackingAndReleaseAllPartitionsConsumer = stopTrackingAndReleaseAllPartitionsConsumer; } + public void setStopTrackingAndReleaseOrPromotePartitionsConsumer(Consumer stopTrackingAndReleaseOrPromotePartitionsConsumer) { + this.stopTrackingAndReleaseOrPromotePartitionsConsumer = stopTrackingAndReleaseOrPromotePartitionsConsumer; + } + public void setStopTrackingAndReleasePartitionsConsumer(Consumer> stopTrackingAndReleasePartitionsConsumer) { this.stopTrackingAndReleasePartitionsConsumer = stopTrackingAndReleasePartitionsConsumer; } @@ -94,6 +99,11 @@ public void stopTrackingAndReleasePartitionsFor(ResourceID producingTaskExecutor stopTrackingAndReleaseAllPartitionsConsumer.accept(producingTaskExecutorId); } + @Override + public void stopTrackingAndReleaseOrPromotePartitionsFor(ResourceID producingTaskExecutorId) { + stopTrackingAndReleaseOrPromotePartitionsConsumer.accept(producingTaskExecutorId); + } + @Override public boolean isTrackingPartitionsFor(ResourceID producingTaskExecutorId) { return isTrackingPartitionsForFunction.apply(producingTaskExecutorId); From fe15dd5c6deb98597060d1a13c31e7f4b1f64192 Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Thu, 24 Oct 2019 11:56:57 +0200 Subject: [PATCH 373/746] [hotfix] Remove unused TaskManagerConfiguration in TaskExecutorTest --- .../apache/flink/runtime/taskexecutor/TaskExecutorTest.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index 7c4d4c96566f..6574620a3896 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -193,8 +193,6 @@ public class TaskExecutorTest extends TestLogger { private Configuration configuration; - private TaskManagerConfiguration taskManagerConfiguration; - private TaskManagerLocation taskManagerLocation; private JobID jobId; @@ -220,7 +218,6 @@ public void setup() throws IOException { null); configuration = new Configuration(); - taskManagerConfiguration = TaskManagerConfiguration.fromConfiguration(configuration); taskManagerLocation = new LocalTaskManagerLocation(); jobId = new JobID(); From 63b5c941d795e065fad0d5ad229cf57b7bb25bd4 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Tue, 29 Oct 2019 11:18:45 +0800 Subject: [PATCH 374/746] [FLINK-14557][python] Clean up the py4j package by removing the unused directory __MACOSX. This closes #10047. --- flink-python/lib/py4j-0.10.8.1-src.zip | Bin 94484 -> 82546 bytes 1 file changed, 0 insertions(+), 0 deletions(-) diff --git a/flink-python/lib/py4j-0.10.8.1-src.zip b/flink-python/lib/py4j-0.10.8.1-src.zip index ce8463b7e17b8aca11f6df556a5d548d84cde4b1..74d5ca481b6decb5d6e940c0815ba1c273f82e08 100644 GIT binary patch delta 36237 zcmZ^}V~i%i*0%e!J#E{zZQC}cZR=@G+qP}Hd)l_GY1_ux@0+vFpYJC}l8VmqzpS)NPGpkMC%@Ym)0{#R70Dz+b001Rfu>akm z0aP)~Qe;M8Yx4g&b@>#z88fof&l;vNB{uxe^C}z_QtkW-saB#rTyRIXo4YH0mJ<- z?SCAT*2U1&L;wJwB>(^{0AA7s8P5L(_W$DzD!C5A|7UMrs3x!a^c2*s0RU{ef&kF| zV;{hZ0sNoD2;fitUlIej)U$Khlt}!ZF=UoCTn_pLJ{jBOF-hYWPrjjC*p@=Y%UuVD zha!rC0nG%U%E-F^e!fWa0+XVV$(Fqy8RpOB`%Ui4;&ypv?Fp!K9FepT3!zT=?3NSPYS$evo% zi?GtB3%S&*z3@tC+m*&aRMIa?y?w=gC?%NgTJHAzgAI@AdJasMJaIR?3J{ z&n<7vL@PN5XS?js1FRmDkKL#llrgEPgm;&VS5OR_6x@RYGj^qmX*}hOOr!D9x~CR& zQ)+zr{wY_O)u4c=#Vw2Z;zweBd;o7;-<=_vXg&_m<8!Ly=vNN3X4DuKVM|sD#z*g5 z)$3%?M@jVQ;Eaq?%81O-e^XDeREZFeswdP~qq{`?ddWu9hCKMQu2yPO=&w@|`<;m1 zggv$4OHXqRY}KP29&*(J6)dq*rY`>d7d=V$sCW4Yb_e;7HM>FTu$a30)}%?ry^>z8 z-Y;9I%`pM(%{cbcaGc3;derI@B9SFpQ3okR5hpiTxpajWnVnSY4dzA9H|C%gtstFK zNMXsozbeP9=#d@OX8LK2BFw2NpwEH6&^Q73@Nd9s;F8P}pZ6kLn)W-RItO+~s_N55 zxASxj(qtin5qn7WYm?-Bug$GzhyUAlK3x<`aa?)P&Z>+0F6I%vV8a!naF&u6B}&?s z54WmYv<}?^nL$wHpT}age048h%BGY9_ma#K^l~-5>hmMmj{x@MTOzb&(D&g8ksi1| zl@!huzyrPQ;9tj`9y-IS+LLq|Q&hgIc^YpkzsA5@!ZGu~QC7jz;Fh)gGRw(l2D`Jo5g& z&RP2@f`{!&yl#3%40G(@c30nMAfo;%8#If`Ta8&&sLup>H-K|v8p zaxf{%#l-s}ilL3rV0+IOcwMW$j3Dk2BWV4uT&woNO6OW4qvf%*pY3O)w><0&WB)2t z1y6^}rTq$842O|~Z`QT+-rL(P!B*0uPg~FA$etf- z!HWr@`%Swh(symND&?{!ejkA{tdJfIYA8Mnrx%(SxW zRw#D!vm-X{v9DCc%Zk^`(r8$zAk+0_8*ui0 za&?LRu$B9kDNb9I;spZrO@-FA^yNg(3`T28&Z4(6=|JG8aea*0wmpO~Fk8r5cli=A zU0Zqi@6A#?Y~@uRSH&;;Bk(}zP*mvoKGxinVQcK&oxwfzd=HOPwlP}37Ef^>GSS8u ziZQbtIR(yw=^m)@0A47&Lj=VXI?xK?gqlR)k1zzZC|H^`dj=_5w}6+6v%nz|lsDK? z>Y=L;s_PmGb}W{$JV=pbN675~W4-XP+v+}YF4$EBQ|@~)#U0ztbAxEf`BAl=e`zaC z)zXRq8>yr6qTCG4>FJ)At1CMf-mim)e%=sBNdV0OrzR$6S{^SB2TA-=GqAav7Bsy| zW;no2C|TWTDKZF>^4ta@s329O4Q)7)GnX2|==MxtS{O(}zXRdIHVbzbpdjK zzvxk5W=}Gpu2D7#?-tq|j2(cU%$!9nbhTl(9L8+u@DNu5eq`i5p(^l&eW1y;(>{R+A!^%U}a=G(Xljaet@DFrc{ zKq(IEVj05k5hUpD^lD-rMiKFbwfSWb45g&=tuxFYq@anh46l4U`;j-l%az)uE2$iebLTFi14r|?~}PDAOM?;=d1rMC!)H&GQw041t;rVCZW?|jBOIg?sD zBPH6J!*idvZV?FcP3JumOQ2zX(d{a4`HI)LVSr{TZ;%GjUthu4(n@0f{vCEQdiS=! zF64V;FhcnoD|BR)vrfk@o1_C|5&Yf#C&Oc3llC8yr>r$DHxAz(pgc>N0kLi^rBA|H zk=Vk=xhS<(c(-Gq6)7IF6v}(7zuoL#W`$VQ850A5}aT$nDD;d zPc_aUv8FB$G%#GN1eB#j{xwm71>KQVx%8Xh)BYGil|i_CX{ zGx+38JM|y}x)iaw{p{aB7=tSfcp{Qt%rWE-BroU01@==!?H0v}XKpe|>l8Y5d@XeY z9Fr32O$y-$pN%*p3x9tKGDmcN!@G^fI$-5%e}Tf`isBd$nDo1EjKG51P1lRX7;J z)Csx+Q%g*Z;0Ch^bIJDvUz&HT)3pU@G3i%#kRq#Vm}RnBIjwRIXxrpG91uB%?louL zcV`QCf6u7vR0*2Q{1q+%{z(xm-xWhz5t;AI-=DGrt^=7XvSh&^5qoIBx7K7E714Ehc=T z0qBouN?#2UYb?t{?gX-h-D4aP)2;%2*r8)dqK0Zm%S+wRtQ-Pk<5=)gPxE^swzC(N z1gmgBi)?4QrKoPXOozxh;JW0%H!K24x^^uBBFrgK76o8k(2kZ*>Gm5*nP(;$fqfG` z!icStv408lave9u8*d*aO$-W$jyT<=4#*x&dVNQqrBGU0n?Q-jOAUFSL{e9ROvRrY z;3Z|$(A>a5VO@`z7DA;fzHsPViUA?QlPXT8P->!IZ4>~DTqW+O)fB?+Dy~gSPj`)8 zwT_qXQez}v?@_BIQ?2j{UFutx6^ zV60VLzwc0p$`Hc+T6wx`DX-dYxWaAY_%%aPt)q16vU!XQ7%+1$%|ivbfD{bxUB#XZ1J&JSdJCG(}%yKbyu+Rf}=g=Af+)KySzO zyDVt5!rWY3FQ1;s*SL^l1tM$b|8}S#GS5^-WL6#?x0* zO%{PrmlD#f4lQXZ8h$N5ll2fHKNzJ3JMX34*8cog#?ajeHmUaZ1gGnEO9by2u&c$- zAw{gy&kn-oi~b%b8o$IrSWydp0rRm+;eaOlo*!t<8%= zshSgOSx(umJaeY&3=|kMH4efPl5Tud_`^fr8UU5Aui*qer0EG+l?iC2TO3$*5W|Tv zzuo&Sd*5j!G?;4inST)E!rm>IGOVB3*QXHB=mZLdvt5~8WmXyl?_PE%qg|Ia5lr)D z&8&eT%hOlYAp{(H;=mGR&0)K1Mq_+FM^ELU8@s2L%|Rgn1rF)ZJ_SoQn`k00EaT@C zaomoSY`=mu+nN{YY)7Yf;qTAmt)2k%SwljsV2o@_&+~2Mt1f({c--_e=TpgNlJ)8> zH$@+zN~8WjH3~{LXFS-W{#(JvU+hh_IyPocK#TOl7!FJj+|~#wH(#CwpKd6R1_*gUkh0FGhzG!dn^ICgtTzp58{%8qN8=i1z~zKr9W zSyJweOeIf(fD@?v2Yaojx+p739n|~o!TN<9;2!oT_C8N|w#_;eb9|ml9pL=Xy<$!%ggTS( zRJ=mOi)m~gdCXNsX}?(??jI{=PF{`8s)f^R5PCwM?z3gQZVWYMBd>DJ0cBn#t*zO7NdT4O43>!8RjubM(09l-Sp55XVF zY{uoV7&drMQYd2_4i}p z;@>gIKGt-ARLk1Y&UAMPH(@wok6emf+|^35hrc}qebU<~dW(5@>D1cxEUj8r0J==C zNl2Vv-mULlF7Hj$N35gp26@yD#Wp;|0~bl%9z+;1bHvBx6Y&t+v8r1ds<# zg(JMJD#3?!-Hbm*KR+E!46f(@V*YdUbED_C*;1mz235Ik4Qz&s>|l9TAgpC zYBV@$>!`Z;5%*Y5|J3a2Uj6B+6CCyXDja;*&t5Jq8o+>qpxC)r`?RH`04DS?rPNVO ztl_>0b)|TrGV$Sr?rrBIAN7Bd9(POCABs&8fKaffjlss@DMO&2T4XB^(`JlBS?*xD z+IXqSjm>5^>8qd2Z! zm6g8e>_TQaHNhuF+@1o6NBOP3HliFhL|RpFyIXsNgS#X_iXfL}JL_iQc29TTB|qKId?O zCiPdvPl&!YCO?r1sxGMiSee>nH5-r3o#IgXNr`sw1r!NNZ4C9=dp%98#FivUoBWy) z=3+Y%ynIPw-y9IfIyIum(XY7Vi7QGB2CCE5JzF-iDH`B4fD91*pr)Z0&R0jHPp8O} zQo8WUj&11r4;Cz29D9?AV(`UF2nnF=cUHoZBi=E?OfTc0y^|(AU&oC`lacn(1;{EF zhD{{&?|x2KQhR?`r$?_-tdL(_Fv^F;mQV~`zpkd4dU^akp3G$>Umlcl5u9#xq~Uqz zMFBMNn-0$UK*OOj$`c1jP4La~$E7!IZDvYK4NnK!2W57g-#>E`eB#3SxXc6(4*M|Php zL#&2aY{1K?_Clth=~4C69{JQqlwm7{T)21u>9!Q=MM*}ids-nMt_HwIaq(F<(?1E6 z+uU0Mvu(QKR2^JqhPsH?7C={6J>*LE^`@5ILSwGYm*7t3cyOaXw8?4_a(dAQjx-c7(C}4@!n(h-3FKMt|Wu;1nAEk zmks%D3q7pgKd3S|zd5dPbxcWJM$ZDVJi6?FrxyI0bNKhz)TZ*A7&SKiNwFNaOO|u7 zzi-Gec;=$gIt0GquZyawN7mggxuvVjDONABYGw!@RF~xOBERipd)I0lY>-{2+KxN3 zd>cypSmD_pk18zeD=&SxCcrkiLZY@#iQqBs_)0xRj>XpHj?$xSsyb4o^C}H|-n00D zhurc_Iw?GvLkX256+{FDaP?1T7lo$_AoyuOF5N-VcR0448_jkk&z-2m05qWi)gJj@ zZ$k790hB9d2m~%It z`QBTUZ9LJ;AUf~c!A0@_H&Jky(vLtehA< zubxp+5U|>O8`$gN-nv|wsqh0l(n!2=UT$0*d?4^e09%B!!%@jpukNmS7hg-ayiJ9upiQ?9tXkpb=<3Cw!$ns;vTZmay5|EqhXa|FDx zAKX8iHR4~JQ#!?MysaG@U_fdGMa5y*A|2=fb)SxPH-Ftwfp|~6%0B{f&!-~e1#x@) z$py?DmF0r)WbNmnYeft7o?}jE3TgjQnkwB4tKSUZYX+mj@FQbk>loDHhh&`VEwe7d zLfq`LZmOZN4Hoxd4iz=;E1|vI<7_=fPmde5aXh7v5@%mI66^? z)rWs-$er5^?~a{2W*7nlx$fcnnJ^9LAkD;rr`jT;#T5Km@Q8YNU=02wRVv-|NA|=R z2s;z~OlOlb)J&2-T|RsT_0$DP2fHrZkGEJ&b66gqZ7xic>r%P$7QIia)Q+21meF`V zJbH^tI~jCP{6%j_03b`l_W9k@>HBCH>1XUGdN!666FGl0vY`j`h9HRU_&GZpKf8V0 z`#d_kyFJ~zw7ppuw(&rWdPpjxgs>HJf_#RvRx|*uNn-zgBYaCZc7UKkJm?R8x9^DUIj>I=8XsboB@xSe4- z84^h|<;Nq-U8^B{C7`-(x#}?N3tQNsq&yspiesCJM311Xqjl)Q3CgbMoby5cz|hBB z4SH$e6@0}6vfP=s3<*g)Z)M?L4Qp#+UW*fvpqbB3=oABAYLpjr8saynr`_daBB8zp z%WGm5);s%JR%lG`4?*u@vD`yLe;};2jf_g;*V~%j5VVV#cGQXD;$WAU`+Y(m%N_mJ zB_=dH;T+Ws|5Kn&v8r&r5l7Hb36di6`#LGC-A70{EKKxpJ8{i*4ww;Jl{lwUXh_u_ zL%9p59upw_GPlY-iQd)*6nTV&@@?GbHIwz345?v%k`x44rk>#_h#k=Tp_M;_o^qY&eE=yjhX3p`YJ;x@?*BwbXu zkYi?)m6sng^>!4Qj=E9o6K}hKUPv1SGNqU&GHRoxf;}&nfFK0g=HNe<`4_~zRwh4} zVka;IY1KNgvq5*;K`OKcU)60$Oh(^ymmQ{CwJBbZ1}Ho0{sKt$(1R$W@ulY-lWOUu zcxYf!s(UwFQkeaI9GIkIj8{!xNbT6CLS3^~voRK+ix|;zA6__%RIM&l6fRBfdJE}~ zkj_LIl9|!>Tk;^nlI*It@yH$`3@BV#T2G)INA&II%@LWaih2uDjA?lM8?74L-e@fe zV%7HJ{#SnjR{A{i%w~CxKWWxV=f$`?*HA8hh;jYUu_~2u4~ip7frHVZJ(fk!T|H1G zIn)yzzAW_2R-mr`^v2VOI~nHt&eee*66i#Tu`A8AhGhXWYC~S@hyZcUi7__(SZ(jc zmIzhz1#K_zl!YzWtB-h4^C$rpoA>^L(ZZ_zwtS&+&|Jj=IJ7D;@EM4tM;lrHEo(d% zL1A{{&LRuXMy>{7z?SN-UFu?)vkmNUUtmFd!Oye-Csk%Z*hR{QiX(33!|7k{N^fbBuLC#2F0 z3Sd$cBXwpylh51~QEurhs>J^}S3eIiAG`gA}sv%~HxbKS+W&xVk_4<{fj2bj()q zk%R?Rc{MfORFej66d)-Qpn4w^AS_hvkb^nI88o&h)GF6-D@~qqR=p?EA<=qEL3qPh zf|^3d$bgkYlQ30@M>>UY&;jGZkn6gzXGzm$^VsBdF#!}KwTYxQIoHdK zk8D=VAuD?J@78|TlP5|iR>O9c{B(Onh`F~d` z8*$z`Jp-&lV_B!moPhq`*5-Iw(HzpJUgYVe-8QgMLl*`axHI-yDIN3JsrOs;6Qr!S zoejMr3mlU8PLl>MwU}(NRCz<*5S*|*r@?29(r+Y{+%y({Nx6`RS`pG7EtP*Qx^Y*4 z3NhlLRG0p%zB947Mv7rj>z{8h(m>ctL#hm4O>$ni@(#pWi3NrkICW+aAVKH-*{j@h zAy!|jg-AIVzUMEUY8PZAAk9`BM3Tj-LM)ub(hr@SO!2M?i2JvAK;531RkTm}Sl=tL zl4wGi$k*%JbzNTqA_{d&O)KUr3}o?zF)qe(u=Om0v5(G&VVRH%Mwj^b%!P(*@f5(h>yXy0M89G@UfWQ>4R2gkm76$K^oTH)+-T{ zr5D17Q~}yEk2E9whDeK!h7Ghpo3?Ft$an401X$V;H4M&UTKv@$jp_{I0Vs~zY z4)t6zht!n(0d$L=O1(ETaW*XNNr2^G^~VYIlN^cF@$g!&Pbv5nBkGh3v?|6{FNtf6 z3rNNr`e^)!mNtv35TA$SYId!nq^|&8TG*_Em4Q&xu$z$G&8pO$Z`)>USWp&gMqzl! z%Ouk5q^;&1U1iqEO`9f%C2M#MvMS~J$-k-sFvEqC=_mT9>U99>w_T!N=Y_EAMAhz8 zqDevY=YpNA4jeDu7i{DPxi+C}$M2&AM$(SjRXV7VcKr$Rs5CKHn8#TuN~h3FjBj_D z*{`?M=EUCz!M)MIq_IAVU>=_vnRYkwSj8WQ%c2lfF}FvPxA#|`U)kEj?NYZz{{7`c z<}GK=$s7}q9h}F5hcq^K{Aa?*nN`#bPy|8?rR(y%w237NtF8%0Y3W?{&w zHn;KM1-UkCExjQXP+cuzF(HL6F#pvPTmCFVI@!6m)n}l@#mF}dcgf@p-j9lxtpcK9 zQM~D5g16wF!PPWji~=4sd*VyF+YPX5yFN~(J43Iv{^qzBXgEv`Cxh?yN_Jz>pfU(} z3quZaK6P1#OCL9rYBIHRoXlC0puphU)3{?*+?SaNi-=O`30t}ad`PkCFq80TZ@54w z={C4}%G5{4W!@)F)pgR(femT#=K=XM)x!19byPuOh#>f!`($R;6y5E!kd>|I)_(c4 zzGE{Q;eGDu?T4>bM>0Yxbl;x5;2Zd-E29vGj2!2(Eo}Y*f%2U7Jn(_UZmp{$t~(hj z{Bgl58$pbx=-$Sb4&sKrf}Lzf8*cW)l$c`%1!wj1b*~qJ{8K`Yg@fT=ood%wS z6GVoqC5)u5gpywkt}%}F$)lK7FgyS(hi@oGkw|?tU(jf5t)RaobSlA5jZuQqaA%01 zcGaj+`hIZd!8jg}zFMQ$)mRoevO(f?H|+-pb-Qq{E@&G3Ey^#8WUXcS5tV3RX(&?1 zl193i2h(QMWP*`%D!wh*F9EEOe1hWbwfn8)A*$F(J9mKUbLxfxbYLi0qwcE;HXA>j z`eDIFJuD&7elfjdsjg7_sy+_b;^!WCjpMfBW8k(#okJZq&FHZ|oajee%cV574N#0B zGEG8^3#^?6TBB~G@|)A&CEHr2@!55r_s+g%JzNob?hQhHW}z^upa4_;nuYC}@JQl? zK%>j?TUJt5E%7FuD=&6{b=Y{|LKqN&W?@!r%s$T`_HKR{n0+H4h>`8*)sKlU+vWa} zi;nCI3w3dIbTzT~7iMbD@bkFmg*Tj0U8)3{wCmxYt}Zwg)R*~e3HI6?oVvx_*Y^?9 zPI1e5KUS+zKy?;X+6P4Cv`vRI=dRV@d^pmLJy8x-d)m1)@p2>|oPG}R@wkOo)pji! zixDN|>`~+*8jXS8YR{~S9400ftdJRc3Sz&1k2I^(YLvb`@z2b7Lh+`<)O6P+}r$I6v&9>acxG8g^lEfP87;2Ek+{q>u2lkhtfLQ7- zU(2htPB-{5D(B|vEdj^evlwR=&=PC7RRSF51G5w3!$t?KeLr) zTPj6hbDuAy#RBmNV$UJy#&tDV8P2t>{O3FHzz*Onf2o2tpi~90DO02R&9&vbNWpaI zBvC|mWc7u@+Y4kD#^w9B*Wc`xsw=}n5Nh+Y8nYZDCtQTr+X6pV~EV{a#Ou=2EQ%}R5K>SY|NBM?I3;~S(_Im-mh>RUBozTm9G zMhrs8hq_C=-X@lW#A7g%n0?>cw?WqwhHDYY7Xvk5d( zuA8|iS^9Kz_*YCd@cfC03EOUWfBkN387(Tl0xl2@R`vvBiEvdcc(RTvzV83MK^Kh3 zr1qS>x)%Xk*L|@qDC!LavMA?xO#V3Sy(&5?Y=n3T(9p9mbEbM-gws^cbYG z1miK|8)y2L=Mde)nrUf9C@k>1)yue)?p*0}Qc4>7*UA5ters3w++QN7reF-ub8_gsexMHz(Z-7HhWM zmB|!j<)|VQl}YvBR-&Hd+U(|&pFo6%l~s20<5$7_24Gt^ zUeGC2h`eS+LRvn5ca`g^->U7+^1sz%U|>dS_*ARtk86kDpF7xC$dfWCAIY68VUz6K z4-FBp=sh0J=7tKvE8_GE`yvQ$z?>pHd8Uwo6aCXV$^O9PqcxOqc@530gH7z;z;H82 z3Bc0pkZ6A02dTS=Q`2P_5q6g~JMa_2p(Gf?jAMw-3oqJVNV99aBPMh$ zgt|Z$9*a=HsVOvI`ZoR#{q(cW=$`uB>X3;6g=@}*5c-L&tZ3EmrEJ3SQq6@Do0)k6 zI77rJ{+MXdH+}Audr+8RY@nZ#R_%m#VM0I@n~Nnh9OlE*!qkcWGWXJ&m{}|I51q?1inUB$@>%ksL(>GMeNUfG?t>nj{JMI9Iw1w8xY-uzeEBFghVOH z4k8HS9yF-y(trCTRqgPdKTmfHH_HA}V3z(oMx1l`eam@uba8K4O=8bo zY5*ky0{5)nz9R())6G1XL;N+em~;~xcE4B0?G{e64l0;h*zJCvgmUE&gVXlISQ}3q z`aZoJX>WV`Bbmmp4`hD1C)m0hhbnDtd$~+9e^x$w@#IN#wVPi|7*Kmwm?JzHKGg(T zOA*-R={3AMaf$S zR`?40#fcNjf15`K(T6-+n4l1@9rW=S?K}={=j1>)S?C@4Lt5U!5YD`_AnDIBM^jLUeNfKakar!RJlIu|L)&!gPM^xJgSK1e}ve9 z+8tshm1!eT0~;6Ec<33y(y(Bq zu?DIHezP*(cptl3mvmNbC?ZgBc!v;@?1T4^`}p?{eH)sIxOFGLhyBD#NPkr}9l@I%OfN!Vy?0EY02#L(Y9a={-$^wo?op;UPkJdn*-Hmr|##c z>@7ORsIS!!xa$~PT7X=_;IMB)ZmPCqX!QbUUfZ(W!Uk&fp9vFGro-H-3c9Eomq@%=;3K${~AW`dhi$YoAS6V7yLxs(Rhrc&ypCet0_~RjxW@`^y3!fQ9%t3M~Xw`2}90j8pIxy^K(LCF*eJ55!j(TYe@vMdpy3{8= zv3^LoGUn4nc5;d$uhY`V8b8K>2_BZu-Y|6 zXK=+}P=RnYB~Fe6w@hwN(}hc86-uGN@>pn;W@C9gW4H*CPNHy6KlrH@Q`nctAKaU^ zv~_Yzf#D;;ksbfNUhBL+N&}IxvQJXh=99=I3ydEd|A6&c?){6En-2+52NV@+Sj(Ekg2l~BlI%4kB zj`|xC+<3>P@Sf<~LO68hB?TEs)s0ECj-4nR<>F^M6fR$#0@Zr<{nD||?2B#kHuT5V z?jpGVNE+g;nZl)sp3c6X-(4_;rw|}ujcgr#hAP@d{j|i%((#T1#@827kNAXo^Qt8R z5}WE!;z`9+8up*T2-%?XuQtqFweV*e%iwV^GHqcg)xVlK=vAht;bg$@%B?(;Siw51 z|M1`mMffqP06m{F)p?8?e!XZpU50AGoM1L9*b!Yd70iIG#s22(9_+be$O=nT$fe0Vi>z+C(9&q`+vC54 zX6j412)Hpu4K{;J&h-JtC3;k$4VYqUyZT#1pP*u%(?73LW;VWTE{ZC7A7E!H0>Wgg z_k&~?z@~?uhrzEa2pbQg?-DUGUKh|-#?K@RDtLwqhQVV^`VV+^+1H`%N zc;Kh$jQX0Ory|O4*UwGk1v$32i=7;z7OpzdZ|wQHi(`*U>|rJcsB35aX|ssWvxthZ zZSi3ROp$|zH-255zP%oww(UQ@H}L8m({a@Vzz(|?Xdvgn!f%9b1+7?m0mj^TD3Xpr z0skxT{id{!4`e-Rhc`*`kpMcL8fOC}aFi!FWY?L*XzWJvyML>@1aE=COaTTr1R73@ zdsV+C|KRf}l3loo)Z{N~d^nxPq@;8ut%uPVeJlJ399llLX)3BIHx|Or{slV44?}Jg z0}GPe?H_~+9uRhj7^fx(zZYeWg$C(E136xJmSQh&O@F%|5HdxDZ0pGfF9oCu-`4)% z-eUZATVrcaKG|9sB^1zY0vPJ{oD>j#ejX}7<#i$KQvK-F77y;OdNrp6sa$=Zb7r`yU37qqbjSH*_tEQ=vq|R!Q`d<_S)JW*jKtoT>-d3sZTJ& z?E~nl40gxfoMyWiZ`1uReHcFsW`c$^-ap;{uw%T)oh?kiH(2q&ILVu+f#TmTj1wX7 zz_g|*I8=+&aqR%RFrX#wZBO*yW6z#0f|VpBkE?%LK?oJf4|b)J490dA**HT?3H$qt zP#KMngAxzK50{!&-+WABZ|?-1E%OKW2oy2MX9o?6OQZ7}ZMV@;@ia5|2gslXX$wGHV16p1lCoQ5DW{v@NfY*B(YeffpLudedJ=KyP<-dYi}?m<^{=bnitTI{W&GW`A97Gvyf;xFstpJDm~~Ujq_? z&#b4VaC7JY5@(}O#`}Sr$|tx+rTwD~Kx2(|E^&GMV|uHuYs82_&$V55)Xh8Bu}nV> z0fBE@4z6PAq~wIh=8^%wKmM@yr*!h(vnCsOp|YQOTvkFgP&jRK!RadQY&M&6Gh`jB z)TXtEQBYvr%Z{o--gSzP)q?7L+~(u~0QL(4Xfv09H)dcn!)B+NK|*4@YqcqSun`m4;*xb^>W$7CC#Oa@oCrmdjv0^v zS8oQaTvMf4O`*>Im}qLfZ9J-`CwW$JHXL-2ZSqOCX)QkW9uV#+*Wg4SP;S_Jyh=#5 zmqlm$utH|1e`by^V;3{AGXAAT9Jz3mO7EF$hu4n`oFYrxbI}tW!PU`WE4N)`Z0k{B zf&;U?gJ8#kC+CE=Q^!O3&*u{pOJWCSI;*|Src;3N`18e!3|WwX)ob`*CUR7?---zV zG??Y8xe5-v#($=XjUwwzX>msO=cOM<#lAC-C8a9lsMPFryK4~SvPf;Pv!-U{HMe)5 z&sJR8fdv}2(#zSE<~i;FLpyDSO38;wS=qx^X!ff8Aj&FMv*+|WE^U@#TrEyV;kI&z zgl3d;=jtAAcOe0{Z`p6qS)5np-Z1V{y@}mCp!xL)W*Ozrr$B-hc3pV^lMsc%b)f3n zf@s!f4NJ8tH+b^E5|;`1Zvy{0Q8glVT!u;x;Bl@t@w0`&jvY5YL(`4alDh+zL1*{@ zzEru*SiVT8XDa_6Uz541%jiK6ubkND4uYJkA*|*01yh+Tss8loVPw*|#ziU+Jy2vW z+NR{E4Q>$NVpgZJ&$^Ix%dbG#2~)A|U+dFb$VuZUW``EyczvdZ5|Iy(frfzk{lo=n z;Js`x8D&BeMe+V_9J6`z@8vZX57*wS@F^6M9NuLe_uF5&-{jA`$|v(*SQc?JizH)? z>x_dXmc(a54!pS+pR}CID#&`Qn8mK?*z>u~*<}3g%&#Ulm-(-=e<2$85DEsfysCez zvoc^VBhdwgpJuG%K+xpLIIp*AbPQ(n0X0VC_K0P}7M+c=c{(~fol*eaVc-{;0ylFAtL>ta_GhgZBPoAHDaYZMHw~+gPC>{v zNr;KbqQQ5Xqq!tqsMAjQ?kbnV7re{OIHPr%mfzpqE^#DL6W)qb4SJ(n7RT_WfxWcJ zSJOYOxGde5C19DV*Ij)p*jEWv7vHHJCVG|Ghvi2fwH~#zmCN49Mi6gTqK!dD$Mg+P zO9p*9SA)(>oAENDdO4|l#2qfu2we_cJ>@Mmr|k3A_hxRKY!&j0RxGe|%g8V+5UQE= z_KA8aMp@c*V>(#fO=eJ56%?#;KykMM7&F`m^L%n7l^sJs&foOevTXxk6@_zXsOQOp ziLtk~TgaMG!;JaN1c%LCTHLInFftFH0Aa50-BbVf+wHBDyX>WLLslYJ5Ooc`*yo8B z@{eN@C(dbFW(}-Ayt-5A1_Zo?cJPCG^FPNjBWG0eEanY?cQI!7$hQ_uphYm-kErC} zIIog;Ul@eGvxO>aB)^(HoNOyNgi*!qae1m(R zbM)K$?Ehh{VKHmXbG_G-+_UEQUemGLDEm;oD|xzfCi;U@oHm5T4>!7LSAgTw$}XSm z`T?cwt_SXHy@Ao9UpXtvq^f_mllbx>brejqABg z`3JQPHa=mWP5Cd;JR~(AY-6C97p5iC6a{Vc3lEliT(hTH-8kxDqQ<|IQoR$K)RzDr zicx`(T(xxX2Esf2Z=^b$N^nB2Fy`VWRb}YN1a)DX5W1e9!!);cdQ;K)eS8ZEqakW+ z;K9~f`?@(Q1CC7uCCJ9OD~85aU50hlcT0&n{<6Dqxqg?Q$h`B-Aw*g61}k9c?86b{^XAG?)Zx&U8aj__;@537`qT5km8ewF6xj3kOf%x2)=oQQHj1 z1!t2~6XEj8B<&o)WNzm{&mZ&S&m0h^K9XjM>)}ki ze!%gugrJG!e2D$dtJ?2Dz6n9L*$amoO+A}dH6VF{`VrtI(pM*E z;3QP`U9dcPYk9bOCb(s8UU*0^k=SQ9hiVvj$`^NB`b7IBvkoIyVjR8seC=vX0cyeJ zf8^lF_X8g+W9y5L{&;Tq!j~RQR95%rLdza-}b+%Ow}xPN!&F@E*K3P ze4ng@79{q3Go)?0MH0rO)(n#=1`;te`kI62LnyVP?8Uc%Pd}vwM-SCdH2I<0Eu_!q z_&a-lbFyHVZglibhEfALc63C1+;sv;SCF@aT^*=HR&CAxOHz4bs>xrCEh|7J%^)djx)how1xTjx7p=1LsD2<>|6@Yeg-yrjC{DMyF$;^i0>K4_Z}3A}*Y;y)FLD|?=s68rgBzW9-N!dS6H z;n|0dK#p%o5+)E8UZRh!HO@GoeapuUc6%#qm$a2EN4?^ccjBL#aL4k_>pz%ftn&N3 zQw==Dj2EKpQDxc@de%5p?t}9Yzlhg{-0nTTsOV0_l?cb~Kp--%!hnzAWhEuC{Lq`p zCKH*4E|fR=lyqVJc?K0^uK@o)esR~J$l;J{R6EL4K42nX#&vc@Het`zsHq`Gf#Z4 z^zyV@y<}d?v>E&$$Ysqge2&DB*NWP!=uu^i?^Leo`Vc8Tu~H{Z?t^UV2@Z^7tXYn` zHtsG=N3{pZEh1gQNy*dPNkH~|Z~BT|3emT|*S7WuaY=3IG+7UjbkwlF*F&Ti)Nz!~ z^A#)qB*&o*X!(8-rPz5>#9_=*vSFW;7BF>%Q97vc{M6h+jKXY#*InTII?~pNM>{5a z0lA4;&4!r%*viQ2pm0AA(4b_0AMZ|UlB?vVg{5hWacp|I53ibZ(-?$8Y?t_vs-G!7 znOnEn%fC5ksCW-k1kqwVcCjV_lM$)}2rv9qeeuyfPbCC<3w}7`;Jw}Kz=`K4U$ptU z_Bxme+WfA9??}3muwu=^VR4Z+US?=E?eT_Ks&Ew%yS{x0_O}z&4VxI}pn45)s7Q;0L4uv8b#UblJ6Zw+eq;_fb^>0iC zZ8c3hW1kQkgc(Hxy2^EO&nZqc)rrn@Vj+%tyIBsH@*zP46i$Umh2njnS(a3 zLxZgNCcIG#$?gg})Nt{0M0^*na*c3QRJ~3bjo}Kay!GXp+ebCAo6H+5XAwO3dmP=h z`v|z%+a71+faQ}B=QjQMWUMo7$iDpq-5UaI>M)UbE&*{|P_`uPnao-hPW@nPVI1LB z#Dg~!FNd-LNN~dv1r_M(C|kC$uk{L(FUdTJj~s-pT6RY%|E%4%Bfh;>AV*j}CN{iv zGXt4$T+FrG3du}4QO+loojhOB*dGEJp2tLaVl9Y0(puhvxGGP$L|^yS+zA!!R;<5N zC;b!!N_LPtXp9E|8FYu&#<83UP<0U%xU)v!G`Bo$4Xr?r{)xDE^ zV2k3($t1zj;S>zFuy%NY?r`;r)!IJIA5D>Y|BBpsp)$!8rJ3HWRhKjnqyQW2PnNcf-XHThQtt{(5^3(M! z;c=2{SNVA#3U>`f2pjC+CkbH5UDwz10K+U6%#hJ&kTfAxPj+Maj8_n;W|~tPS)Vas zA>^=QgA7Se78{`UdvmROhrIkzP4gNx>06eP`csA-0yp?Hu|2d!WP|*0v#(zf^2bhD zSyS-YmkPiP-{vrfS)Gs7lAH>iWmdE@3*bV)B9*G!BMhmG zC~y$&L@PubXAi^A=6}!j@+!d0DuRj8Evvrh0wf!4f2!}K_qj1!Hx9$Fp)heuXVRA1 zA8Kf&PuWcY#~XSj=NnNQ6|gB2jT@GQyt2IWhsjEzBYmgV_X(vG`!cI=FqJeNpY9;d zkRVez)}omX`G|ye_L-N5_xd0^^UWoB2=VQaS2?U~62{;#Zg>n?Z4D_LH7oO2Keayr z0%*IyZ6$~-@YXZBc!NPEpH z&dw3wlXW3AKF35SrAc!rEU22*!E%ep02=ps_(r!#L`ek0M<$p#E+AeNFe-V*y`1vZ zX>SOc4e9@P_nl$hhvT*O4yk`SuRMyMjjf8f+YfixmFV5c(2F4Bu=>F zEkii%O#3eMn>Q}oJP3u59mfyS_^?_ksdl`Sq(9=-aSxPqtdSa-@AtHel!d%wSwN}P zErSFMyslLr%pZ*K6DC*!Tn1)#sdMn>4v>;1nAEAc{d8D zKSLqnXSlq|6nZpvcudeo)mf)ppWj65CWhwbb!-GR7thO`i51GD>8R4SWZ(+ zuv{|{cQ{;ua!|>lo)I6XHQK{nwFvt5mfuC|BRA0^;$Z+Q&TQP-s&zf-g<2JfNfSi^ zY3zD*GWu6h{6)m001OpHD)Rw69iQXfsj0x48om%<_u{4x9r&?l2af|v%-5SoPju8= z5jCUewV5|eL=x3iJ=MfA0@47|s?qgbEgo6d2+yi8Cgg^eH)4HF`u%46K=_~$7)!+V zX?pQE#tKeiYPVZO|3_!VYXd>klPntP6ypdk=+0iF-Y?&=UN`0%v|8n#RDOEF`vXg% zaB}^GO;{}z?g)$P)s35$9&#JoW9F;d_uq_8ee7v8(APycd(Dx_+8VuVpirT%W!L<| z-V||@t+eZFLJ4~YtX}xM{0N$>_ej+43kaborA+mixHNRVD+(P|SPQP79p`77`!EK` zQz&+p6}*51m8&Lq*gL_q!DTJ(o6}T)^24(1)vy7c4mk2uvW9OvSxoLzNkUl}2Ou_nJ|Tg5zR4kY!ORftkS`Wo_6zS7n@c@{pGlUbxzCOoMq2aKBRjPicU$L4sI zss)erW+Ng*pZf$BO&auW4#{+U_n?aT`3*B8qxi#MXTXvspZk^T%c6O{joGt*^B7|` z37U2lXl~n8+x0du$z8x{e|6<@12rPlP8CmY6*zd>?l^V~dE=eWbQb#(Rr>c`1+SQz zZ4{s1kLTsqVTc$;ht1upj=C8|>up5Ue3HcL$2yoT2GI?eQ!Ea@fIacLtQYEFq~M>x z+Z}+8DC>fIe7=;eMcsFjsU66Peguf5u(7Zt6F2D-!T4Y&>yM

    g8f)12twGq64l*UU+joOL#Lr9=;dR zj4i>ZVhl1=@!%3&OYo0I8 zO%j9($;(_(b!=e~@KdcYNo_LF=r1>BzdPjWGZTQ=G4t$tJfhftNM=Qn6&!e~X#ya{ zHbSGcPGRpO-`W$7lXPZCo88h=8%phB`@t&+ty8Kigsle%!9T=bBG!^a0$lDv)$@`@ z`0u`=W+Jo`{2;Lw*LBsgfFzbhNm?9)P^BG+nyR%D2z6Gx?O|aI8ux&_At$gI@^M=5 zsJj8(`*c_Wy2AEzxw>YX#f}eEmGOpWyFSr>+e5Zb+zObp?;7tyig{f@{R$Of$h5 zzVV6pfhR6CVEE(dQYK^^JctGZCR9f zK0?D2n7zId!V8xvr)q(2l>9Y zT^Us@!st#+oD>-7{fg<{j>5B1z6(#IQ%%Oh zLVe5H+1Du}&dmO$m4iU>bTt&qL;yliL_XQaxmO-4-irQ~*Cr@!)L zk(e+k9!0IzR~xc#f$`x%InWHW8x#ahg*Ev<-Wou2-Wp+)yVO6ENF%@Z`hn~enTbe* z0{$R<79$;Uq-IIRl+4pw9#XgGPEdYP{Rq)KlYrmYRiD))ctT%|Kb$IT_S&zn2Z{UW zG$`=Qy`I;mZQ43Xes%Xw`yO!3`yJRrPNx<4eTiN)Q8G&EMk*Y-VEJf+K%W~l+t_my zs==m(S<`V|B@d%)6w-?Jh=>Q149IakD=olb~7} zQN_07#i8IrK1b5=!t@|?j4^&BL_$okPpop*Wn#uAQoR1gFWy?aYp1sKmaEn3<9@wh zp?QOU#>L**1n4#TzBp^;dQj$JhHur?LHqDG9*Wc@FMAnLnM8u>ToS-b5dZ7ESLd_# z0_O)Rr>gB4MCqUT1&D}JvPKZ=wLYY8Nk`#BNnbc$x`i}Vkz3!L(2dLN4&bQ2d;=j8 z`n5eWx*Cg2)Mu1A|3shNR}4=|#&gQ6duQU?L*%L27f1_)kAP(EC}5#Vg-otDSWnpFVZJNqS?C9k7BP0zhmDY5M-(0C zWi-5f$+V{^^vrM?dGAy?Z#WgWu7DYB6_Z)>hIdi=szjkzV$$7dh8zb0{pz%thWA9V zu88HX*ox)5l0Wvz0)EFL7^6f(@uNTqn;LCmH|dA3hSP%DmBCs%qXSWM9DEp(&O<)( z*W5zJGbAvfGtT2tXAyN8(Gc9I5H~w*LrWRjr_r6CpRG7(u!8169pu*mnI62Zr;3gI zlJQAb|Aj{4GsEGF;ay9a7PEnj;*>NwU+{q>{t&1XG;#%(g95$>eAZRSXVP7u-k+%E zf*G~DE~Y~{sFo=rIRupUV_HEP-D%dOUC+J#gkHO5?hTPr#T_k2qILe8iR&NB)(hV& zTQ1`8bXsz{-FtX$y=(YnFb&bQrbzVDSf$Pd3Ed)l*$IK#y|_5+0dt9rE3Ty0B~Svs zVMI8CZzfA+KbiWeDJ1$c^(p$ZLNB;6W2xU`kuvVKKgMg2GoFCN?ffD=g2N0bzP2ZQ zEIQaEq5{F5@`#3f*vY^V?T0P?_&W zdRi`hN5twne?8?o&0vkFD=DKRzl?`PUNF928Xp4}Gr=a`pVGrXgRZOaEoT3Rq64G% zI&m7;C*v%YaNX^ohTbqHn1i17+9BssQN-N0^t{s$v?GUpN+Urp)1r66vrXsGZe_J` z8`~rgMneM8EA@hgU`bxEm_M(i@Dg$LvqK<<_Us;HMSR{VS+X-yr)A)WF-SrX@73bW zs0D#f3gT?>oMx7cw_bd|>V5Jot?b!c-$VGwOio)G9}IgZC`lQX-IV`j&RcT(>)lk&`Sgo~SeNUxzM~LK>-hpBT#S z64`%%j$&Q;0(5qPb>HGf((ije0VWngAA=Z9Vu4}qBln$oV-jt3_n0BXWudV6&dE~?nuyHAL$|RLKAdf5utw z-Rb(~A?rpf!%bSN5WuJdC9NAr(A?}dBWF;4l;k7ULzH`Hn1*5uW zHKiF8N}Nq-u#d$52|qSsQBy#I$zMW52LNVxehEKzbarrcFmbT`S7>me+Th=W_SSrh zL_wGQS7>m)$q#9#Ov>I502poYtUqG;`AOMd@tRax-pA2~SVf`lqI>`}6fx))C z{SGY9i3tGUfejQ65&UfbAzOre;Iqnjivuse~PbmK1es(BLx5us{sIFaMz8tN&hjvT43LKiF5Yx0Ry&` zE$?LtIIvoVEz4r|^+ET-b#-E#8}+Co34C0Na<1%4w`s1$?Ro6GQJ^sTn!@+m1>2fs zaWa3GuEure4+_;6u8|Gv-3aE)19ubavj{%bMmW8_zTK9cASw#Pe)?~5z7=u@`p4UE z-+rDlbHEV;laa1I*Xp&Dvb0OFm!8S^WFjorK!+%2q#c*7Hv+4?KvXx(Ew(64j3LW0n zW-jmlAwH<5h#$M;a4Zpj6q}uV{tL+ob#_P8>u~IM8gQT-6>*Jgv#J0QB$#Uk#cZYl z{YAZv(2mlfyB{tCiKRIt$zH&VRw2#YDWP0dx8$~$si1s~=@fr`y{Y$#GddpU+ATY0 z%zd_k=ED6^l%vxoKg4~~**>_C6~-Q3y?F->CoSkOa0+`3vABE9MuYug&qE}AyxmNZ zd!{{oA`AlLisc`1q`E3)greozLv}H8EQd1Q;ENN8Xh2AC)C%Hq9#tHZf&eFA8L7fs zZ#AHYiRmRVqCb1a1_lObmyJ+i--23mW52?Aew}U>d~5P*`h%1Vta|p)p>6HMB}s&6 z42G|`{`(4QhDPKZ<}z1w-X(Xxq?+(qd#gNW_kF8xcP5Kc$V;6{jrT~{Uh|XpEJV? z51QW+PWShrP&Ht%VE661ieJ@x6JL64lYalvuO)rwgWTHAuz{*;4MH5crM*K5ZVo1U zv}*Ydx*t%|Jz@A^rP$iO;|jMxQ4^-vZcL1bx%h?r*q%4vQ$Vstx z+$Krn-s2N;mQYNj&-1(43N#oz0POvPvtpZg7A9hWrkkoA#V$#xHNgRNT;8r+jQ6Dr zHMf_@oFO3;ic|~{1SgLO((EL6$;mtYyv#fcpqS4DV&d?6?zsMGx|M!?ru}b7x_~I2 zTM|Yrp)_JGe!2!;N0e6ssEV+6gSt=_c8=2ImduW5b9o}kF|6chfv7#IMf-ACsLW`| z#`F?Q5Nv_Nv?RSL&v3hg7_h}z-gt5H@(SSMh2dK%@|@x7B;PxXp4pF8yL<77RRnp6 zgZMIKVkn^WcW_Ek7I+Ca0>cmW85$4wY+$EE z@;owGM@=C{BJjaHBg$SjXRAE;)Riq2=KrjP05_ zpL<6Bf|v4<+v_SiYK0)C=6vcQa6N{A5Oh|7llqt47qGt?C6x_%z{eDwX$vkd2)NjDu` z#g;G(#XBvkfCHS3DDpFI~{zO)%Zam#8b&2lg4-8~_*Vc$Cg%=n{kBct2 zL1anU^`Fg6(a{kV)3xH*=*8mbP%d0VU@27wiZW{Y&E=b^jj+sEoBPc{&3g$I#U?kg zA4tR1nb}mod{yNT9Wdp|st8Laj2iu8% z+*sQqX|N8@FosQ`LHT|c?8`j%uEKyGZXsQu8L3vA^o7n{pUi@lL72w!>_aD~n!~6D z3>M6iwa`68=O>fN7j26PXF_ZXGX#5*#~_vpkBSU<@Q}~xJIjzf>wzY?1mm;Jy+c@P zXE%)%u<|DHN+u)IwCmHVrTWQ(en>s3edW6IEG6m!yuq`HrnBsC+aA8`bv)j#Db?H< zN=@ar7+CAL!t=gql&4yMZJ2-lS|*;0GKoE32Hnh|>FcwB2^7L$)mKU$N(*0P7sixb ztKFQ{P{22>P4EWUjsy=K90yZ^Xqd!`#*{kLH<>(dzYlTv#+i=5Yi9u~lpiOHgNkCw z(8kTsvw;CO)Xh1T0~qNU4DOJhYB+hz!fk`)Xie*Srz6ze!fS$wm6vlY8{46~gDkpP zGP3$`8PW~=w;0brjN$Y}XYz#Yd+U~Al{;GaosJp&STmpcQ{X2%3Y1JiP_h}!C&{D+ zpuza_d(3(}$36y0E+@Dtdd|mAMgZ~Q7H{D5M zKh6X6>B)3Qw9?89(NzcvP$4HAwvBeX50c}s(GK-Xh_kdw31T$*_nYt94Q64|BTr^p zYQl0^Z?&e94WQy9h2InXgY-7^w zrv4teblq_*x0lw7|LtyF!Wr!WPg1b|bfdFSQ+--b_tXqCMzP?W+GwUbCVD(Z2KrzJ zO@sAnIopPYMLC05#>bdtY@xFtW;9PhuMMEsOCK#a@QsdCr@<`_hG1f5+Da|vqo7R& zsZ30K8rgd4*X2l(MiA^-FE;T0S?3&@Rbmd5ksMS~+rlwA+Voc%C?7-sGJj`xwOqB8 z8ztdlKbeZrFQ7`~@)Kj+LW6)q2@N9E7fX8b=s4Nic*#Jrw>6!j@Twp*-XL84V zsM!Kyd3?}Xc~;@+ceF{pj2Za=9weviu&RXn2h0Ko1=mLGNkCIt|)7x%ztJD;kQ= z9#YgZ6f;4p%;r@^=}Tw3KKvhy zp3AGay;oMg7lWFrf*wPkkP9%qCrV+<-l#G1xSDOe%&ZAJDI8YAE8yV&)I%jHCJYMm zEq3UfHmNKREt6NuL?EbN_(4ayo|#9|akX*aZpGg#0k?69lG#=1gEw4i1QBNbnV~?H zzx|^ts783nF?gmSj42>eymaC8on)Ru|FwbrvV@@w4Qfiy%$HP&Lx&h^t2v)*;_*`A z!OlyPUgt&nBI|YaW1NAdG-oT1_7HCA(#V~wMfQWp!0+atD~`0Hf=AA&+$v;k8erJ1 zBP>-Wqa%=ajl+cJ#;}!&L$@$>#D{xc#NZ<;gH*ntkRAqP{Wu=~$Wvjd-0NMS!6&Jo zo~h2VU5!e{96@ti5`Qv*w2LY_kil5gU&K3zSI@Q6Je6Ic`}%;khjRS09AfzLj^&aa z|NHfdYCEsBxyX3B`4_EYWQFSOapViTxOmY9rwk~A8f0-xUmHCbsgrz955;)V6edDm zOo6W5YUN3boK2g%i4g1dw@}kw)5Tu4=m~x(7vZ1i{TMZJbZ*o@8#`W>gQ(YJk-BMH zQ10};iA z^m~pW-hi(^u5(aYt2Gh}1;igI;ZZN@>U2Yg5imwdZE8WW9eDxpCo~X3_4=jvHvEI2 zDL_XX5b7wU5rgAiv^;|kSZxEa+VsF6&cl^6!VuXz3{_T<+8f@@Zd}60Gayp68*q&$ z_;H>~TJf^PQ~}df1AUQlFy>^UFcBAJ-3v?EPhUIJPP9OLRx~j&MT$}5N?1HFrV)w1 z6n2&h-z+?=X}cNe({+7#eWIbT3f~^c_$z;udf)vgro`Fh(bpQ&X*%U7qjLkYQ=1#r zJLJrsxk?u0l*~Ld73@cpaIcviVloKPN~?PW0&Iqp7c)Z-t3Aogkv zd#~KGFP3!Ry|7_wa;ig{mx>FWDF7u8sk1_LwX<%ZyPM0%j+#U4!#i8E3}5@z@Z*f) zc|5PPx}{r@0}&lHn^g6VZ(}fA6ua9I_7~KWe#QEcLbZLJW?`||lVN;@qGa@;GO!?} zhZ8(Vl$UuPYI&or%4_;-HJ}Z^B-O$;b!APCkf#;iDuJ+Z_6Y()7+g6mRoZ1uzIFK- zjaA5LBv(AwkL4V@TvVhYui2Wz?M1XlU*@^NZI2)MNR?$j+80+m)uD1*RoBKoNRHD+V)3nb{0_!#L;yLmmC z$dt9oKFoL9A}E>f64{oHE-C5b@zw`l#!ukoh9ZI0DcQ<%3f-@xM}xXL(T}#(!NC`v zZFX06c6VCGJ;DVs=aV6ls)4#PS=x&^wMTRfO}U>{c^Ns8KYLvfehHfL`_>{0`EtOrVZm*^nW_!nPl3rEHQDmh@Ba^TtdJmnMFO(**rRQsyMDSVGz?|)Az0SGm z9k!V4eRvVLp%s3gk|B6E_W8Wv{>4ZecM5Rt8TqOItzo0n=bZW~5dGr3LJ`#05kT(H zHXDST$v6KIyV}O(Y5pi-jay>xtPi1p=&ZR<^+NU*hd0~9ogw$%-W`v$pMk^5C#)M> zc~~V1W^OA7^QD8iS*j_^nFD$^qR9-!N55BJYUq}sW}-TDozgLuFWWBd7H1tbLF-!@ zLBDRlpXUvQ!~gc~GpH23j+qFnZUakhwXf59(18wJHyq!vTJrvgEEtkLG0Kp77(H!D zerpYzv%L9bz1pX$n4}op1Wm4$4k#bR7KxU0aHM9SMr15>|H#RUSd!k%wO6ni9tg~C ztg7Z~nhZJ7D@4;A&&+f#qxpOVaY*+z|VfcnzI(GtX^UI|~EaQ1wwe617; zTI9Y{^-Qhi2vU}_Dep~%snSN{v)Qu&TQ_jb{+?XIP$pf^Wc{< zL7LRWiQrL^sm-J_qCB~;vxj6BAH~W`{jo)Mm3?P^MwZ|PqQMxg@JjDRO#XPh8?gF1 zSK^h5HqZLU&yHrzJnGJ}3!~+>7B(bu)i_r}_1$T-8&ssA=)>D+H`>CFrjAvYE9r={ zlmnZpMS;7R>BH1LsnbJ_TlK*v#_*h&Gu^%YVmko z4*9f)r6`huI`kzbjA`7Yy3uzNR#PHyZBQkz1IZ$wPgAUQ{S8lE7R$rv`aqwwM6k&+ zllJrx3&p~|Bj{t0F08Yu4i^iIE-Hd2n=@0U3CPjD7C`X|#G)Iz>&Tg=-C8DJWt`+L#^@hwoOSOirZAO0MCRdJoo{fL?<&BDdLqma zBjE=u7@;LJdPVA$rjz2bIp(q9y*-UqX~Z1+B$@`{sSAzzQq0Ufcv!~))_RiPLe-OV z%{M)MS(b5crY>9K;_V6wRjiS-)T8q|-_)rvS`g%Z;niv);g~*N!<3HrNvMiXZ$(o( z>W{vmFC3Wt3=FAGj@TV?)xAlr?DpGf0UDKl$w>nd(`!fk5e{R7>I>rF;N15Ir;8ge+>}0hPUEj}eu?<89Ms$gvuO=kS(a*?zM5U-?eHrN+w9eXJg9_r6cr@!HOCgRH@;GRc z)@?68b&BO}ks|0m&^5v~X$u&O5Q-BTCAnLv&d2h%G%@hgo?h4Kn(HP}yVlPoi9Y)Z zI#GL=-b2hrbhDRZ#Sw>y2;1fyjrx4>MRC=Uqea1t;vEGT*am-9&OfXV$-Vr7U4h)g{b-g9`x* z`JfGR(wWGXx(4?cx*?&;YHCY*a)@_c&>L8zWfe=NWvTjkB2H>T(n?y@P5HGOt@p?fK;a!4$!E|R@gr{04{{p2xZonui@y;Np3F>ar zB>r0ttyg+M7?{UI#mIPE*;P5+VfL5OkdeoIZFZDTt4~oK9m%Mr9ZjJPp`;pG@rw&qC zobUY+k6^w&F7Y+3)Y+}w`3A4?4@(M+DW)PKtcBL;Gy`;M%DU7I=@BUtO+YiTdD7}p zhV!7+Hzzs?3_Xg?p*NHf;zaNInm13SS6by*s=%S~7iE(I3(bDbQXp$m4KK?zEa|V^ zUC1QVp@0`vFIDkyW?O`cJcXRd>7`j#=BSTs<`!uE^Go#1qID8@KE8NK32Sx7$}l%K zMB{*_+1M(5LxA9mzgPk^tOH5r=6>Qr?aw$6*mq+I=IvH;Ov}ea%D>NCWOd*pJBvVs zNZt_do#yh?zQweoFJ-VB(_o=xOdjv6!G{vM=*HTWY+_%sm==VGof4w;KV3M_-4+s= zgUK++@l5aHztOp^YbpDZ?S)8BI~7pddwyEKia4sW{Q!8xez8iV%nph?TM8w?h81^- zyne07;##MoyrIK#q@4Oi^Uf`li50Scd-LrhBBL*DPW9e|&ZeTdb`;CL9-7%+7ol(n zv#!k5H)Pc>8U{jfLDe-i5bU~*DaJWs`;6>dJ)AbXJh*p?<)3oZ(!9nS^p59Ut8ld9 zbjU3{U!rcTZi2jWpbtS_7e>N{f)b)1-X2n+zq~4AUMbdFBU89;YoALxhA2#WpB7*+ z*t&*c>qC~;`aDVuIWYL^R$Xbusi<0NPY(m6TvNASyQamO7AAXu^Q?(@w{;erYZ)?m zgt$zdrzK6`NaPC+(Gxw_IX&2KYDHvFNBfV#A0VCWf=T3Rl&GLU)3ZiFAK(X9_nDkO zC|Ycu;f2?1qHJW@mo(VTZBLY&Lc5PzpySu1kSND@SI2?aT38!` zTZdgNVO$1S9R@bTPW;>N?NOJz)&iJTdu~WbLxiigLnhdI!$}q}l zpnu*(QKL97o{fYA0n6^`pa(fV)y0)+UTIm&Q=?)b>_s|fNi@J3V?v;691}qc1=A;M z^So8c4H3R321!U#w|LT`)JB%QhBlH|iRn7TM{S{AfL#AEh_Qt6Dad;Oq7ExON{tGT zi|rH5s(sxrSLGoV%JG)`Kx4gS)SW(~><}_0_?(9qxuCA(GlV(C-~wL~2Dguf0>f8r z9Hbw{%grG!rQRq-5S^!XZAtFl*ghJGGrn<#hycrADUi~j|Jvvt&p>LKUyg7yhgUYJ~&~-72r$^>C;LlM#+RG#-(-oj`1no z$(*3xjYZH^6@^#fcvxSPyHz%5Xw{@y+0Uo2QxFD;c6hs(>3@XB*w;bNMKegPDUrU*P{d)Wi zrB&!zF+6I8-@5uCzXG?U9~R$frOCv=T6+UfA9E}o<%KpE_;8)IZJvd>Tt1!rqSd7~QIZQ~CA z1Vwu};V$lOvTwLB0f<&sIIip+v}ulSAQhOZK5La`ezJqR(xmFL+)rVjv1nOK+FJij zn889?Rag3*RA0en{+frY`Ake<2zg;?DafGDQ>63VY2b z5eXA2gBHH@8l&P^z*`?|!dcf^U=Aj$64g3CFUs(1qKgu*nROM?Yh;p^YuPe(!`W5R zBt5gZpxA3XLQ0KV&|5p(;tyod-%()G&fL-=EpV49HlqXwnFwav0Dqf(D5va zINm4tLuHQ!P^hKEO`?za66TP?k-`(Dol4S91N!2R z)w_csQEWB@?jUA`J8m=rTkq7I*sHPzga;G~>{QkpLJ;5D&69*MNr#5DxZ!}_DGGU8 zC~3Q5of|Hh$VMPdol|<~JS%u#Ou#eWs-UB?mpf%@A@ONzx3^c;8*YT-hkt>n)yKAw z6y)Hx+Ppz+X3Rb1_#?ga?s}u>J>2j3cvitPs7tx<-8_&lkaXMw6t=V*JO?LK0aj+^ z4Bzys@sSRj)~(e2T}oc%TdsjGUv=y*;fXI`uox!yF&TD}7NeOxbONW}hmLF8 z&{`I18`6$l((nz^o!H^HE1(5J33V14)Vg z89S<&QBk#kB&saV6LW16Y8M z4~UoaU<1VX2bKD6ZlLmid?3D{Rrt7u+He4t*?9&4F#aLK{8NEj_HRejgAaK89|d%< zf;3vs$uPJl5?~cDJeKrM5cmf@CjPey;4fbSj{UhrtL%R6iwt2P=|8z176yX9Ui@GD z;nV23zolPfg989WkN^N&@HM|k0P+7x0a_Bs3HRqki5-4kR7v?yn}1J=)q5%f8zDfD zP=A1j%WSU{O6EKXaKRmYdi)!DbTe~yv2w8g*ZgOf4*d-=2QV=B^K}LWBmyMVEKIB{ zOx#SY)L;-12tfq-$2Iu00~Nv2|Nemn;DG*of(kekdHU1QlieDON1KIP^~)|0~b?t59h0p9X)Ud%^ae=-z)A z&Og9j(640gud>TYf6D&K`GQrSIN!fj{rgpo@Mou=*5skcQpTU6|Du4QMgC!J|DpJF z0Q?RC|7w#YGs#{Li2HLsp0@ViTKi*@{FVLvRVXME5DzASe+oU3!2clh zz|Z3Gf0?kqmW{t!621c@Jr$Av_i#OxkG~BU1_bc`>y44YZ^a~dxc+GZ{`@EKdl~tw zH+AU${{l(a7C_{mi|c<7{=K^VRTu|0sl@_F^uGxGwaENcXbdsQ$r4EVzizURj)tGuK(Kfztye3dWd5GbI1K!#e%o-Qx*G{F@XlZ4}E&CLIJP`*F{I{Kyc9o F_EgblXf8E2vku(96Z>qn7(TufLv1+=xqPJ9zZcb6lK66FhF4b zo;1-7k|6(bpn&{`L|@<1&eB<5pV7hNAL_%u%Jx-3^}p))Fhu|whQa`RN$#)E{_ph< zJGg%{5z`8k{}(q9*uQrA`f^`I6;w3-=P3R$l7Eikzm_5WtL#6`;NP{t|7{BR|C^Xm z|NlH25Mu*OV{-!|gDW{%wJT(`FhxJGI2>?l6nxNYG7_lkYGxEj0#T8GU$SvAxWYDp zMM0>fj6iy3PI-!kVq9XGx< z(b10C(OJpqiPc$&wb9-|#i_}GRoc4g+{x7io-~Ioe&;kD|yZ=^nlVVK&ncY8<^>2YPwzqXKbp97;X14zhZ36cM z>@PG#B}C6dFc1)2*nfpKDfXZ3aQ`c4{|D#)CtUwd3;6${_TP%@pWT4|pIJlwCoVHf zJ3|{w&;Jy&|L6yGgbUz*fvLRY2hD+1p&5=H4oTCXP;@zUIbHDClo;MsAf}$RQcF^A8!nGDkNt!cT0Xw_9Gtgl97=4G zj;I?ywOOG8+{C=HR8xcwW}EyK66v3*+Qt(uBU*J$Bloy7^sr5{$4*bEhF$eGniszx z1AHsX&y3}fXFuY(q`a~khvghy)~RO27Ibc5&Rw#-Z`U1X8Z;RK%}ncP%{|s-%Xv4C zXKSdJ5yyCWN=8=OZQyaN)F@5osaymr?d)!Tgd|=95HqU$RIdF~lZ7tqXu zfxMF|-pL?yectJBrm5Z^d&8e=%gf_xHD#V$&v4y(hmmgwf;?kYH>`AcYmJ0j-suzS z=VcSK>|gAXF=Y+DW7Uj$w9+^*VUIHpGiF%TTHPjTpE*ogHfnS~^jhJ~yCi=$jJWN^ zin+J|kmYKC8ls!_#$;LPNlno6T$xxRIsdylhfhXE*NP|PCTftGOG`z$<;IbtzS?Lz z0TolMUW^O$x+L(U8n6YGxEZy1Nd|3NtO*WNhueVkVV0fH>!wgGZsYgTKD;GtjAHU7 z=+UGHWhnRxQ2Xcb(pBFW{>b9cTwTaPP82r-^MHTKKB4SF)LO`B*w z*s9mVGtF<`(q2Z`7c(@}Z6S-}O%OlrzRqp(IC*<~_H*sFR-KT(M{rxcdn7cweSc3D znm&I7%|P)e#wBXG-~YZ*6xOfFBa`B1CHhh@0a`wC$EjWM!%qEBT@0M&r&_sdORpf{ zfN%9=kjl|}QZHTq~tOI&a~F@dBvOU0wffTiE( zN`KSJ)NDWjAiG-rsQ1p^;}Tni&ddRj!x84#o{@lL5Hs^)2<~GJH`xb*nm|HGoP}pF z-{474?ye1kq;a#dG(`V4l!BPfd}2`48Mr$E=Eu*|pG2b7OxbUf zzy=S2&?<1hL$)r&&ySyz(>K^robwqwjsDbCEbx9lI$IC_baLDEvCjms5Cb0$*Z0R# zFE?j*_CiYxoy@|>NCQkvGm-hb*T-~6S4N)(Z}-QMeb=}#cb=W?>+wfz_|aYNvv zxbSnXy!{cLz&UZz;UcV$5}567uq@w48@kH=g+PFcqWM86sSOJOae>VL?Plo?xvn3~ za*Bk9+8`VhYAFkbpX6#28{V%*ixK`gA>VI(Of@x1+rPF@ht+3Rm%O(9B=|B*&+tI%zm}4M6)n%I|+VctaRq?cg=9`XUzj15AETOCD-Z1PvrURH|!Js1?Tskende zYT2@9-tV;gTKRN(JBxk%eB9H#8=3n&MDLQyPpFZB_-R%%<4o&DI$rd2P+;qr2<~#+ z26(z>dX?DZ)-o#*(2-i*_j+evV%{Cz z=F1Oz(5z98r7D)apfE^M-mx*7LGW~1EUV(+Kc0t~B<;tN{5Hlw!2M&u4i=u}Tt-bK zr38e8fcL^jH}fY{W)?0@e9ci35i~)AHKm>CH804;m%~Xw%ETqV(){3zRUy$dAN6gv zn2>l!721mU9xCJKl*Dju1&E(-&O9Z7QqIe@QX+qUfYImRYQ|mD&}9f24_LW!M&yrg zIoE;p5IT^#?4Sp6`@?2w3XvY5d_O#LWMv;=feVy75)u%huTQXy&7e3fiVf?OM&5Y? zn{rp-DgY_Omeh8?@HuW|11|Dw+E}3deid#q>DAoJ^N|&ML2!(kZ7og*2rl5K(yX?V zZ@uG8S2kS?&ekpp{tZj6I1)@VKtSM9>BLv8QVbq+T=JYHfcm11++7VxJ~sAI0ckc7 zc;9urF~2qSbdEBz@ccNTtB(Fk!Dhe`*xAvQBJeag%Y9c#J|B&MK;(chHBZCzIGLK% zE&4-(tfegzwT?u(d?p2eUho=Qf@|o6aYKqeN^qm2mkAbFhIVM~aKeRF)B-0&0lp z5Nft(cS=%cPHf4Sep)eMiBy~AAkMCmU3G%T{N$L$bW<_Bo{e9$veu{XBFMDS1F;bM zL8FN!b8xYZNv7%>w6;|)6Su8CJPQyBJR8vr`HqGJw6~8tR&-3N$-Uzmy@?1@IuLK6 zp(OWqEm^7qKQ-QKU(F+T(29Uyqlrzy4A6IPdguD>;T5fx>6;Bavf|Z)sGYthEC$+6 z-nzBH*zKp{vTRo(Of?}iIN1esmlXZRKW z$nBA) zy3)B;&-4k6Vn~bx9hPa>EVO`kN;P&*(3^4vZDaG8;NvuuA@wu2&P`DK*^qTAMk&J} zMpFxg{KBHOpR}xW#wQAxgbXorN_Tc)iEJKRu0ZfWAL$)dTq>fGozuDQM2qqa)4=*! zleS=JD4a~lezP@8wpNV{UXOX?8 zPKWdBYs_~iAB*}RHg<8%4sDCIay9kEZr)~|OxOoTrwvMR>NgnI=NN?pUVVChvWL}2m+N+l#EkzqkB>JOY||!sdt`0?s{Gz4 zcKuC*y1(O=aD&4JNXcYPKe$?rw=qv6A2GWZNWjQV_B?8G!RE%T0otJGAR zOtlnsghwK3jXa4y2u#+2qyq0{&N+OdrAEK~BdMi8T{sG-gOC5yFvXMmTgS_9!6ub z8x7fQbVx=e@H)FI0FGP6n9PvwP`^hlN6RKhFO?D5iOLO6MiNw8BFCbw1j~Vux7uRy zi-@@0qF|tFHxIMN7~@FG`KcGJY?^L#5zej~ZYcV<0K3+Co>w|wStD!Me4Zs0376!u z00xB|+YTvhnFfzsy0X{I2PeB!4woy91oNE&g8|PN5MRTGB#&qoOMB4{1;?$cDk2glZp8Eq(LZNLKT&CDRF zg&!CLaiETysTb^@#^sFk_28G5cB@Pc^F2m}22fAf`4S*-Etvf1TOpTQ(<}(^S`-Q9 z4)`~2%i}-dk&nrikSD)tGZ7JcQzb{YmAw_1rc`@2+8{>dx0H|*+(EF8@cJN5nBBOu z84`m?VfEVzVl*xxdV?$#qK2kELHX3Af4NEqT$GldqBBFB5U^g4&)abjESB&Xg?_pB zw&MBboSZbJ_H>(pvkihn<8_S7G$nk2XIorM)?iI!Yca&TDL*j7!~a_=BNbI%XrT24Yc8|K}nAb za2c(Lxtiw4o+L^zLxGLNORX($KYzTNKf%RQ9+SgXUzzE~O+dwR@rwJ{I?3DizsLbS zIcfJXSo4%HSuxkFt#NNOQQMcTmqa319=)gdww&)@ow%@}`@R9n!tA`NXkLuz?BEFa z-bAXUevt&{D8&w$wn|z?XYQ_BVL7b?c-T9{*C&!y zHsc7!*D&5rtez70IQST01?z*0AH{9c$L(>YC;P0$W_Jgt&RA-CQaJL?cTH=@ytHoZ z^|v@kelHu_EI6r^uCUw6fjxJ#aE;xJ&VSdZP%@#%1ebJ^wN3I-mBy}!m3~44?j^BA z9FH-Rs4-mA^=NFz9rQK4L9T-9=)=qK^;SB!9k!A#=-D3!gjbsSj*tT}qU(|#1F^FR ziTzK054KdnUrPiIVGJL$wL8CfAt%^f(X<4bXJw-#Ltk5tj=bXE{hr4jSuGVoJ$xsj zl)kA{PCkfkr6>44mmN2-Ac~P-Cn+I3NjEwjGDaf~As~3Ak;*5WgJg>5xfec)xN9sO z_Yg;Tw5u%MJ53%bLxtyLJh4T7St!+dqTpP0}%>3QCIf;!v%dMmINh0|1`ekojt^j&rcP@hR=-o)AGSvU*;T-u0OLKGpPSP1e&PUGL}iV`pdp)EaKh zo4JQd<%uO@$i^Q@VF33XxSv!rKhgLpjb+$kJ}{tlvy|&nQ|8C;`%mcUZ`P&aY9iZpl_jHw!Cztl9=<8cv!Ek-o%NukRK(&x&krQTj7u9GmmHxl{_jaXxg zQf2nOt*&3gQrK7uIW`!B+3+9soK$VDS`vAiORB<9^ui_YMgu_?0Ia^+ozu(7d-a=J z`Nm`ftbpSa!a{vW#&xR9f}mkuld2XGUhjqF!6e}@w#1W_9?dbf`#Tc+Sw?XVVYEKM zK=ka}{4L9J5uePv!c-5jw{q_1MyMQN4fN_>hN;o*s92*!WP^hv>-2vylM%^|<}A zLK2EL0voTzkATCFzt>c7Tunslt1?a12>w1~fYq;xN~=ggK%ri_;TtCZMI>!TeAlzR z1hT=2E3p6sN(&HbTbsMBA) zX6+h*s&GmXRNvuc%RTh*`Ugn+6ZcazCx+_QwgwW&0blX+IUn*Ye{@CR2qm?Rn0TGpFx9uxIvV>6@BUf$Xm?rj0<5hcn5M2Sf!hEyfq&(bwGEGBaS=5AiRI^z8$ z{*$8_zzUqQMU$h_2WO%R8ovTVIHEJ6<@N7h7}Qb$|0pS#hoc?ev@Q|owLG2#)9i`Y zivR-eX3LT^10h$~35zFj)w$+ni};`F#V6l#YmF($koZqmA6i0n0}ua0!k8|XJb*R@%@k7Y#76awap?op7b%A z#_isrealTgL)uGQ!t-!TRtw`QPFq|12)|;Z(1lE{AkVvS@o@M|8(WG@QKR2`KVd7z z;#v?9Q-)@Gkyx+3nMsNBsi~OEDaSY{>2ly`RklmrLEz^HY$5K#Hs5y?5~oZ{v`hRj z24Dih)!4^%O@G^U_as0dxR!E#gApD$#B7N|%V)Y(} ze!X=`bqECkl{w7csxSb4^-2)O6&~O0lYd*SuUlXqx|!p%G_B-KbWvQtu1g(mM6C?7 z1y|=zvG{EHvU9Ze{1mms$yz9Acn#N51?-)@1&AOSfc7?hzi;@>?xutbRDzh8HvdUY z+g8#nSzebuxvk_y3*74U0niE> zc3RjxO`KLylYf{^GAoROB&NIG3TOmz%ces;m;8C}k6i?Ptb=t=>xpNoLRd_pm%g?m z-yu>pXWF<As_G#tvtvaRVt;}NvV&|=6d>F0ACDxI!?|nh^-0LDYHfV2 z9p2qEz?POaCZu`n;GsKi5bJZlcqzg-^I&lpK*RllOG) zsUtn&Q_^qp=_{^{>)mR^We&`WyoODy59{sHf82XvpOF{ZshmxDrhA}S0^l9uubuo} z8M(ROrS7^BuWWI#t(3UMqvsxK?4x3?lR-+mZ=^uP?{HD24FcG+WD5o(!qJkd)hujd zIPZ|W5D0~q)>6;KoqT#T{7~iue4gGK)X8)VCP|MZ-F?cF9dGs&fo00_MdN1~K97k= zv5ZABxgE*qUCck^cYe0-z@6sz`=PIcN5nCg+oB`F#_z9~L=Z1t@n~@nl25AH!|<*p zgHI0|kERzTntT5hDGHga_!-0jRX!bkJpvty|pn zDX{{RRDroA>52k8sk-JzwBOje5e;+u$;y9nngx(hX@QqwwtUc<#1ZC-{RVY+j>f76 zyO1d~J(=AzmT0U0aj6z;G5xIsoWBxICIT!$a zO+9rF*Neo-$^L}d0O%TnlFhtq4Sh?fdq;#qeKWQ_?TLK9y{&Y`gnDOcPRvUgJ>BrT zXnW+4AKS11F;Hrp(ux)GZ%aKfB-k9rO*hHVjyWBxpzi(wsVwk#1d@bCJd|bU|NgAq zfu8wP|a;GfYvaUpR-jZ3Xl#VQvlh$QAtY8LOe2$!6V8i7{ z;$r+xQxnU*g^Rwgf$dqcv8Q@YrmR*;~P?|B-%YJ@>_%l+h zvqZ>%ciIGnHnQHFy?Cg$S0AIhImrp8o@bK_;a?m=|D@iAlD4WV1WtiuL z2`9S_!iw7ZWzlV!Ed zhhmx4z_Fyt{8C9Wba~NSSh(3Tydl<~{q4&e3nUoNbV=pO)>W8c&^k?&y>_84=Y$cEU1%&kY$9)jZ@w z1~~dg@~Il?91Jy4`p7g7TaKCaCl$#L8{D{!im@){<#DKqjSysblOIIT;#$)ui-? z8CCvI&2E1asv1^l)&(V=U4liqG#EXOL>l5-y_0T<8Tu46Hiuj7=|OCg1~R zzdiAf3hDWuD1ic%TSkru76SbsZ8O44f&O#|KMb}~$3YKf(-YqJ+Cvz-!}g{GSl#-U ztjdPDhIn?67hzZTU69;uI85906X)TxiD)Gl`aUZ0Yr0Q^xQ`mg=yzn(S-o*a>~2Vz1pDb|V(*1? zaUdw&sorCtqC=@yt=7`ZK~S=J^o^0Z?tUaOVJ+~1y!9DFy%1;9PJCO5xBly(!s{i6 zk$J=IDVKG;%H;?Qn}YUEV;@j)B<`E(V@?OhTZ_8l)Cm?A=bqm&_0F5>F$URZM| zZ_-+84Kr;MN{s1Lw8{iZuPjPvQJDx*c(gYig(ljCs_esHCcHpJjz|nvWSM81R$Zi% zsFd?XV}IONti6YjsK>%KF=^LX7E^FutDh(m!d3<~P~2rk(h~qCcg4?q2tZAy-HLs~ z=Z9|GB(%0JzUfrMTGjC6L8T@7ScMc;l8QMJv|MY2+<#L0;V zYus_j!>Gb_tuX*Jf2~9k|2j?)7Y-1TD3bXq2 zO^(#Ap!x`Ia!vVGA8HiQ)9WF+2`DyUi1@ZG0n8svs@T)khQ$u`I_Fl^|4?* zvGlLjOIlRK$CV0%wznXT--nnTAS-ZkJ%XF6+@?U6gSVmn7*y}#m2BqOqIPoc*?>1K?YCvAzI$cR zDchQ4d>@a_%!i2rT%lJ#y{?zK=-kR;ofgeLxv|wJHr&=Ilt|0cb9Mt59vsdaZ~CZ2 z`HM^4WMr8wWMTHM#VkCp|F}m86{Rx$i~)bG)C`PHnDBg)(m2RSI zxJ>ti%m2ISq)8f6ReK0y_R2@sr~`km3L=k#HKBbDk1u3s{u&Ab zvc4RA+i|=teO`(>LuNKo%+t(ToZ4TaGPly46{#r-b1;)(7y(1 zR_wR{W%GSM#XIV7nZ4OcMbY<_wq2=+LCr|nOqMpSG)^qxQy%xXqY5eZF|zSdIHHN! zS?s{n=^$9=e3u)0EwNym2cFlFAk}3IP~9Y2JS}F!c6mVe`7?Z}0IyyK-LjrtZ+zNf z&rO_*@%qOshcxVz-kR>-#4D(G`=J4AX!aH1(q+H=_Dl`!g?BQR{J<0 zIV<>SUm{OA6xTZj6?O&A{B2h)iMX+X^V^4HyI;R?w#cmwf1h0upXXb+;FX* z^iUxk4vUWh0=}wB2?!!8?$rKXtNVXIV{jFovkimKGw>@;d1jYCDWhg5h=E_i+{J84 zkO$liqxQzq7`NBDVENz(K`1_~d=^xJeN*&bwX!pcyjCkzlzB>7BH4B6 zhWIj-o0S|WF0m8LZ#9wl$l5UrEJyJNq?3B=C&RgDOEJ`jC4RBvzW`C{OMURRX77d= zm)NMe((pqgQLi)0+2)N`B*4kzT>#np?dxYzn3%NK{^HJ620iUaAc_fOJwkf&Fs+$+ zop{6b8byotC1*s#hJsD@A5WppE*zliGjQjb3j{{7x5LmsvW$nHkD0|HK!`uB;mahfA&Ksuv%nf(VKCA*9XEpYXQ7U; z>c|pRRM9FffA{SUmgDMdkVL|5iA&iNw64JvAL+=9D6M81bC~XlCi&zFtfT2{Olmhb@ma8MhvTZF>$A9q=Md+8WV~MT+3uMZ1{TUe1$b zDn*o3U}`sIk%2eLxy0Rfco-0hK(JsNu@Nz9$NlM3$l}dj$12ER9@nNr%N6etQOQ!E zqR2M?!DfbVF4-pzkvjYd#+}oBPMsJy;uG*Cu8!{&>?Rvm*jcNQ1rfYPMio(#fM~0z zG{FRa4;ZR2vplp`_?q9lyXGs^Rc87_mHF2_0iBNkIHXQ3y-;{3ZR-Y=ZSD6xDe`EF zjH%kK0m$RF-GbQBWUMIa1rx9++$&Dq%HPuJ-?n>{eB=t9s<|~-w)=|Jyq~(jIvQ9N zmka9I8bdgQdd9a3=YCG(-qb|aOIe!dDR6v9&H&TS*^SEq=z@=!V_&(ge$ko09s1wh zM?H~iV;PGq)SwDpOSG-eEJi2~fI*Tqsg(S<*2gZd$IOSQkuVu2@I3A&^D^vn4O*HQ z&s*q$y`89u=coL!Sp12U)z(38eCR((thI0@5NX1>H{#qnGW#;9u#k_!fl+rSLtTyk zSr3eFms`3*ER^Sr!)x#v(+&FP3T+5z4vvj}*X%=wuR+q5n3~2LmEva5bLs z^h9s-3~ze!7L@pjg5teYPS@v}%AO0FZv!B=z=Ehgyr<2@c9RP?F+wdHWT$ zv_iuL+}}u;;W_vG&$haenlJ)!X;EgYZWRWMIc# zh5dap2rGUrJWfV;WW8L0Y%!C8>y%5yQbA9i#F~iwhn+r zrW2TJXfYC3!p}qwk0|+k5Nohh`{)`=d$4n?t3k5XP`sw)TFfq#?tc1>Y}jOzorZ>u zD}1?KaBsHV$_)#qUUeyj04437LF>|6u_kCN=xM>hMe=W~2ma8h>?R0$~a@hI!K zuW**48~Wu`m%4Rt%gQ5BWw_JE%G<+Ii6(IbSJ5jyInC-A#_Np1i2aZ<=cExbfhX?J zvw%7dAJzFhr0zQVc)LsSpNL*?aX94yJ6QZ_H#+^dFQUNN$Tyw_iC7B{W*GCxdT90G z|A`{1xVogIzrNh>yY1>9-T+p3xOLYArg@mi>n5V9BV-y2RY!4y=OkwKf1WJ_&siX+ zRZ9$aX5ogOwn=~4rzDL~dO@krG&h9^lMqXX16A5B(3-EwUTw$icNlf(KZm14e;b)^;8x*K_shWw@z{iZ8D9Zz=#S66I&-(Px^^#n@44=ViH8l#x-)j#ccLQl?^O)Ugql1*v+(Z*YjNIB zib9U&U8ncEb=Mj9X;r*qhV{y!ukDZX_q@t>MNFccwN*ERiAVw359JavhRh_`EQKwW zw!!Wo@$VR%-Afnq4r}4}7x(uW&Cefn=hlZmHv;sN<I2vZvHBIXIG91FtUm1~Fxk=BAyM=BJ&vvFsGG-1be=mEHbAk%EeC*%P zu5%G7d&_;`M;8DB4syw~{^aXoTk@$Id9TjHjVAmWtm#**YWx{LaeDU(Vk)Z{lixv4 z>dynH@sadwzpR_>a)b&Fa!cs98WZxyJ}K2C$KI&-Y9sPTw3zP(UQfFj?-tn4mm&z< zefK1;3s(_K@?{y3D0?~TzQ^Js3PNWzS5|X-`ATk7a5E3YdGWUelPBIsscdf*mlYJ< zcPh3hJxPCul0z$m{vJLyj>mauwY!r6Cv4mD{pI-hT@g*XqFw@IjftI^bILRPvo!qr zW|a+m08@|Gp#Qy_IAN4ir%#uS)uLdhCXEN!J2ff~K53U*-ZNNFtqs8JXTd4*UsY0P z&Qe0-Q*8xqJ>J{$S9WIuISuc|$vsc^Wxn6lTR00Ylbh8Dd3gY-o4(1-DkCVgg`V^@ zs)PMW&0q;)L6VRZ5$c#taH`l7WD9>*e_NkdeNOqqh=v;~&J*${wxFw;Anre#8bN(+ z0nuX>Ng_J7mai=q96)KjQn@iHtv*`lN%1Gm!jS-skwZBN`z37s@MN6DJLK6TGNzXU zCgsFuD^yEW9o~>e+d6)9HkP#=^G3;0sk%Ynm%ic8L$B-3QQ2?qqCJH)_cS}!d)Nel zxVKw3WW+l8%v)J3*p-7ZsrMV18E+I5os?#nOrz2Xa7_EyvT@i#;V@jv^YEe%PKTIn z7fk~Q>UP`wAG_X%y6EqOz6AwLstOO{LTbT>zpi*`8jNcqBc@j$rR8?yes4c$EJ)j~ zg5D$(w%NiHrrw48W{7`Q(?WiLZ8P<*lNS-0!M{zKX#^Rjt>T1?9F?NUqyrUubT~R= zC;k;InWH5cBIH|1$+=Zo*@WRG)Il|o+o=GM78w3v@2X-=U<x`nYF%iBcz zvz4ll5M&+qV;zh|_~&=f?}+(r5gi>N6N?rm*Rtt7HAP`Z6{U1kKUF1K@jFz=QQ}r? zJlBzx1c?$Np9*n#i6@1~I1NEwF8ySPSZxD^{XcVjT5BU@fjRJI>f<4rF_DTCrOsNQ zY$4HRu(lvo5YHHM`Sn9@nb9t4 zLbO0(oLtaHDw`vjV!mPk;<>EHaa8`0WU*Ipx@<{T(;9uzY{ybThy(dU`_CYHONlJ9 z9kezp9Jykc)vAYJq1u&i4vcDsWZZI6^#-@~BK7=I4c2E*(jBHR+fGHYMO+a8`GLpb z`=*G~qC5~b7j@{5Nk-5gk3A^~M{4kbDY0WoLTnfewt}`Z#NEOqGAIG*!FKbO)M*iv z;<@Q41M4^I0%>T+`x6rEiz_l|FU%|gY+;4V{A#7xM?b{PP^A{@(D zldFVBYHCW$go^nWqC;panW-Wsrx3@&RxrjG4vJgr$n4bVI1# zewZxwLO{+ve^}d08OVqi-JWpAwjRjXJfz7#+@*_;XE6OEn61Uxj;um1 zfZ)DRg>DZU081#)&71=cICr4~Xc#{ryoVvLPiGmnA@` zHX$|in3xO`<^;Hkfw=~S!Eu6Z-hS-^S?OdHmsr1MD);Jl2GT*da%$q@d_}S>B!92@+P&ZY$OH^ zds*sFv}NX-yBC{d-K4qn0TI>T1=XRq7F(dBRzrs&+SZ zlDUXF?oYrav?ZqPWJt7h%~T^PA_2XF77KO+`(41*pjQ}v(5fG5^|L*3_yiHL+&cQt z!w=!Qq)3$MjyL9smp7x&og)LUffb#wPi1UQ&%^WYj+D%AE#u;9(CbZXy+R1JXKf*R)V+gL z`cw+b;PLB*IqfV_SI^~vfh4-61kVy;;>Zl1*3bM*Fcdi0#zLeO+BH;^g1lc zj;HPY-)vk{RIpivPP)($nf6DJ9xzB0Qc_{EPZ+qE{-(^+(JP(4K?lDQ5YuG+?cqs=5ox&;yuPg9b3M z`ooHh@_=a=_<@LS);Gd#{V#6VTrz;=z-#Ep#z^u75&YcUq9gBIOX_Sw&+`zZ9+a3HVBskFvC4= zmt{6=e!Dqueq7dezoYgK+Z##44o4Z=eO>B zA1mNhIwQZGIu>7EghDIJK~;;Ch~98~KS`-ISEez=CN&dXPNADO&jW;9&$WNhj6`67%9iYhqw zhXCAuWzyZl6UNbjr=O;+?A@N-LJdey9Z6jd9d`CJMhAzwuiWVd!W>Xt^7k|h4HL4` zfgq?-u}E&bD>qE|^PwX3`oQ4vElk|axYy7ENxutt6fODAxM)sEEP12-HZHbF*>7gt zSQ?hlRuR`C$~{RON`hS5f}p0@3Ql*In{(KM;-vyE#o35*D1txF;|2;g(`el| z(3=-R#1hy2MFDl~_ad~_lV(ZK_aL#0U-OK(gNlVF9b^Nk-RA8?-jg z$O-MTK@)8#H4=_`!fErsu5^Q$v=ez#43`8?k2Lx$-a+|Brd#PaJ@pNg{Z;#sHh0d^ z4;`b5ZM`t2fo^%tV!Q175cZ6~QscSIduox{XY!2%?sy@6@al@e?8ojilOMxjF=REICc&<9-H9gcwfkBm99Gvwd?!6j0{2A zjPS_HnuvxK=lD&Od{eIw*R?EHCuz|`*TE(Vday)osmLX1O({j}nv*L{tZ;(9xRoXA_sT--42EWVvJV!k^f@l$nZMFBI0H6`6W3gpAW{c_YWm|f} z5}M;S)7mB?itgH+h%QbXf$n{5E$r~Up$fZyf<_Hk)Sjk6jpk50bOE2hGi_lYT>MD= z7+bf%SHj=@vAq;}vRK?^50bc08$)O|fE4BVB92c4VB5_unDT>s=i_dm;d}I5%d}pq zt*QmLtK7cc1z2^m9-tW*N_jp+S#^%%R9*^{O~sC)Z#CKJiY#M|1cwNV4(cwj$9%P2 zk>WT1?t^3`lJY6T^C+FtkANV&^51DRQ=M&1`eI5!fM>Z%r_-iX!)bRbN?Q$fdK7lD ztMMZi(g}>Y4UE1&hxRV%1P>0+&1c_2+hn!1fg$32AK2*>wrArODC5X*bQU=L+A+4; zmN}yYyL-1@JHBqFHg#4k1)_C47?;zeLSe=`?mr)v?+jTdPgLly@&iJ{BwBC~QmuWv zk}`P2br{xce|8olIjDdxF*jetO&PWc84jL<=@=`8eD>)vV&<+OUmG-Jn@1r=NQ z1~VHq-EPJAo}jA*$KQrDt%Z;iCxuzA%w1Yp<2Qrz-c|9LQHrMi$o$q>wRIBn(zPb^ zTs2l#{(9t{kZTc4qv3aEL!hs7Ey-_`>l}Ca-ax2zjrEsLhii`ZNv28p&j-!Y|kBF=Nx|>fZ9T=u#BsPCw{z zEr5Q@By=4zG#rL{tJ7{CF<`bWuXduYXU0lEp0*?kbz9Jx=%7kDZElF1i$Mh>)EgB0 zP&S~m`%ef*N+O0#Agz3!UAhS3qYna$vCsEt25SkoKBOu7-kCIQ+gqfG{`yatcejdH z3*Y;43AbIc8;30Y7wuZ8LV=d^g1Xb{HDJFn*g>w4wQzR?=MiI%<|I6_NB#A1cQ2>w z%Dsng0txw4Elvijt6T_rH3ilw!6BIkJ;s+i^5ez)H5XdxVd!&R(4J^L9J41za-H!?^84L_%R5>&AAAB9pO1d+&E01epCU{Q-Eri&@c$3$-ZHArCEeO)VF7}> zySoK<3+@iV-QB~&Jvc0a26qn@G&m%92<}dB_k3jUoj&Pwr+1$|=k(VvWAKkr&#EhT z)l;={&pBBNROMYh@7}gKSB{*z)txy(H62xJ`Zt_Bt3HPxAuFG)+A9Zs?o$@598TpS zd2tyJNVC5U1e;aA!u7{()w)L@ww-jrf5j)~>K733UYD|rNL(YbdVji7NN0OSi-)SX zy1+OU^Yb-rtSwh^+>n3O(F5x7ov9kbP2h?;@jG;cl>ztK;A7i5vTlhFY8o45@>M)v zPX>B$;6$ENVeS0ZnFeGW&+Qr}8v}_GNw358=5YwFwRcA}g3?;Y6x-eolnfGey@)Dt z3r{G1>e2=$kH3UYYiG1$CNxWxMLm!K4n(}Oiaio^VozVlp$-tBUyoN|G{C%7O$?A# zShsf>NII7da3K1MjH{<{mR0?^4FPJ$jD!c`>h1OR_WDL?mO>piBm+kWHfi^O^` zB)Qtj7?F0@s7Viupe%d+SqUasUqFqHTK!5^A=V~6ZmOCt?)B7Xx&SPwI;Fyi7tcnp zE(mjYs=HYnpG`gX1)OKA508Ko^?-A}p& z;9XJLQVhbduNk+stkv!=S`dc$>~uODjvHY}Pz8$1H|FGA0M5;krG(RsR2a+IW>$6~ zH*Yi@l=C_P?Vs{klI$iOIf@hvdP!e+O{h(%b6z=|w*SbrO5K4ERz-dt=(R}wE?MjS`@%yp9yI5302Kf*?*1+cVYa<8a2)<-BOm49|87V0A_cDdnW5E;Ke(+spgZ{{Y*rh)C(eNf5Z9LelA@Yklp+eey zZtNj9kP8o@G?|vDxULnPiQ34~S4M-%NW5-DYkBAGa1%{bZ}rphA#}(96ZYa-6{E5TjpISKnzepwGGb% zE8D{?s~smHMBERYWDNtLgd-6zv;}b%^MJiMF-#QIEJcd>bh3P-6uGzPW<(NgRPAd$ z{HJKTs1-XNEj(OiUzzWvYv&@`x2t!{xq+jAcjxV}X6($fj4)U}QeRj%){S)5!qo{R zVxj|_5N`>OKBIivTW%+cD)~yZVgypfLlP1Vd^QQ9Fb&xA5_@=A<$yhC@wLZMca3q9 z<0Y{BS$?M=>iDbyK4^Fe9JGJ4*~UMd8bz0KCu+nu74Xm}M=N%=Rojxo3+)eTl~p3ahSPx zJQ8hc@-ShQ#?f~9Y!9F9xY@g@p{wRXd(OY?G+ML7q%#~)1V`Bj_#?aBy{bFNR--nz z_@XoK?VR4aQJ9Q@S|hz}W28A?Lqd{NkHFpVqC^T}Ik6J7N3NVTJB&@A)5yI6q>MY^ z4-Y%Jmi4TSB#ozYe360@VjrAJxg;N!NBa4_vfCG)L5Fx^Hc-}XF5#h^p<0h)+RTC_P2odPsjGKph7n(zj` zTFeb9WXUx^r!EOVQ&-5_%NA+FCNgJ%R{Vf|)%D|-dAZSpE6s;-!cV70{>1~0 z2b96AF4kpU8lJb83R!AuNDga@;~e-)U7@0PLpPVDQEa}DzMxjcbEcOE7t?k8C4BA!A*GmQO z6~@=6R2(2jAqZQzK`xUoo9EUOT88gaT}4w4rOY4Z*MrQLT(jOj75ZSHT+Vu zCPd%DNT&!=S1T#zcp%gC9Nt9MvA&N7&$%aMwB;_q#)3C};>#kCGGVg~2jI-Zv?w=i z8ff(@nH$f|GIe+qF;9)VmW-6Wj}a6DD*yV)Ubl}%%q`|{zk<=v&x8{=_4#!)ofS)Z z;AtLmS)08OYuD2+!HWSmd~!-(!tJ~?(9#009SeNuO!+eP9Eiu*m5zc!QXVIO!K3(uGN={D5gdqEaV^zOi`0M-$N zKS+-e4nt;ZE@~z(qP_S&%fp9$F{{%9jh3+6R0rxwQ9FL42imc*C}|J){fDMqKZo;2 z39%A-I@~1m;xZzkNSl9&WF4OOE3AGXJkqS!@zrweDKK~I@hpt}>_+NClzXdoJ@jxK z3=Sg?#ExAE;ll8X-bfDY3#eENK-8V$`t!soM(04y?tKG(6;Oxd1 zFRC0LK1YuzBk|X-t{r{ZX4XD!Je7t!5(OxDz((tnLj@^czOgVcBOyed@ve?NV6)v) ztwMv94FGn?=#UCGd(||kOD-=vRO zM%!&^p1HhSacc;M`{GW8+S^NCSPL0hK#7TdK@{^|85ep-_uUPA?ij-A#aBMJ)Hs6z z=9JfBB82Q#%v!g`P)*usgf394dwY9)(uf!OEmuIRugi1mcNmGR41uSO`R<{_S)faf=kyB zuP$;N^`Ucn+Xj*ks?{c@?`WyJ=(`}CrK_HnSF(9X7KGVw1Tt8=oZfeJ)gE78Lv(?O z>{J7_sduWixk3X5vv8MJ<6uI`?l57=4BJ8wthAWApJB%2v(XbnBU2BpK-pgqk+q7t z(~BEedTR2G+o!V{lppsl)S}Qp6rI84uH%*t^@V5nDlWIbqdKY7WxZ2C&ilyHlLl_s z#cT{{71y`iakEc_!cEqWJ#GLqth*>&iVFh{QU-VryvS@VKfT`ezPjXVC})p}$HFkE zpbZM0#7=|L^6~I`Doq@=Ll+0Gd>|7^I{tJoNZf!{;jFpBjN9`i-H7tO<|uQhTEZa6 z1&Mf$DSdA~z2!)cF12t5ZUd+LtyU((bfNEhg{wc^w9K#rMO$&Jg|g{=Af;WCQZoQ} z;$1u;o!}D97~Y7QVnKW8G!4xvgF}|gC0~_80!t%g6by^Y@_M^u4@rV!zkida>B(XE z*8!N^Rj~5@u}+D;?DtmmJWs!#jz%j*W@9@%_J4%nl*EiEAhpSjFb((K_FdHH#84T? z&>CNe_7}L$oAy}pfmhpoKGY4#nmdUE_U&8I+=*1T+;174G~I@$Wk5DaE-<@mPK(|l zoS%+e5u8;Hvt&HCI^Yqh<$U&8TN3WsORLlyJuZu&qsG28Xu9pQiylz;J;*8}r)|>m zy=z&9r?*$N@*e-j=abU}eK(-*&4-S1YxEi0Ct;1J+qZL}ux=ZGw-Du33(Nx<*sD1B4?c-0lFwpZn9OI{j9N?);i0p zA#0$TCUd0GqarvaAi-NR7uI4FpJHSdC7tboQ_QY}eGLzRxix`V%vQHZK=clHLn*7C z4ssQ8z7 ziA)kR#c(}>;8caGs@PL8_2h$4_}fcM3fYE8R-y^V4K8x#q0;98p^HIyL_h}tw>76C zts$|C5P5`pj$QP1K4_n+`*sOfgZZp#_u&ab; zLYMA;TXkF-h#CY$t(FGPz@JlhLDq_# z+Mk7Ouu(Q{$8>(}<`+Hx$PD!7z6ZXYkuXcEFp1a%R}DRc6iSRdn4Mfk0)<8rn2O{l zeu`LY^%PWbRS1#7H5c({BmBBD;yUDIIcHM0aDLq~V}MAugC+>Sa0tk))DNI|Z^JLw zYFu!*T``I*dwy6g15Vbf0?cvm%BWjEULs)U+eEuWmvt7rv~2ZSEfY z{QPvH#4mT$pX?%s@3^3u^LDgi=Up&g6-K!ep^#B~u)~t6iR1d7$i;xi72Z`-Dw^qeb?{D;46dcON6lxwaeo zv@vqQdL&>bfpIQ+3vZK2g2GXw+;o}Jl$h2H8J2w9>e+O?=)rn77jzkAj5y|DD0J4u zB0NC~b+L8?*Eu;c6(?}71KJdlbnoJ=8g{!?(C}9waGXL<>O)=Ez256WaPz5O{<5WQ*ECP&~EP`f40u}R5Of0S+cO*;;NHT z!34e>^L*niC8K<<@MlnWohE+-2Y{-=6cSDRIne9D7Fzm&F{@&l%or#poIG(v@+msIY8QE!bdK zSZdkqn#U$o{xdD6`H(^L!X#bm5<|YQmm7!EHCD6bhvh)~jygha^%ib5>P~gE>eP0p~ki{v4)+PFBzCPzFaodj2(&_o zc|5@hfO*eg4&``Kf$vhUuiLjKuUAUozcb4D+Pnvtvtj_c?RErf0N(LusX0Xs@D}Re zCiCNIlbc#h#2TUw)Y|hH^N{kQHQf|50@vYU4qu|Cb-f?KW-`B-uNF#jV|@>oEGYPj zM)ml4Vu{x|-;!{nEEkkkNk_m@ytZuN{EMQn0|F3{S^mNrvD9NY5mT3;MJE=;?iJ z9n9AvHO#4YSnd$%=FI}% zu|Jn?dJkKymt-#!{|Z6bx~o(=16?0JHvSF`2+w2kM9(R4l7=*Kag>~g1>nCD3N9ZmZUE!f$;$|$&=Ua>Bo|18HF zfabdY$=gVJ25u8IGqlezHB!k2+!?v3^_(We%HD@U)K1=DWSHvB)XF$^4@E>P z%GX1?|L&=?AXo!6D|hGo1yIN;)`$$26V>nglFpHaBM zCDDl0KOw1I73r?YBfBIke4f4nYw#x0LmsC+W#B#2OSPH4R@npePH!>D_nq#PFaji_ zmuq!RoJH6^%W7zce0x_^bRRbDJg|Vq+-}~Nn$kGl&tHiJ_P-)D72@L8O63 z94w&jj!4AODr#j#GaK#@CE;lUtc*#HQkiuT3fR7~lzQlRi7MDJ^EX2YhODnQBHM1f z{fM@$_}>a{k|}8@gqZE4kA~?RktN3NNXu~yH=;~T7{8UbEgId!($(8)mHVN*rKg1oju_VBSmxrZA`jfHTP;An(5OxpoPrIbh&*pK?&oR$hn5AMD zu5xRSYjfKTF|LFF)&GzLud! zEuC(IRDB;60^EzG={GASEmRgQCXBkzP2NXttlaQe;Kp;1XeEe25l~WieMRs?O#RTK zWV7us(B<6(B}kRL2wp0EX+_Yl3E+ucQCPy54<}Epw>)!YtTOPCcl%1ho4)c3>bVq zZWoz)5{|rPDBwx3e+|vCfvD0#gZycCbuVfAz2CG_-IM9hdhDD$%t2p~zPXvJoSz3E zYk+~xxd47}od4MX_z&N4;O`#szj<>r>i_0-Y;Ndc>Tc-ygU|7IbKrl)v;9wOj)DKX zp6!Ore{FO87hin`CwmwBM;GTGW|gcusB;1S=+{0PjD-5ldK|i74jvT@Y?Av&i|C(6 z{ChU{zZ(3}Px+rk2K>?I{!f_w?;1*87#Pxb^KoZO^GEmb9}Tau8PdN+ z5!6zoG(tTR6Kx{0t%G_r)kgV|*zW`Wmoqo+-!awx<1Ff&u;wz+!Rvo_+`8 zV(RST{FhvXzfE_5lVy_rula+4VLn=u^Jijz_u3{O^V0n0wXLEKQ^bC(3?s$Bs01S^ zJ1j4&umS#CAOVKo6%^*<2{lR_=eE1eV0E7Yi`IC|SVu-&R-SGW$ z|Fbpyb4dgLZP5+YEPtIB@aPTy7(bAZmx=Y=9zI&?_t=5S%wx#MO0KXU@I8P81G7T< zv66ot=5Ouce+SN}Z)9p`Y+-BYWXeCGnCc_3|H?G^c?_Cg)|Qc{i>bb|i<70Dx&AkTKQR0A8Uy}1ALQS# z4JgXsea~?DIUj`hkr`tF>33$|(?F~W_R&S2z^Lt8r|dqi=Xi@Bnf*Kl-7lGevbg`t zt^bd)|Fp9Hw{r)?{@WocKNrPg5C!O(8$kG-()Y?LX~z7PW|1Ot0fq(v2Bz^>Z zlpZJyBIE`9W%K%;o+8mJ2LH%R3kPON z7#H>P$Z_S+>W^#;?bw{Kpme>&N^OF{Qbtc&u^`Q_q=}T8iT;GlvTQ04A*~O5mZ8fjbFo21F=w+UF+g#7eV0u8 zF~?z8B3COl4~6Je3a5UMQs;OF1f)44UAgzQguMKVx3}A8d;1RuX_@r7R=qnOhMm$^ zJ}jp`0~$`SMwwPABiq92(`lsval9+W?oRn|jHFDG5_i_U=rBA|9%buIjSF`+3Z2)T zLooH9vG2s}9Nv3>jO=TGbu%upD^~t);%ja5LC`QsR%LH zgsM2a?Xe%8-f_YKqr!1GNv#z&Xgl1F1C*J#<-`Wo{+TfwtzPR zaJPxir})N~e5Ck=*p9ofO|G228eA@XsT~_Qo?`<%wM;pA{nYANsw6R)JL^iMNzf~t z0->NBUU=~A<4R@Dcs`OCvHNcKHBGE%+e)YbNhwaeO$&FXf)OII=)Pilb(PQgnqYJ3 zt|31{FtFIW2Xq8>s`RdfH!tV?RnSb02~feu%OE;Zg4Y%MUxc&caPnO;g@%JrYz70HMMHJ6n; zbR}}_G*rKuUfwgR2?Wm)g+r-rId4O$ql7MxU+qj^Ak7G(A3#Ukg*xtDPY?kmkN6%8 z54?m6=ZtMKUWgr_(_f4!W)JbB(0ZvFy|NWx?L`fZ@n>0eOp!<^v~0$o$*tF0l~_h) zHI(LazD#`*;|HxZ<6)z_R%Sa-taYnewL4GI?US87I7GN0_f*jOeBZw|I5?X)GJz&0 z2j%K36gh6}OIZdNlLcE+yb}}ze9(pp0^VAN0LEiCIe2; z;(ELfYY&p?lqxu;%)5u7dbBaJ}@ z8WL+5jSQA7^GhEmHxD0|S>buf+SUFoAHy+wBhNy6UY+~&Y>)_F4!$W6l^)g~%N_o6 zs0~5ZYG7C)i}`^I92|jYq%4jERsw684c75Se2-)Y6=;RJz`;@q_hi1Wm$F8{t?HVW z|MN@={c_Aq*-=X8yZ{)gVZ&)LW{)JedeP^ZP$Z)155{gTKD~(1OKr1)#^!x{hoP1F zwY6E7GXryLt@jfuZXQBFeHLcnoqH8p;EkQVJ@9M(o7X9O<6BTgIkJ!phD5v+A|_I4 zdi`GDy&a3YyF>4oJ+F6%v5+T|CFyI6bzO)y^&6mvThyaGF;EM|(C2PfyAF(2ywR{~ zqjP+W`*+#CT#G7R>7R_zD7#U@SQ~Wc=nCG7j=l77x-awB6EA57YVxIuOwm=76ngPv zW>JCg@%P9-XqH1SaGYJ1N2TMM_24Iw zhFrsf6+g2CCkj*~7cCxH@-Oe@zgTeB;JlKGcU2m_f%gc$0ZCBM2cm~}$R!{nAb%8` z;f_?USlKgc!h6$dkvkF6Z|6uI{DPjPE+^@Qk^PE}>Ls-_7MEX&q6GZHwP*VKPQ(!W zMWe3^@s07c`r8Pa3-|~qnEckF*cAzVIjNX4D#AQPx81;Dq2f~jAzj{aAs_dMew=L} z?Ae>p&5K9O48?l=N0>3NR%m1)Twr-^jEOUeb93XcJO(UGdgimv^GsiD zKBgfU6*}Vu6AZCe#g;2S5ES4PYX$YbcAM_NdCbg`y;AJ$FzDxu(w0gaDT$k#RmXFB zYT!kwq4mB=^)_wkzNiz zdLEP(SmUR4^$_pv=kjlIqNq3--EQ-?MDN@PBV`qUvWipeHjXt4YXRauBJKUXm+g5K z_O8C0ZA6(=TVs`m>w)+qBfJ=05e1QSZDru&`))w0vJh<5rx{MgelOTotv1_KTo9n3 zOHRqc71%d*+KygGZmEvJBKXxFGpvl)G)eb!L}wKFk(FpuzQSewWS+}P4qM}_LOvgR zx6UUC91d(b@&x-TbVW(p<@A==CUTjz3YwoXAk`+=Zx(F1gG`p*oBoTL#<<~IdQ^;y z5eV2jzN6)@ES-W#MgVG?1$V=tUd1zc$$fw#a>F) zbA3Z7w+dCOR#Fn3@SuI3@ZnmxPKI-EMWIw4jN*>{gVc_%zt}@vr#{hfTFr;yFuEtQ z$w2Oz%5fB=B(eiW02qUTYAI9umys^bNpNoqErjr@n)v8R@h|x#A#Tq&@C=?BRXvaP zad{S@M_>NzRFt!t=Z!?h1_s?tmix9}a>uHl|D-<{+r_o+M`kgf*#Vhk9-7ctb3HM3 zjwbeopt6-0GzeI&LCGxE85u7`tltZ!hQ0w3lDYa8g-X^PQTPXBcBY7a4K<&Ek4^|$ zu4o@y+d<1~rYwC8>5RK7hGtB2eM=|5^m_aKq2$}sEGz_4LZ(jQ-l+D@<4SZc^deGj z`8dLDZ?O*2Y1Q#ejxEA&?z&Hto}1}6lzsDienX740Y{+9qjJZsv`Vg$lP$J#o(v!0 z7cZ6=2HS~=bE&%DgpfH9+Cb9T-@v&m?0p#qMNw#l5yZ;u94kKg;*+6vj?u%JtS*o~kCz=0cQ8 z4--%0CSmx*sFRyhldF5a`x$q)*x0cXSnzsguRIl2`PFU6r!R)et+T?;GZ#qlyK0uc z=lz;00v>&D5t}iZ$HY;zZ;P~@wzd26=gm=8l)e0OPJA~jeBL4lf(VL&)t-b<^lilQozJ{6IYhud;O~x&2sgsO(!z3 zHkzF6%IHoi&O8Vhip%;u(n2pfy!i@6_fqi}+`H3L93Y{4!Y`7085JxG620jcJ35b&Xc70>@XQ=+ZcHXdtb446TS#XodUJQwXQ(Lxqy4vZ5LOgm`*v! zc(s71Wi@ld&$_3bkH=&}eX`y`d(vCWOvhtIlmwSkEN4tr&q>L}!w zbM_ELNv$X3)(Qnb$b){%ZAFK{p++Z&vN3bw9|AJlPcIFk+Qe$uI4iE2DpOUCv z{$YHb%P7B;MU17KNVyUN8{CQ+rlOx7ZlT9gI)msqueQvq$gBLZQL+JMuJ8qp{I@sJM@8<^ayAI#pb9^yXX6 zCWmF1ret?Z8riT$iganvZ`hDX)+;T*C;HVZuh3u4&Ly1PSEEDjoI2!lwbZ`bRJ1hUxUxs{SLE>5ti*K$H*t-d5P zL-WiTzC2Tn4Q}q-um;NXiv?0e7$|U4T1HEy9%1`Bht<6m4r*p$ba840Lj!AG^ef@s ztCkB4eNoN!+=ll9wHNKgsPpY;ZrE;3F+_-&~&J)0Ntmt0)4!5A3jfVlG5_k?M2K zV>r>zc;FckKM)@H=H|@CgU#Cb~hVjd1G2XW6z{=$n(3o*Catv%8lVTNg>0 zat;Mgchwgqqn8KBjfBv}pBanS-C&RjskQeg?AJKp7_IxWufnIq4rFxE2Q9G)w!H5M zRJ9~jo7>WxUe*IH*BSxFdFFC4rf0^pe5`Th5Q$loJbEC^{H-Hq-kp|Na%was!FOE^ zG{;gs@V~L84kIY`x~Lu6>P7u3EI2i|)%Z|=Q1$Bl=;FMWI%Hn!68d2zAK|h<;Cvqj z4#w6ZRP83rEvszyy=}jv5u<=kMwdYdZB5>N&FYOoFA>qvzPUw&Xy zCNvTvA1kI$K8TWit4wGLdwI!{x?$3%}@kgz2?e>gFjv7s~PQh9D1!DdklezL9gGRM<&lg)C? zN2{-=4)vPtIsfft`$?fo%BE1+N3iq2WRJregQnv*KZw_APlxj3VSq z&yA8?q(-S0y7HKtfJhhz{CU6j} zUlIvL0G{?`*`c*j5n>jsViq5+^;&`CX;F2<(F|%tT-@S%{Rw0ws!+|shb#-~kDflA zA}H3GeOnzH8k-K%JZuR;yh`_sDBXj`2jw)Y&)7IG&naf8XzCn+a!##09xDIa5r^_dMGI2J29H3 z@v>0A_&PhwMY@?J4?@4X7PhnuqUyvO)Qy?~`RX&1qC=p%V2EaarnNVwJ7S747Oh#R zj1G|_6cZz*cXYenSxRjkR6F#@cJ&&2Z))4%hP()~{9&q3Y}n)9c6}Nbj(+zpaPys{ zEl||fTxuWdRbA+-Iw=8!I?gofTmw8S`)8kr*UQh0)wDpnb;-`fX0=hA>?EKPj;9{E zncDH?hSa-P$zGyb=EhOOQq(cy4Lk=PMmCOl0EZz`#Y1l!YPoEteQ=ZGfv^%I>y1Pa z;7Y%nb=gK@O}dKKFpE&K2@7GOLU;rEDv**SI^!lno~9_<#G&fqeLDPsO#Fw!#K0Y_ z^p8|SWC&5vib=0O4BO81<#`Ki1R`vOPohLm!O zpH6r)q@&5`t1@lnOgSczQ&0egv7F8uK=-Wt6x~(^8Uo?YGCch}IjMZhW(%R0ei9fA z>5Y+wDayi28X{@#Vn%CLCXBrjOqvR%r0}dlh>k_ps31ZRAp`QEzS>spowV*gC86ERA@*glw^$N8=p5`32}Hz} zZq^n7F%aIcnY$(~GxA%$Ay?tnG}F($#~-0*Sykeui#2>-fv2Ctx8bWH(BYA|nw$Z_ zUVRrEB!8_3V62E6P;ju+>zH%X@CiO59t=FrOCXOrpz8a|d@a`{?daL)TnwD>^m<0# zO;P`RRcJp>gQo;7Qx$K#v{gDY%&y19a8Dn8K{5^NgCm2zw&q8j>H9Hek`x9^;n!aZ zr+a2`<5`|)c&$hn^BJqk_c$Bp%K7st1A;zP)i#T4mQz6@Shj7jHE}E(rYB5DGwe>* z=cu2DFUC;DO{`}=Yyj@r$HIU#5Qn>4OUVQy6>#&r=qN4NJzFlHphAw8>Yb-liEEiH zB$kH*ZR@*1w(bX;$tEpdAHN$Aw9cCXSF^;%{gg*~I-I@gRauY`3F;}&y{2XI3O*GI zHf<~Ec$3{{vB$R00+4~=8g-|?fyn5eZDoEifVK4WryR#;(OBiY@IU~fhyfl&8YuikFk;_A*`()84Eq7izbS|-meuuVH!7TpC z6O6Qhp~J+)KDtSnY-I8(X5~oVpf%S11msG27JE1P`0Ixu@0ca@@TXS@0*uO`TVa7I z&!;r=+1i6gQU&8dZ2j-t=T#zwqUxE17@b#}Z-2JlsFbjvve ze>FZfD#B*1N4|37IX&cw88*uok1)v|dU%!07tsmLCicMJ1T3b0iLFV{rC`dx@2^)X6+)VXvfO^NyQl2SsvoLR=J5)#<6SfExDS{- z)cQIGVh%&J=yFWf&|Ag4vv>KZi?V0A@AlFNHj_wF;FUx*l5qkhE2Zt>Kd)CPx65{W zofpuNWEYSGaR5KT4%XaLeY#LsZULo3PmF)ifU8+`N*Q!EBqZTkmVQ(7;e8X?RN=WX zu@U{{s|)#Qd!+mM|KPXCdwko=ZM zb!xBSMMZb!YBcP{wgSUCYACL)EDDt%J%H5*h8SYaN*eg8Gj*#gbt`an01i%eW6ki` z(XmPefKZv7Yll1wy`Wy;IW3+PV+p78J&jRre;c{WSdy5J*=cQa z0rXj1kjT1Z^NE>!=}kNa6?3P9DFuG@3E6jn7#p#EEW2BP+}%R1r}jcFB>EUSc1eNr zfw|`D`QXG!n4TZoYeSp0a(5Pe6Q(vW6Av=ljd>s7d%A-UF^AM5kESHhNFYkyu(#%X zP%gY-wd$FHzLSGV$?;YyNU>?>_%Fl6n<<)}Ird7ziQGM27~S@8hP<>!iIEkQkRM;F zOsInwIz)t~wH=OJShrBVTmq(w_HmTe9z~l0qW#BGiJP*_9NdqIRT|5zi(YJ6mg_ya zP$uI*8bX*T*LN7L0TPPA57@BPAKCmX&JD>?1oX)p5KNI<=~av&nXW;TuMDJe%o2*wYD4F=Zn5RVy$8?W%m+yAXhW=2tP$Af=Raf9cW}LP zf#87z7l(hXgEhq$*fRtc%ik{YLQB(wz?lT*Q4S`eh9%HYoOUYV<}b$JM;6}H$HVy! zQJJoo6*Z7P0mDjHvpS$>RbkAkC#z=R(c+($o-LNtvQ*PSWkG_DQLU`eDr6A7i~v}( zA$9V`fA3B|_Y>&T+JL048EN;Vt(Srk-5Zv}WThIjA`zD6?8rsZJu@gH1p6TSmkDs% zz%iZCydqKFQAnl95P}I}gtkHi-Lw7@l-BAdVO&zR?fN3cHTYry29BZ?%BkygJde-) z2()5ooUGjCj{aiuT~~{=M#w|AD_u|On>8D`6n2&_Qb0ajB8V)jmsbG zTfh?87%S{P7`AqbPBM)bc{x<~;!W800Go=`cE)=SuOJwex2+DankS*0wlrvCq5bxy zi;WhnH&~uNF@zOekUN}!1%+xhf#q$xPtD+6!ikhN_J*3v5%&>oypnG`Rc5dPseHi+ zqQgb-Z1jW(xTJO_4$yb);ULf^ZA6F0d^Gc_Ndr^VJDTm9=-ihM_k?Bm{3*FA2Do6k zFkOyoV+#a(&6}~>u@cSwNgv$!gVW|m3IhDLGD=KZGi-#l9&UGpFPeu7c{s?9Z9{Fd zbG&HYronSrBggPy9i-X{Gyb23Pk07Wi~U>+I?_6w>n1|z_#tSO)0fBe7@J|bia$~4 zq{qZqKeIO)XA#FyL{LJ5VC)LBeg|X^QY_@8SW?fHl$mXM6!xWH^oI|G@&a93*FD~o zVR_Vdj~g9{zpkYHn6CuxU#b}KK}^ih0vx=F5n{)138jl_RIwRLAt$_oo13ZE~OmBc|f& z7oWk(9sVX?bzUbM$bhb%TXPw+@pz+Z*Pg&5sf^FBnr+kg=W5#gxrYijr@7y1^2$97WGL_CaJci_ zD4C&P8rD*$m)g273n_Uz_t9kKF0MGDd%Szir6Rg00A8V4ruF1U8#+9B79h`84&gdAjIr)OX`g7?rsPn=?uT>?F`$_f0xTJL zBwMN0jgh=5NAS>&8y#o@@@~w@9&VxNZ8JerIw z=meW&&gDs^;vBiN5N$Eayz%Z_jipvN)hM?}@x}tdPO@-#p>)9kQycTu-IZ}sB;PTU z@2s9^Skg1Tr3ir`b>!nP%s$P2KtxnFlp>LQss(;Y!ah9FQx~B>=ond0c-OLb%9>;A zrfc(b93FRbg&Tn%wJGw81znm1E3LvO+l?Es7%@0!iYL?M#KI%8aNKorr!eOab{U2d z-JhQ%^|nZ-CWw*A0jYW4I;fLX^S#Sc6w34h53&*qZVjIgevJb|>S~irM zHzyJ$3bVC$IUG{4d?OXbW=W1wIl+;){3$FOB@{MC15yh)XPftIzkDfU#_*}xF`-SZ z+(7lpxsUUFd7xpw&2y;T=uY)`+zRx)C^TE~Mwa5i=dkAj4?u`ek)&)wp*+!C`;20^ zN-Syjl*ofmY2Iyk+1}7!L!`8Ah*7yMp^bFF#?j`C{1DXi8@bolN&H%Bis;uEH(sT> z%Za57bzqbxiMXf{7YS>Ie}Jhut4V(!LBhQ;XS$pfak|AS9{;|DI(Itl;qnVNJbCn%Y_^v#wNR|)2`AMQW!`^bdkuj@*4dkI84 zJMYLbZs2LbPbM=`3EhU(ie)N~2xUsppLoW}?^X%DuE`Y1+(jFNQ%$wLmZI#bsbM9m zQ#eBP$V)&*W^yY=VJpBtMK_3eVQYp&bPtUg)ir?OvdsiM>UCDxK_HY^=Ea5NhOT@W_bw|*8QpJ z-7KL{gdA#yFHRlWpti;&$kO|^qa>XBKT70A`=!3>SXC)H0)A6BIPIky4w8U@!PorQ zJo9ru3GFYNXF$ii0F0m8O#ao5=6|)J`G2R6Qkw5?`zYnE&%br(a$ITzoIG~u+C4UH z{kp~I*L{?~buw9)+Bp28mFXY$82yLzQU2VX_Skxw@Z$C_ou}Vh+>&nnp&yCqbYo*Mv zOsm>|Y^B73{YT9Hq>;_m@NbIjpFu|apV7$nbAfzAX3zh($V#994&RW?N1+|sJ|a7M z`aeMCZ2B)%*?%yy(Z7St1_ogIO`EYgaE@v8s5Bt_pCEIwasF2p=6?vXpJf7%@(d7* z0N^kBj6d`M4U`Cgs;7_0WUR;R${%q(^Zc>Oejer50u5VJTYD!@eH&B5zvB`R107YtsepM`}Nh^6SoUP__U7?fcQ< z&xI`iH#(O8`O)H0G5n)^oZ)$XA8z(pTJu2 z0672eVerN*eoe_%5PNWl4Iu!=cUD}| zzX=r^a)|+l-&kcYBRB3nvf6&~V_p3`6xXj=NeKbSzO(vc@PBxk|GyAL{r5L1)Sp;A zilRVOLV&-hqQrj31X?2nj6A*oB~f~gK0dBK@xNMqKM(S2Rn#|8hrX$Yp{;|B=?|)? zKc4a@y#6CpQ9r-79`*hpWMKfw_qzJNtHY23F20FUhSy*AowEQY z#QuK-7MSV z^+y=tA7OiZSAgm3%l+WB@Lf3UY-(fnjoLTyovE#wCOp^{MV{`T0{fj8u=0Of;Q7x2 z`<)l&qn77qVBZ#I7R!Gv%Lg zsKAE-@`Ld=#liU70&@OV4c|2U-wg)-=igER6H_$eUxNk}i|h>y3@vO^ z)Bkt9Bo^4z{z|&~7b^)jQzvIjdpqC{dudMDhi^Jou*euD83x+nk0X&W8dYkFoTF^) zqqU=Ks$0KXA(T#e;^RHCdfb+NI7j{bpvS8FX@N27|MAQp9|`>9ng4T={~Ac(7n}9< zZJTW~qiqv|&TkWX0`qnKBs=tRa-aTcfwW6MFQUZ?91@4 zevA1}Zx#S>42Z`&6&P5;dIky*3hZ&||Nis;b#ir~aTQVc4z;o^O;VvrHfxhKZM12M zsU&E&ElGDJ6-$t|^e0w@tu9q!V^eCSR3Fv_v0^Er0WSruh}%k`Vjp~J6<_=>w8f|h zLW{qB%7ge)DnaL)xpVJ1bMNe4ObLX-eD|9(=bV{2=iaX{SHvHfjg-^Kgx){5(3Lk<`Pz%_{IJazya6 zh(OI3FU$W8O>Ed8{^A%0!1J0J#6o%82qD<4BAiMzExPn~5};bO7z45tIqVr~P~7nU zdG*+da@O-kCZQh^nYvc(k!knf(TcQnd1_LA@crDrLo!B)!Em>yrUxA5*v;U6%-NG2 zD5U^aIS^&DrtM8H$RJnMLu$ra^@0p|WxewBvj?Bh@(_B$s0>nuz@ZE~7{j@)1csGH zm~glqx88qwR-T5wTk3gtFw9Hc0k=?UfEY0Hxa37%E}_bS^Au{;xm9s~X<#wyEswQL zOvpn==0VTPGKQCXy_vsO$h%`YldWQdUmT26(qC^+PfhJBAC^g63j!sBqYo44Pqium zD(->hA-`)_v>R&p%9pMtM$^^g{`SRd4r)}eO4G7|A>u^johnumIeR=Ysf)# zpi~cZ=K4F?oly>%D}H`DprQ}3CQagqLTTVsNJWg=7>)VUp4LOsCdLs_52krK%CXdk z=8I3Cu}#dWM448{AgEA_^n_HN^kfX4b-*R&+lZ;ghCmRN4&s6M`Ul?3d#A!cgd7mH z5e8FYJP>EkB~LS^HKc85gJaf#Oi!EA>FElmjags((1X5@2}FBdw$0YC!As6NsU->f z*oz!(RlEEn>qS#b|MmJxg4OBXSvo-^zK#cZkY46<~TqzAw9 zZ>|^JX-&174Sx4q7o#xGN0;SfLqW<#ioE1Pl@}h;$1izMhcWb^+d`cTqp6eK#pqqT z;-VLmh0)|}c*v$GL6vs7kwa0Xli`1T)xWSoL~bSpQYbl$rsVqA;B&Jsa%w}eBv=CD zSx>-Dxu+u+sh6x^k2v|g-?4JM({+GJ^$}zc<kTYrHG6PUV9g|xA%U=ZvP@- zot;gMM7^@HELbB3I`i+%R+c^Lc;qH$twVdldlrw!gmRH zGI_H};2i`*uaU!Xde1DYOOL;NknC)rC%g8!ux-zeFS}cXYWSz+$NlT=Y+Zh=(eI{* z4tdtJ$T_`Z9sYrlXmJ}^kF$Gtd0lp_r|xElX0+d*5sRB?xaK?kEB~9=-#vASvCefe zG2GM8^39r_FW>C1t@Q656;fMAk8ZT5^3DE+{1BX9X-!xrHi?rTfgSc6|LCZ-+5fOk zsD`Xb7>%Np?++&jUCdM?HXlqk36!=l`LXMjA)%U|X(PDW@jnyktg`EPUyo>uqgl~r^XFD;Q_Mr7p%ClPe-T&sn~ z7HxUm!$75E8CEOjqhn@xp{Yer8GxXtj68#Qe#FTOoiejzHYi$2w8_K|Wrs%nj|!sG zL>$^ilo19~M*L9r)+QGt+yPlvN=KUwk7!fvoc5rf!*}VI3Vurw{Zc)eEqzeXr{wr9 zeRZrZxmb1H{S5z+qG)ED_h*}h>b0b0#=~UGRu?n7$KvLsn-l)0MUi352Db^-)JV&W zFKA(hlbKdnZgd4eF<31LYL}=Jw=x&*-)ZBA4TR*9DyrwSuFk6sx9t+T+CU5F+GZ6L z?WrdNi8>1pgDHu`_YPi_*Pu3LINM;dgk4@ Date: Tue, 29 Oct 2019 16:23:45 +0100 Subject: [PATCH 375/746] [FLINK-14561] Don't write FLINK_PLUGINS_DIR env variable to Configuration This commit reads the FLINK_PLUGINS_DIR env variable directly instead of first writing it first to the Flink Configuration and then reading it from there. This has the advantage that the plugins env variable which is intended for the local process only is not being leaked to other processes. The latter can happen if we start a new Flink process based on the read Flink Configuration (e.g. Yarn, Mesos TaskExecutor). This closes #10037. --- .../flink/configuration/ConfigConstants.java | 3 + .../configuration/GlobalConfiguration.java | 25 ------ .../flink/core/plugin/PluginConfig.java | 19 ++--- .../flink/core/plugin/PluginConfigTest.java | 80 +++++++++++++++++++ .../overlays/FlinkDistributionOverlay.java | 17 ++-- .../FlinkDistributionOverlayTest.java | 25 +----- .../flink/yarn/YarnClusterDescriptor.java | 15 +--- 7 files changed, 108 insertions(+), 76 deletions(-) create mode 100644 flink-core/src/test/java/org/apache/flink/core/plugin/PluginConfigTest.java diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java index 510c11aa585e..d84b5626000c 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 @@ -2012,6 +2012,9 @@ public final class ConfigConstants { /** The environment variable name which contains the location of the plugins folder. */ public static final String ENV_FLINK_PLUGINS_DIR = "FLINK_PLUGINS_DIR"; + /** The default Flink plugins directory if none has been specified via {@link #ENV_FLINK_PLUGINS_DIR}. */ + public static final String DEFAULT_FLINK_PLUGINS_DIRS = "plugins"; + /** The environment variable name which contains the location of the bin directory. */ public static final String ENV_FLINK_BIN_DIR = "FLINK_BIN_DIR"; diff --git a/flink-core/src/main/java/org/apache/flink/configuration/GlobalConfiguration.java b/flink-core/src/main/java/org/apache/flink/configuration/GlobalConfiguration.java index be0ffa00f84e..564fe199d09b 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/GlobalConfiguration.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/GlobalConfiguration.java @@ -130,34 +130,9 @@ public static Configuration loadConfiguration(final String configDir, @Nullable configuration.addAll(dynamicProperties); } - return enrichWithEnvironmentVariables(configuration); - } - - private static Configuration enrichWithEnvironmentVariables(Configuration configuration) { - enrichWithEnvironmentVariable(ConfigConstants.ENV_FLINK_PLUGINS_DIR, configuration); return configuration; } - private static void enrichWithEnvironmentVariable(String environmentVariable, Configuration configuration) { - String valueFromEnv = System.getenv(environmentVariable); - - if (valueFromEnv == null) { - return; - } - - String valueFromConfig = configuration.getString(environmentVariable, valueFromEnv); - - if (!valueFromEnv.equals(valueFromConfig)) { - throw new IllegalConfigurationException( - "The given configuration file already contains a value (" + valueFromEnv + - ") for the key (" + environmentVariable + - ") that would have been overwritten with (" + valueFromConfig + - ") by an environment with the same name."); - } - - configuration.setString(environmentVariable, valueFromEnv); - } - /** * Loads a YAML-file of key-value pairs. * diff --git a/flink-core/src/main/java/org/apache/flink/core/plugin/PluginConfig.java b/flink-core/src/main/java/org/apache/flink/core/plugin/PluginConfig.java index 49ffbaf2c9b9..b430bdf89575 100644 --- a/flink-core/src/main/java/org/apache/flink/core/plugin/PluginConfig.java +++ b/flink-core/src/main/java/org/apache/flink/core/plugin/PluginConfig.java @@ -55,23 +55,20 @@ public String[] getAlwaysParentFirstPatterns() { public static PluginConfig fromConfiguration(Configuration configuration) { return new PluginConfig( - getPluginsDirPath(configuration), + getPluginsDir().map(File::toPath), CoreOptions.getParentFirstLoaderPatterns(configuration)); } - private static Optional getPluginsDirPath(Configuration configuration) { - String pluginsDir = configuration.getString(ConfigConstants.ENV_FLINK_PLUGINS_DIR, null); - if (pluginsDir == null) { - LOG.info("Environment variable [{}] is not set", ConfigConstants.ENV_FLINK_PLUGINS_DIR); - return Optional.empty(); - } + public static Optional getPluginsDir() { + String pluginsDir = System.getenv().getOrDefault( + ConfigConstants.ENV_FLINK_PLUGINS_DIR, + ConfigConstants.DEFAULT_FLINK_PLUGINS_DIRS); + File pluginsDirFile = new File(pluginsDir); if (!pluginsDirFile.isDirectory()) { - LOG.warn("Environment variable [{}] is set to [{}] but the directory doesn't exist", - ConfigConstants.ENV_FLINK_PLUGINS_DIR, - pluginsDir); + LOG.warn("The plugins directory [{}] does not exist.", pluginsDirFile); return Optional.empty(); } - return Optional.of(pluginsDirFile.toPath()); + return Optional.of(pluginsDirFile); } } diff --git a/flink-core/src/test/java/org/apache/flink/core/plugin/PluginConfigTest.java b/flink-core/src/test/java/org/apache/flink/core/plugin/PluginConfigTest.java new file mode 100644 index 000000000000..fbb4dc433d95 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/core/plugin/PluginConfigTest.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.core.plugin; + +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.core.testutils.CommonTestUtils; +import org.apache.flink.util.TestLogger; + +import org.apache.flink.shaded.guava18.com.google.common.collect.ImmutableMap; + +import org.junit.After; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.util.Map; + +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; + +/** + * Tests for the {@link PluginConfig} utility class. + */ +public class PluginConfigTest extends TestLogger { + + @ClassRule + public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private static Map oldEnvVariables; + + @BeforeClass + public static void setup() { + oldEnvVariables = System.getenv(); + } + + @After + public void teardown() { + if (oldEnvVariables != null) { + CommonTestUtils.setEnv(oldEnvVariables, true); + } + } + + @Test + public void getPluginsDir_existingDirectory_returnsDirectoryFile() throws IOException { + final File pluginsDirectory = temporaryFolder.newFolder(); + final Map envVariables = ImmutableMap.of(ConfigConstants.ENV_FLINK_PLUGINS_DIR, pluginsDirectory.getAbsolutePath()); + CommonTestUtils.setEnv(envVariables); + + assertThat(PluginConfig.getPluginsDir().get(), is(pluginsDirectory)); + } + + @Test + public void getPluginsDir_nonExistingDirectory_returnsEmpty() { + final Map envVariables = ImmutableMap.of(ConfigConstants.ENV_FLINK_PLUGINS_DIR, new File(temporaryFolder.getRoot().getAbsoluteFile(), "should_not_exist").getAbsolutePath()); + CommonTestUtils.setEnv(envVariables); + + assertFalse(PluginConfig.getPluginsDir().isPresent()); + } + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/overlays/FlinkDistributionOverlay.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/overlays/FlinkDistributionOverlay.java index 860ed9f20a2b..cba4bee5b89e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/overlays/FlinkDistributionOverlay.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/overlays/FlinkDistributionOverlay.java @@ -20,11 +20,14 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import org.apache.flink.core.plugin.PluginConfig; import org.apache.flink.runtime.clusterframework.ContainerSpecification; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.io.File; import java.io.IOException; @@ -32,7 +35,6 @@ import static org.apache.flink.configuration.ConfigConstants.ENV_FLINK_CONF_DIR; import static org.apache.flink.configuration.ConfigConstants.ENV_FLINK_HOME_DIR; 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.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkState; @@ -57,13 +59,14 @@ public class FlinkDistributionOverlay extends AbstractContainerOverlay { final File flinkBinPath; final File flinkConfPath; final File flinkLibPath; + @Nullable final File flinkPluginsPath; - public FlinkDistributionOverlay(File flinkBinPath, File flinkConfPath, File flinkLibPath, File flinkPluginsPath) { + FlinkDistributionOverlay(File flinkBinPath, File flinkConfPath, File flinkLibPath, @Nullable File flinkPluginsPath) { this.flinkBinPath = checkNotNull(flinkBinPath); this.flinkConfPath = checkNotNull(flinkConfPath); this.flinkLibPath = checkNotNull(flinkLibPath); - this.flinkPluginsPath = checkNotNull(flinkPluginsPath); + this.flinkPluginsPath = flinkPluginsPath; } @Override @@ -75,12 +78,9 @@ public void configure(ContainerSpecification container) throws IOException { addPathRecursively(flinkBinPath, TARGET_ROOT, container); addPathRecursively(flinkConfPath, TARGET_ROOT, container); addPathRecursively(flinkLibPath, TARGET_ROOT, container); - if (flinkPluginsPath.isDirectory()) { + if (flinkPluginsPath != null) { addPathRecursively(flinkPluginsPath, TARGET_ROOT, container); } - else { - LOG.warn("The plugins directory '" + flinkPluginsPath + "' doesn't exist."); - } } public static Builder newBuilder() { @@ -94,6 +94,7 @@ public static class Builder { File flinkBinPath; File flinkConfPath; File flinkLibPath; + @Nullable File flinkPluginsPath; /** @@ -107,7 +108,7 @@ public Builder fromEnvironment(Configuration globalConfiguration) { flinkBinPath = getObligatoryFileFromEnvironment(ENV_FLINK_BIN_DIR); flinkConfPath = getObligatoryFileFromEnvironment(ENV_FLINK_CONF_DIR); flinkLibPath = getObligatoryFileFromEnvironment(ENV_FLINK_LIB_DIR); - flinkPluginsPath = getObligatoryFileFromEnvironment(ENV_FLINK_PLUGINS_DIR); + flinkPluginsPath = PluginConfig.getPluginsDir().orElse(null); return this; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/overlays/FlinkDistributionOverlayTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/overlays/FlinkDistributionOverlayTest.java index 9f9067c85a77..46806afdbba1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/overlays/FlinkDistributionOverlayTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/overlays/FlinkDistributionOverlayTest.java @@ -29,7 +29,6 @@ import java.io.File; import java.io.IOException; -import java.nio.file.Paths; import java.util.HashMap; import java.util.Map; @@ -39,6 +38,7 @@ import static org.apache.flink.configuration.ConfigConstants.ENV_FLINK_PLUGINS_DIR; import static org.apache.flink.runtime.clusterframework.overlays.FlinkDistributionOverlay.TARGET_ROOT; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.fail; public class FlinkDistributionOverlayTest extends ContainerOverlayTestBase { @@ -68,25 +68,6 @@ public void testConfigure() throws Exception { testConfigure(binFolder, libFolder, pluginsFolder, confFolder, files); } - @Test - public void testConfigureWithMissingPlugins() throws Exception { - File binFolder = tempFolder.newFolder("bin"); - File libFolder = tempFolder.newFolder("lib"); - File pluginsFolder = Paths.get(tempFolder.getRoot().getAbsolutePath(), "s0m3_p4th_th4t_sh0uld_n0t_3x1sts").toFile(); - File confFolder = tempFolder.newFolder("conf"); - - Path[] files = createPaths( - tempFolder.getRoot(), - "bin/config.sh", - "bin/taskmanager.sh", - "lib/foo.jar", - "lib/A/foo.jar", - "lib/B/foo.jar", - "lib/B/bar.jar"); - - testConfigure(binFolder, libFolder, pluginsFolder, confFolder, files); - } - private void testConfigure( File binFolder, File libFolder, @@ -127,7 +108,9 @@ public void testBuilderFromEnvironment() throws Exception { assertEquals(binFolder.getAbsolutePath(), builder.flinkBinPath.getAbsolutePath()); assertEquals(libFolder.getAbsolutePath(), builder.flinkLibPath.getAbsolutePath()); - assertEquals(pluginsFolder.getAbsolutePath(), builder.flinkPluginsPath.getAbsolutePath()); + final File flinkPluginsPath = builder.flinkPluginsPath; + assertNotNull(flinkPluginsPath); + assertEquals(pluginsFolder.getAbsolutePath(), flinkPluginsPath.getAbsolutePath()); assertEquals(confFolder.getAbsolutePath(), builder.flinkConfPath.getAbsolutePath()); } 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 18dcb3c891ca..0bcc52b20b91 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 @@ -36,6 +36,7 @@ import org.apache.flink.configuration.RestOptions; import org.apache.flink.configuration.SecurityOptions; import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.core.plugin.PluginConfig; import org.apache.flink.core.plugin.PluginUtils; import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; @@ -101,11 +102,11 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Optional; 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; @@ -1581,16 +1582,8 @@ private void addLibFoldersToShipFiles(Collection effectiveShipFiles) { } 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."); - } - } + final Optional pluginsDir = PluginConfig.getPluginsDir(); + pluginsDir.ifPresent(effectiveShipFiles::add); } ContainerLaunchContext setupApplicationMasterContainer( From e873ceb2d8001986142f395f912dc11e55755e2d Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 30 Oct 2019 11:02:33 +0100 Subject: [PATCH 376/746] [hotfix] Fix checkstyle violations in FlinkDistributionOverlayTest --- .../overlays/FlinkDistributionOverlayTest.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/overlays/FlinkDistributionOverlayTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/overlays/FlinkDistributionOverlayTest.java index 46806afdbba1..a0eb80000129 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/overlays/FlinkDistributionOverlayTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/overlays/FlinkDistributionOverlayTest.java @@ -82,7 +82,7 @@ private void testConfigure( pluginsFolder); overlay.configure(containerSpecification); - for(Path file : files) { + for (Path file : files) { checkArtifact(containerSpecification, new Path(TARGET_ROOT, file.toString())); } } @@ -102,7 +102,7 @@ public void testBuilderFromEnvironment() throws Exception { map.put(ENV_FLINK_LIB_DIR, libFolder.getAbsolutePath()); map.put(ENV_FLINK_PLUGINS_DIR, pluginsFolder.getAbsolutePath()); map.put(ENV_FLINK_CONF_DIR, confFolder.getAbsolutePath()); - CommonTestUtils.setEnv(map); + CommonTestUtils.setEnv(map); FlinkDistributionOverlay.Builder builder = FlinkDistributionOverlay.newBuilder().fromEnvironment(conf); @@ -133,8 +133,7 @@ public void testBuilderFromEnvironmentBad(String obligatoryEnvironmentVariable) try { FlinkDistributionOverlay.Builder builder = FlinkDistributionOverlay.newBuilder().fromEnvironment(conf); fail(); - } - catch(IllegalStateException e) { + } catch (IllegalStateException e) { // expected } } From 7422dd0f9f22db24e1a3fc8abc02eca09a354213 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 31 Oct 2019 11:06:15 +0100 Subject: [PATCH 377/746] [hotfix] Fix checkstyle violations in FlinkDistributionOverlay --- .../overlays/FlinkDistributionOverlay.java | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/overlays/FlinkDistributionOverlay.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/overlays/FlinkDistributionOverlay.java index cba4bee5b89e..c4b1a96ccda8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/overlays/FlinkDistributionOverlay.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/overlays/FlinkDistributionOverlay.java @@ -23,9 +23,6 @@ import org.apache.flink.core.plugin.PluginConfig; import org.apache.flink.runtime.clusterframework.ContainerSpecification; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import javax.annotation.Nullable; import java.io.File; @@ -41,26 +38,24 @@ /** * Overlays Flink into a container, based on supplied bin/conf/lib directories. * - * The overlayed Flink is indistinguishable from (and interchangeable with) + *

    The overlayed Flink is indistinguishable from (and interchangeable with) * a normal installation of Flink. For a docker image-based container, it should be * possible to bypass this overlay and rely on the normal installation method. * - * The following files are copied to the container: + *

    The following files are copied to the container: * - flink/bin/ * - flink/conf/ * - flink/lib/ */ public class FlinkDistributionOverlay extends AbstractContainerOverlay { - private static final Logger LOG = LoggerFactory.getLogger(FlinkDistributionOverlay.class); - static final Path TARGET_ROOT = new Path("flink"); - final File flinkBinPath; - final File flinkConfPath; - final File flinkLibPath; + private final File flinkBinPath; + private final File flinkConfPath; + private final File flinkLibPath; @Nullable - final File flinkPluginsPath; + private final File flinkPluginsPath; FlinkDistributionOverlay(File flinkBinPath, File flinkConfPath, File flinkLibPath, @Nullable File flinkPluginsPath) { this.flinkBinPath = checkNotNull(flinkBinPath); @@ -100,7 +95,7 @@ public static class Builder { /** * Configures the overlay using the current environment. * - * Locates Flink using FLINK_???_DIR environment variables as provided to all Flink processes by config.sh. + *

    Locates Flink using FLINK_???_DIR environment variables as provided to all Flink processes by config.sh. * * @param globalConfiguration the current configuration. */ From 156cec0403b6650926d82b37e6088a0676bf0d7e Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 26 Oct 2019 13:36:49 +0200 Subject: [PATCH 378/746] [hotfix] Introduce ContainerSpecification#createDynamicProperty --- .../ContainerSpecification.java | 30 ++++++++++++++----- 1 file changed, 22 insertions(+), 8 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContainerSpecification.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContainerSpecification.java index 508a28cad9fa..8f4492a8e9c1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContainerSpecification.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContainerSpecification.java @@ -192,15 +192,29 @@ public static String formatSystemProperties(Configuration jvmArgs) { if(sb.length() > 0) { sb.append(" "); } - boolean quoted = entry.getValue().contains(" "); - if(quoted) { - sb.append("\""); - } - sb.append("-D").append(entry.getKey()).append('=').append(entry.getValue()); - if(quoted) { - sb.append("\""); - } + final String dynamicProperty = createDynamicProperty(entry.getKey(), entry.getValue()); + sb.append(dynamicProperty); } return sb.toString(); } + + /** + * Create a dynamic property from the given key and value of the format {@code -Dkey=value}. + * + * @param key of the dynamic property + * @param value of the dynamic property + * @return dynamic property + */ + public static String createDynamicProperty(String key, String value) { + final String keyPart = "-D" + key + '='; + final String valuePart; + + if (value.contains(" ")) { + valuePart = "\"" + value + "\""; + } else { + valuePart = value; + } + + return keyPart + valuePart; + } } From 9c7e7bdb1ecb923ecee1ab318b638e5ef9b7f2ba Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 26 Oct 2019 13:38:23 +0200 Subject: [PATCH 379/746] [hotfix] Fix checkstyle violations in ContainerSpecification --- .../ContainerSpecification.java | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContainerSpecification.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContainerSpecification.java index 8f4492a8e9c1..0f6b028c92a3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContainerSpecification.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContainerSpecification.java @@ -32,14 +32,14 @@ * Encapsulates a container specification, including artifacts, environment variables, * system properties, and Flink configuration settings. * - * The specification is mutable. + *

    The specification is mutable. * - * Note that the Flink configuration settings are considered dynamic overrides of whatever + *

    Note that the Flink configuration settings are considered dynamic overrides of whatever * static configuration file is present in the container. For example, a container might be * based on a Docker image with a normal Flink installation with customized settings, which these * settings would (partially) override. * - * Artifacts are copied into a sandbox directory within the container, which any Flink process + *

    Artifacts are copied into a sandbox directory within the container, which any Flink process * launched in the container is assumed to use as a working directory. This assumption allows * for relative paths to be used in certain environment variables. */ @@ -51,13 +51,13 @@ public class ContainerSpecification implements java.io.Serializable { private final List artifacts; - private final Map environmentVariables; + private final Map environmentVariables; private final Configuration dynamicConfiguration; public ContainerSpecification() { this.artifacts = new LinkedList<>(); - this.environmentVariables = new HashMap(); + this.environmentVariables = new HashMap(); this.systemProperties = new Configuration(); this.dynamicConfiguration = new Configuration(); } @@ -142,7 +142,9 @@ public String toString() { '}'; } - public static Builder newBuilder() { return new Builder(); } + public static Builder newBuilder() { + return new Builder(); + } public static class Builder { @@ -188,8 +190,8 @@ public Artifact build() { */ public static String formatSystemProperties(Configuration jvmArgs) { StringBuilder sb = new StringBuilder(); - for(Map.Entry entry : jvmArgs.toMap().entrySet()) { - if(sb.length() > 0) { + for (Map.Entry entry : jvmArgs.toMap().entrySet()) { + if (sb.length() > 0) { sb.append(" "); } final String dynamicProperty = createDynamicProperty(entry.getKey(), entry.getValue()); From 87d0a3c26054ebe7aa414612a26750c166903f8e Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 26 Oct 2019 13:42:05 +0200 Subject: [PATCH 380/746] [hotfix] Rename and move MesosEntrypointUtils to o.a.f.mesos.util.MesosUtils --- .../mesos/entrypoint/MesosJobClusterEntrypoint.java | 9 +++++---- .../mesos/entrypoint/MesosSessionClusterEntrypoint.java | 9 +++++---- .../flink/mesos/entrypoint/MesosTaskExecutorRunner.java | 3 ++- .../MesosEntrypointUtils.java => util/MesosUtils.java} | 9 ++++----- 4 files changed, 16 insertions(+), 14 deletions(-) rename flink-mesos/src/main/java/org/apache/flink/mesos/{entrypoint/MesosEntrypointUtils.java => util/MesosUtils.java} (97%) diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java index 6c1f486ba9b4..e01c094602f0 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java @@ -25,6 +25,7 @@ import org.apache.flink.mesos.runtime.clusterframework.services.MesosServices; import org.apache.flink.mesos.runtime.clusterframework.services.MesosServicesUtils; import org.apache.flink.mesos.util.MesosConfiguration; +import org.apache.flink.mesos.util.MesosUtils; import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.runtime.clusterframework.ContainerSpecification; import org.apache.flink.runtime.concurrent.FutureUtils; @@ -84,14 +85,14 @@ protected void initializeServices(Configuration config) throws Exception { final String hostname = config.getString(JobManagerOptions.ADDRESS); // Mesos configuration - schedulerConfiguration = MesosEntrypointUtils.createMesosSchedulerConfiguration(config, hostname); + schedulerConfiguration = MesosUtils.createMesosSchedulerConfiguration(config, hostname); // services mesosServices = MesosServicesUtils.createMesosServices(config, hostname); // TM configuration - taskManagerParameters = MesosEntrypointUtils.createTmParameters(config, LOG); - taskManagerContainerSpec = MesosEntrypointUtils.createContainerSpec(config, dynamicProperties); + taskManagerParameters = MesosUtils.createTmParameters(config, LOG); + taskManagerContainerSpec = MesosUtils.createContainerSpec(config, dynamicProperties); } @Override @@ -137,7 +138,7 @@ public static void main(String[] args) { } Configuration dynamicProperties = BootstrapTools.parseDynamicProperties(cmd); - Configuration configuration = MesosEntrypointUtils.loadConfiguration(dynamicProperties, LOG); + Configuration configuration = MesosUtils.loadConfiguration(dynamicProperties, LOG); MesosJobClusterEntrypoint clusterEntrypoint = new MesosJobClusterEntrypoint(configuration, dynamicProperties); diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java index 4033d7d83d5e..0f308e29e376 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java @@ -25,6 +25,7 @@ import org.apache.flink.mesos.runtime.clusterframework.services.MesosServices; import org.apache.flink.mesos.runtime.clusterframework.services.MesosServicesUtils; import org.apache.flink.mesos.util.MesosConfiguration; +import org.apache.flink.mesos.util.MesosUtils; import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.runtime.clusterframework.ContainerSpecification; import org.apache.flink.runtime.concurrent.FutureUtils; @@ -83,14 +84,14 @@ protected void initializeServices(Configuration config) throws Exception { final String hostname = config.getString(JobManagerOptions.ADDRESS); // Mesos configuration - mesosConfig = MesosEntrypointUtils.createMesosSchedulerConfiguration(config, hostname); + mesosConfig = MesosUtils.createMesosSchedulerConfiguration(config, hostname); // services mesosServices = MesosServicesUtils.createMesosServices(config, hostname); // TM configuration - taskManagerParameters = MesosEntrypointUtils.createTmParameters(config, LOG); - taskManagerContainerSpec = MesosEntrypointUtils.createContainerSpec(config, dynamicProperties); + taskManagerParameters = MesosUtils.createTmParameters(config, LOG); + taskManagerContainerSpec = MesosUtils.createContainerSpec(config, dynamicProperties); } @Override @@ -135,7 +136,7 @@ public static void main(String[] args) { } Configuration dynamicProperties = BootstrapTools.parseDynamicProperties(cmd); - Configuration configuration = MesosEntrypointUtils.loadConfiguration(dynamicProperties, LOG); + Configuration configuration = MesosUtils.loadConfiguration(dynamicProperties, LOG); MesosSessionClusterEntrypoint clusterEntrypoint = new MesosSessionClusterEntrypoint(configuration, dynamicProperties); diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosTaskExecutorRunner.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosTaskExecutorRunner.java index 4c0ec9771d47..d87ab7bb8518 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosTaskExecutorRunner.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosTaskExecutorRunner.java @@ -23,6 +23,7 @@ import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.plugin.PluginUtils; import org.apache.flink.mesos.runtime.clusterframework.MesosConfigKeys; +import org.apache.flink.mesos.util.MesosUtils; import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.security.SecurityConfiguration; @@ -75,7 +76,7 @@ public static void main(String[] args) throws Exception { Configuration dynamicProperties = BootstrapTools.parseDynamicProperties(cmd); LOG.debug("Mesos dynamic properties: {}", dynamicProperties); - configuration = MesosEntrypointUtils.loadConfiguration(dynamicProperties, LOG); + configuration = MesosUtils.loadConfiguration(dynamicProperties, LOG); } catch (Throwable t) { LOG.error("Failed to load the TaskManager configuration and dynamic properties.", t); diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosEntrypointUtils.java b/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosUtils.java similarity index 97% rename from flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosEntrypointUtils.java rename to flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosUtils.java index 19979e4a2e5d..38175eecc385 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosEntrypointUtils.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosUtils.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,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.mesos.entrypoint; +package org.apache.flink.mesos.util; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; @@ -24,7 +24,6 @@ import org.apache.flink.mesos.configuration.MesosOptions; import org.apache.flink.mesos.runtime.clusterframework.MesosConfigKeys; import org.apache.flink.mesos.runtime.clusterframework.MesosTaskManagerParameters; -import org.apache.flink.mesos.util.MesosConfiguration; import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.runtime.clusterframework.ContainerSpecification; import org.apache.flink.runtime.clusterframework.overlays.CompositeContainerOverlay; @@ -46,9 +45,9 @@ import scala.concurrent.duration.FiniteDuration; /** - * Utils for Mesos entry points. + * Utils for Mesos. */ -public class MesosEntrypointUtils { +public class MesosUtils { /** * Loads and validates the Mesos scheduler configuration. From f3af53332f03ec0393b23a8d972ce803354798d4 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 26 Oct 2019 13:43:18 +0200 Subject: [PATCH 381/746] [hotfix] Rename ContainerSpecification#dynamicConfiguration to flinkConfiguration --- .../clusterframework/LaunchableMesosWorker.java | 6 +++--- .../java/org/apache/flink/mesos/util/MesosUtils.java | 2 +- .../clusterframework/ContainerSpecification.java | 12 ++++++------ .../clusterframework/overlays/HadoopConfOverlay.java | 2 +- .../clusterframework/overlays/KeytabOverlay.java | 2 +- .../clusterframework/overlays/SSLStoreOverlay.java | 4 ++-- .../overlays/HadoopConfOverlayTest.java | 2 +- .../clusterframework/overlays/KeytabOverlayTest.java | 2 +- .../overlays/SSLStoreOverlayTest.java | 4 ++-- 9 files changed, 18 insertions(+), 18 deletions(-) diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java index 7fca751f1505..9a4edc2a972d 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java @@ -150,7 +150,7 @@ public double getDisk() { @Override public int getPorts() { - return extractPortKeys(containerSpec.getDynamicConfiguration()).size(); + return extractPortKeys(containerSpec.getFlinkConfiguration()).size(); } @Override @@ -209,7 +209,7 @@ public Protos.TaskInfo launch(Protos.SlaveID slaveId, MesosResourceAllocation al final Configuration dynamicProperties = new Configuration(); // incorporate the dynamic properties set by the template - dynamicProperties.addAll(containerSpec.getDynamicConfiguration()); + dynamicProperties.addAll(containerSpec.getFlinkConfiguration()); // build a TaskInfo with assigned resources, environment variables, etc final Protos.TaskInfo.Builder taskInfo = Protos.TaskInfo.newBuilder() @@ -244,7 +244,7 @@ public Protos.TaskInfo launch(Protos.SlaveID slaveId, MesosResourceAllocation al } // take needed ports for the TM - Set tmPortKeys = extractPortKeys(containerSpec.getDynamicConfiguration()); + Set tmPortKeys = extractPortKeys(containerSpec.getFlinkConfiguration()); List portResources = allocation.takeRanges("ports", tmPortKeys.size(), roles); taskInfo.addAllResources(portResources); Iterator portsToAssign = tmPortKeys.iterator(); diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosUtils.java b/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosUtils.java index 38175eecc385..494233c42f6d 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosUtils.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosUtils.java @@ -125,7 +125,7 @@ public static ContainerSpecification createContainerSpec(Configuration configura ContainerSpecification spec = new ContainerSpecification(); // propagate the AM dynamic configuration to the TM - spec.getDynamicConfiguration().addAll(dynamicProperties); + spec.getFlinkConfiguration().addAll(dynamicProperties); applyOverlays(configuration, spec); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContainerSpecification.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContainerSpecification.java index 0f6b028c92a3..1848950d63ab 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContainerSpecification.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContainerSpecification.java @@ -53,13 +53,13 @@ public class ContainerSpecification implements java.io.Serializable { private final Map environmentVariables; - private final Configuration dynamicConfiguration; + private final Configuration flinkConfiguration; public ContainerSpecification() { this.artifacts = new LinkedList<>(); this.environmentVariables = new HashMap(); this.systemProperties = new Configuration(); - this.dynamicConfiguration = new Configuration(); + this.flinkConfiguration = new Configuration(); } /** @@ -79,8 +79,8 @@ public Map getEnvironmentVariables() { /** * Get the dynamic configuration. */ - public Configuration getDynamicConfiguration() { - return dynamicConfiguration; + public Configuration getFlinkConfiguration() { + return flinkConfiguration; } /** @@ -96,7 +96,7 @@ protected Object clone() throws CloneNotSupportedException { clone.artifacts.addAll(this.artifacts); clone.environmentVariables.putAll(this.environmentVariables); clone.systemProperties.addAll(this.systemProperties); - clone.dynamicConfiguration.addAll(this.dynamicConfiguration); + clone.flinkConfiguration.addAll(this.flinkConfiguration); return clone; } @@ -105,7 +105,7 @@ public String toString() { return "ContainerSpecification{" + "environmentVariables=" + environmentVariables + ", systemProperties=" + systemProperties + - ", dynamicConfiguration=" + dynamicConfiguration + + ", dynamicConfiguration=" + flinkConfiguration + ", artifacts=" + artifacts + '}'; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/overlays/HadoopConfOverlay.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/overlays/HadoopConfOverlay.java index c45cd02eed52..7fb827482257 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/overlays/HadoopConfOverlay.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/overlays/HadoopConfOverlay.java @@ -69,7 +69,7 @@ public void configure(ContainerSpecification container) throws IOException { File hdfsSitePath = new File(hadoopConfDir, "hdfs-site.xml"); container.getEnvironmentVariables().put("HADOOP_CONF_DIR", TARGET_CONF_DIR.toString()); - container.getDynamicConfiguration().setString(ConfigConstants.PATH_HADOOP_CONFIG, TARGET_CONF_DIR.toString()); + container.getFlinkConfiguration().setString(ConfigConstants.PATH_HADOOP_CONFIG, TARGET_CONF_DIR.toString()); container.getArtifacts().add(ContainerSpecification.Artifact .newBuilder() diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/overlays/KeytabOverlay.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/overlays/KeytabOverlay.java index c8c87d44e87f..8f6eb9f97f33 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/overlays/KeytabOverlay.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/overlays/KeytabOverlay.java @@ -60,7 +60,7 @@ public void configure(ContainerSpecification container) throws IOException { .setDest(TARGET_PATH) .setCachable(false) .build()); - container.getDynamicConfiguration().setString(SecurityOptions.KERBEROS_LOGIN_KEYTAB, TARGET_PATH.getPath()); + container.getFlinkConfiguration().setString(SecurityOptions.KERBEROS_LOGIN_KEYTAB, TARGET_PATH.getPath()); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/overlays/SSLStoreOverlay.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/overlays/SSLStoreOverlay.java index 84d407b5687e..e814355d52a3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/overlays/SSLStoreOverlay.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/overlays/SSLStoreOverlay.java @@ -64,7 +64,7 @@ public void configure(ContainerSpecification container) throws IOException { .setDest(TARGET_KEYSTORE_PATH) .setCachable(false) .build()); - container.getDynamicConfiguration().setString(SecurityOptions.SSL_KEYSTORE, TARGET_KEYSTORE_PATH.getPath()); + container.getFlinkConfiguration().setString(SecurityOptions.SSL_KEYSTORE, TARGET_KEYSTORE_PATH.getPath()); } if(truststore != null) { container.getArtifacts().add(ContainerSpecification.Artifact.newBuilder() @@ -72,7 +72,7 @@ public void configure(ContainerSpecification container) throws IOException { .setDest(TARGET_TRUSTSTORE_PATH) .setCachable(false) .build()); - container.getDynamicConfiguration().setString(SecurityOptions.SSL_TRUSTSTORE, TARGET_TRUSTSTORE_PATH.getPath()); + container.getFlinkConfiguration().setString(SecurityOptions.SSL_TRUSTSTORE, TARGET_TRUSTSTORE_PATH.getPath()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/overlays/HadoopConfOverlayTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/overlays/HadoopConfOverlayTest.java index c3ea41bd5d1d..8acfd601b321 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/overlays/HadoopConfOverlayTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/overlays/HadoopConfOverlayTest.java @@ -52,7 +52,7 @@ public void testConfigure() throws Exception { overlay.configure(spec); assertEquals(TARGET_CONF_DIR.getPath(), spec.getEnvironmentVariables().get("HADOOP_CONF_DIR")); - assertEquals(TARGET_CONF_DIR.getPath(), spec.getDynamicConfiguration().getString(ConfigConstants.PATH_HADOOP_CONFIG, null)); + assertEquals(TARGET_CONF_DIR.getPath(), spec.getFlinkConfiguration().getString(ConfigConstants.PATH_HADOOP_CONFIG, null)); checkArtifact(spec, new Path(TARGET_CONF_DIR, "core-site.xml")); checkArtifact(spec, new Path(TARGET_CONF_DIR, "hdfs-site.xml")); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/overlays/KeytabOverlayTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/overlays/KeytabOverlayTest.java index 947d2ffe8614..3ed76ca44d97 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/overlays/KeytabOverlayTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/overlays/KeytabOverlayTest.java @@ -46,7 +46,7 @@ public void testConfigure() throws Exception { ContainerSpecification spec = new ContainerSpecification(); overlay.configure(spec); - assertEquals(TARGET_PATH.getPath(), spec.getDynamicConfiguration().getString(SecurityOptions.KERBEROS_LOGIN_KEYTAB)); + assertEquals(TARGET_PATH.getPath(), spec.getFlinkConfiguration().getString(SecurityOptions.KERBEROS_LOGIN_KEYTAB)); checkArtifact(spec, TARGET_PATH); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/overlays/SSLStoreOverlayTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/overlays/SSLStoreOverlayTest.java index ce48ce442c68..fc6f9082be2f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/overlays/SSLStoreOverlayTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/overlays/SSLStoreOverlayTest.java @@ -46,10 +46,10 @@ public void testConfigure() throws Exception { ContainerSpecification spec = new ContainerSpecification(); overlay.configure(spec); - assertEquals(TARGET_KEYSTORE_PATH.getPath(), spec.getDynamicConfiguration().getString(SecurityOptions.SSL_KEYSTORE)); + assertEquals(TARGET_KEYSTORE_PATH.getPath(), spec.getFlinkConfiguration().getString(SecurityOptions.SSL_KEYSTORE)); checkArtifact(spec, TARGET_KEYSTORE_PATH); - assertEquals(TARGET_TRUSTSTORE_PATH.getPath(), spec.getDynamicConfiguration().getString(SecurityOptions.SSL_TRUSTSTORE)); + assertEquals(TARGET_TRUSTSTORE_PATH.getPath(), spec.getFlinkConfiguration().getString(SecurityOptions.SSL_TRUSTSTORE)); checkArtifact(spec, TARGET_TRUSTSTORE_PATH); } From 80043ae1ee0c080e8f866e27a98cb04e9d93339a Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 26 Oct 2019 13:46:56 +0200 Subject: [PATCH 382/746] [hotfix] Remove clone method from ContainerSpecification --- .../clusterframework/ContainerSpecification.java | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContainerSpecification.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContainerSpecification.java index 1848950d63ab..d0126fc4f60f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContainerSpecification.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContainerSpecification.java @@ -90,16 +90,6 @@ public Configuration getSystemProperties() { return systemProperties; } - @Override - protected Object clone() throws CloneNotSupportedException { - ContainerSpecification clone = new ContainerSpecification(); - clone.artifacts.addAll(this.artifacts); - clone.environmentVariables.putAll(this.environmentVariables); - clone.systemProperties.addAll(this.systemProperties); - clone.flinkConfiguration.addAll(this.flinkConfiguration); - return clone; - } - @Override public String toString() { return "ContainerSpecification{" + From b7d0abfe74088aaba55af03c0a44dc720d729cd8 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 26 Oct 2019 13:59:19 +0200 Subject: [PATCH 383/746] [hotfix][mesos] Remove dynamicProperties from ContainerSpecification The dynamic properties are already contained in the Flink configuration. --- .../mesos/entrypoint/MesosJobClusterEntrypoint.java | 11 +++-------- .../entrypoint/MesosSessionClusterEntrypoint.java | 11 +++-------- .../java/org/apache/flink/mesos/util/MesosUtils.java | 7 +++---- 3 files changed, 9 insertions(+), 20 deletions(-) diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java index e01c094602f0..23190adebf38 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java @@ -36,7 +36,6 @@ import org.apache.flink.runtime.util.EnvironmentInformation; import org.apache.flink.runtime.util.JvmShutdownSafeguard; import org.apache.flink.runtime.util.SignalHandler; -import org.apache.flink.util.Preconditions; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLineParser; @@ -62,8 +61,6 @@ public class MesosJobClusterEntrypoint extends JobClusterEntrypoint { .addOption(BootstrapTools.newDynamicPropertiesOption()); } - private final Configuration dynamicProperties; - private MesosConfiguration schedulerConfiguration; private MesosServices mesosServices; @@ -72,10 +69,8 @@ public class MesosJobClusterEntrypoint extends JobClusterEntrypoint { private ContainerSpecification taskManagerContainerSpec; - public MesosJobClusterEntrypoint(Configuration config, Configuration dynamicProperties) { + public MesosJobClusterEntrypoint(Configuration config) { super(config); - - this.dynamicProperties = Preconditions.checkNotNull(dynamicProperties); } @Override @@ -92,7 +87,7 @@ protected void initializeServices(Configuration config) throws Exception { // TM configuration taskManagerParameters = MesosUtils.createTmParameters(config, LOG); - taskManagerContainerSpec = MesosUtils.createContainerSpec(config, dynamicProperties); + taskManagerContainerSpec = MesosUtils.createContainerSpec(config); } @Override @@ -140,7 +135,7 @@ public static void main(String[] args) { Configuration dynamicProperties = BootstrapTools.parseDynamicProperties(cmd); Configuration configuration = MesosUtils.loadConfiguration(dynamicProperties, LOG); - MesosJobClusterEntrypoint clusterEntrypoint = new MesosJobClusterEntrypoint(configuration, dynamicProperties); + MesosJobClusterEntrypoint clusterEntrypoint = new MesosJobClusterEntrypoint(configuration); ClusterEntrypoint.runClusterEntrypoint(clusterEntrypoint); } diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java index 0f308e29e376..6258df694c2b 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java @@ -35,7 +35,6 @@ import org.apache.flink.runtime.util.EnvironmentInformation; import org.apache.flink.runtime.util.JvmShutdownSafeguard; import org.apache.flink.runtime.util.SignalHandler; -import org.apache.flink.util.Preconditions; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLineParser; @@ -61,8 +60,6 @@ public class MesosSessionClusterEntrypoint extends SessionClusterEntrypoint { .addOption(BootstrapTools.newDynamicPropertiesOption()); } - private final Configuration dynamicProperties; - private MesosConfiguration mesosConfig; private MesosServices mesosServices; @@ -71,10 +68,8 @@ public class MesosSessionClusterEntrypoint extends SessionClusterEntrypoint { private ContainerSpecification taskManagerContainerSpec; - public MesosSessionClusterEntrypoint(Configuration config, Configuration dynamicProperties) { + public MesosSessionClusterEntrypoint(Configuration config) { super(config); - - this.dynamicProperties = Preconditions.checkNotNull(dynamicProperties); } @Override @@ -91,7 +86,7 @@ protected void initializeServices(Configuration config) throws Exception { // TM configuration taskManagerParameters = MesosUtils.createTmParameters(config, LOG); - taskManagerContainerSpec = MesosUtils.createContainerSpec(config, dynamicProperties); + taskManagerContainerSpec = MesosUtils.createContainerSpec(config); } @Override @@ -138,7 +133,7 @@ public static void main(String[] args) { Configuration dynamicProperties = BootstrapTools.parseDynamicProperties(cmd); Configuration configuration = MesosUtils.loadConfiguration(dynamicProperties, LOG); - MesosSessionClusterEntrypoint clusterEntrypoint = new MesosSessionClusterEntrypoint(configuration, dynamicProperties); + MesosSessionClusterEntrypoint clusterEntrypoint = new MesosSessionClusterEntrypoint(configuration); ClusterEntrypoint.runClusterEntrypoint(clusterEntrypoint); } diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosUtils.java b/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosUtils.java index 494233c42f6d..0966dfde9abb 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosUtils.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosUtils.java @@ -119,15 +119,14 @@ public static MesosTaskManagerParameters createTmParameters(Configuration config return taskManagerParameters; } - public static ContainerSpecification createContainerSpec(Configuration configuration, Configuration dynamicProperties) + public static ContainerSpecification createContainerSpec(Configuration flinkConfiguration) throws Exception { // generate a container spec which conveys the artifacts/vars needed to launch a TM ContainerSpecification spec = new ContainerSpecification(); - // propagate the AM dynamic configuration to the TM - spec.getFlinkConfiguration().addAll(dynamicProperties); + spec.getFlinkConfiguration().addAll(flinkConfiguration); - applyOverlays(configuration, spec); + applyOverlays(flinkConfiguration, spec); return spec; } From c0f3c8044a319c3630c390f8b1603a7edf076181 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 26 Oct 2019 14:08:08 +0200 Subject: [PATCH 384/746] [FLINK-14074][mesos] Forward configuration to Mesos TaskExecutor This commit forwards the Flink configuration to a Mesos TaskExecutor by creating the ContainerSpecification with it. This will forward all options which are dynamically configured until the ResourceManager has been started to the Mesos TaskExecutor. This closes #10002. --- .../entrypoint/MesosJobClusterEntrypoint.java | 14 +------ .../MesosSessionClusterEntrypoint.java | 14 +------ .../MesosResourceManagerFactory.java | 19 +++++----- .../apache/flink/mesos/util/MesosUtils.java | 4 +- .../LaunchableMesosWorkerTest.java | 37 +++++++++++++++++++ .../ContainerSpecification.java | 6 +++ 6 files changed, 56 insertions(+), 38 deletions(-) diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java index 23190adebf38..90858509b963 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java @@ -21,13 +21,11 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.mesos.runtime.clusterframework.MesosResourceManagerFactory; -import org.apache.flink.mesos.runtime.clusterframework.MesosTaskManagerParameters; import org.apache.flink.mesos.runtime.clusterframework.services.MesosServices; import org.apache.flink.mesos.runtime.clusterframework.services.MesosServicesUtils; import org.apache.flink.mesos.util.MesosConfiguration; import org.apache.flink.mesos.util.MesosUtils; import org.apache.flink.runtime.clusterframework.BootstrapTools; -import org.apache.flink.runtime.clusterframework.ContainerSpecification; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; import org.apache.flink.runtime.entrypoint.JobClusterEntrypoint; @@ -65,10 +63,6 @@ public class MesosJobClusterEntrypoint extends JobClusterEntrypoint { private MesosServices mesosServices; - private MesosTaskManagerParameters taskManagerParameters; - - private ContainerSpecification taskManagerContainerSpec; - public MesosJobClusterEntrypoint(Configuration config) { super(config); } @@ -84,10 +78,6 @@ protected void initializeServices(Configuration config) throws Exception { // services mesosServices = MesosServicesUtils.createMesosServices(config, hostname); - - // TM configuration - taskManagerParameters = MesosUtils.createTmParameters(config, LOG); - taskManagerContainerSpec = MesosUtils.createContainerSpec(config); } @Override @@ -108,9 +98,7 @@ protected DefaultDispatcherResourceManagerComponentFactory createDispatcherResou return DefaultDispatcherResourceManagerComponentFactory.createJobComponentFactory( new MesosResourceManagerFactory( mesosServices, - schedulerConfiguration, - taskManagerParameters, - taskManagerContainerSpec), + schedulerConfiguration), FileJobGraphRetriever.createFrom(configuration)); } diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java index 6258df694c2b..d7223564b4d9 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java @@ -21,13 +21,11 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.mesos.runtime.clusterframework.MesosResourceManagerFactory; -import org.apache.flink.mesos.runtime.clusterframework.MesosTaskManagerParameters; import org.apache.flink.mesos.runtime.clusterframework.services.MesosServices; import org.apache.flink.mesos.runtime.clusterframework.services.MesosServicesUtils; import org.apache.flink.mesos.util.MesosConfiguration; import org.apache.flink.mesos.util.MesosUtils; import org.apache.flink.runtime.clusterframework.BootstrapTools; -import org.apache.flink.runtime.clusterframework.ContainerSpecification; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; import org.apache.flink.runtime.entrypoint.SessionClusterEntrypoint; @@ -64,10 +62,6 @@ public class MesosSessionClusterEntrypoint extends SessionClusterEntrypoint { private MesosServices mesosServices; - private MesosTaskManagerParameters taskManagerParameters; - - private ContainerSpecification taskManagerContainerSpec; - public MesosSessionClusterEntrypoint(Configuration config) { super(config); } @@ -83,10 +77,6 @@ protected void initializeServices(Configuration config) throws Exception { // services mesosServices = MesosServicesUtils.createMesosServices(config, hostname); - - // TM configuration - taskManagerParameters = MesosUtils.createTmParameters(config, LOG); - taskManagerContainerSpec = MesosUtils.createContainerSpec(config); } @Override @@ -107,9 +97,7 @@ protected DefaultDispatcherResourceManagerComponentFactory createDispatcherResou return DefaultDispatcherResourceManagerComponentFactory.createSessionComponentFactory( new MesosResourceManagerFactory( mesosServices, - mesosConfig, - taskManagerParameters, - taskManagerContainerSpec)); + mesosConfig)); } public static void main(String[] args) { diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManagerFactory.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManagerFactory.java index 52a8ef12d534..d9615367c7ae 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManagerFactory.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManagerFactory.java @@ -21,6 +21,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.mesos.runtime.clusterframework.services.MesosServices; import org.apache.flink.mesos.util.MesosConfiguration; +import org.apache.flink.mesos.util.MesosUtils; import org.apache.flink.runtime.clusterframework.ContainerSpecification; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.entrypoint.ClusterInformation; @@ -35,6 +36,9 @@ import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -43,23 +47,17 @@ */ public class MesosResourceManagerFactory extends ActiveResourceManagerFactory { + private static final Logger LOG = LoggerFactory.getLogger(MesosResourceManagerFactory.class); + @Nonnull private final MesosServices mesosServices; @Nonnull private final MesosConfiguration schedulerConfiguration; - @Nonnull - private final MesosTaskManagerParameters taskManagerParameters; - - @Nonnull - private final ContainerSpecification taskManagerContainerSpec; - - public MesosResourceManagerFactory(@Nonnull MesosServices mesosServices, @Nonnull MesosConfiguration schedulerConfiguration, @Nonnull MesosTaskManagerParameters taskManagerParameters, @Nonnull ContainerSpecification taskManagerContainerSpec) { + public MesosResourceManagerFactory(@Nonnull MesosServices mesosServices, @Nonnull MesosConfiguration schedulerConfiguration) { this.mesosServices = mesosServices; this.schedulerConfiguration = schedulerConfiguration; - this.taskManagerParameters = taskManagerParameters; - this.taskManagerContainerSpec = taskManagerContainerSpec; } @Override @@ -79,6 +77,9 @@ public ResourceManager createActiveResourceManager( highAvailabilityServices, rpcService.getScheduledExecutor()); + final MesosTaskManagerParameters taskManagerParameters = MesosUtils.createTmParameters(configuration, LOG); + final ContainerSpecification taskManagerContainerSpec = MesosUtils.createContainerSpec(configuration); + return new MesosResourceManager( rpcService, getEndpointId(), diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosUtils.java b/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosUtils.java index 0966dfde9abb..946567f23df3 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosUtils.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosUtils.java @@ -122,9 +122,7 @@ public static MesosTaskManagerParameters createTmParameters(Configuration config public static ContainerSpecification createContainerSpec(Configuration flinkConfiguration) throws Exception { // generate a container spec which conveys the artifacts/vars needed to launch a TM - ContainerSpecification spec = new ContainerSpecification(); - - spec.getFlinkConfiguration().addAll(flinkConfiguration); + ContainerSpecification spec = ContainerSpecification.from(flinkConfiguration); applyOverlays(flinkConfiguration, spec); diff --git a/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorkerTest.java b/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorkerTest.java index 48a436cb9954..3d53160bc42f 100644 --- a/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorkerTest.java +++ b/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorkerTest.java @@ -19,15 +19,29 @@ package org.apache.flink.mesos.runtime.clusterframework; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.mesos.configuration.MesosOptions; +import org.apache.flink.mesos.scheduler.LaunchableTask; +import org.apache.flink.mesos.util.MesosResourceAllocation; +import org.apache.flink.mesos.util.MesosUtils; +import org.apache.flink.runtime.clusterframework.ContainerSpecification; import org.apache.flink.util.TestLogger; +import org.apache.mesos.Protos; import org.junit.Test; import java.util.Arrays; +import java.util.Collections; import java.util.HashSet; import java.util.Set; +import scala.Option; + +import static org.apache.flink.mesos.Utils.ports; +import static org.apache.flink.mesos.Utils.range; import static org.apache.flink.mesos.configuration.MesosOptions.PORT_ASSIGNMENTS; +import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.assertThat; @@ -66,4 +80,27 @@ public void canGetNoPortKeys() { assertThat(portKeys, is(equalTo(LaunchableMesosWorker.TM_PORT_KEYS))); } + @Test + public void launch_withNonDefaultConfiguration_forwardsConfigurationValues() { + final Configuration configuration = new Configuration(); + configuration.setString(MesosOptions.MASTER_URL, "foobar"); + final MemorySize memorySize = new MemorySize(1337L); + configuration.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, memorySize.toString()); + + final LaunchableTask launchableTask = new LaunchableMesosWorker( + ignored -> Option.empty(), + MesosTaskManagerParameters.create(configuration), + ContainerSpecification.from(configuration), + Protos.TaskID.newBuilder().setValue("test-task-id").build(), + MesosUtils.createMesosSchedulerConfiguration(configuration, "localhost")); + + final Protos.TaskInfo taskInfo = launchableTask.launch( + Protos.SlaveID.newBuilder().setValue("test-slave-id").build(), + new MesosResourceAllocation(Collections.singleton(ports(range(1000, 2000))))); + + assertThat( + taskInfo.getCommand().getValue(), + containsString(ContainerSpecification.createDynamicProperty(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE.key(), memorySize.toString()))); + } + } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContainerSpecification.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContainerSpecification.java index d0126fc4f60f..605039b8176c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContainerSpecification.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContainerSpecification.java @@ -175,6 +175,12 @@ public Artifact build() { } } + public static ContainerSpecification from(Configuration flinkConfiguration) { + final ContainerSpecification containerSpecification = new ContainerSpecification(); + containerSpecification.getFlinkConfiguration().addAll(flinkConfiguration); + return containerSpecification; + } + /** * Format the system properties as a shell-compatible command-line argument. */ From 3a16b2b972fc449412e1f3a0e123bee4951e597d Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 26 Oct 2019 18:31:38 +0200 Subject: [PATCH 385/746] [hotfix][mesos] Add more logging information to LaunchableMesosWorker --- .../mesos/runtime/clusterframework/LaunchableMesosWorker.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java index 9a4edc2a972d..895c66a608e5 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java @@ -339,6 +339,8 @@ public Protos.TaskInfo launch(Protos.SlaveID slaveId, MesosResourceAllocation al containerInfo.addAllVolumes(params.containerVolumes()); taskInfo.setContainer(containerInfo); + LOG.debug("Starting TaskExecutor {} with command: {}", slaveId, taskInfo.getCommand().getValue()); + return taskInfo.build(); } From 3a69d5b8af7fd131670fbf4479653b8700c8a3c4 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Tue, 29 Oct 2019 12:18:31 +0800 Subject: [PATCH 386/746] [FLINK-14558][python] Fix ClassNotFoundException of PythonScalarFunctionOperator Load the jar of flink-python with user classloader instead of the system classloader. This closes #10045. --- .../plan/nodes/common/CommonPythonCalc.scala | 14 ++++++++++++-- .../flink/table/plan/nodes/CommonPythonCalc.scala | 12 +++++++++++- .../nodes/datastream/DataStreamPythonCalc.scala | 2 +- 3 files changed, 24 insertions(+), 4 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonCalc.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonCalc.scala index cf59d177b4d6..f37322bf5873 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonCalc.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPythonCalc.scala @@ -23,6 +23,7 @@ import org.apache.calcite.sql.`type`.SqlTypeName import org.apache.flink.api.dag.Transformation import org.apache.flink.streaming.api.operators.OneInputStreamOperator import org.apache.flink.streaming.api.transformations.OneInputTransformation +import org.apache.flink.table.api.TableException import org.apache.flink.table.dataformat.BaseRow import org.apache.flink.table.functions.python.{PythonFunction, PythonFunctionInfo, SimplePythonFunction} import org.apache.flink.table.planner.calcite.FlinkTypeFactory @@ -36,8 +37,17 @@ import scala.collection.mutable trait CommonPythonCalc { + private def loadClass(className: String): Class[_] = { + try { + Class.forName(className, false, Thread.currentThread.getContextClassLoader) + } catch { + case ex: ClassNotFoundException => throw new TableException( + "The dependency of 'flink-python' is not present on the classpath.", ex) + } + } + private lazy val convertLiteralToPython = { - val clazz = Class.forName("org.apache.flink.api.common.python.PythonBridgeUtils") + val clazz = loadClass("org.apache.flink.api.common.python.PythonBridgeUtils") clazz.getMethod("convertLiteralToPython", classOf[RexLiteral], classOf[SqlTypeName]) } @@ -95,7 +105,7 @@ trait CommonPythonCalc { udfInputOffsets: Array[Int], pythonFunctionInfos: Array[PythonFunctionInfo], forwardedFields: Array[Int])= { - val clazz = Class.forName(PYTHON_SCALAR_FUNCTION_OPERATOR_NAME) + val clazz = loadClass(PYTHON_SCALAR_FUNCTION_OPERATOR_NAME) val ctor = clazz.getConstructor( classOf[Array[PythonFunctionInfo]], classOf[RowType], 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 2acf2dc25eeb..deccc5448285 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 @@ -19,6 +19,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.api.TableException import org.apache.flink.table.functions.python.{PythonFunction, PythonFunctionInfo, SimplePythonFunction} import org.apache.flink.table.functions.utils.ScalarSqlFunction @@ -27,8 +28,17 @@ import scala.collection.mutable trait CommonPythonCalc { + protected def loadClass(className: String): Class[_] = { + try { + Class.forName(className, false, Thread.currentThread.getContextClassLoader) + } catch { + case ex: ClassNotFoundException => throw new TableException( + "The dependency of 'flink-python' is not present on the classpath.", ex) + } + } + private lazy val convertLiteralToPython = { - val clazz = Class.forName("org.apache.flink.api.common.python.PythonBridgeUtils") + val clazz = loadClass("org.apache.flink.api.common.python.PythonBridgeUtils") clazz.getMethod("convertLiteralToPython", classOf[RexLiteral], classOf[SqlTypeName]) } 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 d2511c8e4575..a610ee560d00 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 @@ -116,7 +116,7 @@ class DataStreamPythonCalc( inputRowType: RowType, outputRowType: RowType, udfInputOffsets: Array[Int]) = { - val clazz = Class.forName(PYTHON_SCALAR_FUNCTION_OPERATOR_NAME) + val clazz = loadClass(PYTHON_SCALAR_FUNCTION_OPERATOR_NAME) val ctor = clazz.getConstructor( classOf[Array[PythonFunctionInfo]], classOf[RowType], From 8522f3b0b93a6ae92f0497888a3200e2db91fd6f Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Wed, 30 Oct 2019 14:51:39 +0100 Subject: [PATCH 387/746] [FLINK-14544][runtime] Fixing race condition during task cancellation when closing mailbox / enqueuing poison letter. The race condition occurred because StreamTask#cancel will stop TaskMailboxProcessor#runMainLoop by sending a poison letter. That loop, however, can also be left by receiving EOI from upstream tasks, such that the normal cleanup in StreamTask#invoke will close the mailbox(processor). Sending a (poison) letter with a closed mailbox will trigger an MailboxStateException, which acts as a safety net and reports illegal mailbox accesses in respect to its state. To avoid the situation, the enqueuing of the poison letter is more now robust. It checks for the mailbox being in the correct state (OPEN) or else will not enqueue the poison letter, which would not have an effect anyways as the mailbox loop has already been terminated. Since the check and the enqueuing of the letter are non-atomic, another (much more unlikely) race condition would have been introduced. Hence, there is now a convenience method to TaskMailbox to run code under the internal lock, such that MailboxProcessor can atomically check if mailbox is still open and enqueuing poison letter. --- .../runtime/tasks/mailbox/TaskMailbox.java | 11 ++++++++ .../tasks/mailbox/TaskMailboxImpl.java | 10 +++++++ .../mailbox/execution/MailboxProcessor.java | 8 +++++- .../tasks/mailbox/TaskMailboxImplTest.java | 28 +++++++++++++++++++ .../execution/TaskMailboxProcessorTest.java | 10 +++++++ 5 files changed, 66 insertions(+), 1 deletion(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java index 6238f595b51d..78311cb61109 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java @@ -74,4 +74,15 @@ enum State { * Returns true if the mailbox contains mail. */ boolean hasMail(); + + /** + * Runs the given code exclusively on this mailbox. No synchronized operations can be run concurrently to the + * given runnable (e.g., {@link #put(Mail)} or modifying lifecycle methods). + * + *

    Use this methods when you want to atomically execute code that uses different methods (e.g., check for + * state and then put message if open). + * + * @param runnable the runnable to execute + */ + void runExclusively(Runnable runnable); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java index 645d57df525e..61889507a3b7 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java @@ -245,4 +245,14 @@ public List close() { public State getState() { return state; } + + @Override + public void runExclusively(Runnable runnable) { + lock.lock(); + try { + runnable.run(); + } finally { + lock.unlock(); + } + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java index 0dcb8a257046..d627f75c00cc 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java @@ -169,7 +169,13 @@ public void reportThrowable(Throwable throwable) { * This method must be called to end the stream task when all actions for the tasks have been performed. */ public void allActionsCompleted() { - sendPriorityLetter(mailboxPoisonLetter, "poison letter"); + mailbox.runExclusively(() -> { + // keep state check and poison letter enqueuing atomic, such that no intermediate #close may cause a + // MailboxStateException in #sendPriorityLetter. + if (mailbox.getState() == TaskMailbox.State.OPEN) { + sendPriorityLetter(mailboxPoisonLetter, "poison letter"); + } + }); } private void sendPriorityLetter(Runnable priorityLetter, String descriptionFormat, Object... descriptionArgs) { diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImplTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImplTest.java index 61256122dd71..0b6653ec060b 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImplTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImplTest.java @@ -34,6 +34,7 @@ import java.util.function.Consumer; import static org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox.MAX_PRIORITY; +import static org.junit.Assert.assertEquals; /** * Unit tests for {@link TaskMailboxImpl}. @@ -307,4 +308,31 @@ public void testPutWithPriorityAndReadingFromMainMailbox() throws Exception { Assert.assertSame(mailB, taskMailbox.take(TaskMailbox.MIN_PRIORITY)); Assert.assertSame(mailD, taskMailbox.take(TaskMailbox.MIN_PRIORITY)); } + + /** + * Testing that we cannot close while running exclusively. + */ + @Test + public void testRunExclusively() throws InterruptedException { + CountDownLatch exclusiveCodeStarted = new CountDownLatch(1); + + final int numMails = 10; + + // send 10 mails in an atomic operation + new Thread(() -> + taskMailbox.runExclusively(() -> { + exclusiveCodeStarted.countDown(); + for (int index = 0; index < numMails; index++) { + try { + taskMailbox.put(new Mail(() -> {}, 1, "mailD")); + Thread.sleep(1); + } catch (Exception e) { + } + } + })).start(); + + exclusiveCodeStarted.await(); + // make sure that all 10 messages have been actually enqueued. + assertEquals(numMails, taskMailbox.close().size()); + } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/TaskMailboxProcessorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/TaskMailboxProcessorTest.java index 466c4f7e7df6..cf58da39388f 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/TaskMailboxProcessorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/TaskMailboxProcessorTest.java @@ -201,6 +201,16 @@ public void runDefaultAction(DefaultActionContext context) { mailboxThread.checkException(); } + /** + * Testing that canceling after closing will not lead to an exception. + */ + @Test + public void testCancelAfterClose() { + MailboxProcessor mailboxProcessor = new MailboxProcessor((ctx) -> {}); + mailboxProcessor.close(); + mailboxProcessor.allActionsCompleted(); + } + private static MailboxProcessor start(MailboxThread mailboxThread) { mailboxThread.start(); final MailboxProcessor mailboxProcessor = mailboxThread.getMailboxProcessor(); From a966e44b525e694dd90be30206ed83ee341ed9a4 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Wed, 30 Oct 2019 18:23:43 +0800 Subject: [PATCH 388/746] [FLINK-14439][runtime] Enable RestartPipelinedRegionStrategy to leverage JM tracked partition availability This enables better failover experience when using the DefaultScheduler This closes #10043. --- .../executiongraph/failover/flip1/FailoverStrategy.java | 5 ++++- .../failover/flip1/RestartPipelinedRegionStrategy.java | 7 +++++-- .../apache/flink/runtime/scheduler/DefaultScheduler.java | 2 +- .../org/apache/flink/runtime/scheduler/SchedulerBase.java | 5 +++++ 4 files changed, 15 insertions(+), 4 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverStrategy.java index 23d4f7f92504..798d88b62207 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverStrategy.java @@ -48,8 +48,11 @@ interface Factory { * Instantiates the {@link FailoverStrategy}. * * @param topology of the graph to failover + * @param resultPartitionAvailabilityChecker to check whether a result partition is available * @return The instantiated failover strategy. */ - FailoverStrategy create(FailoverTopology topology); + FailoverStrategy create( + FailoverTopology topology, + ResultPartitionAvailabilityChecker resultPartitionAvailabilityChecker); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionStrategy.java index e118f7af1ea4..a4cb2def9d63 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartPipelinedRegionStrategy.java @@ -267,8 +267,11 @@ public void removeResultPartitionFromFailedState(IntermediateResultPartitionID r public static class Factory implements FailoverStrategy.Factory { @Override - public FailoverStrategy create(final FailoverTopology topology) { - return new RestartPipelinedRegionStrategy(topology); + public FailoverStrategy create( + final FailoverTopology topology, + final ResultPartitionAvailabilityChecker resultPartitionAvailabilityChecker) { + + return new RestartPipelinedRegionStrategy(topology, resultPartitionAvailabilityChecker); } } } 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 b46eb7ff2610..ccb59eaed514 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 @@ -141,7 +141,7 @@ public DefaultScheduler( this.executionFailureHandler = new ExecutionFailureHandler( getFailoverTopology(), - failoverStrategyFactory.create(getFailoverTopology()), + failoverStrategyFactory.create(getFailoverTopology(), getResultPartitionAvailabilityChecker()), restartBackoffTimeStrategy); this.schedulingStrategy = schedulingStrategyFactory.createInstance(this, getSchedulingTopology(), getJobGraph()); this.executionSlotAllocator = checkNotNull(executionSlotAllocatorFactory).createInstance(getInputsLocationsRetriever()); 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 27c2856fe3d3..6a372daa6ebb 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 @@ -51,6 +51,7 @@ import org.apache.flink.runtime.executiongraph.IntermediateResult; import org.apache.flink.runtime.executiongraph.JobStatusListener; import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverTopology; +import org.apache.flink.runtime.executiongraph.failover.flip1.ResultPartitionAvailabilityChecker; import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory; import org.apache.flink.runtime.executiongraph.restart.RestartStrategyResolving; @@ -335,6 +336,10 @@ protected void failJob(Throwable cause) { return schedulingTopology; } + protected final ResultPartitionAvailabilityChecker getResultPartitionAvailabilityChecker() { + return getExecutionGraph().getResultPartitionAvailabilityChecker(); + } + protected final InputsLocationsRetriever getInputsLocationsRetriever() { return inputsLocationsRetriever; } From 6a25bc59503397b9c959d957624c654ba5b3d8e5 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Wed, 30 Oct 2019 18:32:55 +0800 Subject: [PATCH 389/746] [FLINK-14440][tests] Annotate BatchFineGrainedRecoveryITCase to enable scheduler NG testing for it This closes #10044. --- .../flink/test/recovery/BatchFineGrainedRecoveryITCase.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/BatchFineGrainedRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/BatchFineGrainedRecoveryITCase.java index bf12e111823b..ec379a146ca1 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/BatchFineGrainedRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/BatchFineGrainedRecoveryITCase.java @@ -54,6 +54,7 @@ import org.apache.flink.runtime.rest.messages.job.SubtaskExecutionAttemptDetailsInfo; import org.apache.flink.runtime.util.ExecutorThreadFactory; import org.apache.flink.test.util.TestEnvironment; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.Collector; import org.apache.flink.util.ConfigurationException; import org.apache.flink.util.ExceptionUtils; @@ -64,6 +65,7 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -120,6 +122,7 @@ * lost results. * */ +@Category(AlsoRunWithSchedulerNG.class) public class BatchFineGrainedRecoveryITCase extends TestLogger { private static final Logger LOG = LoggerFactory.getLogger(BatchFineGrainedRecoveryITCase.class); From d00fe06d8f548892a7f8c95dfda43c70224464c7 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Sun, 13 Oct 2019 21:09:15 +0200 Subject: [PATCH 390/746] [FLINK-14502] Change CustomCommandLine cluster client methods to get a configuration --- .../generated/deployment_configuration.html | 16 + .../generated/yarn_config_configuration.html | 45 +++ docs/ops/config.md | 4 + docs/ops/config.zh.md | 4 + .../client/cli/AbstractCustomCommandLine.java | 9 +- .../apache/flink/client/cli/CliFrontend.java | 19 +- .../flink/client/cli/CustomCommandLine.java | 36 +- .../apache/flink/client/cli/DefaultCLI.java | 12 +- .../flink/client/cli/DefaultCLITest.java | 14 +- .../cli/util/DummyCustomCommandLine.java | 12 +- .../program/rest/RestClusterClientTest.java | 6 +- .../configuration/DeploymentOptions.java | 35 ++ .../apache/flink/api/scala/FlinkShell.scala | 10 +- .../gateway/local/ExecutionContext.java | 23 +- .../yarn/CliFrontendRunWithYarnTest.java | 12 +- .../org/apache/flink/yarn/YARNITCase.java | 2 - .../flink/yarn/YarnConfigurationITCase.java | 2 - .../org/apache/flink/yarn/YarnTestBase.java | 1 - .../NonDeployingYarnClusterDescriptor.java | 3 +- .../flink/yarn/YarnClusterDescriptor.java | 33 +- .../flink/yarn/cli/FlinkYarnSessionCli.java | 347 +++++++++++------- .../flink/yarn/cli/YarnConfigUtils.java | 87 +++++ .../yarn/configuration/YarnConfigOptions.java | 48 +++ .../flink/yarn/AbstractYarnClusterTest.java | 1 - .../flink/yarn/FlinkYarnSessionCliTest.java | 60 ++- .../flink/yarn/YarnClusterDescriptorTest.java | 2 - 26 files changed, 563 insertions(+), 280 deletions(-) create mode 100644 docs/_includes/generated/deployment_configuration.html create mode 100644 flink-core/src/main/java/org/apache/flink/configuration/DeploymentOptions.java create mode 100644 flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnConfigUtils.java diff --git a/docs/_includes/generated/deployment_configuration.html b/docs/_includes/generated/deployment_configuration.html new file mode 100644 index 000000000000..bdac7576987d --- /dev/null +++ b/docs/_includes/generated/deployment_configuration.html @@ -0,0 +1,16 @@ + + + + + + + + + + + + + + + +
    KeyDefaultDescription
    execution.attached
    falseSpecifies if the pipeline is submitted in attached or detached mode.
    diff --git a/docs/_includes/generated/yarn_config_configuration.html b/docs/_includes/generated/yarn_config_configuration.html index a28f15bdb05d..6cf3945db7e1 100644 --- a/docs/_includes/generated/yarn_config_configuration.html +++ b/docs/_includes/generated/yarn_config_configuration.html @@ -7,6 +7,11 @@ + +

    $internal.yarn.dynamic-properties
    + (none) + **DO NOT USE** Specify YARN dynamic properties. +
    yarn.application-attempt-failures-validity-interval
    10000 @@ -22,11 +27,36 @@ "0" With this configuration option, users can specify a port, a range of ports or a list of ports for the Application Master (and JobManager) RPC port. By default we recommend using the default value (0) to let the operating system choose an appropriate port. In particular when multiple AMs are running on the same physical host, fixed port assignments prevent the AM from starting. For example when running Flink on YARN on an environment with a restrictive firewall, this option allows specifying a range of allowed ports. + +
    yarn.application.id
    + (none) + The YARN application id of the running yarn cluster. This is the YARN cluster where the pipeline is going to be executed. + + +
    yarn.application.name
    + (none) + A custom name for your YARN application. + + +
    yarn.application.node-label
    + (none) + Specify YARN node label for the YARN application. +
    yarn.application.priority
    -1 A non-negative integer indicating the priority for submitting a Flink YARN application. It will only take effect if YARN priority scheduling setting is enabled. Larger integer corresponds with higher priority. If priority is negative or set to '-1'(default), Flink will unset yarn priority setting and use cluster default priority. Please refer to YARN's official documentation for specific settings required to enable priority scheduling for the targeted YARN version. + +
    yarn.application.queue
    + (none) + The YARN queue on which to put the current pipeline. + + +
    yarn.application.type
    + (none) + A custom type for your YARN application.. +
    yarn.appmaster.rpc.address
    (none) @@ -47,6 +77,11 @@ -1 The number of virtual cores (vcores) per YARN container. By default, the number of vcores is set to the number of slots per TaskManager, if set, or to 1, otherwise. In order for this parameter to be used your cluster must have CPU scheduling enabled. You can do this by setting the `org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler`. + +
    yarn.flink-dist-jar
    + (none) + The location of the Flink dist jar. +
    yarn.heartbeat.container-request-interval
    500 @@ -57,6 +92,11 @@ 5 Time between heartbeats with the ResourceManager in seconds. + +
    yarn.log-config-file
    + (none) + The location of the log config file, e.g. the path to your log4j.properties for log4j. +
    yarn.maximum-failed-containers
    (none) @@ -72,6 +112,11 @@ (none) When a Flink job is submitted to YARN, the JobManager’s host and the number of available processing slots is written into a properties file, so that the Flink client is able to pick those details up. This configuration parameter allows changing the default location of that file (for example for environments sharing a Flink installation between users). + +
    yarn.ship-directories
    + (none) + A semicolon-separated list of directories to be shipped to the YARN cluster. +
    yarn.tags
    (none) diff --git a/docs/ops/config.md b/docs/ops/config.md index 756153f14dd8..6a8f4cec4df4 100644 --- a/docs/ops/config.md +++ b/docs/ops/config.md @@ -64,6 +64,10 @@ These parameters configure the default HDFS used by Flink. Setups that do not sp {% include generated/core_configuration.html %} +### Execution + +{% include generated/deployment_configuration.html %} + ### JobManager {% include generated/job_manager_configuration.html %} diff --git a/docs/ops/config.zh.md b/docs/ops/config.zh.md index d965e838adb6..1274f6d16e56 100644 --- a/docs/ops/config.zh.md +++ b/docs/ops/config.zh.md @@ -64,6 +64,10 @@ These parameters configure the default HDFS used by Flink. Setups that do not sp {% include generated/core_configuration.html %} +### Execution + +{% include generated/deployment_configuration.html %} + ### JobManager {% include generated/job_manager_configuration.html %} diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java b/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java index 0cae6e72e6f9..805d58a75da9 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java @@ -69,13 +69,8 @@ public void addGeneralOptions(Options baseOptions) { baseOptions.addOption(zookeeperNamespaceOption); } - /** - * Override configuration settings by specified command line options. - * - * @param commandLine containing the overriding values - * @return Effective configuration with the overridden configuration settings - */ - protected Configuration applyCommandLineOptionsToConfiguration(CommandLine commandLine) throws FlinkException { + @Override + public Configuration applyCommandLineOptionsToConfiguration(CommandLine commandLine) throws FlinkException { final Configuration resultingConfiguration = new Configuration(configuration); if (commandLine.hasOption(addressOption.getOpt())) { 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 85892279c3fe..c49374a15529 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 @@ -195,9 +195,9 @@ protected void run(String[] args) throws Exception { } final CustomCommandLine customCommandLine = getActiveCustomCommandLine(commandLine); - + final Configuration executorConfig = customCommandLine.applyCommandLineOptionsToConfiguration(commandLine); try { - runProgram(customCommandLine, commandLine, runOptions, program); + runProgram(customCommandLine, executorConfig, runOptions, program); } finally { program.deleteExtractedLibraries(); } @@ -205,13 +205,13 @@ protected void run(String[] args) throws Exception { private void runProgram( CustomCommandLine customCommandLine, - CommandLine commandLine, + Configuration executorConfig, RunOptions runOptions, PackagedProgram program) throws ProgramInvocationException, FlinkException { - final ClusterDescriptor clusterDescriptor = customCommandLine.createClusterDescriptor(commandLine); + final ClusterDescriptor clusterDescriptor = customCommandLine.createClusterDescriptor(executorConfig); try { - final T clusterId = customCommandLine.getClusterId(commandLine); + final T clusterId = customCommandLine.getClusterId(executorConfig); final ClusterClient client; @@ -221,7 +221,7 @@ private void runProgram( final JobGraph jobGraph = PackagedProgramUtils.createJobGraph(program, configuration, parallelism); - final ClusterSpecification clusterSpecification = customCommandLine.getClusterSpecification(commandLine); + final ClusterSpecification clusterSpecification = customCommandLine.getClusterSpecification(executorConfig); client = clusterDescriptor.deployJobCluster( clusterSpecification, jobGraph, @@ -242,7 +242,7 @@ private void runProgram( } else { // also in job mode we have to deploy a session cluster because the job // might consist of multiple parts (e.g. when using collect) - final ClusterSpecification clusterSpecification = customCommandLine.getClusterSpecification(commandLine); + final ClusterSpecification clusterSpecification = customCommandLine.getClusterSpecification(executorConfig); client = clusterDescriptor.deploySessionCluster(clusterSpecification); // if not running in detached mode, add a shutdown hook to shut down cluster if client exits // there's a race-condition here if cli is killed before shutdown hook is installed @@ -912,9 +912,10 @@ private JobID parseJobId(String jobIdString) throws CliArgsException { * @throws FlinkException if something goes wrong */ private void runClusterAction(CustomCommandLine activeCommandLine, CommandLine commandLine, ClusterAction clusterAction) throws FlinkException { - final ClusterDescriptor clusterDescriptor = activeCommandLine.createClusterDescriptor(commandLine); + final Configuration executorConfig = activeCommandLine.applyCommandLineOptionsToConfiguration(commandLine); + final ClusterDescriptor clusterDescriptor = activeCommandLine.createClusterDescriptor(executorConfig); - final T clusterId = activeCommandLine.getClusterId(commandLine); + final T clusterId = activeCommandLine.getClusterId(executorConfig); if (clusterId == null) { throw new FlinkException("No cluster id was specified. Please specify a cluster to which " + diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java index e93997490e8d..9458de2f8946 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java @@ -20,6 +20,7 @@ import org.apache.flink.client.deployment.ClusterDescriptor; import org.apache.flink.client.deployment.ClusterSpecification; +import org.apache.flink.configuration.Configuration; import org.apache.flink.util.FlinkException; import org.apache.commons.cli.CommandLine; @@ -59,37 +60,40 @@ public interface CustomCommandLine { void addGeneralOptions(Options baseOptions); /** - * Create a {@link ClusterDescriptor} from the given configuration, configuration directory - * and the command line. + * Override configuration settings by specified command line options. * - * @param commandLine containing command line options relevant for the ClusterDescriptor - * @return ClusterDescriptor - * @throws FlinkException if the ClusterDescriptor could not be created + * @param commandLine containing the overriding values + * @return the effective configuration with the overridden configuration settings */ - ClusterDescriptor createClusterDescriptor(CommandLine commandLine) throws FlinkException; + Configuration applyCommandLineOptionsToConfiguration(CommandLine commandLine) throws FlinkException; /** - * Returns the cluster id if a cluster id was specified on the command line, otherwise it - * returns null. + * Create a {@link ClusterDescriptor} from the given configuration. * - *

    A cluster id identifies a running cluster, e.g. the Yarn application id for a Flink - * cluster running on Yarn. + * @param configuration containing the configuration options relevant for the {@link ClusterDescriptor} + * @return the corresponding {@link ClusterDescriptor}. + */ + ClusterDescriptor createClusterDescriptor(Configuration configuration); + + /** + * Returns the cluster id if a cluster id is specified in the provided configuration, otherwise it returns {@code null}. + * + *

    A cluster id identifies a running cluster, e.g. the Yarn application id for a Flink cluster running on Yarn. * - * @param commandLine containing command line options relevant for the cluster id retrieval + * @param configuration containing the configuration options relevant for the cluster id retrieval * @return Cluster id identifying the cluster to deploy jobs to or null */ @Nullable - T getClusterId(CommandLine commandLine); + T getClusterId(Configuration configuration); /** * Returns the {@link ClusterSpecification} specified by the configuration and the command * line options. This specification can be used to deploy a new Flink cluster. * - * @param commandLine containing command line options relevant for the ClusterSpecification - * @return ClusterSpecification for a new Flink cluster - * @throws FlinkException if the ClusterSpecification could not be created + * @param configuration containing the configuration options relevant for the {@link ClusterSpecification} + * @return the corresponding {@link ClusterSpecification} for a new Flink cluster */ - ClusterSpecification getClusterSpecification(CommandLine commandLine) throws FlinkException; + ClusterSpecification getClusterSpecification(Configuration configuration); default CommandLine parseCommandLineOptions(String[] args, boolean stopAtNonOptions) throws CliArgsException { final Options options = new Options(); diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java index e9ed9af9533c..f21f755e19b5 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java @@ -22,7 +22,6 @@ import org.apache.flink.client.deployment.StandaloneClusterDescriptor; import org.apache.flink.client.deployment.StandaloneClusterId; import org.apache.flink.configuration.Configuration; -import org.apache.flink.util.FlinkException; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; @@ -55,21 +54,18 @@ public void addGeneralOptions(Options baseOptions) { } @Override - public StandaloneClusterDescriptor createClusterDescriptor( - CommandLine commandLine) throws FlinkException { - final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(commandLine); - - return new StandaloneClusterDescriptor(effectiveConfiguration); + public StandaloneClusterDescriptor createClusterDescriptor(Configuration configuration) { + return new StandaloneClusterDescriptor(configuration); } @Override @Nullable - public StandaloneClusterId getClusterId(CommandLine commandLine) { + public StandaloneClusterId getClusterId(Configuration configuration) { return StandaloneClusterId.getInstance(); } @Override - public ClusterSpecification getClusterSpecification(CommandLine commandLine) { + public ClusterSpecification getClusterSpecification(Configuration configuration) { return new ClusterSpecification.ClusterSpecificationBuilder().createClusterSpecification(); } } diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java index 23fe69c9211e..73f967d755ee 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java @@ -64,11 +64,10 @@ public void testConfigurationPassing() throws Exception { final String[] args = {}; CommandLine commandLine = defaultCLI.parseCommandLineOptions(args, false); + final Configuration executorConfig = defaultCLI.applyCommandLineOptionsToConfiguration(commandLine); - final ClusterDescriptor clusterDescriptor = - defaultCLI.createClusterDescriptor(commandLine); - - final ClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(commandLine)); + final ClusterDescriptor clusterDescriptor = defaultCLI.createClusterDescriptor(executorConfig); + final ClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(executorConfig)); final URL webInterfaceUrl = new URL(clusterClient.getWebInterfaceURL()); @@ -97,11 +96,10 @@ public void testManualConfigurationOverride() throws Exception { final String[] args = {"-m", manualHostname + ':' + manualPort}; CommandLine commandLine = defaultCLI.parseCommandLineOptions(args, false); + final Configuration executorConfig = defaultCLI.applyCommandLineOptionsToConfiguration(commandLine); - final ClusterDescriptor clusterDescriptor = - defaultCLI.createClusterDescriptor(commandLine); - - final ClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(commandLine)); + final ClusterDescriptor clusterDescriptor = defaultCLI.createClusterDescriptor(executorConfig); + final ClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(executorConfig)); final URL webInterfaceUrl = new URL(clusterClient.getWebInterfaceURL()); diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java index 12bea7493f18..0bd7dc1ab97b 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java @@ -22,6 +22,7 @@ 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.util.Preconditions; import org.apache.commons.cli.CommandLine; @@ -60,18 +61,23 @@ public void addGeneralOptions(Options baseOptions) { } @Override - public ClusterDescriptor createClusterDescriptor(CommandLine commandLine) { + public Configuration applyCommandLineOptionsToConfiguration(CommandLine commandLine) { + return new Configuration(); + } + + @Override + public ClusterDescriptor createClusterDescriptor(Configuration configuration) { return new DummyClusterDescriptor<>(clusterClient); } @Override @Nullable - public String getClusterId(CommandLine commandLine) { + public String getClusterId(Configuration configuration) { return "dummy"; } @Override - public ClusterSpecification getClusterSpecification(CommandLine commandLine) { + public ClusterSpecification getClusterSpecification(Configuration configuration) { return new ClusterSpecification.ClusterSpecificationBuilder().createClusterSpecification(); } } diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java index f234e07726f2..2ffa0b684f1c 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java @@ -554,10 +554,10 @@ public void testRESTManualConfigurationOverride() throws Exception { final String[] args = {"-m", manualHostname + ':' + manualPort}; CommandLine commandLine = defaultCLI.parseCommandLineOptions(args, false); + final Configuration executorConfig = defaultCLI.applyCommandLineOptionsToConfiguration(commandLine); - final StandaloneClusterDescriptor clusterDescriptor = defaultCLI.createClusterDescriptor(commandLine); - - final RestClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(commandLine)); + final StandaloneClusterDescriptor clusterDescriptor = defaultCLI.createClusterDescriptor(executorConfig); + final RestClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(executorConfig)); URL webMonitorBaseUrl = clusterClient.getWebMonitorBaseUrl().get(); assertThat(webMonitorBaseUrl.getHost(), equalTo(manualHostname)); diff --git a/flink-core/src/main/java/org/apache/flink/configuration/DeploymentOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/DeploymentOptions.java new file mode 100644 index 000000000000..1addcc40d702 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/configuration/DeploymentOptions.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.configuration; + +import org.apache.flink.annotation.PublicEvolving; + +import static org.apache.flink.configuration.ConfigOptions.key; + +/** + * The {@link ConfigOption configuration options} relevant for all Executors. + */ +@PublicEvolving +public class DeploymentOptions { + + public static final ConfigOption ATTACHED = + key("execution.attached") + .defaultValue(false) + .withDescription("Specifies if the pipeline is submitted in attached or detached mode."); +} diff --git a/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala b/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala index ab388c745a6b..04f68ffb8b56 100644 --- a/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala +++ b/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala @@ -258,10 +258,11 @@ object FlinkShell { val commandLine = CliFrontendParser.parse(commandLineOptions, args.toArray, true) val customCLI = frontend.getActiveCustomCommandLine(commandLine) + val executorConfig = customCLI.applyCommandLineOptionsToConfiguration(commandLine); - val clusterDescriptor = customCLI.createClusterDescriptor(commandLine) + val clusterDescriptor = customCLI.createClusterDescriptor(executorConfig) - val clusterSpecification = customCLI.getClusterSpecification(commandLine) + val clusterSpecification = customCLI.getClusterSpecification(executorConfig) val cluster = clusterDescriptor.deploySessionCluster(clusterSpecification) @@ -288,12 +289,13 @@ object FlinkShell { configuration, CliFrontend.loadCustomCommandLines(configuration, configurationDirectory)) val customCLI = frontend.getActiveCustomCommandLine(commandLine) + val executorConfig = customCLI.applyCommandLineOptionsToConfiguration(commandLine); val clusterDescriptor = customCLI - .createClusterDescriptor(commandLine) + .createClusterDescriptor(executorConfig) .asInstanceOf[ClusterDescriptor[Any]] - val clusterId = customCLI.getClusterId(commandLine) + val clusterId = customCLI.getClusterId(executorConfig) val cluster = clusterDescriptor.retrieve(clusterId) 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 023efbc210e7..d02859772f91 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 @@ -111,14 +111,14 @@ public class ExecutionContext { private final Map> tableSinks; private final Map functions; private final Configuration flinkConfig; - private final CommandLine commandLine; + private final Configuration executorConfig; private final CustomCommandLine activeCommandLine; private final RunOptions runOptions; private final T clusterId; private final ClusterSpecification clusterSpec; public ExecutionContext(Environment defaultEnvironment, SessionContext sessionContext, List dependencies, - Configuration flinkConfig, Options commandLineOptions, List> availableCommandLines) { + Configuration flinkConfig, Options commandLineOptions, List> availableCommandLines) throws FlinkException { this.sessionContext = sessionContext.copy(); // create internal copy because session context is mutable this.mergedEnv = Environment.merge(defaultEnvironment, sessionContext.getEnvironment()); this.dependencies = dependencies; @@ -155,11 +155,12 @@ public ExecutionContext(Environment defaultEnvironment, SessionContext sessionCo }); // convert deployment options into command line options that describe a cluster - commandLine = createCommandLine(mergedEnv.getDeployment(), commandLineOptions); + final CommandLine commandLine = createCommandLine(mergedEnv.getDeployment(), commandLineOptions); activeCommandLine = findActiveCommandLine(availableCommandLines, commandLine); + executorConfig = activeCommandLine.applyCommandLineOptionsToConfiguration(commandLine); runOptions = createRunOptions(commandLine); - clusterId = activeCommandLine.getClusterId(commandLine); - clusterSpec = createClusterSpecification(activeCommandLine, commandLine); + clusterId = activeCommandLine.getClusterId(executorConfig); + clusterSpec = activeCommandLine.getClusterSpecification(executorConfig); } public SessionContext getSessionContext() { @@ -182,8 +183,8 @@ public T getClusterId() { return clusterId; } - public ClusterDescriptor createClusterDescriptor() throws Exception { - return activeCommandLine.createClusterDescriptor(commandLine); + public ClusterDescriptor createClusterDescriptor() { + return activeCommandLine.createClusterDescriptor(executorConfig); } public EnvironmentInstance createEnvironmentInstance() { @@ -244,14 +245,6 @@ private static RunOptions createRunOptions(CommandLine commandLine) { } } - private static ClusterSpecification createClusterSpecification(CustomCommandLine activeCommandLine, CommandLine commandLine) { - try { - return activeCommandLine.getClusterSpecification(commandLine); - } catch (FlinkException e) { - throw new SqlExecutionException("Could not create cluster specification for the given deployment.", e); - } - } - private Catalog createCatalog(String name, Map catalogProperties, ClassLoader classLoader) { final CatalogFactory factory = TableFactoryService.find(CatalogFactory.class, catalogProperties, classLoader); 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 28a8d5aac4ef..e16abc017f64 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 @@ -23,12 +23,10 @@ import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; -import org.apache.flink.util.FlinkException; import org.apache.flink.yarn.cli.FlinkYarnSessionCli; import org.apache.flink.yarn.util.FakeClusterClient; 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.conf.YarnConfiguration; import org.junit.AfterClass; @@ -90,7 +88,6 @@ public void testRun() throws Exception { private static class TestingFlinkYarnSessionCli extends FlinkYarnSessionCli { private final ClusterClient clusterClient; - private final String configurationDirectory; private TestingFlinkYarnSessionCli( Configuration configuration, @@ -99,18 +96,15 @@ private TestingFlinkYarnSessionCli( String longPrefix) throws Exception { super(configuration, configurationDirectory, shortPrefix, longPrefix); - this.clusterClient = new FakeClusterClient(); - this.configurationDirectory = configurationDirectory; + this.clusterClient = new FakeClusterClient(configuration); } @Override - public YarnClusterDescriptor createClusterDescriptor(CommandLine commandLine) - throws FlinkException { - YarnClusterDescriptor parent = super.createClusterDescriptor(commandLine); + public YarnClusterDescriptor createClusterDescriptor(Configuration configuration) { + YarnClusterDescriptor parent = super.createClusterDescriptor(configuration); return new NonDeployingYarnClusterDescriptor( parent.getFlinkConfiguration(), (YarnConfiguration) parent.getYarnClient().getConfig(), - configurationDirectory, parent.getYarnClient(), clusterClient); } 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 e894f654a5ba..6cf6fb62c64b 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 @@ -22,7 +22,6 @@ import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.AkkaOptions; -import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmaster.JobResult; @@ -72,7 +71,6 @@ public void testPerJobMode() throws Exception { try (final YarnClusterDescriptor yarnClusterDescriptor = new YarnClusterDescriptor( configuration, getYarnConfiguration(), - System.getenv(ConfigConstants.ENV_FLINK_CONF_DIR), yarnClient, true)) { 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 7459bac03047..2aa476a74fb9 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 @@ -19,7 +19,6 @@ package org.apache.flink.yarn; import org.apache.flink.api.common.time.Time; -import org.apache.flink.client.cli.CliFrontend; import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.PackagedProgram; @@ -98,7 +97,6 @@ public void testFlinkContainerMemory() throws Exception { final YarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor( configuration, yarnConfiguration, - CliFrontend.getConfigurationDirectoryFromEnv(), yarnClient, true); 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 6481420d2017..80f6efc72560 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 @@ -307,7 +307,6 @@ YarnClusterDescriptor createYarnClusterDescriptor(org.apache.flink.configuration final YarnClusterDescriptor yarnClusterDescriptor = new YarnClusterDescriptor( flinkConfiguration, YARN_CONFIGURATION, - CliFrontend.getConfigurationDirectoryFromEnv(), yarnClient, true); yarnClusterDescriptor.setLocalJarPath(new Path(flinkUberjar.toURI())); 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 26719362c803..48e05f8a8e5e 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 @@ -40,10 +40,9 @@ public class NonDeployingYarnClusterDescriptor extends YarnClusterDescriptor { public NonDeployingYarnClusterDescriptor( Configuration flinkConfiguration, YarnConfiguration yarnConfiguration, - String configurationDirectory, YarnClient yarnClient, ClusterClient clusterClient) { - super(flinkConfiguration, yarnConfiguration, configurationDirectory, yarnClient, true); + super(flinkConfiguration, yarnConfiguration, yarnClient, true); this.clusterClient = Preconditions.checkNotNull(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 0bcc52b20b91..e64b3bc895bb 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 @@ -47,6 +47,7 @@ import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; import org.apache.flink.util.ShutdownHookUtil; +import org.apache.flink.yarn.cli.YarnConfigUtils; import org.apache.flink.yarn.configuration.YarnConfigOptions; import org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint; import org.apache.flink.yarn.entrypoint.YarnSessionClusterEntrypoint; @@ -130,8 +131,6 @@ public class YarnClusterDescriptor implements ClusterDescriptor { private String yarnQueue; - private String configurationDirectory; - private Path flinkJarPath; private String dynamicPropertiesEncoded; @@ -153,7 +152,6 @@ public class YarnClusterDescriptor implements ClusterDescriptor { public YarnClusterDescriptor( Configuration flinkConfiguration, YarnConfiguration yarnConfiguration, - String configurationDirectory, YarnClient yarnClient, boolean sharedYarnClient) { @@ -163,8 +161,6 @@ public YarnClusterDescriptor( this.flinkConfiguration = Preconditions.checkNotNull(flinkConfiguration); this.userJarInclusion = getUserJarInclusionMode(flinkConfiguration); - - this.configurationDirectory = Preconditions.checkNotNull(configurationDirectory); } @VisibleForTesting @@ -236,9 +232,6 @@ private void isReadyForDeployment(ClusterSpecification clusterSpecification) thr 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"); } @@ -710,22 +703,10 @@ private ApplicationReport startAppMaster( 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."); - } + final List logConfigFiles = YarnConfigUtils + .decodeListFromConfig(configuration, YarnConfigOptions.APPLICATION_LOG_CONFIG_FILES, File::new); + if (logConfigFiles != null) { + systemShipFiles.addAll(logConfigFiles); } addEnvironmentFoldersToShipFiles(systemShipFiles); @@ -945,8 +926,8 @@ private ApplicationReport startAppMaster( final ContainerLaunchContext amContainer = setupApplicationMasterContainer( yarnClusterEntrypoint, - hasLogback, - hasLog4j, + containsFileWithEnding(systemShipFiles, CONFIG_FILE_LOGBACK_NAME), + containsFileWithEnding(systemShipFiles, CONFIG_FILE_LOG4J_NAME), hasKrb5, clusterSpecification.getMasterMemoryMB()); 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 8d6916af802b..61e3d589db4d 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 @@ -26,6 +26,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ConfigurationUtils; import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.DeploymentOptions; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.MemorySize; @@ -37,7 +38,6 @@ import org.apache.flink.util.ExceptionUtils; 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.YarnClusterDescriptor; import org.apache.flink.yarn.configuration.YarnConfigOptions; @@ -52,7 +52,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.client.api.YarnClient; -import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.util.ConverterUtils; import org.slf4j.Logger; @@ -72,12 +71,13 @@ import java.lang.reflect.UndeclaredThrowableException; import java.net.URLDecoder; import java.nio.charset.Charset; -import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.Set; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -87,6 +87,7 @@ import static org.apache.flink.client.cli.CliFrontendParser.SHUTDOWN_IF_ATTACHED_OPTION; import static org.apache.flink.client.cli.CliFrontendParser.YARN_DETACHED_OPTION; import static org.apache.flink.configuration.HighAvailabilityOptions.HA_CLUSTER_ID; +import static org.apache.flink.util.Preconditions.checkNotNull; /** * Class handling the command line interface to the YARN session. @@ -158,8 +159,6 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine when testing we execute this class without a flink-dist jar - if (decodedPath.endsWith(".jar")) { - localJarPath = new Path(new File(decodedPath).toURI()); - } else { - localJarPath = null; - } - } + private YarnClusterDescriptor createDescriptor(Configuration configuration) { + YarnClusterDescriptor yarnClusterDescriptor = new YarnClusterDescriptor(configuration); + final Path localJarPath = getLocalFlinkDistPath(configuration, yarnClusterDescriptor); if (localJarPath != null) { yarnClusterDescriptor.setLocalJarPath(localJarPath); } - List shipFiles = new ArrayList<>(); - // path to directories to ship - if (cmd.hasOption(shipPath.getOpt())) { - String[] shipPaths = cmd.getOptionValues(this.shipPath.getOpt()); - for (String shipPath : shipPaths) { - File shipDir = new File(shipPath); - if (shipDir.isDirectory()) { - shipFiles.add(shipDir); - } else { - LOG.warn("Ship directory {} is not a directory. Ignoring it.", shipDir.getAbsolutePath()); - } - } - } - + final List shipFiles = decodeDirsToShipToCluster(configuration); yarnClusterDescriptor.addShipFiles(shipFiles); - // queue - if (cmd.hasOption(queue.getOpt())) { - yarnClusterDescriptor.setQueue(cmd.getOptionValue(queue.getOpt())); + final String queueName = configuration.getString(YarnConfigOptions.APPLICATION_QUEUE); + if (queueName != null) { + yarnClusterDescriptor.setQueue(queueName); } - final Properties properties = cmd.getOptionProperties(dynamicproperties.getOpt()); + final String dynamicPropertiesEncoded = configuration.getString(YarnConfigOptions.DYNAMIC_PROPERTIES); + yarnClusterDescriptor.setDynamicPropertiesEncoded(dynamicPropertiesEncoded); - for (String key : properties.stringPropertyNames()) { - LOG.info("Dynamic Property set: {}={}", key, GlobalConfiguration.isSensitive(key) ? GlobalConfiguration.HIDDEN_CONTENT : properties.getProperty(key)); + final boolean detached = !configuration.getBoolean(DeploymentOptions.ATTACHED); + yarnClusterDescriptor.setDetachedMode(detached); + + final String appName = configuration.getString(YarnConfigOptions.APPLICATION_NAME); + if (appName != null) { + yarnClusterDescriptor.setName(appName); } - String[] dynamicProperties = properties.stringPropertyNames().stream() - .flatMap( - (String key) -> { - final String value = properties.getProperty(key); + final String appType = configuration.getString(YarnConfigOptions.APPLICATION_TYPE); + if (appType != null) { + yarnClusterDescriptor.setApplicationType(appType); + } - if (value != null) { - return Stream.of(key + dynamicproperties.getValueSeparator() + value); - } else { - return Stream.empty(); - } - }) - .toArray(String[]::new); + final String zkNamespace = configuration.getString(HA_CLUSTER_ID); + if (zkNamespace != null) { + yarnClusterDescriptor.setZookeeperNamespace(zkNamespace); + } - String dynamicPropertiesEncoded = StringUtils.join(dynamicProperties, YARN_DYNAMIC_PROPERTIES_SEPARATOR); + final String nodeLabel = configuration.getString(YarnConfigOptions.NODE_LABEL); + if (nodeLabel != null) { + yarnClusterDescriptor.setNodeLabel(nodeLabel); + } - yarnClusterDescriptor.setDynamicPropertiesEncoded(dynamicPropertiesEncoded); + return yarnClusterDescriptor; + } - if (cmd.hasOption(YARN_DETACHED_OPTION.getOpt()) || cmd.hasOption(DETACHED_OPTION.getOpt())) { - yarnClusterDescriptor.setDetachedMode(true); + private Path getLocalFlinkDistPath(final Configuration configuration, final YarnClusterDescriptor yarnClusterDescriptor) { + final String localJarPath = configuration.getString(YarnConfigOptions.FLINK_DIST_JAR); + if (localJarPath != null) { + return new Path(localJarPath); } - if (cmd.hasOption(name.getOpt())) { - yarnClusterDescriptor.setName(cmd.getOptionValue(name.getOpt())); - } + LOG.info("No path for the flink jar passed. Using the location of " + yarnClusterDescriptor.getClass() + " to locate the jar"); + + // check whether it's actually a jar file --> when testing we execute this class without a flink-dist jar + final String decodedPath = getDecodedJarPath(yarnClusterDescriptor); + return decodedPath.endsWith(".jar") + ? new Path(new File(decodedPath).toURI()) + : null; + } - if (cmd.hasOption(applicationType.getOpt())) { - yarnClusterDescriptor.setApplicationType(cmd.getOptionValue(applicationType.getOpt())); + private String getDecodedJarPath(final YarnClusterDescriptor yarnClusterDescriptor) { + final String encodedJarPath = yarnClusterDescriptor + .getClass().getProtectionDomain().getCodeSource().getLocation().getPath(); + try { + return URLDecoder.decode(encodedJarPath, Charset.defaultCharset().name()); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException("Couldn't decode the encoded Flink dist jar path: " + encodedJarPath + + " Please supply a path manually via the -" + flinkJar.getOpt() + " option."); } + } - if (cmd.hasOption(zookeeperNamespace.getOpt())) { - String zookeeperNamespaceValue = cmd.getOptionValue(this.zookeeperNamespace.getOpt()); - yarnClusterDescriptor.setZookeeperNamespace(zookeeperNamespaceValue); + private Path getLocalFlinkDistPathFromCmd(final CommandLine cmd) { + final String flinkJarOptionName = flinkJar.getOpt(); + if (!cmd.hasOption(flinkJarOptionName)) { + return null; } - if (cmd.hasOption(nodeLabel.getOpt())) { - String nodeLabelValue = cmd.getOptionValue(this.nodeLabel.getOpt()); - yarnClusterDescriptor.setNodeLabel(nodeLabelValue); + String userPath = cmd.getOptionValue(flinkJarOptionName); + if (!userPath.startsWith("file://")) { + userPath = "file://" + userPath; } + return new Path(userPath); + } - return yarnClusterDescriptor; + private List decodeDirsToShipToCluster(final Configuration configuration) { + checkNotNull(configuration); + return YarnConfigUtils.decodeListFromConfig(configuration, YarnConfigOptions.SHIP_DIRECTORIES, File::new); + } + + private void encodeDirsToShipToCluster(final Configuration configuration, final CommandLine cmd) { + checkNotNull(cmd); + checkNotNull(configuration); + + if (cmd.hasOption(shipPath.getOpt())) { + YarnConfigUtils.encodeListToConfig( + configuration, + YarnConfigOptions.SHIP_DIRECTORIES, + cmd.getOptionValues(this.shipPath.getOpt()), + (String path) -> { + final File shipDir = new File(path); + if (shipDir.isDirectory()) { + return path; + } + LOG.warn("Ship directory {} is not a directory. Ignoring it.", shipDir.getAbsolutePath()); + return null; + }); + } } - private ClusterSpecification createClusterSpecification(Configuration configuration, CommandLine cmd) { + private ClusterSpecification createClusterSpecification(Configuration configuration) { // JobManager Memory final int jobManagerMemoryMB = ConfigurationUtils.getJobManagerHeapMemory(configuration).getMebiBytes(); @@ -433,47 +424,30 @@ public void addGeneralOptions(Options baseOptions) { } @Override - public YarnClusterDescriptor createClusterDescriptor(CommandLine commandLine) throws FlinkException { - final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(commandLine); - - return createDescriptor( - effectiveConfiguration, - yarnConfiguration, - configurationDirectory, - commandLine); + public YarnClusterDescriptor createClusterDescriptor(Configuration configuration) { + return createDescriptor(configuration); } @Override @Nullable - public ApplicationId getClusterId(CommandLine commandLine) { - if (commandLine.hasOption(applicationId.getOpt())) { - return ConverterUtils.toApplicationId(commandLine.getOptionValue(applicationId.getOpt())); - } else if (isYarnPropertiesFileMode(commandLine)) { - return yarnApplicationIdFromYarnProperties; - } else { - return null; - } + public ApplicationId getClusterId(Configuration configuration) { + final String clusterId = configuration.getString(YarnConfigOptions.APPLICATION_ID); + return clusterId != null ? ConverterUtils.toApplicationId(clusterId) : null; } @Override - public ClusterSpecification getClusterSpecification(CommandLine commandLine) throws FlinkException { - final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(commandLine); - - return createClusterSpecification(effectiveConfiguration, commandLine); + public ClusterSpecification getClusterSpecification(Configuration configuration) { + return createClusterSpecification(configuration); } @Override - protected Configuration applyCommandLineOptionsToConfiguration(CommandLine commandLine) throws FlinkException { + public Configuration applyCommandLineOptionsToConfiguration(CommandLine commandLine) throws FlinkException { // we ignore the addressOption because it can only contain "yarn-cluster" final Configuration effectiveConfiguration = new Configuration(configuration); - if (commandLine.hasOption(zookeeperNamespaceOption.getOpt())) { - String zkNamespace = commandLine.getOptionValue(zookeeperNamespaceOption.getOpt()); - effectiveConfiguration.setString(HA_CLUSTER_ID, zkNamespace); - } - - final ApplicationId applicationId = getClusterId(commandLine); + applyDescriptorOptionToConfig(commandLine, effectiveConfiguration); + final ApplicationId applicationId = getApplicationId(commandLine); if (applicationId != null) { final String zooKeeperNamespace; if (commandLine.hasOption(zookeeperNamespace.getOpt())){ @@ -483,6 +457,7 @@ protected Configuration applyCommandLineOptionsToConfiguration(CommandLine comma } effectiveConfiguration.setString(HA_CLUSTER_ID, zooKeeperNamespace); + effectiveConfiguration.setString(YarnConfigOptions.APPLICATION_ID, ConverterUtils.toString(applicationId)); } if (commandLine.hasOption(jmMemory.getOpt())) { @@ -512,6 +487,89 @@ protected Configuration applyCommandLineOptionsToConfiguration(CommandLine comma } } + private ApplicationId getApplicationId(CommandLine commandLine) { + if (commandLine.hasOption(applicationId.getOpt())) { + return ConverterUtils.toApplicationId(commandLine.getOptionValue(applicationId.getOpt())); + } else if (isYarnPropertiesFileMode(commandLine)) { + return yarnApplicationIdFromYarnProperties; + } + return null; + } + + private void applyDescriptorOptionToConfig(final CommandLine commandLine, final Configuration configuration) { + checkNotNull(commandLine); + checkNotNull(configuration); + + final Path localJarPath = getLocalFlinkDistPathFromCmd(commandLine); + if (localJarPath != null) { + configuration.setString(YarnConfigOptions.FLINK_DIST_JAR, localJarPath.toString()); + } + + encodeDirsToShipToCluster(configuration, commandLine); + + if (commandLine.hasOption(queue.getOpt())) { + final String queueName = commandLine.getOptionValue(queue.getOpt()); + configuration.setString(YarnConfigOptions.APPLICATION_QUEUE, queueName); + } + + final String dynamicPropertiesEncoded = encodeDynamicProperties(commandLine); + configuration.setString(YarnConfigOptions.DYNAMIC_PROPERTIES, dynamicPropertiesEncoded); + + final boolean detached = commandLine.hasOption(YARN_DETACHED_OPTION.getOpt()) || commandLine.hasOption(DETACHED_OPTION.getOpt()); + configuration.setBoolean(DeploymentOptions.ATTACHED, !detached); + + if (commandLine.hasOption(name.getOpt())) { + final String appName = commandLine.getOptionValue(name.getOpt()); + configuration.setString(YarnConfigOptions.APPLICATION_NAME, appName); + } + + if (commandLine.hasOption(applicationType.getOpt())) { + final String appType = commandLine.getOptionValue(applicationType.getOpt()); + configuration.setString(YarnConfigOptions.APPLICATION_TYPE, appType); + } + + if (commandLine.hasOption(zookeeperNamespace.getOpt())) { + String zookeeperNamespaceValue = commandLine.getOptionValue(zookeeperNamespace.getOpt()); + configuration.setString(HA_CLUSTER_ID, zookeeperNamespaceValue); + } else if (commandLine.hasOption(zookeeperNamespaceOption.getOpt())) { + String zookeeperNamespaceValue = commandLine.getOptionValue(zookeeperNamespaceOption.getOpt()); + configuration.setString(HA_CLUSTER_ID, zookeeperNamespaceValue); + } + + if (commandLine.hasOption(nodeLabel.getOpt())) { + final String nodeLabelValue = commandLine.getOptionValue(this.nodeLabel.getOpt()); + configuration.setString(YarnConfigOptions.NODE_LABEL, nodeLabelValue); + } + + discoverAndEncodeLogConfigFiles(configuration); + } + + private void discoverAndEncodeLogConfigFiles(final Configuration configuration) { + final Set logFiles = discoverLogConfigFiles(); + YarnConfigUtils.encodeListToConfig(configuration, YarnConfigOptions.APPLICATION_LOG_CONFIG_FILES, logFiles, File::getPath); + } + + private Set discoverLogConfigFiles() { + final Set logConfigFiles = new HashSet<>(); + + File logbackFile = new File(configurationDirectory + File.separator + CONFIG_FILE_LOGBACK_NAME); + final boolean hasLogback = logbackFile.exists(); + if (hasLogback) { + logConfigFiles.add(logbackFile); + } + + File log4jFile = new File(configurationDirectory + File.separator + CONFIG_FILE_LOG4J_NAME); + final boolean hasLog4j = log4jFile.exists(); + if (hasLog4j) { + logConfigFiles.add(log4jFile); + if (hasLogback) { + LOG.warn("The configuration directory ('" + configurationDirectory + "') contains both LOG4J and " + + "Logback configuration files. Please delete or rename one of them."); + } + } + return logConfigFiles; + } + private boolean isYarnPropertiesFileMode(CommandLine commandLine) { boolean canApplyYarnProperties = !commandLine.hasOption(addressOption.getOpt()); @@ -573,7 +631,8 @@ public int run(String[] args) throws CliArgsException, FlinkException { return 0; } - final YarnClusterDescriptor yarnClusterDescriptor = createClusterDescriptor(cmd); + final Configuration configuration = applyCommandLineOptionsToConfiguration(cmd); + final YarnClusterDescriptor yarnClusterDescriptor = createClusterDescriptor(configuration); try { // Query cluster for metrics @@ -590,7 +649,7 @@ public int run(String[] args) throws CliArgsException, FlinkException { clusterClient = yarnClusterDescriptor.retrieve(yarnApplicationId); } else { - final ClusterSpecification clusterSpecification = getClusterSpecification(cmd); + final ClusterSpecification clusterSpecification = getClusterSpecification(configuration); clusterClient = yarnClusterDescriptor.deploySessionCluster(clusterSpecification); @@ -772,6 +831,26 @@ private void logAndSysout(String message) { System.out.println(message); } + private String encodeDynamicProperties(final CommandLine cmd) { + final Properties properties = cmd.getOptionProperties(dynamicproperties.getOpt()); + final String[] dynamicProperties = properties.stringPropertyNames().stream() + .flatMap( + (String key) -> { + final String value = properties.getProperty(key); + + LOG.info("Dynamic Property set: {}={}", key, GlobalConfiguration.isSensitive(key) ? GlobalConfiguration.HIDDEN_CONTENT : value); + + if (value != null) { + return Stream.of(key + dynamicproperties.getValueSeparator() + value); + } else { + return Stream.empty(); + } + }) + .toArray(String[]::new); + + return StringUtils.join(dynamicProperties, YARN_DYNAMIC_PROPERTIES_SEPARATOR); + } + public static Map getDynamicProperties(String dynamicPropertiesEncoded) { if (dynamicPropertiesEncoded != null && dynamicPropertiesEncoded.length() > 0) { Map properties = new HashMap<>(); @@ -958,20 +1037,4 @@ public static File getYarnPropertiesLocation(@Nullable String yarnPropertiesFile return new File(propertiesFileLocation, YARN_PROPERTIES_FILE + currentUser); } - - private YarnClusterDescriptor getClusterDescriptor( - Configuration configuration, - YarnConfiguration yarnConfiguration, - String configurationDirectory) { - final YarnClient yarnClient = YarnClient.createYarnClient(); - yarnClient.init(yarnConfiguration); - yarnClient.start(); - - return new YarnClusterDescriptor( - configuration, - yarnConfiguration, - configurationDirectory, - yarnClient, - false); - } } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnConfigUtils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnConfigUtils.java new file mode 100644 index 000000000000..edb02096e35f --- /dev/null +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnConfigUtils.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.yarn.cli; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Utilities for parsing {@link org.apache.flink.configuration.ConfigOption configuration options}. + */ +public class YarnConfigUtils { + // TODO: 16.10.19 test this and test LOG FILES discovery. + private static final String COLLECTION_DELIMITER = ";"; + + public static void encodeListToConfig( + final Configuration configuration, + final ConfigOption key, + final Collection value, + final Function mapper) { + encodeListToConfig(configuration, key, value.stream(), mapper); + } + + public static void encodeListToConfig( + final Configuration configuration, + final ConfigOption key, + final T[] value, + final Function mapper) { + encodeListToConfig(configuration, key, Arrays.stream(value), mapper); + } + + private static void encodeListToConfig( + final Configuration configuration, + final ConfigOption key, + final Stream values, + final Function mapper) { + + checkNotNull(values); + checkNotNull(key); + checkNotNull(configuration); + + final String encodedString = values.map(mapper).filter(Objects::nonNull).collect(Collectors.joining(COLLECTION_DELIMITER)); + if (encodedString != null && !encodedString.isEmpty()) { + configuration.setString(key, encodedString); + } + } + + public static List decodeListFromConfig( + final Configuration configuration, + final ConfigOption key, + final Function mapper) { + + checkNotNull(configuration); + checkNotNull(key); + + final String encodedString = configuration.getString(key); + return encodedString != null + ? Arrays.stream(encodedString.split(COLLECTION_DELIMITER)).map(mapper).collect(Collectors.toList()) + : Collections.emptyList(); + } +} diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java b/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java index 035bb0497087..d8611ccb9615 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java @@ -204,6 +204,54 @@ public class YarnConfigOptions { .defaultValue("") .withDescription("A comma-separated list of tags to apply to the Flink YARN application."); + // ----------------------- YARN CLI OPTIONS ------------------------------------ + + public static final ConfigOption APPLICATION_LOG_CONFIG_FILES = + key("yarn.log-config-file") + .noDefaultValue() + .withDescription("The location of the log config file, e.g. the path to your log4j.properties for log4j."); + + public static final ConfigOption DYNAMIC_PROPERTIES = + key("$internal.yarn.dynamic-properties") + .noDefaultValue() + .withDescription("**DO NOT USE** Specify YARN dynamic properties."); + + public static final ConfigOption SHIP_DIRECTORIES = + key("yarn.ship-directories") + .noDefaultValue() + .withDescription("A semicolon-separated list of directories to be shipped to the YARN cluster."); + + public static final ConfigOption FLINK_DIST_JAR = + key("yarn.flink-dist-jar") + .noDefaultValue() + .withDescription("The location of the Flink dist jar."); + + public static final ConfigOption APPLICATION_ID = + key("yarn.application.id") + .noDefaultValue() + .withDescription("The YARN application id of the running yarn cluster." + + " This is the YARN cluster where the pipeline is going to be executed."); + + public static final ConfigOption APPLICATION_QUEUE = + key("yarn.application.queue") + .noDefaultValue() + .withDescription("The YARN queue on which to put the current pipeline."); + + public static final ConfigOption APPLICATION_NAME = + key("yarn.application.name") + .noDefaultValue() + .withDescription("A custom name for your YARN application."); + + public static final ConfigOption APPLICATION_TYPE = + key("yarn.application.type") + .noDefaultValue() + .withDescription("A custom type for your YARN application.."); + + public static final ConfigOption NODE_LABEL = + key("yarn.application.node-label") + .noDefaultValue() + .withDescription("Specify YARN node label for the YARN application."); + // ------------------------------------------------------------------------ /** This class is not meant to be instantiated. */ diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java index 9009a53b8771..1ba20083178c 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java @@ -67,7 +67,6 @@ public void testClusterClientRetrievalOfFinishedYarnApplication() throws Excepti final YarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor( new Configuration(), yarnConfiguration, - temporaryFolder.newFolder().getAbsolutePath(), yarnClient, false); 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 23388751133a..ae566b4b2c06 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,8 @@ public void testDynamicProperties() throws Exception { 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"}); - YarnClusterDescriptor flinkYarnDescriptor = cli.createClusterDescriptor(cmd); + Configuration executorConfig = cli.applyCommandLineOptionsToConfiguration(cmd); + YarnClusterDescriptor flinkYarnDescriptor = cli.createClusterDescriptor(executorConfig); Assert.assertNotNull(flinkYarnDescriptor); @@ -109,10 +110,10 @@ public void testCorrectSettingOfMaxSlots() throws Exception { "yarn"); final CommandLine commandLine = yarnCLI.parseCommandLineOptions(params, true); + final Configuration executorConfig = yarnCLI.applyCommandLineOptionsToConfiguration(commandLine); - YarnClusterDescriptor descriptor = yarnCLI.createClusterDescriptor(commandLine); - - final ClusterSpecification clusterSpecification = yarnCLI.getClusterSpecification(commandLine); + final YarnClusterDescriptor descriptor = yarnCLI.createClusterDescriptor(executorConfig); + final ClusterSpecification clusterSpecification = yarnCLI.getClusterSpecification(executorConfig); // each task manager has 3 slots but the parallelism is 7. Thus the slots should be increased. assertEquals(3, clusterSpecification.getSlotsPerTaskManager()); @@ -131,8 +132,9 @@ public void testCorrectSettingOfDetachedMode() throws Exception { "yarn"); final CommandLine commandLine = yarnCLI.parseCommandLineOptions(params, true); + final Configuration executorConfig = yarnCLI.applyCommandLineOptionsToConfiguration(commandLine); - YarnClusterDescriptor descriptor = yarnCLI.createClusterDescriptor(commandLine); + YarnClusterDescriptor descriptor = yarnCLI.createClusterDescriptor(executorConfig); // each task manager has 3 slots but the parallelism is 7. Thus the slots should be increased. assertTrue(descriptor.isDetachedMode()); @@ -151,8 +153,9 @@ public void testZookeeperNamespaceProperty() throws Exception { "yarn"); CommandLine commandLine = yarnCLI.parseCommandLineOptions(params, true); + Configuration executorConfig = yarnCLI.applyCommandLineOptionsToConfiguration(commandLine); - YarnClusterDescriptor descriptor = yarnCLI.createClusterDescriptor(commandLine); + YarnClusterDescriptor descriptor = yarnCLI.createClusterDescriptor(executorConfig); assertEquals(zkNamespaceCliInput, descriptor.getZookeeperNamespace()); } @@ -170,8 +173,9 @@ public void testNodeLabelProperty() throws Exception { "yarn"); CommandLine commandLine = yarnCLI.parseCommandLineOptions(params, true); + Configuration executorConfig = yarnCLI.applyCommandLineOptionsToConfiguration(commandLine); - YarnClusterDescriptor descriptor = yarnCLI.createClusterDescriptor(commandLine); + YarnClusterDescriptor descriptor = yarnCLI.createClusterDescriptor(executorConfig); assertEquals(nodeLabelCliInput, descriptor.getNodeLabel()); } @@ -194,8 +198,9 @@ public void testResumeFromYarnPropertiesFile() throws Exception { "yarn"); final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {}, true); + final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); - final ApplicationId clusterId = flinkYarnSessionCli.getClusterId(commandLine); + final ApplicationId clusterId = flinkYarnSessionCli.getClusterId(executorConfig); assertEquals(TEST_YARN_APPLICATION_ID, clusterId); } @@ -229,8 +234,9 @@ public void testResumeFromYarnID() throws Exception { "yarn"); final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {"-yid", TEST_YARN_APPLICATION_ID.toString()}, true); + final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); - final ApplicationId clusterId = flinkYarnSessionCli.getClusterId(commandLine); + final ApplicationId clusterId = flinkYarnSessionCli.getClusterId(executorConfig); assertEquals(TEST_YARN_APPLICATION_ID, clusterId); } @@ -245,8 +251,9 @@ public void testResumeFromYarnIDZookeeperNamespace() throws Exception { "yarn"); final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {"-yid", TEST_YARN_APPLICATION_ID.toString()}, true); + final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); - final YarnClusterDescriptor clusterDescriptor = flinkYarnSessionCli.createClusterDescriptor(commandLine); + final YarnClusterDescriptor clusterDescriptor = flinkYarnSessionCli.createClusterDescriptor(executorConfig); final Configuration clusterDescriptorConfiguration = clusterDescriptor.getFlinkConfiguration(); @@ -266,8 +273,9 @@ public void testResumeFromYarnIDZookeeperNamespaceOverride() throws Exception { final String overrideZkNamespace = "my_cluster"; final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {"-yid", TEST_YARN_APPLICATION_ID.toString(), "-yz", overrideZkNamespace}, true); + final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); - final YarnClusterDescriptor clusterDescriptor = flinkYarnSessionCli.createClusterDescriptor(commandLine); + final YarnClusterDescriptor clusterDescriptor = flinkYarnSessionCli.createClusterDescriptor(executorConfig); final Configuration clusterDescriptorConfiguration = clusterDescriptor.getFlinkConfiguration(); @@ -288,7 +296,9 @@ public void testYarnIDOverridesPropertiesFile() throws Exception { "y", "yarn"); final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {"-yid", TEST_YARN_APPLICATION_ID_2.toString() }, true); - final ApplicationId clusterId = flinkYarnSessionCli.getClusterId(commandLine); + final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); + + final ApplicationId clusterId = flinkYarnSessionCli.getClusterId(executorConfig); assertEquals(TEST_YARN_APPLICATION_ID_2, clusterId); } @@ -315,8 +325,9 @@ public void testCommandLineClusterSpecification() throws Exception { "yarn"); CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(args, false); + Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); - final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(commandLine); + final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(executorConfig); assertThat(clusterSpecification.getMasterMemoryMB(), is(jobManagerMemory)); assertThat(clusterSpecification.getTaskManagerMemoryMB(), is(taskManagerMemory)); @@ -345,8 +356,9 @@ public void testConfigurationClusterSpecification() throws Exception { "yarn"); CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(args, false); + Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); - final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(commandLine); + final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(executorConfig); assertThat(clusterSpecification.getMasterMemoryMB(), is(jobManagerMemory)); assertThat(clusterSpecification.getTaskManagerMemoryMB(), is(taskManagerMemory)); @@ -366,8 +378,9 @@ public void testHeapMemoryPropertyWithoutUnit() throws Exception { "yarn"); final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(args, false); + final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); - final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(commandLine); + final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(executorConfig); assertThat(clusterSpecification.getMasterMemoryMB(), is(1024)); assertThat(clusterSpecification.getTaskManagerMemoryMB(), is(2048)); @@ -385,7 +398,9 @@ public void testHeapMemoryPropertyWithUnitMB() throws Exception { "y", "yarn"); final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(args, false); - final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(commandLine); + final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); + + final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(executorConfig); assertThat(clusterSpecification.getMasterMemoryMB(), is(1024)); assertThat(clusterSpecification.getTaskManagerMemoryMB(), is(2048)); @@ -403,7 +418,9 @@ public void testHeapMemoryPropertyWithArbitraryUnit() throws Exception { "y", "yarn"); final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(args, false); - final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(commandLine); + final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); + + final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(executorConfig); assertThat(clusterSpecification.getMasterMemoryMB(), is(1024)); assertThat(clusterSpecification.getTaskManagerMemoryMB(), is(2048)); @@ -425,8 +442,9 @@ public void testHeapMemoryPropertyWithOldConfigKey() throws Exception { "yarn"); final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[0], false); + final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); - final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(commandLine); + final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(executorConfig); assertThat(clusterSpecification.getMasterMemoryMB(), is(2048)); assertThat(clusterSpecification.getTaskManagerMemoryMB(), is(4096)); @@ -444,8 +462,9 @@ public void testHeapMemoryPropertyWithConfigDefaultValue() throws Exception { "yarn"); final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[0], false); + final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); - final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(commandLine); + final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(executorConfig); assertThat(clusterSpecification.getMasterMemoryMB(), is(1024)); assertThat(clusterSpecification.getTaskManagerMemoryMB(), is(1024)); @@ -461,8 +480,9 @@ public void testMultipleYarnShipOptions() throws Exception { "yarn"); final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(args, false); + final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); - YarnClusterDescriptor flinkYarnDescriptor = flinkYarnSessionCli.createClusterDescriptor(commandLine); + YarnClusterDescriptor flinkYarnDescriptor = flinkYarnSessionCli.createClusterDescriptor(executorConfig); 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 143749b049a1..69af7c386e65 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 @@ -525,7 +525,6 @@ public void testYarnClientShutDown() { yarnClusterDescriptor = new YarnClusterDescriptor( new Configuration(), yarnConfiguration, - temporaryFolder.getRoot().getAbsolutePath(), closableYarnClient, false); @@ -542,7 +541,6 @@ private YarnClusterDescriptor createYarnClusterDescriptor(Configuration configur return new YarnClusterDescriptor( configuration, yarnConfiguration, - temporaryFolder.getRoot().getAbsolutePath(), yarnClient, true); } From 71b056f04552700c425350e4ed25eff0a4357ba3 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Tue, 22 Oct 2019 22:24:27 +0200 Subject: [PATCH 391/746] [FLINK-14501] Change the log config file discovery --- .../flink/yarn/YarnClusterDescriptor.java | 12 +++--- .../flink/yarn/cli/FlinkYarnSessionCli.java | 39 ++++++++----------- .../yarn/configuration/YarnConfigOptions.java | 2 +- 3 files changed, 23 insertions(+), 30 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 e64b3bc895bb..1b33324556c2 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 @@ -47,7 +47,6 @@ import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; import org.apache.flink.util.ShutdownHookUtil; -import org.apache.flink.yarn.cli.YarnConfigUtils; import org.apache.flink.yarn.configuration.YarnConfigOptions; import org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint; import org.apache.flink.yarn.entrypoint.YarnSessionClusterEntrypoint; @@ -703,10 +702,9 @@ private ApplicationReport startAppMaster( systemShipFiles.add(file.getAbsoluteFile()); } - final List logConfigFiles = YarnConfigUtils - .decodeListFromConfig(configuration, YarnConfigOptions.APPLICATION_LOG_CONFIG_FILES, File::new); - if (logConfigFiles != null) { - systemShipFiles.addAll(logConfigFiles); + final String logConfigFilePath = configuration.getString(YarnConfigOptions.APPLICATION_LOG_CONFIG_FILE); + if (logConfigFilePath != null) { + systemShipFiles.add(new File(logConfigFilePath)); } addEnvironmentFoldersToShipFiles(systemShipFiles); @@ -926,8 +924,8 @@ private ApplicationReport startAppMaster( final ContainerLaunchContext amContainer = setupApplicationMasterContainer( yarnClusterEntrypoint, - containsFileWithEnding(systemShipFiles, CONFIG_FILE_LOGBACK_NAME), - containsFileWithEnding(systemShipFiles, CONFIG_FILE_LOG4J_NAME), + logConfigFilePath != null && logConfigFilePath.endsWith(CONFIG_FILE_LOGBACK_NAME), + logConfigFilePath != null && logConfigFilePath.endsWith(CONFIG_FILE_LOG4J_NAME), hasKrb5, clusterSpecification.getMasterMemoryMB()); 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 61e3d589db4d..a831ab23a0f7 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 @@ -73,11 +73,10 @@ import java.nio.charset.Charset; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; -import java.util.Set; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -541,33 +540,29 @@ private void applyDescriptorOptionToConfig(final CommandLine commandLine, final configuration.setString(YarnConfigOptions.NODE_LABEL, nodeLabelValue); } - discoverAndEncodeLogConfigFiles(configuration); + discoverLogConfigFile().ifPresent( + file -> configuration.setString(YarnConfigOptions.APPLICATION_LOG_CONFIG_FILE, file.getPath()) + ); } - private void discoverAndEncodeLogConfigFiles(final Configuration configuration) { - final Set logFiles = discoverLogConfigFiles(); - YarnConfigUtils.encodeListToConfig(configuration, YarnConfigOptions.APPLICATION_LOG_CONFIG_FILES, logFiles, File::getPath); - } - - private Set discoverLogConfigFiles() { - final Set logConfigFiles = new HashSet<>(); + private Optional discoverLogConfigFile() { + Optional logConfigFile = Optional.empty(); - File logbackFile = new File(configurationDirectory + File.separator + CONFIG_FILE_LOGBACK_NAME); - final boolean hasLogback = logbackFile.exists(); - if (hasLogback) { - logConfigFiles.add(logbackFile); + final File log4jFile = new File(configurationDirectory + File.separator + CONFIG_FILE_LOG4J_NAME); + if (log4jFile.exists()) { + logConfigFile = Optional.of(log4jFile); } - File log4jFile = new File(configurationDirectory + File.separator + CONFIG_FILE_LOG4J_NAME); - final boolean hasLog4j = log4jFile.exists(); - if (hasLog4j) { - logConfigFiles.add(log4jFile); - if (hasLogback) { - LOG.warn("The configuration directory ('" + configurationDirectory + "') contains both LOG4J and " + - "Logback configuration files. Please delete or rename one of them."); + final File logbackFile = new File(configurationDirectory + File.separator + CONFIG_FILE_LOGBACK_NAME); + if (logbackFile.exists()) { + if (logConfigFile.isPresent()) { + LOG.warn("The configuration directory ('" + configurationDirectory + "') already contains a LOG4J config file." + + "If you want to use logback, then please delete or rename the log configuration file."); + } else { + logConfigFile = Optional.of(logbackFile); } } - return logConfigFiles; + return logConfigFile; } private boolean isYarnPropertiesFileMode(CommandLine commandLine) { diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java b/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java index d8611ccb9615..8ef72936eed7 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java @@ -206,7 +206,7 @@ public class YarnConfigOptions { // ----------------------- YARN CLI OPTIONS ------------------------------------ - public static final ConfigOption APPLICATION_LOG_CONFIG_FILES = + public static final ConfigOption APPLICATION_LOG_CONFIG_FILE = key("yarn.log-config-file") .noDefaultValue() .withDescription("The location of the log config file, e.g. the path to your log4j.properties for log4j."); From 476b25a5a399244f7596a79704deabc970241f79 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Tue, 22 Oct 2019 20:12:51 +0200 Subject: [PATCH 392/746] [FLINK-14501] Add the DeploymentOptions.TARGET --- docs/_includes/generated/deployment_configuration.html | 5 +++++ .../apache/flink/client/cli/AbstractCustomCommandLine.java | 2 ++ .../org/apache/flink/configuration/DeploymentOptions.java | 5 +++++ .../java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java | 1 + 4 files changed, 13 insertions(+) diff --git a/docs/_includes/generated/deployment_configuration.html b/docs/_includes/generated/deployment_configuration.html index bdac7576987d..2c652e4c1448 100644 --- a/docs/_includes/generated/deployment_configuration.html +++ b/docs/_includes/generated/deployment_configuration.html @@ -12,5 +12,10 @@ false Specifies if the pipeline is submitted in attached or detached mode. + +

    execution.target
    + (none) + The deployment target for the execution, e.g. "local" for local execution. + diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java b/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java index 805d58a75da9..cba103661245 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java @@ -19,6 +19,7 @@ package org.apache.flink.client.cli; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.DeploymentOptions; import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.configuration.UnmodifiableConfiguration; import org.apache.flink.util.FlinkException; @@ -72,6 +73,7 @@ public void addGeneralOptions(Options baseOptions) { @Override public Configuration applyCommandLineOptionsToConfiguration(CommandLine commandLine) throws FlinkException { final Configuration resultingConfiguration = new Configuration(configuration); + resultingConfiguration.setString(DeploymentOptions.TARGET, getId()); if (commandLine.hasOption(addressOption.getOpt())) { String addressWithPort = commandLine.getOptionValue(addressOption.getOpt()); diff --git a/flink-core/src/main/java/org/apache/flink/configuration/DeploymentOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/DeploymentOptions.java index 1addcc40d702..a032205cb6ab 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/DeploymentOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/DeploymentOptions.java @@ -28,6 +28,11 @@ @PublicEvolving public class DeploymentOptions { + public static final ConfigOption TARGET = + key("execution.target") + .noDefaultValue() + .withDescription("The deployment target for the execution, e.g. \"local\" for local execution."); + public static final ConfigOption ATTACHED = key("execution.attached") .defaultValue(false) 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 a831ab23a0f7..2aaa6fb00e73 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 @@ -443,6 +443,7 @@ public ClusterSpecification getClusterSpecification(Configuration configuration) public Configuration applyCommandLineOptionsToConfiguration(CommandLine commandLine) throws FlinkException { // we ignore the addressOption because it can only contain "yarn-cluster" final Configuration effectiveConfiguration = new Configuration(configuration); + effectiveConfiguration.setString(DeploymentOptions.TARGET, getId()); applyDescriptorOptionToConfig(commandLine, effectiveConfiguration); From 16d67e835d634e5cbbf9f37e76e4ab3985ba4b8f Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Tue, 22 Oct 2019 20:22:36 +0200 Subject: [PATCH 393/746] [FLINK-14501] Add the ClusterClientFactory and make it discoverable --- .../deployment/ClusterClientFactory.java | 63 ++++++++++++++++ .../ClusterClientServiceLoader.java | 36 +++++++++ .../DefaultClusterClientServiceLoader.java | 75 +++++++++++++++++++ 3 files changed, 174 insertions(+) create mode 100644 flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterClientFactory.java create mode 100644 flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterClientServiceLoader.java create mode 100644 flink-clients/src/main/java/org/apache/flink/client/deployment/DefaultClusterClientServiceLoader.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterClientFactory.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterClientFactory.java new file mode 100644 index 000000000000..36647b6adc19 --- /dev/null +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterClientFactory.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.client.deployment; + +import org.apache.flink.configuration.Configuration; + +import javax.annotation.Nullable; + +/** + * A factory containing all the necessary information for creating clients to Flink clusters. + */ +public interface ClusterClientFactory { + + /** + * Returns {@code true} if the current {@link ClusterClientFactory} is compatible with the provided configuration, + * {@code false} otherwise. + */ + boolean isCompatibleWith(Configuration configuration); + + /** + * Create a {@link ClusterDescriptor} from the given configuration. + * + * @param configuration containing the configuration options relevant for the {@link ClusterDescriptor} + * @return the corresponding {@link ClusterDescriptor}. + */ + ClusterDescriptor createClusterDescriptor(Configuration configuration); + + /** + * Returns the cluster id if a cluster id is specified in the provided configuration, otherwise it returns {@code null}. + * + *

    A cluster id identifies a running cluster, e.g. the Yarn application id for a Flink cluster running on Yarn. + * + * @param configuration containing the configuration options relevant for the cluster id retrieval + * @return Cluster id identifying the cluster to deploy jobs to or null + */ + @Nullable + ClusterID getClusterId(Configuration configuration); + + /** + * Returns the {@link ClusterSpecification} specified by the configuration and the command + * line options. This specification can be used to deploy a new Flink cluster. + * + * @param configuration containing the configuration options relevant for the {@link ClusterSpecification} + * @return the corresponding {@link ClusterSpecification} for a new Flink cluster + */ + ClusterSpecification getClusterSpecification(Configuration configuration); +} diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterClientServiceLoader.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterClientServiceLoader.java new file mode 100644 index 000000000000..51eef13d009a --- /dev/null +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterClientServiceLoader.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.deployment; + +import org.apache.flink.configuration.Configuration; + +/** + * An interface used to discover the appropriate {@link ClusterClientFactory cluster client factory} based on the + * provided {@link Configuration}. + */ +public interface ClusterClientServiceLoader { + + /** + * Discovers the appropriate {@link ClusterClientFactory} based on the provided configuration. + * + * @param configuration the configuration based on which the appropriate factory is going to be used. + * @return the appropriate {@link ClusterClientFactory}. + */ + ClusterClientFactory getClusterClientFactory(final Configuration configuration); +} diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/DefaultClusterClientServiceLoader.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/DefaultClusterClientServiceLoader.java new file mode 100644 index 000000000000..574aeaf7d234 --- /dev/null +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/DefaultClusterClientServiceLoader.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.client.deployment; + +import org.apache.flink.configuration.Configuration; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.ServiceLoader; +import java.util.stream.Collectors; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A service provider for {@link ClusterClientFactory cluster client factories}. + */ +public class DefaultClusterClientServiceLoader implements ClusterClientServiceLoader { + + private static final Logger LOG = LoggerFactory.getLogger(DefaultClusterClientServiceLoader.class); + + private static final ServiceLoader defaultLoader = ServiceLoader.load(ClusterClientFactory.class); + + @Override + public ClusterClientFactory getClusterClientFactory(final Configuration configuration) { + checkNotNull(configuration); + + final List compatibleFactories = new ArrayList<>(); + final Iterator factories = defaultLoader.iterator(); + while (factories.hasNext()) { + try { + final ClusterClientFactory factory = factories.next(); + if (factory != null && factory.isCompatibleWith(configuration)) { + compatibleFactories.add(factory); + } + } catch (Throwable e) { + if (e.getCause() instanceof NoClassDefFoundError) { + LOG.info("Could not load factory due to missing dependencies."); + } else { + throw e; + } + } + } + + if (compatibleFactories.size() > 1) { + final List configStr = + configuration.toMap().entrySet().stream() + .map(e -> e.getKey() + "=" + e.getValue()) + .collect(Collectors.toList()); + + throw new IllegalStateException("Multiple compatible client factories found for:\n" + String.join("\n", configStr) + "."); + } + + return compatibleFactories.isEmpty() ? null : (ClusterClientFactory) compatibleFactories.get(0); + } +} From d1aa63fe520d3608757f098ef65f73e939029de3 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Tue, 22 Oct 2019 20:38:28 +0200 Subject: [PATCH 394/746] [FLINK-14501] Add Standalone and Yarn ClusterClientFactories --- .../deployment/StandaloneClientFactory.java | 59 ++++++ ...ink.client.deployment.ClusterClientFactory | 16 ++ .../ClusterClientServiceLoaderTest.java | 167 +++++++++++++++++ ...ink.client.deployment.ClusterClientFactory | 18 ++ .../flink/yarn/YarnClusterClientFactory.java | 173 ++++++++++++++++++ ...ink.client.deployment.ClusterClientFactory | 16 ++ .../yarn/YarnClusterClientFactoryTest.java | 47 +++++ 7 files changed, 496 insertions(+) create mode 100644 flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClientFactory.java create mode 100644 flink-clients/src/main/resources/META-INF/services/org.apache.flink.client.deployment.ClusterClientFactory create mode 100644 flink-clients/src/test/java/org/apache/flink/client/deployment/ClusterClientServiceLoaderTest.java create mode 100644 flink-clients/src/test/resources/META-INF/services/org.apache.flink.client.deployment.ClusterClientFactory create mode 100644 flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClientFactory.java create mode 100644 flink-yarn/src/main/resources/META-INF/services/org.apache.flink.client.deployment.ClusterClientFactory create mode 100644 flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterClientFactoryTest.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClientFactory.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClientFactory.java new file mode 100644 index 000000000000..b441a63a825f --- /dev/null +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClientFactory.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.client.deployment; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.DeploymentOptions; + +import javax.annotation.Nullable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link ClusterClientFactory} for a standalone cluster, i.e. Flink on bare-metal. + */ +public class StandaloneClientFactory implements ClusterClientFactory { + + public static final String ID = "default"; + + @Override + public boolean isCompatibleWith(Configuration configuration) { + checkNotNull(configuration); + return ID.equals(configuration.getString(DeploymentOptions.TARGET)); + } + + @Override + public StandaloneClusterDescriptor createClusterDescriptor(Configuration configuration) { + checkNotNull(configuration); + return new StandaloneClusterDescriptor(configuration); + } + + @Override + @Nullable + public StandaloneClusterId getClusterId(Configuration configuration) { + checkNotNull(configuration); + return StandaloneClusterId.getInstance(); + } + + @Override + public ClusterSpecification getClusterSpecification(Configuration configuration) { + checkNotNull(configuration); + return new ClusterSpecification.ClusterSpecificationBuilder().createClusterSpecification(); + } +} diff --git a/flink-clients/src/main/resources/META-INF/services/org.apache.flink.client.deployment.ClusterClientFactory b/flink-clients/src/main/resources/META-INF/services/org.apache.flink.client.deployment.ClusterClientFactory new file mode 100644 index 000000000000..fd9e4fab19a7 --- /dev/null +++ b/flink-clients/src/main/resources/META-INF/services/org.apache.flink.client.deployment.ClusterClientFactory @@ -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.client.deployment.StandaloneClientFactory \ No newline at end of file diff --git a/flink-clients/src/test/java/org/apache/flink/client/deployment/ClusterClientServiceLoaderTest.java b/flink-clients/src/test/java/org/apache/flink/client/deployment/ClusterClientServiceLoaderTest.java new file mode 100644 index 000000000000..45157a25094f --- /dev/null +++ b/flink-clients/src/test/java/org/apache/flink/client/deployment/ClusterClientServiceLoaderTest.java @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.client.deployment; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.DeploymentOptions; + +import org.junit.Before; +import org.junit.Test; + +import javax.annotation.Nullable; + +import static org.hamcrest.CoreMatchers.allOf; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.core.Is.is; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * Tests for the {@link DefaultClusterClientServiceLoader}. + */ +public class ClusterClientServiceLoaderTest { + + private static final String VALID_TARGET = "existing"; + private static final String AMBIGUOUS_TARGET = "duplicate"; + private static final String NON_EXISTING_TARGET = "non-existing"; + + private static final int VALID_ID = 42; + + private ClusterClientServiceLoader serviceLoaderUnderTest; + + @Before + public void init() { + serviceLoaderUnderTest = new DefaultClusterClientServiceLoader(); + } + + @Test + public void testStandaloneClusterClientFactoryDiscovery() { + final Configuration config = new Configuration(); + config.setString(DeploymentOptions.TARGET, StandaloneClientFactory.ID); + + ClusterClientFactory factory = serviceLoaderUnderTest.getClusterClientFactory(config); + assertTrue(factory instanceof StandaloneClientFactory); + } + + @Test + public void testFactoryDiscovery() { + final Configuration config = new Configuration(); + config.setString(DeploymentOptions.TARGET, VALID_TARGET); + + final ClusterClientFactory factory = serviceLoaderUnderTest.getClusterClientFactory(config); + assertNotNull(factory); + + final Integer id = factory.getClusterId(config); + assertThat(id, allOf(is(notNullValue()), equalTo(VALID_ID))); + } + + @Test(expected = IllegalStateException.class) + public void testMoreThanOneCompatibleFactoriesException() { + final Configuration config = new Configuration(); + config.setString(DeploymentOptions.TARGET, AMBIGUOUS_TARGET); + + serviceLoaderUnderTest.getClusterClientFactory(config); + fail(); + } + + @Test + public void testNoFactoriesFound() { + final Configuration config = new Configuration(); + config.setString(DeploymentOptions.TARGET, NON_EXISTING_TARGET); + + final ClusterClientFactory factory = serviceLoaderUnderTest.getClusterClientFactory(config); + assertNull(factory); + } + + /** + * Test {@link ClusterClientFactory} that is successfully discovered. + */ + public static class ValidClusterClientFactory extends DummyClusterClientFactory { + + public static final String ID = VALID_TARGET; + + @Override + public boolean isCompatibleWith(Configuration configuration) { + return configuration.getString(DeploymentOptions.TARGET).equals(VALID_TARGET); + } + + @Nullable + @Override + public Integer getClusterId(Configuration configuration) { + return VALID_ID; + } + } + + /** + * Test {@link ClusterClientFactory} that has a duplicate. + */ + public static class FirstCollidingClusterClientFactory extends DummyClusterClientFactory { + + public static final String ID = AMBIGUOUS_TARGET; + + @Override + public boolean isCompatibleWith(Configuration configuration) { + return configuration.getString(DeploymentOptions.TARGET).equals(AMBIGUOUS_TARGET); + } + } + + /** + * Test {@link ClusterClientFactory} that has a duplicate. + */ + public static class SecondCollidingClusterClientFactory extends DummyClusterClientFactory { + + public static final String ID = AMBIGUOUS_TARGET; + + @Override + public boolean isCompatibleWith(Configuration configuration) { + return configuration.getString(DeploymentOptions.TARGET).equals(AMBIGUOUS_TARGET); + } + } + + /** + * A base test {@link ClusterClientFactory} that supports no operation and is meant to be extended. + */ + public static class DummyClusterClientFactory implements ClusterClientFactory { + + @Override + public boolean isCompatibleWith(Configuration configuration) { + throw new UnsupportedOperationException(); + } + + @Override + public ClusterDescriptor createClusterDescriptor(Configuration configuration) { + throw new UnsupportedOperationException(); + } + + @Nullable + @Override + public Integer getClusterId(Configuration configuration) { + throw new UnsupportedOperationException(); + } + + @Override + public ClusterSpecification getClusterSpecification(Configuration configuration) { + throw new UnsupportedOperationException(); + } + } +} diff --git a/flink-clients/src/test/resources/META-INF/services/org.apache.flink.client.deployment.ClusterClientFactory b/flink-clients/src/test/resources/META-INF/services/org.apache.flink.client.deployment.ClusterClientFactory new file mode 100644 index 000000000000..930cea96c716 --- /dev/null +++ b/flink-clients/src/test/resources/META-INF/services/org.apache.flink.client.deployment.ClusterClientFactory @@ -0,0 +1,18 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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.client.deployment.ClusterClientServiceLoaderTest$ValidClusterClientFactory +org.apache.flink.client.deployment.ClusterClientServiceLoaderTest$FirstCollidingClusterClientFactory +org.apache.flink.client.deployment.ClusterClientServiceLoaderTest$SecondCollidingClusterClientFactory \ No newline at end of file diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClientFactory.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClientFactory.java new file mode 100644 index 000000000000..a0d9ab23d6fb --- /dev/null +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClientFactory.java @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.client.deployment.ClusterClientFactory; +import org.apache.flink.client.deployment.ClusterSpecification; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ConfigurationUtils; +import org.apache.flink.configuration.DeploymentOptions; +import org.apache.flink.configuration.HighAvailabilityOptions; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.yarn.cli.YarnConfigUtils; +import org.apache.flink.yarn.configuration.YarnConfigOptions; + +import org.apache.hadoop.fs.Path; +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.apache.hadoop.yarn.util.ConverterUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.File; +import java.io.UnsupportedEncodingException; +import java.net.URLDecoder; +import java.nio.charset.Charset; +import java.util.List; +import java.util.Optional; +import java.util.function.Consumer; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link ClusterClientFactory} for a YARN cluster. + */ +public class YarnClusterClientFactory implements ClusterClientFactory { + + private static final Logger LOG = LoggerFactory.getLogger(YarnClusterClientFactory.class); + + public static final String ID = "yarn-cluster"; + + @Override + public boolean isCompatibleWith(Configuration configuration) { + checkNotNull(configuration); + return ID.equals(configuration.getString(DeploymentOptions.TARGET)); + } + + @Override + public YarnClusterDescriptor createClusterDescriptor(Configuration configuration) { + checkNotNull(configuration); + + final YarnClusterDescriptor yarnClusterDescriptor = getClusterDescriptor(configuration); + yarnClusterDescriptor.setDetachedMode(!configuration.getBoolean(DeploymentOptions.ATTACHED)); + + getLocalFlinkDistPath(configuration, yarnClusterDescriptor) + .ifPresent(yarnClusterDescriptor::setLocalJarPath); + + decodeDirsToShipToCluster(configuration) + .ifPresent(yarnClusterDescriptor::addShipFiles); + + handleConfigOption(configuration, YarnConfigOptions.APPLICATION_QUEUE, yarnClusterDescriptor::setQueue); + handleConfigOption(configuration, YarnConfigOptions.DYNAMIC_PROPERTIES, yarnClusterDescriptor::setDynamicPropertiesEncoded); + handleConfigOption(configuration, YarnConfigOptions.APPLICATION_NAME, yarnClusterDescriptor::setName); + handleConfigOption(configuration, YarnConfigOptions.APPLICATION_TYPE, yarnClusterDescriptor::setApplicationType); + handleConfigOption(configuration, YarnConfigOptions.NODE_LABEL, yarnClusterDescriptor::setNodeLabel); + handleConfigOption(configuration, HighAvailabilityOptions.HA_CLUSTER_ID, yarnClusterDescriptor::setZookeeperNamespace); + return yarnClusterDescriptor; + } + + @Nullable + @Override + public ApplicationId getClusterId(Configuration configuration) { + checkNotNull(configuration); + final String clusterId = configuration.getString(YarnConfigOptions.APPLICATION_ID); + return clusterId != null ? ConverterUtils.toApplicationId(clusterId) : null; + } + + @Override + public ClusterSpecification getClusterSpecification(Configuration configuration) { + checkNotNull(configuration); + + // JobManager Memory + final int jobManagerMemoryMB = ConfigurationUtils.getJobManagerHeapMemory(configuration).getMebiBytes(); + + // Task Managers memory + final int taskManagerMemoryMB = ConfigurationUtils.getTaskManagerHeapMemory(configuration).getMebiBytes(); + + int slotsPerTaskManager = configuration.getInteger(TaskManagerOptions.NUM_TASK_SLOTS); + + return new ClusterSpecification.ClusterSpecificationBuilder() + .setMasterMemoryMB(jobManagerMemoryMB) + .setTaskManagerMemoryMB(taskManagerMemoryMB) + .setSlotsPerTaskManager(slotsPerTaskManager) + .createClusterSpecification(); + } + + private Optional> decodeDirsToShipToCluster(final Configuration configuration) { + checkNotNull(configuration); + + final List files = YarnConfigUtils.decodeListFromConfig(configuration, YarnConfigOptions.SHIP_DIRECTORIES, File::new); + return files.isEmpty() ? Optional.empty() : Optional.of(files); + } + + private Optional getLocalFlinkDistPath(final Configuration configuration, final YarnClusterDescriptor yarnClusterDescriptor) { + final String localJarPath = configuration.getString(YarnConfigOptions.FLINK_DIST_JAR); + if (localJarPath != null) { + return Optional.of(new Path(localJarPath)); + } + + LOG.info("No path for the flink jar passed. Using the location of " + yarnClusterDescriptor.getClass() + " to locate the jar"); + + // check whether it's actually a jar file --> when testing we execute this class without a flink-dist jar + final String decodedPath = getDecodedJarPath(yarnClusterDescriptor); + return decodedPath.endsWith(".jar") + ? Optional.of(new Path(new File(decodedPath).toURI())) + : Optional.empty(); + } + + private String getDecodedJarPath(final YarnClusterDescriptor yarnClusterDescriptor) { + final String encodedJarPath = yarnClusterDescriptor + .getClass().getProtectionDomain().getCodeSource().getLocation().getPath(); + try { + return URLDecoder.decode(encodedJarPath, Charset.defaultCharset().name()); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException("Couldn't decode the encoded Flink dist jar path: " + encodedJarPath + + " You can supply a path manually via the command line."); + } + } + + private void handleConfigOption(final Configuration configuration, final ConfigOption option, final Consumer consumer) { + checkNotNull(configuration); + checkNotNull(option); + checkNotNull(consumer); + + final String value = configuration.getString(option); + if (value != null) { + consumer.accept(value); + } + } + + private YarnClusterDescriptor getClusterDescriptor(Configuration configuration) { + final YarnClient yarnClient = YarnClient.createYarnClient(); + final YarnConfiguration yarnConfiguration = new YarnConfiguration(); + + yarnClient.init(yarnConfiguration); + yarnClient.start(); + + return new YarnClusterDescriptor( + configuration, + yarnConfiguration, + yarnClient, + false); + } +} diff --git a/flink-yarn/src/main/resources/META-INF/services/org.apache.flink.client.deployment.ClusterClientFactory b/flink-yarn/src/main/resources/META-INF/services/org.apache.flink.client.deployment.ClusterClientFactory new file mode 100644 index 000000000000..ea1c4de902f7 --- /dev/null +++ b/flink-yarn/src/main/resources/META-INF/services/org.apache.flink.client.deployment.ClusterClientFactory @@ -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.yarn.YarnClusterClientFactory \ No newline at end of file diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterClientFactoryTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterClientFactoryTest.java new file mode 100644 index 000000000000..931313a66754 --- /dev/null +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterClientFactoryTest.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.yarn; + +import org.apache.flink.client.deployment.ClusterClientFactory; +import org.apache.flink.client.deployment.ClusterClientServiceLoader; +import org.apache.flink.client.deployment.DefaultClusterClientServiceLoader; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.DeploymentOptions; + +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.junit.Test; + +import static org.junit.Assert.assertTrue; + +/** + * Test for the {@link YarnClusterClientFactory} discovery. + */ +public class YarnClusterClientFactoryTest { + + @Test + public void testYarnClusterClientFactoryDiscovery() { + final Configuration configuration = new Configuration(); + configuration.setString(DeploymentOptions.TARGET, YarnClusterClientFactory.ID); + + final ClusterClientServiceLoader serviceLoader = new DefaultClusterClientServiceLoader(); + final ClusterClientFactory factory = serviceLoader.getClusterClientFactory(configuration); + + assertTrue(factory instanceof YarnClusterClientFactory); + } +} From e4448748f33474d153da7579f249b9ed452bfae2 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Tue, 22 Oct 2019 22:37:11 +0200 Subject: [PATCH 395/746] [FLINK-14501] Wired ClusterClientFactories to production code --- .../client/cli/AbstractCustomCommandLine.java | 2 +- .../apache/flink/client/cli/CliFrontend.java | 74 +++++---- .../flink/client/cli/CliFrontendParser.java | 14 +- .../flink/client/cli/CustomCommandLine.java | 34 +--- .../apache/flink/client/cli/DefaultCLI.java | 26 +-- .../flink/client/cli/CliFrontendRunTest.java | 21 ++- .../flink/client/cli/CliFrontendTestBase.java | 6 +- .../flink/client/cli/DefaultCLITest.java | 36 +++-- .../cli/util/DummyClusterClientFactory.java | 66 ++++++++ .../util/DummyClusterClientServiceLoader.java | 44 ++++++ .../cli/util/DummyClusterDescriptor.java | 3 +- .../cli/util/DummyCustomCommandLine.java | 36 +---- .../client/cli/util/MockedCliFrontend.java | 5 +- .../ClusterClientServiceLoaderTest.java | 8 +- .../program/rest/RestClusterClientTest.java | 14 +- .../apache/flink/api/scala/FlinkShell.scala | 18 ++- .../gateway/local/ExecutionContext.java | 35 +++-- .../client/gateway/local/LocalExecutor.java | 20 ++- .../client/gateway/local/DependencyTest.java | 4 +- .../gateway/local/LocalExecutorITCase.java | 10 +- .../yarn/CliFrontendRunWithYarnTest.java | 42 +++-- .../flink/yarn/cli/FlinkYarnSessionCli.java | 148 +++--------------- .../flink/yarn/FlinkYarnSessionCliTest.java | 96 +++++++----- 23 files changed, 406 insertions(+), 356 deletions(-) create mode 100644 flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterClientFactory.java create mode 100644 flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterClientServiceLoader.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java b/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java index cba103661245..f32d4f861d54 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/AbstractCustomCommandLine.java @@ -39,7 +39,7 @@ * a ZooKeeper namespace. * */ -public abstract class AbstractCustomCommandLine implements CustomCommandLine { +public abstract class AbstractCustomCommandLine implements CustomCommandLine { protected final Option zookeeperNamespaceOption = new Option("z", "zookeeperNamespace", true, "Namespace to create the Zookeeper sub-paths for high availability mode"); 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 c49374a15529..7e0d28853193 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 @@ -27,8 +27,11 @@ import org.apache.flink.api.dag.Pipeline; import org.apache.flink.client.ClientUtils; import org.apache.flink.client.FlinkPipelineTranslationUtil; +import org.apache.flink.client.deployment.ClusterClientFactory; +import org.apache.flink.client.deployment.ClusterClientServiceLoader; import org.apache.flink.client.deployment.ClusterDescriptor; import org.apache.flink.client.deployment.ClusterSpecification; +import org.apache.flink.client.deployment.DefaultClusterClientServiceLoader; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.client.program.PackagedProgramUtils; @@ -64,7 +67,6 @@ import java.io.File; import java.io.FileNotFoundException; import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; import java.lang.reflect.UndeclaredThrowableException; import java.net.InetSocketAddress; import java.net.URL; @@ -81,6 +83,8 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * Implementation of a simple command line frontend for executing programs. */ @@ -104,7 +108,7 @@ public class CliFrontend { private final Configuration configuration; - private final List> customCommandLines; + private final List customCommandLines; private final Options customCommandLineOptions; @@ -112,17 +116,27 @@ public class CliFrontend { private final int defaultParallelism; + private final ClusterClientServiceLoader clusterClientServiceLoader; + public CliFrontend( Configuration configuration, - List> customCommandLines) { - this.configuration = Preconditions.checkNotNull(configuration); - this.customCommandLines = Preconditions.checkNotNull(customCommandLines); + List customCommandLines) { + this(configuration, new DefaultClusterClientServiceLoader(), customCommandLines); + } + + public CliFrontend( + Configuration configuration, + ClusterClientServiceLoader clusterClientServiceLoader, + List customCommandLines) { + this.configuration = checkNotNull(configuration); + this.customCommandLines = checkNotNull(customCommandLines); + this.clusterClientServiceLoader = checkNotNull(clusterClientServiceLoader); FileSystem.initialize(configuration, PluginUtils.createPluginManagerFromRootFolder(configuration)); this.customCommandLineOptions = new Options(); - for (CustomCommandLine customCommandLine : customCommandLines) { + for (CustomCommandLine customCommandLine : customCommandLines) { customCommandLine.addGeneralOptions(customCommandLineOptions); customCommandLine.addRunOptions(customCommandLineOptions); } @@ -194,24 +208,27 @@ protected void run(String[] args) throws Exception { throw new CliArgsException("Could not build the program from JAR file.", e); } - final CustomCommandLine customCommandLine = getActiveCustomCommandLine(commandLine); + final CustomCommandLine customCommandLine = getActiveCustomCommandLine(commandLine); final Configuration executorConfig = customCommandLine.applyCommandLineOptionsToConfiguration(commandLine); try { - runProgram(customCommandLine, executorConfig, runOptions, program); + runProgram(executorConfig, runOptions, program); } finally { program.deleteExtractedLibraries(); } } private void runProgram( - CustomCommandLine customCommandLine, Configuration executorConfig, RunOptions runOptions, PackagedProgram program) throws ProgramInvocationException, FlinkException { - final ClusterDescriptor clusterDescriptor = customCommandLine.createClusterDescriptor(executorConfig); + + final ClusterClientFactory clusterClientFactory = clusterClientServiceLoader.getClusterClientFactory(executorConfig); + checkNotNull(clusterClientFactory); + + final ClusterDescriptor clusterDescriptor = clusterClientFactory.createClusterDescriptor(executorConfig); try { - final T clusterId = customCommandLine.getClusterId(executorConfig); + final T clusterId = clusterClientFactory.getClusterId(executorConfig); final ClusterClient client; @@ -221,7 +238,7 @@ private void runProgram( final JobGraph jobGraph = PackagedProgramUtils.createJobGraph(program, configuration, parallelism); - final ClusterSpecification clusterSpecification = customCommandLine.getClusterSpecification(executorConfig); + final ClusterSpecification clusterSpecification = clusterClientFactory.getClusterSpecification(executorConfig); client = clusterDescriptor.deployJobCluster( clusterSpecification, jobGraph, @@ -242,7 +259,7 @@ private void runProgram( } else { // also in job mode we have to deploy a session cluster because the job // might consist of multiple parts (e.g. when using collect) - final ClusterSpecification clusterSpecification = customCommandLine.getClusterSpecification(executorConfig); + final ClusterSpecification clusterSpecification = clusterClientFactory.getClusterSpecification(executorConfig); client = clusterDescriptor.deploySessionCluster(clusterSpecification); // if not running in detached mode, add a shutdown hook to shut down cluster if client exits // there's a race-condition here if cli is killed before shutdown hook is installed @@ -395,7 +412,7 @@ protected void list(String[] args) throws Exception { showAll = listOptions.showAll(); } - final CustomCommandLine activeCommandLine = getActiveCustomCommandLine(commandLine); + final CustomCommandLine activeCommandLine = getActiveCustomCommandLine(commandLine); runClusterAction( activeCommandLine, @@ -513,7 +530,7 @@ protected void stop(String[] args) throws Exception { logAndSysout((advanceToEndOfEventTime ? "Draining job " : "Suspending job ") + "\"" + jobId + "\" with a savepoint."); - final CustomCommandLine activeCommandLine = getActiveCustomCommandLine(commandLine); + final CustomCommandLine activeCommandLine = getActiveCustomCommandLine(commandLine); runClusterAction( activeCommandLine, commandLine, @@ -550,7 +567,7 @@ protected void cancel(String[] args) throws Exception { return; } - final CustomCommandLine activeCommandLine = getActiveCustomCommandLine(commandLine); + final CustomCommandLine activeCommandLine = getActiveCustomCommandLine(commandLine); final String[] cleanedArgs = cancelOptions.getArgs(); @@ -635,7 +652,7 @@ protected void savepoint(String[] args) throws Exception { return; } - final CustomCommandLine activeCommandLine = getActiveCustomCommandLine(commandLine); + final CustomCommandLine activeCommandLine = getActiveCustomCommandLine(commandLine); if (savepointOptions.isDispose()) { runClusterAction( @@ -908,21 +925,22 @@ private JobID parseJobId(String jobIdString) throws CliArgsException { * @param activeCommandLine to create the {@link ClusterDescriptor} from * @param commandLine containing the parsed command line options * @param clusterAction the cluster action to run against the retrieved {@link ClusterClient}. - * @param type of the cluster id + * @param type of the cluster id * @throws FlinkException if something goes wrong */ - private void runClusterAction(CustomCommandLine activeCommandLine, CommandLine commandLine, ClusterAction clusterAction) throws FlinkException { + private void runClusterAction(CustomCommandLine activeCommandLine, CommandLine commandLine, ClusterAction clusterAction) throws FlinkException { final Configuration executorConfig = activeCommandLine.applyCommandLineOptionsToConfiguration(commandLine); - final ClusterDescriptor clusterDescriptor = activeCommandLine.createClusterDescriptor(executorConfig); - final T clusterId = activeCommandLine.getClusterId(executorConfig); + final ClusterClientFactory clusterClientFactory = clusterClientServiceLoader.getClusterClientFactory(executorConfig); + final ClusterDescriptor clusterDescriptor = clusterClientFactory.createClusterDescriptor(executorConfig); + final ClusterID clusterId = clusterClientFactory.getClusterId(executorConfig); if (clusterId == null) { throw new FlinkException("No cluster id was specified. Please specify a cluster to which " + "you would like to connect."); } else { try { - final ClusterClient clusterClient = clusterDescriptor.retrieve(clusterId); + final ClusterClient clusterClient = clusterDescriptor.retrieve(clusterId); try { clusterAction.runAction(clusterClient); @@ -1052,7 +1070,7 @@ public static void main(final String[] args) { final Configuration configuration = GlobalConfiguration.loadConfiguration(configurationDirectory); // 3. load the custom command lines - final List> customCommandLines = loadCustomCommandLines( + final List customCommandLines = loadCustomCommandLines( configuration, configurationDirectory); @@ -1117,8 +1135,8 @@ static void setJobManagerAddressInConfig(Configuration config, InetSocketAddress config.setInteger(RestOptions.PORT, address.getPort()); } - public static List> loadCustomCommandLines(Configuration configuration, String configurationDirectory) { - List> customCommandLines = new ArrayList<>(2); + public static List loadCustomCommandLines(Configuration configuration, String configurationDirectory) { + List customCommandLines = new ArrayList<>(2); // Command line interface of the YARN session, with a special initialization here // to prefix all options with y/yarn. @@ -1150,8 +1168,8 @@ public static List> loadCustomCommandLines(Configuration co * @param commandLine The input to the command-line. * @return custom command-line which is active (may only be one at a time) */ - public CustomCommandLine getActiveCustomCommandLine(CommandLine commandLine) { - for (CustomCommandLine cli : customCommandLines) { + public CustomCommandLine getActiveCustomCommandLine(CommandLine commandLine) { + for (CustomCommandLine cli : customCommandLines) { if (cli.isActive(commandLine)) { return cli; } @@ -1164,7 +1182,7 @@ public CustomCommandLine getActiveCustomCommandLine(CommandLine commandLine) * @param className The fully-qualified class name to load. * @param params The constructor parameters */ - private static CustomCommandLine loadCustomCommandLine(String className, Object... params) throws IllegalAccessException, InvocationTargetException, InstantiationException, ClassNotFoundException, NoSuchMethodException { + private static CustomCommandLine loadCustomCommandLine(String className, Object... params) throws Exception { Class customCliClass = Class.forName(className).asSubclass(CustomCommandLine.class); diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java index 5639cb545cec..97ba33e05979 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java @@ -306,7 +306,7 @@ private static Options getSavepointOptionsWithoutDeprecatedOptions(Options optio /** * Prints the help for the client. */ - public static void printHelp(Collection> customCommandLines) { + public static void printHelp(Collection customCommandLines) { System.out.println("./flink [OPTIONS] [ARGUMENTS]"); System.out.println(); System.out.println("The following actions are available:"); @@ -321,7 +321,7 @@ public static void printHelp(Collection> customCommandLines System.out.println(); } - public static void printHelpForRun(Collection> customCommandLines) { + public static void printHelpForRun(Collection customCommandLines) { HelpFormatter formatter = new HelpFormatter(); formatter.setLeftPadding(5); formatter.setWidth(80); @@ -349,7 +349,7 @@ public static void printHelpForInfo() { System.out.println(); } - public static void printHelpForList(Collection> customCommandLines) { + public static void printHelpForList(Collection customCommandLines) { HelpFormatter formatter = new HelpFormatter(); formatter.setLeftPadding(5); formatter.setWidth(80); @@ -364,7 +364,7 @@ public static void printHelpForList(Collection> customComma System.out.println(); } - public static void printHelpForStop(Collection> customCommandLines) { + public static void printHelpForStop(Collection customCommandLines) { HelpFormatter formatter = new HelpFormatter(); formatter.setLeftPadding(5); formatter.setWidth(80); @@ -379,7 +379,7 @@ public static void printHelpForStop(Collection> customComma System.out.println(); } - public static void printHelpForCancel(Collection> customCommandLines) { + public static void printHelpForCancel(Collection customCommandLines) { HelpFormatter formatter = new HelpFormatter(); formatter.setLeftPadding(5); formatter.setWidth(80); @@ -394,7 +394,7 @@ public static void printHelpForCancel(Collection> customCom System.out.println(); } - public static void printHelpForSavepoint(Collection> customCommandLines) { + public static void printHelpForSavepoint(Collection customCommandLines) { HelpFormatter formatter = new HelpFormatter(); formatter.setLeftPadding(5); formatter.setWidth(80); @@ -415,7 +415,7 @@ public static void printHelpForSavepoint(Collection> custom * @param runOptions True if the run options should be printed, False to print only general options */ private static void printCustomCliOptions( - Collection> customCommandLines, + Collection customCommandLines, HelpFormatter formatter, boolean runOptions) { // prints options from all available command-line classes diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java index 9458de2f8946..4241b1c9d51f 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CustomCommandLine.java @@ -18,20 +18,16 @@ package org.apache.flink.client.cli; -import org.apache.flink.client.deployment.ClusterDescriptor; -import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.configuration.Configuration; import org.apache.flink.util.FlinkException; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; -import javax.annotation.Nullable; - /** * Custom command-line interface to load hooks for the command-line interface. */ -public interface CustomCommandLine { +public interface CustomCommandLine { /** * Signals whether the custom command-line wants to execute or not. @@ -67,34 +63,6 @@ public interface CustomCommandLine { */ Configuration applyCommandLineOptionsToConfiguration(CommandLine commandLine) throws FlinkException; - /** - * Create a {@link ClusterDescriptor} from the given configuration. - * - * @param configuration containing the configuration options relevant for the {@link ClusterDescriptor} - * @return the corresponding {@link ClusterDescriptor}. - */ - ClusterDescriptor createClusterDescriptor(Configuration configuration); - - /** - * Returns the cluster id if a cluster id is specified in the provided configuration, otherwise it returns {@code null}. - * - *

    A cluster id identifies a running cluster, e.g. the Yarn application id for a Flink cluster running on Yarn. - * - * @param configuration containing the configuration options relevant for the cluster id retrieval - * @return Cluster id identifying the cluster to deploy jobs to or null - */ - @Nullable - T getClusterId(Configuration configuration); - - /** - * Returns the {@link ClusterSpecification} specified by the configuration and the command - * line options. This specification can be used to deploy a new Flink cluster. - * - * @param configuration containing the configuration options relevant for the {@link ClusterSpecification} - * @return the corresponding {@link ClusterSpecification} for a new Flink cluster - */ - ClusterSpecification getClusterSpecification(Configuration configuration); - default CommandLine parseCommandLineOptions(String[] args, boolean stopAtNonOptions) throws CliArgsException { final Options options = new Options(); addGeneralOptions(options); diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java index f21f755e19b5..397d5dd9b4a8 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java @@ -18,20 +18,16 @@ package org.apache.flink.client.cli; -import org.apache.flink.client.deployment.ClusterSpecification; -import org.apache.flink.client.deployment.StandaloneClusterDescriptor; -import org.apache.flink.client.deployment.StandaloneClusterId; +import org.apache.flink.client.deployment.StandaloneClientFactory; import org.apache.flink.configuration.Configuration; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; -import javax.annotation.Nullable; - /** * The default CLI which is used for interaction with standalone clusters. */ -public class DefaultCLI extends AbstractCustomCommandLine { +public class DefaultCLI extends AbstractCustomCommandLine { public DefaultCLI(Configuration configuration) { super(configuration); @@ -45,27 +41,11 @@ public boolean isActive(CommandLine commandLine) { @Override public String getId() { - return "default"; + return StandaloneClientFactory.ID; } @Override public void addGeneralOptions(Options baseOptions) { super.addGeneralOptions(baseOptions); } - - @Override - public StandaloneClusterDescriptor createClusterDescriptor(Configuration configuration) { - return new StandaloneClusterDescriptor(configuration); - } - - @Override - @Nullable - public StandaloneClusterId getClusterId(Configuration configuration) { - return StandaloneClusterId.getInstance(); - } - - @Override - public ClusterSpecification getClusterSpecification(Configuration configuration) { - return new ClusterSpecification.ClusterSpecificationBuilder().createClusterSpecification(); - } } diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java index 7bff564e9c1b..2230917f9006 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java @@ -18,6 +18,8 @@ package org.apache.flink.client.cli; +import org.apache.flink.client.deployment.ClusterClientServiceLoader; +import org.apache.flink.client.deployment.DefaultClusterClientServiceLoader; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.configuration.Configuration; @@ -145,12 +147,23 @@ public void testParallelismWithOverflow() throws Exception { // -------------------------------------------------------------------------------------------- public static void verifyCliFrontend( - AbstractCustomCommandLine cli, + AbstractCustomCommandLine cli, String[] parameters, int expectedParallelism, boolean isDetached) throws Exception { RunTestingCliFrontend testFrontend = - new RunTestingCliFrontend(cli, expectedParallelism, isDetached); + new RunTestingCliFrontend(new DefaultClusterClientServiceLoader(), cli, expectedParallelism, isDetached); + testFrontend.run(parameters); // verifies the expected values (see below) + } + + public static void verifyCliFrontend( + ClusterClientServiceLoader clusterClientServiceLoader, + AbstractCustomCommandLine cli, + String[] parameters, + int expectedParallelism, + boolean isDetached) throws Exception { + RunTestingCliFrontend testFrontend = + new RunTestingCliFrontend(clusterClientServiceLoader, cli, expectedParallelism, isDetached); testFrontend.run(parameters); // verifies the expected values (see below) } @@ -160,11 +173,13 @@ private static final class RunTestingCliFrontend extends CliFrontend { private final boolean isDetached; private RunTestingCliFrontend( - AbstractCustomCommandLine cli, + ClusterClientServiceLoader clusterClientServiceLoader, + AbstractCustomCommandLine cli, int expectedParallelism, boolean isDetached) { super( cli.getConfiguration(), + clusterClientServiceLoader, Collections.singletonList(cli)); this.expectedParallelism = expectedParallelism; this.isDetached = isDetached; diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendTestBase.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendTestBase.java index 8ff426c80579..b384d49eb913 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendTestBase.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendTestBase.java @@ -28,12 +28,10 @@ public abstract class CliFrontendTestBase extends TestLogger { protected Configuration getConfiguration() { - final Configuration configuration = GlobalConfiguration - .loadConfiguration(CliFrontendTestUtils.getConfigDir()); - return configuration; + return GlobalConfiguration.loadConfiguration(CliFrontendTestUtils.getConfigDir()); } - static AbstractCustomCommandLine getCli(Configuration configuration) { + static AbstractCustomCommandLine getCli(Configuration configuration) { return new DefaultCLI(configuration); } } diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java index 73f967d755ee..ba1508840918 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java @@ -18,12 +18,16 @@ package org.apache.flink.client.cli; +import org.apache.flink.client.deployment.ClusterClientFactory; +import org.apache.flink.client.deployment.ClusterClientServiceLoader; import org.apache.flink.client.deployment.ClusterDescriptor; +import org.apache.flink.client.deployment.DefaultClusterClientServiceLoader; import org.apache.flink.client.deployment.StandaloneClusterId; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.RestOptions; +import org.apache.flink.util.FlinkException; import org.apache.commons.cli.CommandLine; import org.hamcrest.Matchers; @@ -33,6 +37,7 @@ import java.net.URL; +import static org.apache.flink.util.Preconditions.checkState; import static org.junit.Assert.assertThat; /** @@ -57,17 +62,12 @@ public void testConfigurationPassing() throws Exception { configuration.setString(RestOptions.ADDRESS, localhost); configuration.setInteger(RestOptions.PORT, port); - @SuppressWarnings("unchecked") - final AbstractCustomCommandLine defaultCLI = - (AbstractCustomCommandLine) getCli(configuration); + final AbstractCustomCommandLine defaultCLI = getCli(configuration); final String[] args = {}; - CommandLine commandLine = defaultCLI.parseCommandLineOptions(args, false); - final Configuration executorConfig = defaultCLI.applyCommandLineOptionsToConfiguration(commandLine); - - final ClusterDescriptor clusterDescriptor = defaultCLI.createClusterDescriptor(executorConfig); - final ClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(executorConfig)); + final CommandLine commandLine = defaultCLI.parseCommandLineOptions(args, false); + final ClusterClient clusterClient = getClusterClient(defaultCLI, commandLine); final URL webInterfaceUrl = new URL(clusterClient.getWebInterfaceURL()); @@ -87,19 +87,14 @@ public void testManualConfigurationOverride() throws Exception { configuration.setString(JobManagerOptions.ADDRESS, localhost); configuration.setInteger(JobManagerOptions.PORT, port); - @SuppressWarnings("unchecked") - final AbstractCustomCommandLine defaultCLI = - (AbstractCustomCommandLine) getCli(configuration); + final AbstractCustomCommandLine defaultCLI = getCli(configuration); final String manualHostname = "123.123.123.123"; final int manualPort = 4321; final String[] args = {"-m", manualHostname + ':' + manualPort}; - CommandLine commandLine = defaultCLI.parseCommandLineOptions(args, false); - final Configuration executorConfig = defaultCLI.applyCommandLineOptionsToConfiguration(commandLine); - - final ClusterDescriptor clusterDescriptor = defaultCLI.createClusterDescriptor(executorConfig); - final ClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(executorConfig)); + final CommandLine commandLine = defaultCLI.parseCommandLineOptions(args, false); + final ClusterClient clusterClient = getClusterClient(defaultCLI, commandLine); final URL webInterfaceUrl = new URL(clusterClient.getWebInterfaceURL()); @@ -107,4 +102,13 @@ public void testManualConfigurationOverride() throws Exception { assertThat(webInterfaceUrl.getPort(), Matchers.equalTo(manualPort)); } + private ClusterClient getClusterClient(AbstractCustomCommandLine defaultCLI, CommandLine commandLine) throws FlinkException { + final ClusterClientServiceLoader serviceLoader = new DefaultClusterClientServiceLoader(); + final Configuration executorConfig = defaultCLI.applyCommandLineOptionsToConfiguration(commandLine); + final ClusterClientFactory clusterFactory = serviceLoader.getClusterClientFactory(executorConfig); + checkState(clusterFactory != null); + + final ClusterDescriptor clusterDescriptor = clusterFactory.createClusterDescriptor(executorConfig); + return clusterDescriptor.retrieve(clusterFactory.getClusterId(executorConfig)); + } } diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterClientFactory.java b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterClientFactory.java new file mode 100644 index 000000000000..d06757672393 --- /dev/null +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterClientFactory.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.cli.util; + +import org.apache.flink.client.deployment.ClusterClientFactory; +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.configuration.DeploymentOptions; + +import javax.annotation.Nullable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link ClusterClientFactory} used for testing. + * @param The type of the id of the cluster. + */ +public class DummyClusterClientFactory implements ClusterClientFactory { + + public static final String ID = "dummy-client-factory"; + + private final ClusterClient clusterClient; + + public DummyClusterClientFactory(ClusterClient clusterClient) { + this.clusterClient = checkNotNull(clusterClient); + } + + @Override + public boolean isCompatibleWith(Configuration configuration) { + return ID.equals(configuration.getString(DeploymentOptions.TARGET)); + } + + @Override + public ClusterDescriptor createClusterDescriptor(Configuration configuration) { + return new DummyClusterDescriptor<>(checkNotNull(clusterClient)); + } + + @Override + @Nullable + public String getClusterId(Configuration configuration) { + return "dummy"; + } + + @Override + public ClusterSpecification getClusterSpecification(Configuration configuration) { + return new ClusterSpecification.ClusterSpecificationBuilder().createClusterSpecification(); + } +} diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterClientServiceLoader.java b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterClientServiceLoader.java new file mode 100644 index 000000000000..f58383c3be49 --- /dev/null +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterClientServiceLoader.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.client.cli.util; + +import org.apache.flink.client.deployment.ClusterClientFactory; +import org.apache.flink.client.deployment.ClusterClientServiceLoader; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.configuration.Configuration; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A test {@link ClusterClientServiceLoader} that returns always a {@link DummyClusterClientFactory}. + */ +public class DummyClusterClientServiceLoader implements ClusterClientServiceLoader { + + private final ClusterClient clusterClient; + + public DummyClusterClientServiceLoader(final ClusterClient clusterClient) { + this.clusterClient = checkNotNull(clusterClient); + } + + @Override + public ClusterClientFactory getClusterClientFactory(final Configuration configuration) { + checkNotNull(configuration); + return new DummyClusterClientFactory<>(clusterClient); + } +} diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java index 7620ae221575..a7af09b0739b 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java @@ -22,7 +22,6 @@ import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; /** @@ -60,7 +59,7 @@ public ClusterClient deployJobCluster( } @Override - public void killCluster(T clusterId) throws FlinkException { + public void killCluster(T clusterId) { throw new UnsupportedOperationException("Cannot terminate a dummy cluster."); } diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java index 0bd7dc1ab97b..4a9465693cea 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyCustomCommandLine.java @@ -19,26 +19,16 @@ package org.apache.flink.client.cli.util; import org.apache.flink.client.cli.CustomCommandLine; -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.util.Preconditions; +import org.apache.flink.configuration.DeploymentOptions; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; -import javax.annotation.Nullable; - /** * Dummy implementation of the {@link CustomCommandLine} for testing purposes. */ -public class DummyCustomCommandLine implements CustomCommandLine { - private final ClusterClient clusterClient; - - public DummyCustomCommandLine(ClusterClient clusterClient) { - this.clusterClient = Preconditions.checkNotNull(clusterClient); - } +public class DummyCustomCommandLine implements CustomCommandLine { @Override public boolean isActive(CommandLine commandLine) { @@ -47,7 +37,7 @@ public boolean isActive(CommandLine commandLine) { @Override public String getId() { - return DummyCustomCommandLine.class.getSimpleName(); + return DummyClusterClientFactory.ID; } @Override @@ -62,22 +52,8 @@ public void addGeneralOptions(Options baseOptions) { @Override public Configuration applyCommandLineOptionsToConfiguration(CommandLine commandLine) { - return new Configuration(); - } - - @Override - public ClusterDescriptor createClusterDescriptor(Configuration configuration) { - return new DummyClusterDescriptor<>(clusterClient); - } - - @Override - @Nullable - public String getClusterId(Configuration configuration) { - return "dummy"; - } - - @Override - public ClusterSpecification getClusterSpecification(Configuration configuration) { - return new ClusterSpecification.ClusterSpecificationBuilder().createClusterSpecification(); + final Configuration configuration = new Configuration(); + configuration.setString(DeploymentOptions.TARGET, DummyClusterClientFactory.ID); + return configuration; } } diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java b/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java index 477293de8906..e76175e5c75b 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java @@ -31,9 +31,10 @@ */ public class MockedCliFrontend extends CliFrontend { - public MockedCliFrontend(ClusterClient clusterClient) throws Exception { + public MockedCliFrontend(ClusterClient clusterClient) { super( new Configuration(), - Collections.singletonList(new DummyCustomCommandLine(clusterClient))); + new DummyClusterClientServiceLoader<>(clusterClient), + Collections.singletonList(new DummyCustomCommandLine())); } } diff --git a/flink-clients/src/test/java/org/apache/flink/client/deployment/ClusterClientServiceLoaderTest.java b/flink-clients/src/test/java/org/apache/flink/client/deployment/ClusterClientServiceLoaderTest.java index 45157a25094f..a08402138713 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/deployment/ClusterClientServiceLoaderTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/deployment/ClusterClientServiceLoaderTest.java @@ -96,7 +96,7 @@ public void testNoFactoriesFound() { /** * Test {@link ClusterClientFactory} that is successfully discovered. */ - public static class ValidClusterClientFactory extends DummyClusterClientFactory { + public static class ValidClusterClientFactory extends BaseTestingClusterClientFactory { public static final String ID = VALID_TARGET; @@ -115,7 +115,7 @@ public Integer getClusterId(Configuration configuration) { /** * Test {@link ClusterClientFactory} that has a duplicate. */ - public static class FirstCollidingClusterClientFactory extends DummyClusterClientFactory { + public static class FirstCollidingClusterClientFactory extends BaseTestingClusterClientFactory { public static final String ID = AMBIGUOUS_TARGET; @@ -128,7 +128,7 @@ public boolean isCompatibleWith(Configuration configuration) { /** * Test {@link ClusterClientFactory} that has a duplicate. */ - public static class SecondCollidingClusterClientFactory extends DummyClusterClientFactory { + public static class SecondCollidingClusterClientFactory extends BaseTestingClusterClientFactory { public static final String ID = AMBIGUOUS_TARGET; @@ -141,7 +141,7 @@ public boolean isCompatibleWith(Configuration configuration) { /** * A base test {@link ClusterClientFactory} that supports no operation and is meant to be extended. */ - public static class DummyClusterClientFactory implements ClusterClientFactory { + public static class BaseTestingClusterClientFactory implements ClusterClientFactory { @Override public boolean isCompatibleWith(Configuration configuration) { diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java index 2ffa0b684f1c..d11b0466d099 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java @@ -22,7 +22,10 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobSubmissionResult; import org.apache.flink.client.cli.DefaultCLI; -import org.apache.flink.client.deployment.StandaloneClusterDescriptor; +import org.apache.flink.client.deployment.ClusterClientFactory; +import org.apache.flink.client.deployment.ClusterClientServiceLoader; +import org.apache.flink.client.deployment.ClusterDescriptor; +import org.apache.flink.client.deployment.DefaultClusterClientServiceLoader; import org.apache.flink.client.deployment.StandaloneClusterId; import org.apache.flink.client.program.DetachedJobExecutionResult; import org.apache.flink.client.program.ProgramInvocationException; @@ -554,10 +557,15 @@ public void testRESTManualConfigurationOverride() throws Exception { final String[] args = {"-m", manualHostname + ':' + manualPort}; CommandLine commandLine = defaultCLI.parseCommandLineOptions(args, false); + + final ClusterClientServiceLoader serviceLoader = new DefaultClusterClientServiceLoader(); final Configuration executorConfig = defaultCLI.applyCommandLineOptionsToConfiguration(commandLine); - final StandaloneClusterDescriptor clusterDescriptor = defaultCLI.createClusterDescriptor(executorConfig); - final RestClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(executorConfig)); + final ClusterClientFactory clusterFactory = serviceLoader.getClusterClientFactory(executorConfig); + checkState(clusterFactory != null); + + final ClusterDescriptor clusterDescriptor = clusterFactory.createClusterDescriptor(executorConfig); + final RestClusterClient clusterClient = (RestClusterClient) clusterDescriptor.retrieve(clusterFactory.getClusterId(executorConfig)); URL webMonitorBaseUrl = clusterClient.getWebMonitorBaseUrl().get(); assertThat(webMonitorBaseUrl.getHost(), equalTo(manualHostname)); diff --git a/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala b/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala index 04f68ffb8b56..84d657e16d60 100644 --- a/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala +++ b/flink-scala-shell/src/main/scala/org/apache/flink/api/scala/FlinkShell.scala @@ -22,7 +22,7 @@ import java.io._ import java.net.URL import org.apache.flink.client.cli.{CliFrontend, CliFrontendParser} -import org.apache.flink.client.deployment.ClusterDescriptor +import org.apache.flink.client.deployment.{ClusterDescriptor, DefaultClusterClientServiceLoader} import org.apache.flink.client.program.ClusterClient import org.apache.flink.configuration.{Configuration, GlobalConfiguration, JobManagerOptions} import org.apache.flink.runtime.minicluster.{MiniCluster, MiniClusterConfiguration} @@ -258,11 +258,12 @@ object FlinkShell { val commandLine = CliFrontendParser.parse(commandLineOptions, args.toArray, true) val customCLI = frontend.getActiveCustomCommandLine(commandLine) - val executorConfig = customCLI.applyCommandLineOptionsToConfiguration(commandLine); - - val clusterDescriptor = customCLI.createClusterDescriptor(executorConfig) + val executorConfig = customCLI.applyCommandLineOptionsToConfiguration(commandLine) - val clusterSpecification = customCLI.getClusterSpecification(executorConfig) + val serviceLoader = new DefaultClusterClientServiceLoader + val clientFactory = serviceLoader.getClusterClientFactory(executorConfig) + val clusterDescriptor = clientFactory.createClusterDescriptor(executorConfig) + val clusterSpecification = clientFactory.getClusterSpecification(executorConfig) val cluster = clusterDescriptor.deploySessionCluster(clusterSpecification) @@ -291,11 +292,12 @@ object FlinkShell { val customCLI = frontend.getActiveCustomCommandLine(commandLine) val executorConfig = customCLI.applyCommandLineOptionsToConfiguration(commandLine); - val clusterDescriptor = customCLI + val serviceLoader = new DefaultClusterClientServiceLoader + val clientFactory = serviceLoader.getClusterClientFactory(executorConfig) + val clusterDescriptor = clientFactory .createClusterDescriptor(executorConfig) .asInstanceOf[ClusterDescriptor[Any]] - - val clusterId = customCLI.getClusterId(executorConfig) + val clusterId = clientFactory.getClusterId(executorConfig) val cluster = clusterDescriptor.retrieve(clusterId) 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 d02859772f91..15331f34579e 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 @@ -28,8 +28,11 @@ 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.ClusterClientFactory; +import org.apache.flink.client.deployment.ClusterClientServiceLoader; import org.apache.flink.client.deployment.ClusterDescriptor; import org.apache.flink.client.deployment.ClusterSpecification; +import org.apache.flink.client.deployment.DefaultClusterClientServiceLoader; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.plugin.TemporaryClassLoaderContext; import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; @@ -93,6 +96,9 @@ import java.util.Map; import java.util.function.Supplier; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + /** * Context for executing table programs. This class caches everything that can be cached across * multiple queries as long as the session context does not change. This must be thread-safe as @@ -112,13 +118,18 @@ public class ExecutionContext { private final Map functions; private final Configuration flinkConfig; private final Configuration executorConfig; - private final CustomCommandLine activeCommandLine; + private final ClusterClientFactory clusterClientFactory; private final RunOptions runOptions; private final T clusterId; private final ClusterSpecification clusterSpec; public ExecutionContext(Environment defaultEnvironment, SessionContext sessionContext, List dependencies, - Configuration flinkConfig, Options commandLineOptions, List> availableCommandLines) throws FlinkException { + Configuration flinkConfig, Options commandLineOptions, List availableCommandLines) throws FlinkException { + this(defaultEnvironment, sessionContext, dependencies, flinkConfig, new DefaultClusterClientServiceLoader(), commandLineOptions, availableCommandLines); + } + + public ExecutionContext(Environment defaultEnvironment, SessionContext sessionContext, List dependencies, + Configuration flinkConfig, ClusterClientServiceLoader clusterClientServiceLoader, Options commandLineOptions, List availableCommandLines) throws FlinkException { this.sessionContext = sessionContext.copy(); // create internal copy because session context is mutable this.mergedEnv = Environment.merge(defaultEnvironment, sessionContext.getEnvironment()); this.dependencies = dependencies; @@ -155,12 +166,17 @@ public ExecutionContext(Environment defaultEnvironment, SessionContext sessionCo }); // convert deployment options into command line options that describe a cluster + final ClusterClientServiceLoader serviceLoader = checkNotNull(clusterClientServiceLoader); final CommandLine commandLine = createCommandLine(mergedEnv.getDeployment(), commandLineOptions); - activeCommandLine = findActiveCommandLine(availableCommandLines, commandLine); + final CustomCommandLine activeCommandLine = findActiveCommandLine(availableCommandLines, commandLine); + executorConfig = activeCommandLine.applyCommandLineOptionsToConfiguration(commandLine); + clusterClientFactory = serviceLoader.getClusterClientFactory(executorConfig); + checkState(clusterClientFactory != null); + runOptions = createRunOptions(commandLine); - clusterId = activeCommandLine.getClusterId(executorConfig); - clusterSpec = activeCommandLine.getClusterSpecification(executorConfig); + clusterId = clusterClientFactory.getClusterId(executorConfig); + clusterSpec = clusterClientFactory.getClusterSpecification(executorConfig); } public SessionContext getSessionContext() { @@ -184,7 +200,7 @@ public T getClusterId() { } public ClusterDescriptor createClusterDescriptor() { - return activeCommandLine.createClusterDescriptor(executorConfig); + return clusterClientFactory.createClusterDescriptor(executorConfig); } public EnvironmentInstance createEnvironmentInstance() { @@ -227,11 +243,10 @@ private static CommandLine createCommandLine(DeploymentEntry deployment, Options } } - @SuppressWarnings("unchecked") - private static CustomCommandLine findActiveCommandLine(List> availableCommandLines, CommandLine commandLine) { - for (CustomCommandLine cli : availableCommandLines) { + private static CustomCommandLine findActiveCommandLine(List availableCommandLines, CommandLine commandLine) { + for (CustomCommandLine cli : availableCommandLines) { if (cli.isActive(commandLine)) { - return (CustomCommandLine) cli; + return cli; } } throw new SqlExecutionException("Could not find a matching deployment."); 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 605a3661849e..9796b4cb80ca 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 @@ -24,7 +24,9 @@ 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.ClusterClientServiceLoader; import org.apache.flink.client.deployment.ClusterDescriptor; +import org.apache.flink.client.deployment.DefaultClusterClientServiceLoader; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; @@ -72,6 +74,8 @@ import java.util.List; import java.util.Map; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * Executor that performs the Flink communication locally. The calls are blocking depending on the * response time to the Flink cluster. Flink jobs are not blocking. @@ -84,10 +88,11 @@ public class LocalExecutor implements Executor { // deployment + private final ClusterClientServiceLoader clusterClientServiceLoader; private final Environment defaultEnvironment; private final List dependencies; private final Configuration flinkConfig; - private final List> commandLines; + private final List commandLines; private final Options commandLineOptions; // result maintenance @@ -159,12 +164,14 @@ public LocalExecutor(URL defaultEnv, List jars, List libraries) { // prepare result store resultStore = new ResultStore(flinkConfig); + + clusterClientServiceLoader = new DefaultClusterClientServiceLoader(); } /** * Constructor for testing purposes. */ - public LocalExecutor(Environment defaultEnvironment, List dependencies, Configuration flinkConfig, CustomCommandLine commandLine) { + public LocalExecutor(Environment defaultEnvironment, List dependencies, Configuration flinkConfig, CustomCommandLine commandLine, ClusterClientServiceLoader clusterClientServiceLoader) { this.defaultEnvironment = defaultEnvironment; this.dependencies = dependencies; this.flinkConfig = flinkConfig; @@ -172,7 +179,8 @@ public LocalExecutor(Environment defaultEnvironment, List dependencies, Con this.commandLineOptions = collectCommandLineOptions(commandLines); // prepare result store - resultStore = new ResultStore(flinkConfig); + this.resultStore = new ResultStore(flinkConfig); + this.clusterClientServiceLoader = checkNotNull(clusterClientServiceLoader); } @Override @@ -562,7 +570,7 @@ private synchronized ExecutionContext getOrCreateExecutionContext(SessionCont if (executionContext == null || !executionContext.getSessionContext().equals(session)) { try { executionContext = new ExecutionContext<>(defaultEnvironment, session, dependencies, - flinkConfig, commandLineOptions, commandLines); + flinkConfig, clusterClientServiceLoader, commandLineOptions, commandLines); } catch (Throwable t) { // catch everything such that a configuration does not crash the executor throw new SqlExecutionException("Could not create execution context.", t); @@ -614,9 +622,9 @@ private static List discoverDependencies(List jars, List librarie return dependencies; } - private static Options collectCommandLineOptions(List> commandLines) { + private static Options collectCommandLineOptions(List commandLines) { final Options customOptions = new Options(); - for (CustomCommandLine customCommandLine : commandLines) { + for (CustomCommandLine customCommandLine : commandLines) { customCommandLine.addRunOptions(customOptions); } return CliFrontendParser.mergeOptions( diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java index aad2da1c898f..64f52331ad90 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java @@ -19,6 +19,7 @@ package org.apache.flink.table.client.gateway.local; import org.apache.flink.client.cli.DefaultCLI; +import org.apache.flink.client.deployment.DefaultClusterClientServiceLoader; import org.apache.flink.configuration.Configuration; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableSchema; @@ -91,7 +92,8 @@ public void testTableFactoryDiscovery() throws Exception { env, Collections.singletonList(dependency), new Configuration(), - new DefaultCLI(new Configuration())); + new DefaultCLI(new Configuration()), + new DefaultClusterClientServiceLoader()); final SessionContext session = new SessionContext("test-session", new Environment()); 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 923d14149ac8..8731b69c2112 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 @@ -23,6 +23,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.client.cli.util.DummyClusterClientServiceLoader; import org.apache.flink.client.cli.util.DummyCustomCommandLine; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.ConfigConstants; @@ -609,7 +610,8 @@ private LocalExecutor createDefaultExecutor(ClusterClient clusterClient) EnvironmentFileUtil.parseModified(DEFAULTS_ENVIRONMENT_FILE, replaceVars), Collections.emptyList(), clusterClient.getFlinkConfiguration(), - new DummyCustomCommandLine(clusterClient)); + new DummyCustomCommandLine(), + new DummyClusterClientServiceLoader(clusterClient)); } private LocalExecutor createModifiedExecutor(ClusterClient clusterClient, Map replaceVars) throws Exception { @@ -617,7 +619,8 @@ private LocalExecutor createModifiedExecutor(ClusterClient clusterClient, EnvironmentFileUtil.parseModified(DEFAULTS_ENVIRONMENT_FILE, replaceVars), Collections.emptyList(), clusterClient.getFlinkConfiguration(), - new DummyCustomCommandLine(clusterClient)); + new DummyCustomCommandLine(), + new DummyClusterClientServiceLoader(clusterClient)); } private LocalExecutor createModifiedExecutor( @@ -626,7 +629,8 @@ private LocalExecutor createModifiedExecutor( EnvironmentFileUtil.parseModified(yamlFile, replaceVars), Collections.emptyList(), clusterClient.getFlinkConfiguration(), - new DummyCustomCommandLine(clusterClient)); + new DummyCustomCommandLine(), + new DummyClusterClientServiceLoader(clusterClient)); } private List retrieveTableResult( 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 e16abc017f64..4e7ece316a54 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,6 +20,8 @@ import org.apache.flink.client.cli.CliFrontendTestBase; import org.apache.flink.client.cli.CliFrontendTestUtils; +import org.apache.flink.client.deployment.ClusterClientFactory; +import org.apache.flink.client.deployment.ClusterClientServiceLoader; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; @@ -36,6 +38,7 @@ import org.junit.rules.TemporaryFolder; import static org.apache.flink.client.cli.CliFrontendRunTest.verifyCliFrontend; +import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.yarn.util.YarnTestUtils.getTestJarPath; /** @@ -67,36 +70,50 @@ public void testRun() throws Exception { configuration.setString(JobManagerOptions.ADDRESS, "localhost"); configuration.setInteger(JobManagerOptions.PORT, 8081); - FlinkYarnSessionCli yarnCLI = new TestingFlinkYarnSessionCli( + final ClusterClientServiceLoader testServiceLoader = + new TestingYarnClusterClientServiceLoader(new FakeClusterClient()); + + final FlinkYarnSessionCli yarnCLI = new FlinkYarnSessionCli( configuration, + testServiceLoader, tmp.getRoot().getAbsolutePath(), "y", - "yarn"); + "yarn", + true); // test detached mode { String[] parameters = {"-m", "yarn-cluster", "-p", "2", "-d", testJarPath}; - verifyCliFrontend(yarnCLI, parameters, 2, true); + verifyCliFrontend(testServiceLoader, yarnCLI, parameters, 2, true); } // test detached mode { String[] parameters = {"-m", "yarn-cluster", "-p", "2", "-yd", testJarPath}; - verifyCliFrontend(yarnCLI, parameters, 2, true); + verifyCliFrontend(testServiceLoader, yarnCLI, parameters, 2, true); } } - private static class TestingFlinkYarnSessionCli extends FlinkYarnSessionCli { + private static class TestingYarnClusterClientServiceLoader implements ClusterClientServiceLoader { + private final ClusterClient clusterClient; - private TestingFlinkYarnSessionCli( - Configuration configuration, - String configurationDirectory, - String shortPrefix, - String longPrefix) throws Exception { - super(configuration, configurationDirectory, shortPrefix, longPrefix); + TestingYarnClusterClientServiceLoader(ClusterClient clusterClient) { + this.clusterClient = checkNotNull(clusterClient); + } - this.clusterClient = new FakeClusterClient(configuration); + @Override + public ClusterClientFactory getClusterClientFactory(Configuration configuration) { + return new TestingYarnClusterClientFactory(clusterClient); + } + } + + private static class TestingYarnClusterClientFactory extends YarnClusterClientFactory { + + private final ClusterClient clusterClient; + + TestingYarnClusterClientFactory(ClusterClient clusterClient) { + this.clusterClient = checkNotNull(clusterClient); } @Override @@ -109,5 +126,4 @@ public YarnClusterDescriptor createClusterDescriptor(Configuration configuration clusterClient); } } - } 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 2aaa6fb00e73..e301a74beb7b 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 @@ -21,10 +21,12 @@ import org.apache.flink.client.cli.AbstractCustomCommandLine; import org.apache.flink.client.cli.CliArgsException; import org.apache.flink.client.cli.CliFrontend; +import org.apache.flink.client.deployment.ClusterClientFactory; +import org.apache.flink.client.deployment.ClusterClientServiceLoader; import org.apache.flink.client.deployment.ClusterSpecification; +import org.apache.flink.client.deployment.DefaultClusterClientServiceLoader; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.ConfigurationUtils; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.DeploymentOptions; import org.apache.flink.configuration.GlobalConfiguration; @@ -39,6 +41,7 @@ import org.apache.flink.util.ExecutorUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.ShutdownHookUtil; +import org.apache.flink.yarn.YarnClusterClientFactory; import org.apache.flink.yarn.YarnClusterDescriptor; import org.apache.flink.yarn.configuration.YarnConfigOptions; @@ -67,13 +70,9 @@ import java.io.InputStream; import java.io.InputStreamReader; import java.io.OutputStream; -import java.io.UnsupportedEncodingException; import java.lang.reflect.UndeclaredThrowableException; -import java.net.URLDecoder; -import java.nio.charset.Charset; import java.util.Collections; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Properties; @@ -91,7 +90,7 @@ /** * Class handling the command line interface to the YARN session. */ -public class FlinkYarnSessionCli extends AbstractCustomCommandLine { +public class FlinkYarnSessionCli extends AbstractCustomCommandLine { private static final Logger LOG = LoggerFactory.getLogger(FlinkYarnSessionCli.class); //------------------------------------ Constants ------------------------- @@ -101,9 +100,6 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine shipFiles = decodeDirsToShipToCluster(configuration); - yarnClusterDescriptor.addShipFiles(shipFiles); - - final String queueName = configuration.getString(YarnConfigOptions.APPLICATION_QUEUE); - if (queueName != null) { - yarnClusterDescriptor.setQueue(queueName); - } - - final String dynamicPropertiesEncoded = configuration.getString(YarnConfigOptions.DYNAMIC_PROPERTIES); - yarnClusterDescriptor.setDynamicPropertiesEncoded(dynamicPropertiesEncoded); - - final boolean detached = !configuration.getBoolean(DeploymentOptions.ATTACHED); - yarnClusterDescriptor.setDetachedMode(detached); - - final String appName = configuration.getString(YarnConfigOptions.APPLICATION_NAME); - if (appName != null) { - yarnClusterDescriptor.setName(appName); - } - - final String appType = configuration.getString(YarnConfigOptions.APPLICATION_TYPE); - if (appType != null) { - yarnClusterDescriptor.setApplicationType(appType); - } - - final String zkNamespace = configuration.getString(HA_CLUSTER_ID); - if (zkNamespace != null) { - yarnClusterDescriptor.setZookeeperNamespace(zkNamespace); - } - - final String nodeLabel = configuration.getString(YarnConfigOptions.NODE_LABEL); - if (nodeLabel != null) { - yarnClusterDescriptor.setNodeLabel(nodeLabel); - } - - return yarnClusterDescriptor; - } - - private Path getLocalFlinkDistPath(final Configuration configuration, final YarnClusterDescriptor yarnClusterDescriptor) { - final String localJarPath = configuration.getString(YarnConfigOptions.FLINK_DIST_JAR); - if (localJarPath != null) { - return new Path(localJarPath); - } - - LOG.info("No path for the flink jar passed. Using the location of " + yarnClusterDescriptor.getClass() + " to locate the jar"); - - // check whether it's actually a jar file --> when testing we execute this class without a flink-dist jar - final String decodedPath = getDecodedJarPath(yarnClusterDescriptor); - return decodedPath.endsWith(".jar") - ? new Path(new File(decodedPath).toURI()) - : null; - } - - private String getDecodedJarPath(final YarnClusterDescriptor yarnClusterDescriptor) { - final String encodedJarPath = yarnClusterDescriptor - .getClass().getProtectionDomain().getCodeSource().getLocation().getPath(); - try { - return URLDecoder.decode(encodedJarPath, Charset.defaultCharset().name()); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException("Couldn't decode the encoded Flink dist jar path: " + encodedJarPath + - " Please supply a path manually via the -" + flinkJar.getOpt() + " option."); - } - } - private Path getLocalFlinkDistPathFromCmd(final CommandLine cmd) { final String flinkJarOptionName = flinkJar.getOpt(); if (!cmd.hasOption(flinkJarOptionName)) { @@ -340,11 +278,6 @@ private Path getLocalFlinkDistPathFromCmd(final CommandLine cmd) { return new Path(userPath); } - private List decodeDirsToShipToCluster(final Configuration configuration) { - checkNotNull(configuration); - return YarnConfigUtils.decodeListFromConfig(configuration, YarnConfigOptions.SHIP_DIRECTORIES, File::new); - } - private void encodeDirsToShipToCluster(final Configuration configuration, final CommandLine cmd) { checkNotNull(cmd); checkNotNull(configuration); @@ -365,22 +298,6 @@ private void encodeDirsToShipToCluster(final Configuration configuration, final } } - private ClusterSpecification createClusterSpecification(Configuration configuration) { - // JobManager Memory - final int jobManagerMemoryMB = ConfigurationUtils.getJobManagerHeapMemory(configuration).getMebiBytes(); - - // Task Managers memory - final int taskManagerMemoryMB = ConfigurationUtils.getTaskManagerHeapMemory(configuration).getMebiBytes(); - - int slotsPerTaskManager = configuration.getInteger(TaskManagerOptions.NUM_TASK_SLOTS); - - return new ClusterSpecification.ClusterSpecificationBuilder() - .setMasterMemoryMB(jobManagerMemoryMB) - .setTaskManagerMemoryMB(taskManagerMemoryMB) - .setSlotsPerTaskManager(slotsPerTaskManager) - .createClusterSpecification(); - } - private void printUsage() { System.out.println("Usage:"); HelpFormatter formatter = new HelpFormatter(); @@ -397,14 +314,14 @@ private void printUsage() { @Override public boolean isActive(CommandLine commandLine) { String jobManagerOption = commandLine.getOptionValue(addressOption.getOpt(), null); - boolean yarnJobManager = ID.equals(jobManagerOption); + boolean yarnJobManager = YarnClusterClientFactory.ID.equals(jobManagerOption); boolean yarnAppId = commandLine.hasOption(applicationId.getOpt()); return yarnJobManager || yarnAppId || (isYarnPropertiesFileMode(commandLine) && yarnApplicationIdFromYarnProperties != null); } @Override public String getId() { - return ID; + return YarnClusterClientFactory.ID; } @Override @@ -422,23 +339,6 @@ public void addGeneralOptions(Options baseOptions) { baseOptions.addOption(applicationId); } - @Override - public YarnClusterDescriptor createClusterDescriptor(Configuration configuration) { - return createDescriptor(configuration); - } - - @Override - @Nullable - public ApplicationId getClusterId(Configuration configuration) { - final String clusterId = configuration.getString(YarnConfigOptions.APPLICATION_ID); - return clusterId != null ? ConverterUtils.toApplicationId(clusterId) : null; - } - - @Override - public ClusterSpecification getClusterSpecification(Configuration configuration) { - return createClusterSpecification(configuration); - } - @Override public Configuration applyCommandLineOptionsToConfiguration(CommandLine commandLine) throws FlinkException { // we ignore the addressOption because it can only contain "yarn-cluster" @@ -628,7 +528,9 @@ public int run(String[] args) throws CliArgsException, FlinkException { } final Configuration configuration = applyCommandLineOptionsToConfiguration(cmd); - final YarnClusterDescriptor yarnClusterDescriptor = createClusterDescriptor(configuration); + final ClusterClientFactory yarnClusterClientFactory = clusterClientServiceLoader.getClusterClientFactory(configuration); + + final YarnClusterDescriptor yarnClusterDescriptor = (YarnClusterDescriptor) yarnClusterClientFactory.createClusterDescriptor(configuration); try { // Query cluster for metrics @@ -645,7 +547,7 @@ public int run(String[] args) throws CliArgsException, FlinkException { clusterClient = yarnClusterDescriptor.retrieve(yarnApplicationId); } else { - final ClusterSpecification clusterSpecification = getClusterSpecification(configuration); + final ClusterSpecification clusterSpecification = yarnClusterClientFactory.getClusterSpecification(configuration); clusterClient = yarnClusterDescriptor.deploySessionCluster(clusterSpecification); 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 ae566b4b2c06..a357126d972d 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 @@ -18,7 +18,10 @@ package org.apache.flink.yarn; +import org.apache.flink.client.deployment.ClusterClientFactory; +import org.apache.flink.client.deployment.ClusterClientServiceLoader; import org.apache.flink.client.deployment.ClusterSpecification; +import org.apache.flink.client.deployment.DefaultClusterClientServiceLoader; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.configuration.JobManagerOptions; @@ -86,7 +89,8 @@ public void testDynamicProperties() throws Exception { "-D", "akka.ask.timeout=5 min", "-D", "env.java.opts=-DappName=foobar", "-D", "security.ssl.internal.key-password=changeit"}); Configuration executorConfig = cli.applyCommandLineOptionsToConfiguration(cmd); - YarnClusterDescriptor flinkYarnDescriptor = cli.createClusterDescriptor(executorConfig); + ClusterClientFactory clientFactory = getClusterClientFactory(executorConfig); + YarnClusterDescriptor flinkYarnDescriptor = (YarnClusterDescriptor) clientFactory.createClusterDescriptor(executorConfig); Assert.assertNotNull(flinkYarnDescriptor); @@ -110,10 +114,10 @@ public void testCorrectSettingOfMaxSlots() throws Exception { "yarn"); final CommandLine commandLine = yarnCLI.parseCommandLineOptions(params, true); - final Configuration executorConfig = yarnCLI.applyCommandLineOptionsToConfiguration(commandLine); - final YarnClusterDescriptor descriptor = yarnCLI.createClusterDescriptor(executorConfig); - final ClusterSpecification clusterSpecification = yarnCLI.getClusterSpecification(executorConfig); + final Configuration executorConfig = yarnCLI.applyCommandLineOptionsToConfiguration(commandLine); + final ClusterClientFactory clientFactory = getClusterClientFactory(executorConfig); + final ClusterSpecification clusterSpecification = clientFactory.getClusterSpecification(executorConfig); // each task manager has 3 slots but the parallelism is 7. Thus the slots should be increased. assertEquals(3, clusterSpecification.getSlotsPerTaskManager()); @@ -132,9 +136,10 @@ public void testCorrectSettingOfDetachedMode() throws Exception { "yarn"); final CommandLine commandLine = yarnCLI.parseCommandLineOptions(params, true); - final Configuration executorConfig = yarnCLI.applyCommandLineOptionsToConfiguration(commandLine); - YarnClusterDescriptor descriptor = yarnCLI.createClusterDescriptor(executorConfig); + final Configuration executorConfig = yarnCLI.applyCommandLineOptionsToConfiguration(commandLine); + final ClusterClientFactory clientFactory = getClusterClientFactory(executorConfig); + final YarnClusterDescriptor descriptor = (YarnClusterDescriptor) clientFactory.createClusterDescriptor(executorConfig); // each task manager has 3 slots but the parallelism is 7. Thus the slots should be increased. assertTrue(descriptor.isDetachedMode()); @@ -153,9 +158,10 @@ public void testZookeeperNamespaceProperty() throws Exception { "yarn"); CommandLine commandLine = yarnCLI.parseCommandLineOptions(params, true); - Configuration executorConfig = yarnCLI.applyCommandLineOptionsToConfiguration(commandLine); - YarnClusterDescriptor descriptor = yarnCLI.createClusterDescriptor(executorConfig); + Configuration executorConfig = yarnCLI.applyCommandLineOptionsToConfiguration(commandLine); + ClusterClientFactory clientFactory = getClusterClientFactory(executorConfig); + YarnClusterDescriptor descriptor = (YarnClusterDescriptor) clientFactory.createClusterDescriptor(executorConfig); assertEquals(zkNamespaceCliInput, descriptor.getZookeeperNamespace()); } @@ -173,9 +179,10 @@ public void testNodeLabelProperty() throws Exception { "yarn"); CommandLine commandLine = yarnCLI.parseCommandLineOptions(params, true); - Configuration executorConfig = yarnCLI.applyCommandLineOptionsToConfiguration(commandLine); - YarnClusterDescriptor descriptor = yarnCLI.createClusterDescriptor(executorConfig); + Configuration executorConfig = yarnCLI.applyCommandLineOptionsToConfiguration(commandLine); + ClusterClientFactory clientFactory = getClusterClientFactory(executorConfig); + YarnClusterDescriptor descriptor = (YarnClusterDescriptor) clientFactory.createClusterDescriptor(executorConfig); assertEquals(nodeLabelCliInput, descriptor.getNodeLabel()); } @@ -198,9 +205,10 @@ public void testResumeFromYarnPropertiesFile() throws Exception { "yarn"); final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {}, true); - final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); - final ApplicationId clusterId = flinkYarnSessionCli.getClusterId(executorConfig); + final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); + final ClusterClientFactory clientFactory = getClusterClientFactory(executorConfig); + final ApplicationId clusterId = clientFactory.getClusterId(executorConfig); assertEquals(TEST_YARN_APPLICATION_ID, clusterId); } @@ -234,9 +242,10 @@ public void testResumeFromYarnID() throws Exception { "yarn"); final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {"-yid", TEST_YARN_APPLICATION_ID.toString()}, true); - final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); - final ApplicationId clusterId = flinkYarnSessionCli.getClusterId(executorConfig); + final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); + final ClusterClientFactory clientFactory = getClusterClientFactory(executorConfig); + final ApplicationId clusterId = clientFactory.getClusterId(executorConfig); assertEquals(TEST_YARN_APPLICATION_ID, clusterId); } @@ -251,9 +260,10 @@ public void testResumeFromYarnIDZookeeperNamespace() throws Exception { "yarn"); final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {"-yid", TEST_YARN_APPLICATION_ID.toString()}, true); - final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); - final YarnClusterDescriptor clusterDescriptor = flinkYarnSessionCli.createClusterDescriptor(executorConfig); + final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); + final ClusterClientFactory clientFactory = getClusterClientFactory(executorConfig); + final YarnClusterDescriptor clusterDescriptor = (YarnClusterDescriptor) clientFactory.createClusterDescriptor(executorConfig); final Configuration clusterDescriptorConfiguration = clusterDescriptor.getFlinkConfiguration(); @@ -274,11 +284,11 @@ public void testResumeFromYarnIDZookeeperNamespaceOverride() throws Exception { final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {"-yid", TEST_YARN_APPLICATION_ID.toString(), "-yz", overrideZkNamespace}, true); final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); + final ClusterClientFactory clientFactory = getClusterClientFactory(executorConfig); - final YarnClusterDescriptor clusterDescriptor = flinkYarnSessionCli.createClusterDescriptor(executorConfig); + final YarnClusterDescriptor clusterDescriptor = (YarnClusterDescriptor) clientFactory.createClusterDescriptor(executorConfig); final Configuration clusterDescriptorConfiguration = clusterDescriptor.getFlinkConfiguration(); - final String clusterId = clusterDescriptorConfiguration.getValue(HighAvailabilityOptions.HA_CLUSTER_ID); assertEquals(overrideZkNamespace, clusterId); } @@ -296,9 +306,11 @@ public void testYarnIDOverridesPropertiesFile() throws Exception { "y", "yarn"); final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[] {"-yid", TEST_YARN_APPLICATION_ID_2.toString() }, true); + final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); + final ClusterClientFactory clientFactory = getClusterClientFactory(executorConfig); + final ApplicationId clusterId = clientFactory.getClusterId(executorConfig); - final ApplicationId clusterId = flinkYarnSessionCli.getClusterId(executorConfig); assertEquals(TEST_YARN_APPLICATION_ID_2, clusterId); } @@ -325,9 +337,10 @@ public void testCommandLineClusterSpecification() throws Exception { "yarn"); CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(args, false); - Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); - final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(executorConfig); + Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); + ClusterClientFactory clientFactory = getClusterClientFactory(executorConfig); + ClusterSpecification clusterSpecification = clientFactory.getClusterSpecification(executorConfig); assertThat(clusterSpecification.getMasterMemoryMB(), is(jobManagerMemory)); assertThat(clusterSpecification.getTaskManagerMemoryMB(), is(taskManagerMemory)); @@ -356,9 +369,10 @@ public void testConfigurationClusterSpecification() throws Exception { "yarn"); CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(args, false); - Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); - final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(executorConfig); + Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); + ClusterClientFactory clientFactory = getClusterClientFactory(executorConfig); + ClusterSpecification clusterSpecification = clientFactory.getClusterSpecification(executorConfig); assertThat(clusterSpecification.getMasterMemoryMB(), is(jobManagerMemory)); assertThat(clusterSpecification.getTaskManagerMemoryMB(), is(taskManagerMemory)); @@ -378,9 +392,10 @@ public void testHeapMemoryPropertyWithoutUnit() throws Exception { "yarn"); final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(args, false); - final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); - final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(executorConfig); + final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); + final ClusterClientFactory clientFactory = getClusterClientFactory(executorConfig); + final ClusterSpecification clusterSpecification = clientFactory.getClusterSpecification(executorConfig); assertThat(clusterSpecification.getMasterMemoryMB(), is(1024)); assertThat(clusterSpecification.getTaskManagerMemoryMB(), is(2048)); @@ -398,9 +413,10 @@ public void testHeapMemoryPropertyWithUnitMB() throws Exception { "y", "yarn"); final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(args, false); - final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); - final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(executorConfig); + final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); + final ClusterClientFactory clientFactory = getClusterClientFactory(executorConfig); + final ClusterSpecification clusterSpecification = clientFactory.getClusterSpecification(executorConfig); assertThat(clusterSpecification.getMasterMemoryMB(), is(1024)); assertThat(clusterSpecification.getTaskManagerMemoryMB(), is(2048)); @@ -418,9 +434,10 @@ public void testHeapMemoryPropertyWithArbitraryUnit() throws Exception { "y", "yarn"); final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(args, false); - final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); - final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(executorConfig); + final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); + final ClusterClientFactory clientFactory = getClusterClientFactory(executorConfig); + final ClusterSpecification clusterSpecification = clientFactory.getClusterSpecification(executorConfig); assertThat(clusterSpecification.getMasterMemoryMB(), is(1024)); assertThat(clusterSpecification.getTaskManagerMemoryMB(), is(2048)); @@ -442,9 +459,10 @@ public void testHeapMemoryPropertyWithOldConfigKey() throws Exception { "yarn"); final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[0], false); - final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); - final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(executorConfig); + final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); + final ClusterClientFactory clientFactory = getClusterClientFactory(executorConfig); + final ClusterSpecification clusterSpecification = clientFactory.getClusterSpecification(executorConfig); assertThat(clusterSpecification.getMasterMemoryMB(), is(2048)); assertThat(clusterSpecification.getTaskManagerMemoryMB(), is(4096)); @@ -462,9 +480,10 @@ public void testHeapMemoryPropertyWithConfigDefaultValue() throws Exception { "yarn"); final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(new String[0], false); - final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); - final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(executorConfig); + final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); + final ClusterClientFactory clientFactory = getClusterClientFactory(executorConfig); + final ClusterSpecification clusterSpecification = clientFactory.getClusterSpecification(executorConfig); assertThat(clusterSpecification.getMasterMemoryMB(), is(1024)); assertThat(clusterSpecification.getTaskManagerMemoryMB(), is(1024)); @@ -480,19 +499,24 @@ public void testMultipleYarnShipOptions() throws Exception { "yarn"); final CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(args, false); - final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); - YarnClusterDescriptor flinkYarnDescriptor = flinkYarnSessionCli.createClusterDescriptor(executorConfig); + final Configuration executorConfig = flinkYarnSessionCli.applyCommandLineOptionsToConfiguration(commandLine); + final ClusterClientFactory clientFactory = getClusterClientFactory(executorConfig); + YarnClusterDescriptor flinkYarnDescriptor = (YarnClusterDescriptor) clientFactory.createClusterDescriptor(executorConfig); assertEquals(2, flinkYarnDescriptor.getShipFiles().size()); } - /////////// // Utils // /////////// + private ClusterClientFactory getClusterClientFactory(final Configuration executorConfig) { + final ClusterClientServiceLoader clusterClientServiceLoader = new DefaultClusterClientServiceLoader(); + return clusterClientServiceLoader.getClusterClientFactory(executorConfig); + } + private File writeYarnPropertiesFile(String contents) throws IOException { File tmpFolder = tmp.newFolder(); String currentUser = System.getProperty("user.name"); From 3c8af452ffa998c6f8bd96d0c042eb71482e9ac2 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Fri, 25 Oct 2019 11:46:26 +0200 Subject: [PATCH 396/746] [hotfix] Do not expose Yarn dynamic options --- .../generated/yarn_config_configuration.html | 5 --- .../flink/yarn/YarnClusterClientFactory.java | 3 +- .../flink/yarn/cli/FlinkYarnSessionCli.java | 3 +- .../yarn/configuration/YarnConfigOptions.java | 5 --- .../YarnConfigOptionsInternal.java | 37 +++++++++++++++++++ 5 files changed, 41 insertions(+), 12 deletions(-) create mode 100644 flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptionsInternal.java diff --git a/docs/_includes/generated/yarn_config_configuration.html b/docs/_includes/generated/yarn_config_configuration.html index 6cf3945db7e1..98655bfd16c0 100644 --- a/docs/_includes/generated/yarn_config_configuration.html +++ b/docs/_includes/generated/yarn_config_configuration.html @@ -7,11 +7,6 @@ - -

    $internal.yarn.dynamic-properties
    - (none) - **DO NOT USE** Specify YARN dynamic properties. -
    yarn.application-attempt-failures-validity-interval
    10000 diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClientFactory.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClientFactory.java index a0d9ab23d6fb..a954be81186f 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClientFactory.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClientFactory.java @@ -28,6 +28,7 @@ import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.yarn.cli.YarnConfigUtils; import org.apache.flink.yarn.configuration.YarnConfigOptions; +import org.apache.flink.yarn.configuration.YarnConfigOptionsInternal; import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.api.records.ApplicationId; @@ -78,7 +79,7 @@ public YarnClusterDescriptor createClusterDescriptor(Configuration configuration .ifPresent(yarnClusterDescriptor::addShipFiles); handleConfigOption(configuration, YarnConfigOptions.APPLICATION_QUEUE, yarnClusterDescriptor::setQueue); - handleConfigOption(configuration, YarnConfigOptions.DYNAMIC_PROPERTIES, yarnClusterDescriptor::setDynamicPropertiesEncoded); + handleConfigOption(configuration, YarnConfigOptionsInternal.DYNAMIC_PROPERTIES, yarnClusterDescriptor::setDynamicPropertiesEncoded); handleConfigOption(configuration, YarnConfigOptions.APPLICATION_NAME, yarnClusterDescriptor::setName); handleConfigOption(configuration, YarnConfigOptions.APPLICATION_TYPE, yarnClusterDescriptor::setApplicationType); handleConfigOption(configuration, YarnConfigOptions.NODE_LABEL, yarnClusterDescriptor::setNodeLabel); 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 e301a74beb7b..0efa6103d6b0 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 @@ -44,6 +44,7 @@ import org.apache.flink.yarn.YarnClusterClientFactory; import org.apache.flink.yarn.YarnClusterDescriptor; import org.apache.flink.yarn.configuration.YarnConfigOptions; +import org.apache.flink.yarn.configuration.YarnConfigOptionsInternal; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.HelpFormatter; @@ -413,7 +414,7 @@ private void applyDescriptorOptionToConfig(final CommandLine commandLine, final } final String dynamicPropertiesEncoded = encodeDynamicProperties(commandLine); - configuration.setString(YarnConfigOptions.DYNAMIC_PROPERTIES, dynamicPropertiesEncoded); + configuration.setString(YarnConfigOptionsInternal.DYNAMIC_PROPERTIES, dynamicPropertiesEncoded); final boolean detached = commandLine.hasOption(YARN_DETACHED_OPTION.getOpt()) || commandLine.hasOption(DETACHED_OPTION.getOpt()); configuration.setBoolean(DeploymentOptions.ATTACHED, !detached); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java b/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java index 8ef72936eed7..021552d5c791 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java @@ -211,11 +211,6 @@ public class YarnConfigOptions { .noDefaultValue() .withDescription("The location of the log config file, e.g. the path to your log4j.properties for log4j."); - public static final ConfigOption DYNAMIC_PROPERTIES = - key("$internal.yarn.dynamic-properties") - .noDefaultValue() - .withDescription("**DO NOT USE** Specify YARN dynamic properties."); - public static final ConfigOption SHIP_DIRECTORIES = key("yarn.ship-directories") .noDefaultValue() diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptionsInternal.java b/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptionsInternal.java new file mode 100644 index 000000000000..4910809fb490 --- /dev/null +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptionsInternal.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.yarn.configuration; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ConfigOption; + +import static org.apache.flink.configuration.ConfigOptions.key; + +/** + * Yarn configuration options that are not meant to be set by the user. + */ +@Internal +public class YarnConfigOptionsInternal { + + public static final ConfigOption DYNAMIC_PROPERTIES = + key("$internal.yarn.dynamic-properties") + .noDefaultValue() + .withDescription("**DO NOT USE** Specify YARN dynamic properties."); + +} From bf955c5a1e29d368d2ae4682c1b46fa1b2bbf17e Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Mon, 28 Oct 2019 20:13:41 +0100 Subject: [PATCH 397/746] [FLINK-14501] Make YarnClusterDescriptor initialization self-contained --- .../flink/yarn/YarnClusterClientFactory.java | 79 +-------------- .../flink/yarn/YarnClusterDescriptor.java | 96 ++++++++++++------- 2 files changed, 61 insertions(+), 114 deletions(-) diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClientFactory.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClientFactory.java index a954be81186f..aa138a723777 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClientFactory.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClientFactory.java @@ -20,34 +20,19 @@ import org.apache.flink.client.deployment.ClusterClientFactory; import org.apache.flink.client.deployment.ClusterSpecification; -import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ConfigurationUtils; import org.apache.flink.configuration.DeploymentOptions; -import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.configuration.TaskManagerOptions; -import org.apache.flink.yarn.cli.YarnConfigUtils; import org.apache.flink.yarn.configuration.YarnConfigOptions; -import org.apache.flink.yarn.configuration.YarnConfigOptionsInternal; -import org.apache.hadoop.fs.Path; 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.apache.hadoop.yarn.util.ConverterUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import javax.annotation.Nullable; -import java.io.File; -import java.io.UnsupportedEncodingException; -import java.net.URLDecoder; -import java.nio.charset.Charset; -import java.util.List; -import java.util.Optional; -import java.util.function.Consumer; - import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -55,8 +40,6 @@ */ public class YarnClusterClientFactory implements ClusterClientFactory { - private static final Logger LOG = LoggerFactory.getLogger(YarnClusterClientFactory.class); - public static final String ID = "yarn-cluster"; @Override @@ -68,23 +51,7 @@ public boolean isCompatibleWith(Configuration configuration) { @Override public YarnClusterDescriptor createClusterDescriptor(Configuration configuration) { checkNotNull(configuration); - - final YarnClusterDescriptor yarnClusterDescriptor = getClusterDescriptor(configuration); - yarnClusterDescriptor.setDetachedMode(!configuration.getBoolean(DeploymentOptions.ATTACHED)); - - getLocalFlinkDistPath(configuration, yarnClusterDescriptor) - .ifPresent(yarnClusterDescriptor::setLocalJarPath); - - decodeDirsToShipToCluster(configuration) - .ifPresent(yarnClusterDescriptor::addShipFiles); - - handleConfigOption(configuration, YarnConfigOptions.APPLICATION_QUEUE, yarnClusterDescriptor::setQueue); - handleConfigOption(configuration, YarnConfigOptionsInternal.DYNAMIC_PROPERTIES, yarnClusterDescriptor::setDynamicPropertiesEncoded); - handleConfigOption(configuration, YarnConfigOptions.APPLICATION_NAME, yarnClusterDescriptor::setName); - handleConfigOption(configuration, YarnConfigOptions.APPLICATION_TYPE, yarnClusterDescriptor::setApplicationType); - handleConfigOption(configuration, YarnConfigOptions.NODE_LABEL, yarnClusterDescriptor::setNodeLabel); - handleConfigOption(configuration, HighAvailabilityOptions.HA_CLUSTER_ID, yarnClusterDescriptor::setZookeeperNamespace); - return yarnClusterDescriptor; + return getClusterDescriptor(configuration); } @Nullable @@ -114,50 +81,6 @@ public ClusterSpecification getClusterSpecification(Configuration configuration) .createClusterSpecification(); } - private Optional> decodeDirsToShipToCluster(final Configuration configuration) { - checkNotNull(configuration); - - final List files = YarnConfigUtils.decodeListFromConfig(configuration, YarnConfigOptions.SHIP_DIRECTORIES, File::new); - return files.isEmpty() ? Optional.empty() : Optional.of(files); - } - - private Optional getLocalFlinkDistPath(final Configuration configuration, final YarnClusterDescriptor yarnClusterDescriptor) { - final String localJarPath = configuration.getString(YarnConfigOptions.FLINK_DIST_JAR); - if (localJarPath != null) { - return Optional.of(new Path(localJarPath)); - } - - LOG.info("No path for the flink jar passed. Using the location of " + yarnClusterDescriptor.getClass() + " to locate the jar"); - - // check whether it's actually a jar file --> when testing we execute this class without a flink-dist jar - final String decodedPath = getDecodedJarPath(yarnClusterDescriptor); - return decodedPath.endsWith(".jar") - ? Optional.of(new Path(new File(decodedPath).toURI())) - : Optional.empty(); - } - - private String getDecodedJarPath(final YarnClusterDescriptor yarnClusterDescriptor) { - final String encodedJarPath = yarnClusterDescriptor - .getClass().getProtectionDomain().getCodeSource().getLocation().getPath(); - try { - return URLDecoder.decode(encodedJarPath, Charset.defaultCharset().name()); - } catch (UnsupportedEncodingException e) { - throw new RuntimeException("Couldn't decode the encoded Flink dist jar path: " + encodedJarPath + - " You can supply a path manually via the command line."); - } - } - - private void handleConfigOption(final Configuration configuration, final ConfigOption option, final Consumer consumer) { - checkNotNull(configuration); - checkNotNull(option); - checkNotNull(consumer); - - final String value = configuration.getString(option); - if (value != null) { - consumer.accept(value); - } - } - private YarnClusterDescriptor getClusterDescriptor(Configuration configuration) { final YarnClient yarnClient = YarnClient.createYarnClient(); final YarnConfiguration yarnConfiguration = new YarnConfiguration(); 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 1b33324556c2..9b1618ee1c56 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 @@ -29,6 +29,7 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.DeploymentOptions; import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.configuration.IllegalConfigurationException; import org.apache.flink.configuration.JobManagerOptions; @@ -47,7 +48,9 @@ import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; import org.apache.flink.util.ShutdownHookUtil; +import org.apache.flink.yarn.cli.YarnConfigUtils; import org.apache.flink.yarn.configuration.YarnConfigOptions; +import org.apache.flink.yarn.configuration.YarnConfigOptionsInternal; import org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint; import org.apache.flink.yarn.entrypoint.YarnSessionClusterEntrypoint; @@ -87,8 +90,11 @@ import java.io.IOException; import java.io.ObjectOutputStream; import java.io.PrintStream; +import java.io.UnsupportedEncodingException; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; +import java.net.URLDecoder; +import java.nio.charset.Charset; import java.nio.file.FileVisitResult; import java.nio.file.Files; import java.nio.file.SimpleFileVisitor; @@ -108,6 +114,7 @@ import static org.apache.flink.configuration.ConfigConstants.ENV_FLINK_LIB_DIR; import static org.apache.flink.runtime.entrypoint.component.FileJobGraphRetriever.JOB_GRAPH_FILE_PATH; +import static org.apache.flink.util.Preconditions.checkNotNull; 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; @@ -128,23 +135,23 @@ public class YarnClusterDescriptor implements ClusterDescriptor { /** Lazily initialized list of files to ship. */ private final List shipFiles = new LinkedList<>(); - private String yarnQueue; + private final String yarnQueue; private Path flinkJarPath; - private String dynamicPropertiesEncoded; + private final String dynamicPropertiesEncoded; private final Configuration flinkConfiguration; - private boolean detached; + private final boolean detached; - private String customName; + private final String customName; - private String zookeeperNamespace; + private final String nodeLabel; - private String nodeLabel; + private final String applicationType; - private String applicationType; + private String zookeeperNamespace; private YarnConfigOptions.UserJarInclusion userJarInclusion; @@ -160,6 +167,51 @@ public YarnClusterDescriptor( this.flinkConfiguration = Preconditions.checkNotNull(flinkConfiguration); this.userJarInclusion = getUserJarInclusionMode(flinkConfiguration); + + getLocalFlinkDistPath(flinkConfiguration).ifPresent(this::setLocalJarPath); + decodeDirsToShipToCluster(flinkConfiguration).ifPresent(this::addShipFiles); + + this.detached = !flinkConfiguration.getBoolean(DeploymentOptions.ATTACHED); + this.yarnQueue = flinkConfiguration.getString(YarnConfigOptions.APPLICATION_QUEUE); + this.dynamicPropertiesEncoded = flinkConfiguration.getString(YarnConfigOptionsInternal.DYNAMIC_PROPERTIES); + this.customName = flinkConfiguration.getString(YarnConfigOptions.APPLICATION_NAME); + this.applicationType = flinkConfiguration.getString(YarnConfigOptions.APPLICATION_TYPE); + this.nodeLabel = flinkConfiguration.getString(YarnConfigOptions.NODE_LABEL); + + // we want to ignore the default value at this point. + this.zookeeperNamespace = flinkConfiguration.getString(HighAvailabilityOptions.HA_CLUSTER_ID, null); + } + + private Optional> decodeDirsToShipToCluster(final Configuration configuration) { + checkNotNull(configuration); + + final List files = YarnConfigUtils.decodeListFromConfig(configuration, YarnConfigOptions.SHIP_DIRECTORIES, File::new); + return files.isEmpty() ? Optional.empty() : Optional.of(files); + } + + private Optional getLocalFlinkDistPath(final Configuration configuration) { + final String localJarPath = configuration.getString(YarnConfigOptions.FLINK_DIST_JAR); + if (localJarPath != null) { + return Optional.of(new Path(localJarPath)); + } + + LOG.info("No path for the flink jar passed. Using the location of " + getClass() + " to locate the jar"); + + // check whether it's actually a jar file --> when testing we execute this class without a flink-dist jar + final String decodedPath = getDecodedJarPath(); + return decodedPath.endsWith(".jar") + ? Optional.of(new Path(new File(decodedPath).toURI())) + : Optional.empty(); + } + + private String getDecodedJarPath() { + final String encodedJarPath = getClass().getProtectionDomain().getCodeSource().getLocation().getPath(); + try { + return URLDecoder.decode(encodedJarPath, Charset.defaultCharset().name()); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException("Couldn't decode the encoded Flink dist jar path: " + encodedJarPath + + " You can supply a path manually via the command line."); + } } @VisibleForTesting @@ -191,10 +243,6 @@ 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"); @@ -215,10 +263,6 @@ public void addShipFiles(List shipFiles) { this.shipFiles.addAll(shipFiles); } - public void setDynamicPropertiesEncoded(String dynamicPropertiesEncoded) { - this.dynamicPropertiesEncoded = dynamicPropertiesEncoded; - } - public String getDynamicPropertiesEncoded() { return this.dynamicPropertiesEncoded; } @@ -286,14 +330,6 @@ private static boolean allocateResource(int[] nodeManagers, int toAllocate) { 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. */ @@ -306,7 +342,7 @@ public String getZookeeperNamespace() { return zookeeperNamespace; } - public void setZookeeperNamespace(String zookeeperNamespace) { + private void setZookeeperNamespace(String zookeeperNamespace) { this.zookeeperNamespace = zookeeperNamespace; } @@ -314,10 +350,6 @@ public String getNodeLabel() { return nodeLabel; } - public void setNodeLabel(String nodeLabel) { - this.nodeLabel = nodeLabel; - } - // ------------------------------------------------------------- // Lifecycle management // ------------------------------------------------------------- @@ -1293,14 +1325,6 @@ public String getClusterDescription() { } } - 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 { From ea6fddeea1fce742b44b3b7b5e1e48ac1815df8d Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Thu, 10 Oct 2019 11:39:51 +0200 Subject: [PATCH 398/746] [FLINK-14377] Parse the ProgramOptions to a Configuration. --- .../generated/deployment_configuration.html | 5 + .../generated/pipeline_configuration.html | 21 +++ .../savepoint_config_configuration.html | 21 +++ docs/ops/config.md | 5 + docs/ops/config.zh.md | 5 + .../apache/flink/client/cli/CliFrontend.java | 65 ++++----- .../flink/client/cli/CliFrontendParser.java | 13 +- .../client/cli/ExecutionConfigAccessor.java | 127 ++++++++++++++++++ .../cli/ExecutionConfigurationUtils.java | 96 +++++++++++++ .../apache/flink/client/cli/InfoOptions.java | 31 ----- .../flink/client/cli/ProgramOptions.java | 7 +- .../apache/flink/client/cli/RunOptions.java | 31 ----- .../cli/CliFrontendPackageProgramTest.java | 93 ++++++++----- .../flink/client/cli/CliFrontendRunTest.java | 32 +++-- .../configuration/DeploymentOptions.java | 6 + .../flink/configuration/PipelineOptions.java | 49 +++++++ .../ConfigOptionsDocGenerator.java | 1 + .../jobgraph/SavepointConfigOptions.java | 49 +++++++ .../jobgraph/SavepointRestoreSettings.java | 17 +++ .../gateway/local/ExecutionContext.java | 16 ++- 20 files changed, 528 insertions(+), 162 deletions(-) create mode 100644 docs/_includes/generated/pipeline_configuration.html create mode 100644 docs/_includes/generated/savepoint_config_configuration.html create mode 100644 flink-clients/src/main/java/org/apache/flink/client/cli/ExecutionConfigAccessor.java create mode 100644 flink-clients/src/main/java/org/apache/flink/client/cli/ExecutionConfigurationUtils.java delete mode 100644 flink-clients/src/main/java/org/apache/flink/client/cli/InfoOptions.java delete mode 100644 flink-clients/src/main/java/org/apache/flink/client/cli/RunOptions.java create mode 100644 flink-core/src/main/java/org/apache/flink/configuration/PipelineOptions.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SavepointConfigOptions.java diff --git a/docs/_includes/generated/deployment_configuration.html b/docs/_includes/generated/deployment_configuration.html index 2c652e4c1448..7f7bdccf8dd1 100644 --- a/docs/_includes/generated/deployment_configuration.html +++ b/docs/_includes/generated/deployment_configuration.html @@ -12,6 +12,11 @@ false Specifies if the pipeline is submitted in attached or detached mode. + +
    execution.shutdown-on-attached-exit
    + false + If the job is submitted in attached mode, perform a best-effort cluster shutdown when the CLI is terminated abruptly, e.g., in response to a user interrupt, such as typing Ctrl + C. +
    execution.target
    (none) diff --git a/docs/_includes/generated/pipeline_configuration.html b/docs/_includes/generated/pipeline_configuration.html new file mode 100644 index 000000000000..36cf71194f95 --- /dev/null +++ b/docs/_includes/generated/pipeline_configuration.html @@ -0,0 +1,21 @@ + + + + + + + + + + + + + + + + + + + + +
    KeyDefaultDescription
    pipeline.classpaths
    (none)A semicolon-separated list of the classpaths to package with the job jars to be sent to the cluster. These have to be valid URLs.
    pipeline.jars
    (none)A semicolon-separated list of the jars to package with the job jars to be sent to the cluster. These have to be valid paths.
    diff --git a/docs/_includes/generated/savepoint_config_configuration.html b/docs/_includes/generated/savepoint_config_configuration.html new file mode 100644 index 000000000000..10318f4c4685 --- /dev/null +++ b/docs/_includes/generated/savepoint_config_configuration.html @@ -0,0 +1,21 @@ + + + + + + + + + + + + + + + + + + + + +
    KeyDefaultDescription
    execution.savepoint.ignore-unclaimed-state
    falseAllow to skip savepoint state that cannot be restored. Allow this if you removed an operator from your pipeline after the savepoint was triggered.
    execution.savepoint.path
    (none)Path to a savepoint to restore the job from (for example hdfs:///flink/savepoint-1537).
    diff --git a/docs/ops/config.md b/docs/ops/config.md index 6a8f4cec4df4..9390dbde5975 100644 --- a/docs/ops/config.md +++ b/docs/ops/config.md @@ -67,6 +67,7 @@ These parameters configure the default HDFS used by Flink. Setups that do not sp ### Execution {% include generated/deployment_configuration.html %} +{% include generated/savepoint_config_configuration.html %} ### JobManager @@ -188,6 +189,10 @@ The configuration keys in this section are independent of the used resource mana {% include generated/environment_configuration.html %} +### Pipeline + +{% include generated/pipeline_configuration.html %} + ### Checkpointing {% include generated/checkpointing_configuration.html %} diff --git a/docs/ops/config.zh.md b/docs/ops/config.zh.md index 1274f6d16e56..d3faea08d35f 100644 --- a/docs/ops/config.zh.md +++ b/docs/ops/config.zh.md @@ -67,6 +67,7 @@ These parameters configure the default HDFS used by Flink. Setups that do not sp ### Execution {% include generated/deployment_configuration.html %} +{% include generated/savepoint_config_configuration.html %} ### JobManager @@ -188,6 +189,10 @@ The configuration keys in this section are independent of the used resource mana {% include generated/environment_configuration.html %} +### Pipeline + +{% include generated/pipeline_configuration.html %} + ### Checkpointing {% include generated/checkpointing_configuration.html %} 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 7e0d28853193..2c7da42d1c42 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 @@ -56,7 +56,6 @@ import org.apache.flink.runtime.util.EnvironmentInformation; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; -import org.apache.flink.util.Preconditions; import org.apache.flink.util.ShutdownHookUtil; import org.apache.commons.cli.CommandLine; @@ -83,6 +82,7 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import static org.apache.flink.client.cli.CliFrontendParser.HELP_OPTION; import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -184,17 +184,18 @@ protected void run(String[] args) throws Exception { final CommandLine commandLine = CliFrontendParser.parse(commandLineOptions, args, true); - final RunOptions runOptions = new RunOptions(commandLine); + final ProgramOptions programOptions = new ProgramOptions(commandLine); + final ExecutionConfigAccessor executionParameters = ExecutionConfigAccessor.fromProgramOptions(programOptions); // evaluate help flag - if (runOptions.isPrintHelp()) { + if (commandLine.hasOption(HELP_OPTION.getOpt())) { CliFrontendParser.printHelpForRun(customCommandLines); return; } - if (!runOptions.isPython()) { + if (!programOptions.isPython()) { // Java program should be specified a JAR file - if (runOptions.getJarFilePath() == null) { + if (executionParameters.getJarFilePath() == null) { throw new CliArgsException("Java program should be specified a JAR file."); } } @@ -202,7 +203,7 @@ protected void run(String[] args) throws Exception { final PackagedProgram program; try { LOG.info("Building program from JAR file"); - program = buildProgram(runOptions); + program = buildProgram(programOptions, executionParameters); } catch (FileNotFoundException e) { throw new CliArgsException("Could not build the program from JAR file.", e); @@ -210,8 +211,9 @@ protected void run(String[] args) throws Exception { final CustomCommandLine customCommandLine = getActiveCustomCommandLine(commandLine); final Configuration executorConfig = customCommandLine.applyCommandLineOptionsToConfiguration(commandLine); + final Configuration executionConfig = executionParameters.getConfiguration(); try { - runProgram(executorConfig, runOptions, program); + runProgram(executorConfig, executionConfig, program); } finally { program.deleteExtractedLibraries(); } @@ -219,7 +221,7 @@ protected void run(String[] args) throws Exception { private void runProgram( Configuration executorConfig, - RunOptions runOptions, + Configuration executionConfig, PackagedProgram program) throws ProgramInvocationException, FlinkException { final ClusterClientFactory clusterClientFactory = clusterClientServiceLoader.getClusterClientFactory(executorConfig); @@ -229,12 +231,12 @@ private void runProgram( try { final T clusterId = clusterClientFactory.getClusterId(executorConfig); - + final ExecutionConfigAccessor executionParameters = ExecutionConfigAccessor.fromConfiguration(executionConfig); final ClusterClient client; // directly deploy the job if the cluster is started in job mode and detached - if (clusterId == null && runOptions.getDetachedMode()) { - int parallelism = runOptions.getParallelism() == -1 ? defaultParallelism : runOptions.getParallelism(); + if (clusterId == null && executionParameters.getDetachedMode()) { + int parallelism = executionParameters.getParallelism() == -1 ? defaultParallelism : executionParameters.getParallelism(); final JobGraph jobGraph = PackagedProgramUtils.createJobGraph(program, configuration, parallelism); @@ -242,7 +244,7 @@ private void runProgram( client = clusterDescriptor.deployJobCluster( clusterSpecification, jobGraph, - runOptions.getDetachedMode()); + executionParameters.getDetachedMode()); logAndSysout("Job has been submitted with JobID " + jobGraph.getJobID()); @@ -263,7 +265,7 @@ private void runProgram( client = clusterDescriptor.deploySessionCluster(clusterSpecification); // if not running in detached mode, add a shutdown hook to shut down cluster if client exits // there's a race-condition here if cli is killed before shutdown hook is installed - if (!runOptions.getDetachedMode() && runOptions.isShutdownOnAttachedExit()) { + if (!executionParameters.getDetachedMode() && executionParameters.isShutdownOnAttachedExit()) { shutdownHook = ShutdownHookUtil.addShutdownHook(client::shutDownCluster, client.getClass().getSimpleName(), LOG); } else { shutdownHook = null; @@ -271,11 +273,9 @@ private void runProgram( } try { - client.setDetached(runOptions.getDetachedMode()); - - LOG.debug("{}", runOptions.getSavepointRestoreSettings()); + client.setDetached(executionParameters.getDetachedMode()); - int userParallelism = runOptions.getParallelism(); + int userParallelism = executionParameters.getParallelism(); LOG.debug("User parallelism is set to {}", userParallelism); if (ExecutionConfig.PARALLELISM_DEFAULT == userParallelism) { userParallelism = defaultParallelism; @@ -323,25 +323,26 @@ protected void info(String[] args) throws CliArgsException, FileNotFoundExceptio final CommandLine commandLine = CliFrontendParser.parse(commandOptions, args, true); - InfoOptions infoOptions = new InfoOptions(commandLine); + final ProgramOptions programOptions = new ProgramOptions(commandLine); + final ExecutionConfigAccessor executionParameters = ExecutionConfigAccessor.fromProgramOptions(programOptions); // evaluate help flag - if (infoOptions.isPrintHelp()) { + if (commandLine.hasOption(HELP_OPTION.getOpt())) { CliFrontendParser.printHelpForInfo(); return; } - if (infoOptions.getJarFilePath() == null) { + if (programOptions.getJarFilePath() == null) { throw new CliArgsException("The program JAR file was not specified."); } // -------- build the packaged program ------------- LOG.info("Building program from JAR file"); - final PackagedProgram program = buildProgram(infoOptions); + final PackagedProgram program = buildProgram(programOptions, executionParameters); try { - int parallelism = infoOptions.getParallelism(); + int parallelism = programOptions.getParallelism(); if (ExecutionConfig.PARALLELISM_DEFAULT == parallelism) { parallelism = defaultParallelism; } @@ -722,7 +723,7 @@ private String triggerSavepoint(ClusterClient clusterClient, JobID jobId, Str * Sends a SavepointDisposalRequest to the job manager. */ private void disposeSavepoint(ClusterClient clusterClient, String savepointPath) throws FlinkException { - Preconditions.checkNotNull(savepointPath, "Missing required argument: savepoint path. " + + checkNotNull(savepointPath, "Missing required argument: savepoint path. " + "Usage: bin/flink savepoint -d "); logAndSysout("Disposing savepoint '" + savepointPath + "'."); @@ -769,15 +770,17 @@ protected void executeProgram(PackagedProgram program, ClusterClient client, * * @return A PackagedProgram (upon success) */ - PackagedProgram buildProgram(ProgramOptions options) throws FileNotFoundException, ProgramInvocationException { - String[] programArgs = options.getProgramArgs(); - String jarFilePath = options.getJarFilePath(); - List classpaths = options.getClasspaths(); + PackagedProgram buildProgram( + final ProgramOptions runOptions, + final ExecutionConfigAccessor executionParameters) throws FileNotFoundException, ProgramInvocationException { + String[] programArgs = runOptions.getProgramArgs(); + String jarFilePath = executionParameters.getJarFilePath(); + List classpaths = executionParameters.getClasspaths(); // Get assembler class - String entryPointClass = options.getEntryPointClassName(); + String entryPointClass = runOptions.getEntryPointClassName(); File jarFile = null; - if (options.isPython()) { + if (runOptions.isPython()) { // If the job is specified a jar file if (jarFilePath != null) { jarFile = getJarFile(jarFilePath); @@ -799,7 +802,7 @@ PackagedProgram buildProgram(ProgramOptions options) throws FileNotFoundExceptio new PackagedProgram(jarFile, classpaths, programArgs) : new PackagedProgram(jarFile, classpaths, entryPointClass, programArgs); - program.setSavepointRestoreSettings(options.getSavepointRestoreSettings()); + program.setSavepointRestoreSettings(executionParameters.getSavepointRestoreSettings()); return program; } @@ -1190,7 +1193,7 @@ private static CustomCommandLine loadCustomCommandLine(String className, Object. // construct class types from the parameters Class[] types = new Class[params.length]; for (int i = 0; i < params.length; i++) { - Preconditions.checkNotNull(params[i], "Parameters for custom command-lines may not be null."); + checkNotNull(params[i], "Parameters for custom command-lines may not be null."); types[i] = params[i].getClass(); } diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java index 97ba33e05979..c0295c707312 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java @@ -192,7 +192,7 @@ public class CliFrontendParser { PYMODULE_OPTION.setArgName("pyModule"); } - private static final Options RUN_OPTIONS = getRunCommandOptions(); + static final Options RUN_OPTIONS = getRunCommandOptions(); private static Options buildGeneralOptions(Options options) { options.addOption(HELP_OPTION); @@ -445,17 +445,6 @@ public static SavepointRestoreSettings createSavepointRestoreSettings(CommandLin // Line Parsing // -------------------------------------------------------------------------------------------- - public static RunOptions parseRunCommand(String[] args) throws CliArgsException { - try { - DefaultParser parser = new DefaultParser(); - CommandLine line = parser.parse(RUN_OPTIONS, args, true); - return new RunOptions(line); - } - catch (ParseException e) { - throw new CliArgsException(e.getMessage()); - } - } - public static CommandLine parse(Options options, String[] args, boolean stopAtNonOptions) throws CliArgsException { final DefaultParser parser = new DefaultParser(); diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/ExecutionConfigAccessor.java b/flink-clients/src/main/java/org/apache/flink/client/cli/ExecutionConfigAccessor.java new file mode 100644 index 000000000000..194a352a34a5 --- /dev/null +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/ExecutionConfigAccessor.java @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.client.cli; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.DeploymentOptions; +import org.apache.flink.configuration.PipelineOptions; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; + +import java.io.File; +import java.net.MalformedURLException; +import java.net.URL; +import java.util.Collections; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Accessor that exposes config settings that are relevant for execution from an underlying {@link Configuration}. + */ +@Internal +public class ExecutionConfigAccessor { + + private final Configuration configuration; + + private ExecutionConfigAccessor(final Configuration configuration) { + this.configuration = checkNotNull(configuration); + } + + /** + * Creates an {@link ExecutionConfigAccessor} based on the provided {@link Configuration}. + */ + public static ExecutionConfigAccessor fromConfiguration(final Configuration configuration) { + return new ExecutionConfigAccessor(checkNotNull(configuration)); + } + + /** + * Creates an {@link ExecutionConfigAccessor} based on the provided {@link ProgramOptions} as provided by the user through the CLI. + */ + public static ExecutionConfigAccessor fromProgramOptions(final ProgramOptions options) { + checkNotNull(options); + + final Configuration configuration = new Configuration(); + configuration.setInteger(CoreOptions.DEFAULT_PARALLELISM, options.getParallelism()); + configuration.setBoolean(DeploymentOptions.ATTACHED, !options.getDetachedMode()); + configuration.setBoolean(DeploymentOptions.SHUTDOWN_IF_ATTACHED, options.isShutdownOnAttachedExit()); + + parseClasspathURLsToConfig(options.getClasspaths(), configuration); + parseJarURLToConfig(options.getJarFilePath(), configuration); + + SavepointRestoreSettings.toConfiguration(options.getSavepointRestoreSettings(), configuration); + + return new ExecutionConfigAccessor(configuration); + } + + private static void parseClasspathURLsToConfig(final List classpathURLs, final Configuration configuration) { + ExecutionConfigurationUtils.urlListToConfig( + classpathURLs, + PipelineOptions.LIST_SEPARATOR, + configuration, + PipelineOptions.CLASSPATHS); + } + + private static void parseJarURLToConfig(final String jarFile, final Configuration configuration) { + if (jarFile == null) { + return; + } + + try { + final URL jarUrl = new File(jarFile).getAbsoluteFile().toURI().toURL(); + final List jarUrlSingleton = Collections.singletonList(jarUrl); + ExecutionConfigurationUtils.urlListToConfig(jarUrlSingleton, PipelineOptions.LIST_SEPARATOR, configuration, PipelineOptions.JARS); + } catch (MalformedURLException e) { + throw new IllegalArgumentException("JAR file path invalid", e); + } + } + + public Configuration getConfiguration() { + return configuration; + } + + public String getJarFilePath() { + final List jarURL = ExecutionConfigurationUtils.urlListFromConfig(configuration, PipelineOptions.JARS, PipelineOptions.LIST_SEPARATOR); + if (jarURL != null && !jarURL.isEmpty()) { + return jarURL.get(0).getPath(); + } + return null; + } + + public List getClasspaths() { + return ExecutionConfigurationUtils.urlListFromConfig(configuration, PipelineOptions.CLASSPATHS, PipelineOptions.LIST_SEPARATOR); + } + + public int getParallelism() { + return configuration.getInteger(CoreOptions.DEFAULT_PARALLELISM); + } + + public boolean getDetachedMode() { + return !configuration.getBoolean(DeploymentOptions.ATTACHED); + } + + public SavepointRestoreSettings getSavepointRestoreSettings() { + return SavepointRestoreSettings.fromConfiguration(configuration); + } + + public boolean isShutdownOnAttachedExit() { + return configuration.getBoolean(DeploymentOptions.SHUTDOWN_IF_ATTACHED); + } +} diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/ExecutionConfigurationUtils.java b/flink-clients/src/main/java/org/apache/flink/client/cli/ExecutionConfigurationUtils.java new file mode 100644 index 000000000000..099984ae72a8 --- /dev/null +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/ExecutionConfigurationUtils.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.cli; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; + +import java.net.MalformedURLException; +import java.net.URL; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Utilities for parsing parameters in the {@link ExecutionConfigAccessor}. + */ +@Internal +class ExecutionConfigurationUtils { + + /** + * Parses a list of {@link URL URLs} to a string and puts it in the provided {@code configuration} as the value of the provided {@code option}. + * @param urls the list of URLs to parse + * @param delimiter the delimiter to be used to separate the members of the list in the string + * @param configuration the configuration object to put the list + * @param option the {@link ConfigOption option} to serve as the key for the list in the configuration + * @return the produced string to be put in the configuration. + */ + static String urlListToConfig( + final List urls, + final String delimiter, + final Configuration configuration, + final ConfigOption option) { + + checkNotNull(urls); + checkNotNull(delimiter); + checkNotNull(configuration); + checkNotNull(option); + + final String str = urls.stream().map(URL::toString).collect(Collectors.joining(delimiter)); + configuration.setString(option, str); + return str; + } + + /** + * Parses a string into a list of {@link URL URLs} from a given {@link Configuration}. + * @param configuration the configuration containing the string-ified list of URLs + * @param option the {@link ConfigOption option} whose value is the list of URLs + * @param delimiter the delimiter used to separate the members of the list in the string + * @return the produced list of URLs. + */ + static List urlListFromConfig( + final Configuration configuration, + final ConfigOption option, + final String delimiter) { + + checkNotNull(configuration); + checkNotNull(option); + checkNotNull(delimiter); + + final String urls = configuration.getString(option); + if (urls == null || urls.length() == 0) { + return Collections.emptyList(); + } + + try (final Stream urlTokens = Arrays.stream(urls.split(delimiter))) { + return urlTokens.map(str -> { + try { + return new URL(str); + } catch (MalformedURLException e) { + throw new IllegalArgumentException("Invalid URL", e); + } + }).collect(Collectors.toList()); + } + } +} diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/InfoOptions.java b/flink-clients/src/main/java/org/apache/flink/client/cli/InfoOptions.java deleted file mode 100644 index 559ce9497c7c..000000000000 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/InfoOptions.java +++ /dev/null @@ -1,31 +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.cli; - -import org.apache.commons.cli.CommandLine; - -/** - * Command line options for the INFO command. - */ -public class InfoOptions extends ProgramOptions { - - public InfoOptions(CommandLine line) throws CliArgsException { - super(line); - } -} diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/ProgramOptions.java b/flink-clients/src/main/java/org/apache/flink/client/cli/ProgramOptions.java index e3a9907f7cf7..be3b8118aa5f 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/ProgramOptions.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/ProgramOptions.java @@ -44,7 +44,7 @@ /** * Base class for command line options that refer to a JAR file program. */ -public abstract class ProgramOptions extends CommandLineOptions { +public class ProgramOptions extends CommandLineOptions { private final String jarFilePath; @@ -67,7 +67,7 @@ public abstract class ProgramOptions extends CommandLineOptions { */ private final boolean isPython; - protected ProgramOptions(CommandLine line) throws CliArgsException { + public ProgramOptions(CommandLine line) throws CliArgsException { super(line); String[] args = line.hasOption(ARGS_OPTION.getOpt()) ? @@ -168,8 +168,7 @@ protected ProgramOptions(CommandLine line) throws CliArgsException { parallelism = ExecutionConfig.PARALLELISM_DEFAULT; } - detachedMode = line.hasOption(DETACHED_OPTION.getOpt()) || line.hasOption( - YARN_DETACHED_OPTION.getOpt()); + detachedMode = line.hasOption(DETACHED_OPTION.getOpt()) || line.hasOption(YARN_DETACHED_OPTION.getOpt()); shutdownOnAttachedExit = line.hasOption(SHUTDOWN_IF_ATTACHED_OPTION.getOpt()); this.savepointSettings = CliFrontendParser.createSavepointRestoreSettings(line); diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/RunOptions.java b/flink-clients/src/main/java/org/apache/flink/client/cli/RunOptions.java deleted file mode 100644 index 08a15d3e90df..000000000000 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/RunOptions.java +++ /dev/null @@ -1,31 +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.cli; - -import org.apache.commons.cli.CommandLine; - -/** - * Command line options for the RUN command. - */ -public class RunOptions extends ProgramOptions { - - public RunOptions(CommandLine line) throws CliArgsException { - super(line); - } -} 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 873ba0041779..2efb8ca9bc25 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 @@ -29,6 +29,7 @@ import org.apache.flink.optimizer.costs.DefaultCostEstimator; import org.apache.flink.util.TestLogger; +import org.apache.commons.cli.CommandLine; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; @@ -78,11 +79,12 @@ public void setup() throws Exception { @Test public void testNonExistingJarFile() throws Exception { - ProgramOptions options = mock(ProgramOptions.class); - when(options.getJarFilePath()).thenReturn("/some/none/existing/path"); + ProgramOptions programOptions = mock(ProgramOptions.class); + ExecutionConfigAccessor executionOptions = mock(ExecutionConfigAccessor.class); + when(executionOptions.getJarFilePath()).thenReturn("/some/none/existing/path"); try { - frontend.buildProgram(options); + frontend.buildProgram(programOptions, executionOptions); fail("should throw an exception"); } catch (FileNotFoundException e) { @@ -92,11 +94,12 @@ public void testNonExistingJarFile() throws Exception { @Test public void testFileNotJarFile() throws Exception { - ProgramOptions options = mock(ProgramOptions.class); - when(options.getJarFilePath()).thenReturn(getNonJarFilePath()); + ProgramOptions programOptions = mock(ProgramOptions.class); + ExecutionConfigAccessor executionOptions = mock(ExecutionConfigAccessor.class); + when(executionOptions.getJarFilePath()).thenReturn(getNonJarFilePath()); try { - frontend.buildProgram(options); + frontend.buildProgram(programOptions, executionOptions); fail("should throw an exception"); } catch (ProgramInvocationException e) { @@ -114,12 +117,15 @@ public void testVariantWithExplicitJarAndArgumentsOption() throws Exception { URL[] classpath = new URL[] { new URL("file:///tmp/foo"), new URL("file:///tmp/bar") }; String[] reducedArguments = new String[] {"--debug", "true", "arg1", "arg2"}; - RunOptions options = CliFrontendParser.parseRunCommand(arguments); - assertEquals(getTestJarPath(), options.getJarFilePath()); - assertArrayEquals(classpath, options.getClasspaths().toArray()); - assertArrayEquals(reducedArguments, options.getProgramArgs()); + CommandLine commandLine = CliFrontendParser.parse(CliFrontendParser.RUN_OPTIONS, arguments, true); + ProgramOptions programOptions = new ProgramOptions(commandLine); + ExecutionConfigAccessor executionOptions = ExecutionConfigAccessor.fromProgramOptions(programOptions); - PackagedProgram prog = frontend.buildProgram(options); + assertEquals(getTestJarPath(), executionOptions.getJarFilePath()); + assertArrayEquals(classpath, executionOptions.getClasspaths().toArray()); + assertArrayEquals(reducedArguments, programOptions.getProgramArgs()); + + PackagedProgram prog = frontend.buildProgram(programOptions, executionOptions); Assert.assertArrayEquals(reducedArguments, prog.getArguments()); Assert.assertEquals(TEST_JAR_MAIN_CLASS, prog.getMainClassName()); @@ -135,12 +141,15 @@ public void testVariantWithExplicitJarAndNoArgumentsOption() throws Exception { URL[] classpath = new URL[] { new URL("file:///tmp/foo"), new URL("file:///tmp/bar") }; String[] reducedArguments = new String[] {"--debug", "true", "arg1", "arg2"}; - RunOptions options = CliFrontendParser.parseRunCommand(arguments); - assertEquals(getTestJarPath(), options.getJarFilePath()); - assertArrayEquals(classpath, options.getClasspaths().toArray()); - assertArrayEquals(reducedArguments, options.getProgramArgs()); + CommandLine commandLine = CliFrontendParser.parse(CliFrontendParser.RUN_OPTIONS, arguments, true); + ProgramOptions programOptions = new ProgramOptions(commandLine); + ExecutionConfigAccessor executionOptions = ExecutionConfigAccessor.fromProgramOptions(programOptions); + + assertEquals(getTestJarPath(), executionOptions.getJarFilePath()); + assertArrayEquals(classpath, executionOptions.getClasspaths().toArray()); + assertArrayEquals(reducedArguments, programOptions.getProgramArgs()); - PackagedProgram prog = frontend.buildProgram(options); + PackagedProgram prog = frontend.buildProgram(programOptions, executionOptions); Assert.assertArrayEquals(reducedArguments, prog.getArguments()); Assert.assertEquals(TEST_JAR_MAIN_CLASS, prog.getMainClassName()); @@ -156,12 +165,15 @@ public void testValidVariantWithNoJarAndNoArgumentsOption() throws Exception { URL[] classpath = new URL[] { new URL("file:///tmp/foo"), new URL("file:///tmp/bar") }; String[] reducedArguments = {"--debug", "true", "arg1", "arg2"}; - RunOptions options = CliFrontendParser.parseRunCommand(arguments); - assertEquals(getTestJarPath(), options.getJarFilePath()); - assertArrayEquals(classpath, options.getClasspaths().toArray()); - assertArrayEquals(reducedArguments, options.getProgramArgs()); + CommandLine commandLine = CliFrontendParser.parse(CliFrontendParser.RUN_OPTIONS, arguments, true); + ProgramOptions programOptions = new ProgramOptions(commandLine); + ExecutionConfigAccessor executionOptions = ExecutionConfigAccessor.fromProgramOptions(programOptions); + + assertEquals(getTestJarPath(), executionOptions.getJarFilePath()); + assertArrayEquals(classpath, executionOptions.getClasspaths().toArray()); + assertArrayEquals(reducedArguments, programOptions.getProgramArgs()); - PackagedProgram prog = frontend.buildProgram(options); + PackagedProgram prog = frontend.buildProgram(programOptions, executionOptions); Assert.assertArrayEquals(reducedArguments, prog.getArguments()); Assert.assertEquals(TEST_JAR_MAIN_CLASS, prog.getMainClassName()); @@ -182,13 +194,16 @@ public void testNonExistingFileWithArguments() throws Exception { URL[] classpath = new URL[] { new URL("file:///tmp/foo"), new URL("file:///tmp/bar") }; String[] reducedArguments = {"--debug", "true", "arg1", "arg2"}; - RunOptions options = CliFrontendParser.parseRunCommand(arguments); - assertEquals(arguments[4], options.getJarFilePath()); - assertArrayEquals(classpath, options.getClasspaths().toArray()); - assertArrayEquals(reducedArguments, options.getProgramArgs()); + CommandLine commandLine = CliFrontendParser.parse(CliFrontendParser.RUN_OPTIONS, arguments, true); + ProgramOptions programOptions = new ProgramOptions(commandLine); + ExecutionConfigAccessor executionOptions = ExecutionConfigAccessor.fromProgramOptions(programOptions); + + assertEquals(arguments[4], executionOptions.getJarFilePath()); + assertArrayEquals(classpath, executionOptions.getClasspaths().toArray()); + assertArrayEquals(reducedArguments, programOptions.getProgramArgs()); try { - frontend.buildProgram(options); + frontend.buildProgram(programOptions, executionOptions); fail("Should fail with an exception"); } catch (FileNotFoundException e) { @@ -200,12 +215,15 @@ public void testNonExistingFileWithArguments() throws Exception { public void testNonExistingFileWithoutArguments() throws Exception { String[] arguments = {"/some/none/existing/path"}; - RunOptions options = CliFrontendParser.parseRunCommand(arguments); - assertEquals(arguments[0], options.getJarFilePath()); - assertArrayEquals(new String[0], options.getProgramArgs()); + CommandLine commandLine = CliFrontendParser.parse(CliFrontendParser.RUN_OPTIONS, arguments, true); + ProgramOptions programOptions = new ProgramOptions(commandLine); + ExecutionConfigAccessor executionOptions = ExecutionConfigAccessor.fromProgramOptions(programOptions); + + assertEquals(arguments[0], executionOptions.getJarFilePath()); + assertArrayEquals(new String[0], programOptions.getProgramArgs()); try { - frontend.buildProgram(options); + frontend.buildProgram(programOptions, executionOptions); } catch (FileNotFoundException e) { // that's what we want @@ -258,13 +276,16 @@ public void testPlanWithExternalClass() throws Exception { URL[] classpath = new URL[] { new URL("file:///tmp/foo"), new URL("file:///tmp/bar") }; String[] reducedArguments = { "true", "arg1", "arg2" }; - RunOptions options = CliFrontendParser.parseRunCommand(arguments); - assertEquals(getTestJarPath(), options.getJarFilePath()); - assertArrayEquals(classpath, options.getClasspaths().toArray()); - assertEquals(TEST_JAR_CLASSLOADERTEST_CLASS, options.getEntryPointClassName()); - assertArrayEquals(reducedArguments, options.getProgramArgs()); + CommandLine commandLine = CliFrontendParser.parse(CliFrontendParser.RUN_OPTIONS, arguments, true); + ProgramOptions programOptions = new ProgramOptions(commandLine); + ExecutionConfigAccessor executionOptions = ExecutionConfigAccessor.fromProgramOptions(programOptions); + + assertEquals(getTestJarPath(), executionOptions.getJarFilePath()); + assertArrayEquals(classpath, executionOptions.getClasspaths().toArray()); + assertEquals(TEST_JAR_CLASSLOADERTEST_CLASS, programOptions.getEntryPointClassName()); + assertArrayEquals(reducedArguments, programOptions.getProgramArgs()); - PackagedProgram prog = spy(frontend.buildProgram(options)); + PackagedProgram prog = spy(frontend.buildProgram(programOptions, executionOptions)); ClassLoader testClassLoader = new ClassLoader(prog.getUserCodeClassLoader()) { @Override diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java index 2230917f9006..918b6dbd2b04 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java @@ -25,6 +25,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.commons.cli.CommandLine; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -81,8 +82,12 @@ public void testRun() throws Exception { // test configure savepoint path (no ignore flag) { String[] parameters = {"-s", "expectedSavepointPath", getTestJarPath()}; - RunOptions options = CliFrontendParser.parseRunCommand(parameters); - SavepointRestoreSettings savepointSettings = options.getSavepointRestoreSettings(); + + CommandLine commandLine = CliFrontendParser.parse(CliFrontendParser.RUN_OPTIONS, parameters, true); + ProgramOptions programOptions = new ProgramOptions(commandLine); + ExecutionConfigAccessor executionOptions = ExecutionConfigAccessor.fromProgramOptions(programOptions); + + SavepointRestoreSettings savepointSettings = executionOptions.getSavepointRestoreSettings(); assertTrue(savepointSettings.restoreSavepoint()); assertEquals("expectedSavepointPath", savepointSettings.getRestorePath()); assertFalse(savepointSettings.allowNonRestoredState()); @@ -91,8 +96,12 @@ public void testRun() throws Exception { // test configure savepoint path (with ignore flag) { String[] parameters = {"-s", "expectedSavepointPath", "-n", getTestJarPath()}; - RunOptions options = CliFrontendParser.parseRunCommand(parameters); - SavepointRestoreSettings savepointSettings = options.getSavepointRestoreSettings(); + + CommandLine commandLine = CliFrontendParser.parse(CliFrontendParser.RUN_OPTIONS, parameters, true); + ProgramOptions programOptions = new ProgramOptions(commandLine); + ExecutionConfigAccessor executionOptions = ExecutionConfigAccessor.fromProgramOptions(programOptions); + + SavepointRestoreSettings savepointSettings = executionOptions.getSavepointRestoreSettings(); assertTrue(savepointSettings.restoreSavepoint()); assertEquals("expectedSavepointPath", savepointSettings.getRestorePath()); assertTrue(savepointSettings.allowNonRestoredState()); @@ -102,12 +111,15 @@ public void testRun() throws Exception { { String[] parameters = { getTestJarPath(), "-arg1", "value1", "justavalue", "--arg2", "value2"}; - RunOptions options = CliFrontendParser.parseRunCommand(parameters); - assertEquals("-arg1", options.getProgramArgs()[0]); - assertEquals("value1", options.getProgramArgs()[1]); - assertEquals("justavalue", options.getProgramArgs()[2]); - assertEquals("--arg2", options.getProgramArgs()[3]); - assertEquals("value2", options.getProgramArgs()[4]); + + CommandLine commandLine = CliFrontendParser.parse(CliFrontendParser.RUN_OPTIONS, parameters, true); + ProgramOptions programOptions = new ProgramOptions(commandLine); + + assertEquals("-arg1", programOptions.getProgramArgs()[0]); + assertEquals("value1", programOptions.getProgramArgs()[1]); + assertEquals("justavalue", programOptions.getProgramArgs()[2]); + assertEquals("--arg2", programOptions.getProgramArgs()[3]); + assertEquals("value2", programOptions.getProgramArgs()[4]); } } diff --git a/flink-core/src/main/java/org/apache/flink/configuration/DeploymentOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/DeploymentOptions.java index a032205cb6ab..a09fd6864563 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/DeploymentOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/DeploymentOptions.java @@ -37,4 +37,10 @@ public class DeploymentOptions { key("execution.attached") .defaultValue(false) .withDescription("Specifies if the pipeline is submitted in attached or detached mode."); + + public static final ConfigOption SHUTDOWN_IF_ATTACHED = + key("execution.shutdown-on-attached-exit") + .defaultValue(false) + .withDescription("If the job is submitted in attached mode, perform a best-effort cluster shutdown " + + "when the CLI is terminated abruptly, e.g., in response to a user interrupt, such as typing Ctrl + C."); } diff --git a/flink-core/src/main/java/org/apache/flink/configuration/PipelineOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/PipelineOptions.java new file mode 100644 index 000000000000..6ae8a3134dc3 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/configuration/PipelineOptions.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.configuration; + +import org.apache.flink.annotation.PublicEvolving; + +import static org.apache.flink.configuration.ConfigOptions.key; + +/** + * The {@link ConfigOption configuration options} for job execution. + */ +@PublicEvolving +public class PipelineOptions { + + public static final String LIST_SEPARATOR = ";"; + + /** + * A list of jar files that contain the user-defined function (UDF) classes and all classes used from within the UDFs. + */ + public static final ConfigOption JARS = + key("pipeline.jars") + .noDefaultValue() + .withDescription("A semicolon-separated list of the jars to package with the job jars to be sent to the cluster. These have to be valid paths."); + + /** + * 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 + */ + public static final ConfigOption CLASSPATHS = + key("pipeline.classpaths") + .noDefaultValue() + .withDescription("A semicolon-separated list of the classpaths to package with the job jars to be sent to the cluster. These have to be valid URLs."); +} diff --git a/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java b/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java index 999f85a17814..deb4e03dd8cd 100644 --- a/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java +++ b/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java @@ -58,6 +58,7 @@ public class ConfigOptionsDocGenerator { static final OptionsClassLocation[] LOCATIONS = new OptionsClassLocation[]{ new OptionsClassLocation("flink-core", "org.apache.flink.configuration"), new OptionsClassLocation("flink-runtime", "org.apache.flink.runtime.shuffle"), + new OptionsClassLocation("flink-runtime", "org.apache.flink.runtime.jobgraph"), new OptionsClassLocation("flink-yarn", "org.apache.flink.yarn.configuration"), new OptionsClassLocation("flink-mesos", "org.apache.flink.mesos.configuration"), new OptionsClassLocation("flink-mesos", "org.apache.flink.mesos.runtime.clusterframework"), diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SavepointConfigOptions.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SavepointConfigOptions.java new file mode 100644 index 000000000000..86fd38903fc6 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SavepointConfigOptions.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobgraph; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.ConfigOption; + +import static org.apache.flink.configuration.ConfigOptions.key; + +/** + * The {@link ConfigOption configuration options} used when restoring from a savepoint. + */ +@PublicEvolving +public class SavepointConfigOptions { + + /** + * The path to a savepoint that will be used to bootstrap the pipeline's state. + */ + public static final ConfigOption SAVEPOINT_PATH = + key("execution.savepoint.path") + .noDefaultValue() + .withDescription("Path to a savepoint to restore the job from (for example hdfs:///flink/savepoint-1537)."); + + /** + * A flag indicating if we allow Flink to skip savepoint state that cannot be restored, + * e.g. because the corresponding operator has been removed. + */ + public static final ConfigOption SAVEPOINT_IGNORE_UNCLAIMED_STATE = + key("execution.savepoint.ignore-unclaimed-state") + .defaultValue(false) + .withDescription("Allow to skip savepoint state that cannot be restored. " + + "Allow this if you removed an operator from your pipeline after the savepoint was triggered."); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SavepointRestoreSettings.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SavepointRestoreSettings.java index 953d11ce36b5..b7fee4e7a9c7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SavepointRestoreSettings.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SavepointRestoreSettings.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.jobgraph; +import org.apache.flink.configuration.Configuration; + import java.io.Serializable; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -131,4 +133,19 @@ public static SavepointRestoreSettings forPath(String savepointPath, boolean all return new SavepointRestoreSettings(savepointPath, allowNonRestoredState); } + // -------------------------- Parsing to and from a configuration object ------------------------------------ + + public static void toConfiguration(final SavepointRestoreSettings savepointRestoreSettings, final Configuration configuration) { + configuration.setBoolean(SavepointConfigOptions.SAVEPOINT_IGNORE_UNCLAIMED_STATE, savepointRestoreSettings.allowNonRestoredState()); + final String savepointPath = savepointRestoreSettings.getRestorePath(); + if (savepointPath != null) { + configuration.setString(SavepointConfigOptions.SAVEPOINT_PATH, savepointPath); + } + } + + public static SavepointRestoreSettings fromConfiguration(final Configuration configuration) { + final String savepointPath = configuration.getString(SavepointConfigOptions.SAVEPOINT_PATH); + final boolean allowNonRestored = configuration.getBoolean(SavepointConfigOptions.SAVEPOINT_IGNORE_UNCLAIMED_STATE); + return savepointPath == null ? SavepointRestoreSettings.none() : SavepointRestoreSettings.forPath(savepointPath, allowNonRestored); + } } 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 15331f34579e..a8e558fbd541 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 @@ -27,7 +27,8 @@ 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.cli.ExecutionConfigAccessor; +import org.apache.flink.client.cli.ProgramOptions; import org.apache.flink.client.deployment.ClusterClientFactory; import org.apache.flink.client.deployment.ClusterClientServiceLoader; import org.apache.flink.client.deployment.ClusterDescriptor; @@ -119,7 +120,7 @@ public class ExecutionContext { private final Configuration flinkConfig; private final Configuration executorConfig; private final ClusterClientFactory clusterClientFactory; - private final RunOptions runOptions; + private final ExecutionConfigAccessor executionParameters; private final T clusterId; private final ClusterSpecification clusterSpec; @@ -174,7 +175,7 @@ public ExecutionContext(Environment defaultEnvironment, SessionContext sessionCo clusterClientFactory = serviceLoader.getClusterClientFactory(executorConfig); checkState(clusterClientFactory != null); - runOptions = createRunOptions(commandLine); + executionParameters = createExecutionParameterProvider(commandLine); clusterId = clusterClientFactory.getClusterId(executorConfig); clusterSpec = clusterClientFactory.getClusterSpecification(executorConfig); } @@ -252,9 +253,10 @@ private static CustomCommandLine findActiveCommandLine(List a throw new SqlExecutionException("Could not find a matching deployment."); } - private static RunOptions createRunOptions(CommandLine commandLine) { + private static ExecutionConfigAccessor createExecutionParameterProvider(CommandLine commandLine) { try { - return new RunOptions(commandLine); + final ProgramOptions programOptions = new ProgramOptions(commandLine); + return ExecutionConfigAccessor.fromProgramOptions(programOptions); } catch (CliArgsException e) { throw new SqlExecutionException("Invalid deployment run options.", e); } @@ -464,8 +466,8 @@ public JobGraph createJobGraph(String name) { parallelism); ClientUtils.addJarFiles(jobGraph, dependencies); - jobGraph.setClasspaths(runOptions.getClasspaths()); - jobGraph.setSavepointRestoreSettings(runOptions.getSavepointRestoreSettings()); + jobGraph.setClasspaths(executionParameters.getClasspaths()); + jobGraph.setSavepointRestoreSettings(executionParameters.getSavepointRestoreSettings()); return jobGraph; } From 20a00050653cd10607b5017ffbb74946053fa962 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Thu, 31 Oct 2019 10:55:54 +0100 Subject: [PATCH 399/746] [hotfix] Rename T to ClusterID in CliFrontend and ExecutionContext --- .../apache/flink/client/cli/CliFrontend.java | 20 +++++++++---------- .../gateway/local/ExecutionContext.java | 12 +++++------ 2 files changed, 16 insertions(+), 16 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 2c7da42d1c42..33fb2533b307 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 @@ -219,20 +219,20 @@ protected void run(String[] args) throws Exception { } } - private void runProgram( + private void runProgram( Configuration executorConfig, Configuration executionConfig, PackagedProgram program) throws ProgramInvocationException, FlinkException { - final ClusterClientFactory clusterClientFactory = clusterClientServiceLoader.getClusterClientFactory(executorConfig); + final ClusterClientFactory clusterClientFactory = clusterClientServiceLoader.getClusterClientFactory(executorConfig); checkNotNull(clusterClientFactory); - final ClusterDescriptor clusterDescriptor = clusterClientFactory.createClusterDescriptor(executorConfig); + final ClusterDescriptor clusterDescriptor = clusterClientFactory.createClusterDescriptor(executorConfig); try { - final T clusterId = clusterClientFactory.getClusterId(executorConfig); + final ClusterID clusterId = clusterClientFactory.getClusterId(executorConfig); final ExecutionConfigAccessor executionParameters = ExecutionConfigAccessor.fromConfiguration(executionConfig); - final ClusterClient client; + final ClusterClient client; // directly deploy the job if the cluster is started in job mode and detached if (clusterId == null && executionParameters.getDetachedMode()) { @@ -422,8 +422,8 @@ protected void list(String[] args) throws Exception { } - private void listJobs( - ClusterClient clusterClient, + private void listJobs( + ClusterClient clusterClient, boolean showRunning, boolean showScheduled, boolean showAll) throws FlinkException { @@ -968,10 +968,10 @@ private void runClusterAction(CustomCommandLine activeCommandLine, C * Internal interface to encapsulate cluster actions which are executed via * the {@link ClusterClient}. * - * @param type of the cluster id + * @param type of the cluster id */ @FunctionalInterface - private interface ClusterAction { + private interface ClusterAction { /** * Run the cluster action with the given {@link ClusterClient}. @@ -979,7 +979,7 @@ private interface ClusterAction { * @param clusterClient to run the cluster action against * @throws FlinkException if something goes wrong */ - void runAction(ClusterClient clusterClient) throws FlinkException; + void runAction(ClusterClient clusterClient) throws FlinkException; } // -------------------------------------------------------------------------------------------- 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 a8e558fbd541..1fb0a3941aca 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 @@ -105,9 +105,9 @@ * multiple queries as long as the session context does not change. This must be thread-safe as * it might be reused across different query submissions. * - * @param cluster id + * @param cluster id */ -public class ExecutionContext { +public class ExecutionContext { private final SessionContext sessionContext; private final Environment mergedEnv; @@ -119,9 +119,9 @@ public class ExecutionContext { private final Map functions; private final Configuration flinkConfig; private final Configuration executorConfig; - private final ClusterClientFactory clusterClientFactory; + private final ClusterClientFactory clusterClientFactory; private final ExecutionConfigAccessor executionParameters; - private final T clusterId; + private final ClusterID clusterId; private final ClusterSpecification clusterSpec; public ExecutionContext(Environment defaultEnvironment, SessionContext sessionContext, List dependencies, @@ -196,11 +196,11 @@ public ClusterSpecification getClusterSpec() { return clusterSpec; } - public T getClusterId() { + public ClusterID getClusterId() { return clusterId; } - public ClusterDescriptor createClusterDescriptor() { + public ClusterDescriptor createClusterDescriptor() { return clusterClientFactory.createClusterDescriptor(executorConfig); } From 9241af09bdbb69a06b18edd115f2d500ba5c14b9 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Thu, 31 Oct 2019 13:53:44 +0100 Subject: [PATCH 400/746] [FLINK-14502] Use the newly introduced list ConfigOption wherever possible. --- .../client/cli/ExecutionConfigAccessor.java | 7 ++-- .../cli/ExecutionConfigurationUtils.java | 40 +++++++------------ .../configuration/DeploymentOptions.java | 3 ++ .../flink/configuration/PipelineOptions.java | 12 ++++-- .../jobgraph/SavepointConfigOptions.java | 2 + .../flink/yarn/cli/YarnConfigUtils.java | 20 +++++----- .../yarn/configuration/YarnConfigOptions.java | 13 +++++- 7 files changed, 52 insertions(+), 45 deletions(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/ExecutionConfigAccessor.java b/flink-clients/src/main/java/org/apache/flink/client/cli/ExecutionConfigAccessor.java index 194a352a34a5..ba5341481c22 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/ExecutionConfigAccessor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/ExecutionConfigAccessor.java @@ -74,7 +74,6 @@ public static ExecutionConfigAccessor fromProgramOptions(final ProgramOptions op private static void parseClasspathURLsToConfig(final List classpathURLs, final Configuration configuration) { ExecutionConfigurationUtils.urlListToConfig( classpathURLs, - PipelineOptions.LIST_SEPARATOR, configuration, PipelineOptions.CLASSPATHS); } @@ -87,7 +86,7 @@ private static void parseJarURLToConfig(final String jarFile, final Configuratio try { final URL jarUrl = new File(jarFile).getAbsoluteFile().toURI().toURL(); final List jarUrlSingleton = Collections.singletonList(jarUrl); - ExecutionConfigurationUtils.urlListToConfig(jarUrlSingleton, PipelineOptions.LIST_SEPARATOR, configuration, PipelineOptions.JARS); + ExecutionConfigurationUtils.urlListToConfig(jarUrlSingleton, configuration, PipelineOptions.JARS); } catch (MalformedURLException e) { throw new IllegalArgumentException("JAR file path invalid", e); } @@ -98,7 +97,7 @@ public Configuration getConfiguration() { } public String getJarFilePath() { - final List jarURL = ExecutionConfigurationUtils.urlListFromConfig(configuration, PipelineOptions.JARS, PipelineOptions.LIST_SEPARATOR); + final List jarURL = ExecutionConfigurationUtils.urlListFromConfig(configuration, PipelineOptions.JARS); if (jarURL != null && !jarURL.isEmpty()) { return jarURL.get(0).getPath(); } @@ -106,7 +105,7 @@ public String getJarFilePath() { } public List getClasspaths() { - return ExecutionConfigurationUtils.urlListFromConfig(configuration, PipelineOptions.CLASSPATHS, PipelineOptions.LIST_SEPARATOR); + return ExecutionConfigurationUtils.urlListFromConfig(configuration, PipelineOptions.CLASSPATHS); } public int getParallelism() { diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/ExecutionConfigurationUtils.java b/flink-clients/src/main/java/org/apache/flink/client/cli/ExecutionConfigurationUtils.java index 099984ae72a8..e25c129a4dc2 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/ExecutionConfigurationUtils.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/ExecutionConfigurationUtils.java @@ -24,11 +24,9 @@ import java.net.MalformedURLException; import java.net.URL; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.stream.Collectors; -import java.util.stream.Stream; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -41,24 +39,21 @@ class ExecutionConfigurationUtils { /** * Parses a list of {@link URL URLs} to a string and puts it in the provided {@code configuration} as the value of the provided {@code option}. * @param urls the list of URLs to parse - * @param delimiter the delimiter to be used to separate the members of the list in the string * @param configuration the configuration object to put the list * @param option the {@link ConfigOption option} to serve as the key for the list in the configuration - * @return the produced string to be put in the configuration. + * @return the produced list of strings to be put in the configuration. */ - static String urlListToConfig( + static List urlListToConfig( final List urls, - final String delimiter, final Configuration configuration, - final ConfigOption option) { + final ConfigOption> option) { checkNotNull(urls); - checkNotNull(delimiter); checkNotNull(configuration); checkNotNull(option); - final String str = urls.stream().map(URL::toString).collect(Collectors.joining(delimiter)); - configuration.setString(option, str); + final List str = urls.stream().map(URL::toString).collect(Collectors.toList()); + configuration.set(option, str); return str; } @@ -66,31 +61,26 @@ static String urlListToConfig( * Parses a string into a list of {@link URL URLs} from a given {@link Configuration}. * @param configuration the configuration containing the string-ified list of URLs * @param option the {@link ConfigOption option} whose value is the list of URLs - * @param delimiter the delimiter used to separate the members of the list in the string * @return the produced list of URLs. */ static List urlListFromConfig( final Configuration configuration, - final ConfigOption option, - final String delimiter) { + final ConfigOption> option) { checkNotNull(configuration); checkNotNull(option); - checkNotNull(delimiter); - final String urls = configuration.getString(option); - if (urls == null || urls.length() == 0) { + final List urls = configuration.get(option); + if (urls == null || urls.isEmpty()) { return Collections.emptyList(); } - try (final Stream urlTokens = Arrays.stream(urls.split(delimiter))) { - return urlTokens.map(str -> { - try { - return new URL(str); - } catch (MalformedURLException e) { - throw new IllegalArgumentException("Invalid URL", e); - } - }).collect(Collectors.toList()); - } + return urls.stream().map(str -> { + try { + return new URL(str); + } catch (MalformedURLException e) { + throw new IllegalArgumentException("Invalid URL", e); + } + }).collect(Collectors.toList()); } } diff --git a/flink-core/src/main/java/org/apache/flink/configuration/DeploymentOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/DeploymentOptions.java index a09fd6864563..9ffa93c7e5d4 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/DeploymentOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/DeploymentOptions.java @@ -30,16 +30,19 @@ public class DeploymentOptions { public static final ConfigOption TARGET = key("execution.target") + .stringType() .noDefaultValue() .withDescription("The deployment target for the execution, e.g. \"local\" for local execution."); public static final ConfigOption ATTACHED = key("execution.attached") + .booleanType() .defaultValue(false) .withDescription("Specifies if the pipeline is submitted in attached or detached mode."); public static final ConfigOption SHUTDOWN_IF_ATTACHED = key("execution.shutdown-on-attached-exit") + .booleanType() .defaultValue(false) .withDescription("If the job is submitted in attached mode, perform a best-effort cluster shutdown " + "when the CLI is terminated abruptly, e.g., in response to a user interrupt, such as typing Ctrl + C."); diff --git a/flink-core/src/main/java/org/apache/flink/configuration/PipelineOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/PipelineOptions.java index 6ae8a3134dc3..6b2c21d2f567 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/PipelineOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/PipelineOptions.java @@ -20,6 +20,8 @@ import org.apache.flink.annotation.PublicEvolving; +import java.util.List; + import static org.apache.flink.configuration.ConfigOptions.key; /** @@ -28,13 +30,13 @@ @PublicEvolving public class PipelineOptions { - public static final String LIST_SEPARATOR = ";"; - /** * A list of jar files that contain the user-defined function (UDF) classes and all classes used from within the UDFs. */ - public static final ConfigOption JARS = + public static final ConfigOption> JARS = key("pipeline.jars") + .stringType() + .asList() .noDefaultValue() .withDescription("A semicolon-separated list of the jars to package with the job jars to be sent to the cluster. These have to be valid paths."); @@ -42,8 +44,10 @@ public class PipelineOptions { * 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 */ - public static final ConfigOption CLASSPATHS = + public static final ConfigOption> CLASSPATHS = key("pipeline.classpaths") + .stringType() + .asList() .noDefaultValue() .withDescription("A semicolon-separated list of the classpaths to package with the job jars to be sent to the cluster. These have to be valid URLs."); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SavepointConfigOptions.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SavepointConfigOptions.java index 86fd38903fc6..240a95c97714 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SavepointConfigOptions.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SavepointConfigOptions.java @@ -34,6 +34,7 @@ public class SavepointConfigOptions { */ public static final ConfigOption SAVEPOINT_PATH = key("execution.savepoint.path") + .stringType() .noDefaultValue() .withDescription("Path to a savepoint to restore the job from (for example hdfs:///flink/savepoint-1537)."); @@ -43,6 +44,7 @@ public class SavepointConfigOptions { */ public static final ConfigOption SAVEPOINT_IGNORE_UNCLAIMED_STATE = key("execution.savepoint.ignore-unclaimed-state") + .booleanType() .defaultValue(false) .withDescription("Allow to skip savepoint state that cannot be restored. " + "Allow this if you removed an operator from your pipeline after the savepoint was triggered."); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnConfigUtils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnConfigUtils.java index edb02096e35f..efd3d45cde30 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnConfigUtils.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnConfigUtils.java @@ -36,12 +36,10 @@ * Utilities for parsing {@link org.apache.flink.configuration.ConfigOption configuration options}. */ public class YarnConfigUtils { - // TODO: 16.10.19 test this and test LOG FILES discovery. - private static final String COLLECTION_DELIMITER = ";"; public static void encodeListToConfig( final Configuration configuration, - final ConfigOption key, + final ConfigOption> key, final Collection value, final Function mapper) { encodeListToConfig(configuration, key, value.stream(), mapper); @@ -49,7 +47,7 @@ public static void encodeListToConfig( public static void encodeListToConfig( final Configuration configuration, - final ConfigOption key, + final ConfigOption> key, final T[] value, final Function mapper) { encodeListToConfig(configuration, key, Arrays.stream(value), mapper); @@ -57,7 +55,7 @@ public static void encodeListToConfig( private static void encodeListToConfig( final Configuration configuration, - final ConfigOption key, + final ConfigOption> key, final Stream values, final Function mapper) { @@ -65,23 +63,23 @@ private static void encodeListToConfig( checkNotNull(key); checkNotNull(configuration); - final String encodedString = values.map(mapper).filter(Objects::nonNull).collect(Collectors.joining(COLLECTION_DELIMITER)); - if (encodedString != null && !encodedString.isEmpty()) { - configuration.setString(key, encodedString); + final List encodedString = values.map(mapper).filter(Objects::nonNull).collect(Collectors.toList()); + if (!encodedString.isEmpty()) { + configuration.set(key, encodedString); } } public static List decodeListFromConfig( final Configuration configuration, - final ConfigOption key, + final ConfigOption> key, final Function mapper) { checkNotNull(configuration); checkNotNull(key); - final String encodedString = configuration.getString(key); + final List encodedString = configuration.get(key); return encodedString != null - ? Arrays.stream(encodedString.split(COLLECTION_DELIMITER)).map(mapper).collect(Collectors.toList()) + ? encodedString.stream().map(mapper).collect(Collectors.toList()) : Collections.emptyList(); } } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java b/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java index 021552d5c791..b4f924764ba0 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java @@ -21,6 +21,8 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.description.Description; +import java.util.List; + import static org.apache.flink.configuration.ConfigOptions.key; import static org.apache.flink.configuration.description.LinkElement.link; import static org.apache.flink.configuration.description.TextElement.code; @@ -208,42 +210,51 @@ public class YarnConfigOptions { public static final ConfigOption APPLICATION_LOG_CONFIG_FILE = key("yarn.log-config-file") + .stringType() .noDefaultValue() .withDescription("The location of the log config file, e.g. the path to your log4j.properties for log4j."); - public static final ConfigOption SHIP_DIRECTORIES = + public static final ConfigOption> SHIP_DIRECTORIES = key("yarn.ship-directories") + .stringType() + .asList() .noDefaultValue() .withDescription("A semicolon-separated list of directories to be shipped to the YARN cluster."); public static final ConfigOption FLINK_DIST_JAR = key("yarn.flink-dist-jar") + .stringType() .noDefaultValue() .withDescription("The location of the Flink dist jar."); public static final ConfigOption APPLICATION_ID = key("yarn.application.id") + .stringType() .noDefaultValue() .withDescription("The YARN application id of the running yarn cluster." + " This is the YARN cluster where the pipeline is going to be executed."); public static final ConfigOption APPLICATION_QUEUE = key("yarn.application.queue") + .stringType() .noDefaultValue() .withDescription("The YARN queue on which to put the current pipeline."); public static final ConfigOption APPLICATION_NAME = key("yarn.application.name") + .stringType() .noDefaultValue() .withDescription("A custom name for your YARN application."); public static final ConfigOption APPLICATION_TYPE = key("yarn.application.type") + .stringType() .noDefaultValue() .withDescription("A custom type for your YARN application.."); public static final ConfigOption NODE_LABEL = key("yarn.application.node-label") + .stringType() .noDefaultValue() .withDescription("Specify YARN node label for the YARN application."); From 09f2f43a1d73c76bf4d3f4a1205269eb860deb14 Mon Sep 17 00:00:00 2001 From: xuyang1706 Date: Fri, 20 Sep 2019 20:23:29 +0800 Subject: [PATCH 401/746] [FLINK-14154][ml] Add the class for multivariate Gaussian Distribution. This closes #9733. --- .../basicstatistic/MultivariateGaussian.java | 142 ++++++++++++++++++ .../MultivariateGaussianTest.java | 74 +++++++++ 2 files changed, 216 insertions(+) create mode 100644 flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/statistics/basicstatistic/MultivariateGaussian.java create mode 100644 flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/common/statistics/basicstatistic/MultivariateGaussianTest.java diff --git a/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/statistics/basicstatistic/MultivariateGaussian.java b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/statistics/basicstatistic/MultivariateGaussian.java new file mode 100644 index 000000000000..a40095e1e1ec --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/main/java/org/apache/flink/ml/common/statistics/basicstatistic/MultivariateGaussian.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.ml.common.statistics.basicstatistic; + +import org.apache.flink.ml.common.linalg.BLAS; +import org.apache.flink.ml.common.linalg.DenseMatrix; +import org.apache.flink.ml.common.linalg.DenseVector; +import org.apache.flink.ml.common.linalg.SparseVector; +import org.apache.flink.ml.common.linalg.Vector; + +import org.apache.flink.shaded.guava18.com.google.common.primitives.Doubles; + +import com.github.fommil.netlib.LAPACK; +import org.netlib.util.intW; + +/** + * This class provides basic functionality for a Multivariate Gaussian (Normal) Distribution. + */ +public class MultivariateGaussian { + + private static final LAPACK LAPACK_INST = LAPACK.getInstance(); + private static final com.github.fommil.netlib.BLAS F2J_BLAS_INST = com.github.fommil.netlib.F2jBLAS.getInstance(); + private static final double EPSILON; + + static { + double eps = 1.0; + while ((1.0 + (eps / 2.0)) != 1.0) { + eps /= 2.0; + } + EPSILON = eps; + } + + private final DenseVector mean; + private final DenseMatrix cov; + + private DenseMatrix rootSigmaInv; + private double u; + + // data buffers for computing pdf + private DenseVector delta; + private DenseVector v; + + /** + * The constructor. + * + * @param mean The mean vector of the distribution. + * @param cov The covariance matrix of the distribution. + */ + public MultivariateGaussian(DenseVector mean, DenseMatrix cov) { + this.mean = mean; + this.cov = cov; + this.delta = DenseVector.zeros(mean.size()); + this.v = DenseVector.zeros(mean.size()); + calculateCovarianceConstants(); + } + + /** + * Returns density of this multivariate Gaussian at given point x . + */ + public double pdf(Vector x) { + return Math.exp(logpdf(x)); + } + + /** + * Returns the log-density of this multivariate Gaussian at given point x . + */ + public double logpdf(Vector x) { + int n = mean.size(); + System.arraycopy(mean.getData(), 0, delta.getData(), 0, n); + BLAS.scal(-1.0, delta); + if (x instanceof DenseVector) { + BLAS.axpy(1., (DenseVector) x, delta); + } else if (x instanceof SparseVector) { + BLAS.axpy(1., (SparseVector) x, delta); + } + BLAS.gemv(1.0, rootSigmaInv, true, delta, 0., v); + return u - 0.5 * BLAS.dot(v, v); + } + + /** + * Compute distribution dependent constants. + * + *

    The probability density function is calculated as: + * pdf(x) = (2*pi)^(-k/2)^ * det(sigma)^(-1/2)^ * exp((-1/2) * (x-mu).t * inv(sigma) * (x-mu)) + * + *

    Here we compute the following distribution dependent constants that can be reused in each pdf computation: + * A) u = log((2*pi)^(-k/2)^ * det(sigma)^(-1/2)^) + * B) rootSigmaInv = sqrt(inv(sigma)) = U * D^(-1/2)^ + * + *

      + *
    • sigma = U * D * U.t + *
    • inv(sigma) = U * inv(D) * U.t = (U * D^(-1/2)^) * (U * D^(-1/2)^).t + *
    • sqrt(inv(sigma)) = U * D^(-1/2)^ + *

    + */ + private void calculateCovarianceConstants() { + int k = this.mean.size(); + int lwork = 3 * k - 1; + double[] matU = new double[k * k]; + double[] work = new double[lwork]; + double[] evs = new double[k]; + intW info = new intW(0); + + System.arraycopy(cov.getData(), 0, matU, 0, k * k); + LAPACK_INST.dsyev("V", "U", k, matU, k, evs, work, lwork, info); + + double maxEv = Doubles.max(evs); + double tol = EPSILON * k * maxEv; + + // log(pseudo-determinant) is sum of the logs of all non-zero singular values + double logPseudoDetSigma = 0.; + for (double ev : evs) { + if (ev > tol) { + logPseudoDetSigma += Math.log(ev); + } + } + + for (int i = 0; i < k; i++) { + double invEv = evs[i] > tol ? Math.sqrt(1.0 / evs[i]) : 0.; + F2J_BLAS_INST.dscal(k, invEv, matU, i * k, 1); + } + this.rootSigmaInv = new DenseMatrix(k, k, matU); + this.u = -0.5 * (k * Math.log(2.0 * Math.PI) + logPseudoDetSigma); + } +} diff --git a/flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/common/statistics/basicstatistic/MultivariateGaussianTest.java b/flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/common/statistics/basicstatistic/MultivariateGaussianTest.java new file mode 100644 index 000000000000..da7342c9b7c2 --- /dev/null +++ b/flink-ml-parent/flink-ml-lib/src/test/java/org/apache/flink/ml/common/statistics/basicstatistic/MultivariateGaussianTest.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.statistics.basicstatistic; + +import org.apache.flink.ml.common.linalg.DenseMatrix; +import org.apache.flink.ml.common.linalg.DenseVector; + +import org.junit.Assert; +import org.junit.Test; + +/** + * Test cases for {@link MultivariateGaussian}. + */ +public class MultivariateGaussianTest { + private static final double TOL = 1.0e-5; + + @Test + public void testUnivariate() throws Exception { + DenseVector x1 = new DenseVector(new double[]{0.0}); + DenseVector x2 = new DenseVector(new double[]{1.5}); + DenseVector mu = DenseVector.zeros(1); + DenseMatrix sigma1 = DenseMatrix.ones(1, 1); + MultivariateGaussian dist1 = new MultivariateGaussian(mu, sigma1); + Assert.assertEquals(dist1.pdf(x1), 0.39894, TOL); + Assert.assertEquals(dist1.pdf(x2), 0.12952, TOL); + + DenseMatrix sigma2 = DenseMatrix.ones(1, 1); + sigma2.scaleEqual(4.0); + MultivariateGaussian dist2 = new MultivariateGaussian(mu, sigma2); + Assert.assertEquals(dist2.pdf(x1), 0.19947, TOL); + Assert.assertEquals(dist2.pdf(x2), 0.15057, TOL); + } + + @Test + public void testMultivariate() throws Exception { + DenseVector mu = DenseVector.zeros(2); + + DenseMatrix sigma1 = DenseMatrix.eye(2); + MultivariateGaussian mg1 = new MultivariateGaussian(mu, sigma1); + Assert.assertEquals(mg1.pdf(DenseVector.zeros(2)), 0.15915, TOL); + Assert.assertEquals(mg1.pdf(DenseVector.ones(2)), 0.05855, TOL); + + DenseMatrix sigma2 = new DenseMatrix(2, 2, new double[]{4.0, -1.0, -1.0, 2.0}); + MultivariateGaussian mg2 = new MultivariateGaussian(mu, sigma2); + Assert.assertEquals(mg2.pdf(DenseVector.zeros(2)), 0.060155, TOL); + Assert.assertEquals(mg2.pdf(DenseVector.ones(2)), 0.033971, TOL); + } + + @Test + public void testMultivariateDegenerate() throws Exception { + DenseVector mu = DenseVector.zeros(2); + DenseMatrix sigma = new DenseMatrix(2, 2, new double[]{1.0, 1.0, 1.0, 1.0}); + MultivariateGaussian mg = new MultivariateGaussian(mu, sigma); + Assert.assertEquals(mg.pdf(DenseVector.zeros(2)), 0.11254, TOL); + Assert.assertEquals(mg.pdf(DenseVector.ones(2)), 0.068259, TOL); + } +} From bf5235e340543b9c4551d2131e8a405bd1e9e0c0 Mon Sep 17 00:00:00 2001 From: tison Date: Fri, 1 Nov 2019 14:51:28 +0800 Subject: [PATCH 402/746] [FLINK-14496][client] Exclude detach flag from ClusterClient This closes #9972 . --- .../org/apache/flink/client/ClientUtils.java | 63 +++++++++++-- .../apache/flink/client/RemoteExecutor.java | 2 +- .../apache/flink/client/cli/CliFrontend.java | 14 +-- .../flink/client/program/ClusterClient.java | 27 ------ .../client/program/ContextEnvironment.java | 12 ++- .../client/program/MiniClusterClient.java | 38 -------- .../program/rest/RestClusterClient.java | 40 --------- .../flink/client/cli/CliFrontendRunTest.java | 4 +- .../flink/client/program/ClientTest.java | 20 ++--- .../program/rest/RestClusterClientTest.java | 14 ++- .../kafka/KafkaConsumerTestBase.java | 90 +++++++------------ .../state/api/SavepointReaderITTestBase.java | 6 +- .../api/SavepointReaderKeyedStateITCase.java | 5 +- .../state/api/SavepointWriterITCase.java | 5 +- .../jobmanager/JMXJobManagerMetricTest.java | 4 +- .../AbstractQueryableStateTestBase.java | 42 +++------ .../runtime/webmonitor/WebFrontendITCase.java | 7 +- .../environment/StreamContextEnvironment.java | 9 +- .../RemoteStreamExecutionEnvironmentTest.java | 33 +++++-- .../client/gateway/local/ProgramDeployer.java | 10 +-- .../JobMasterStopWithSavepointIT.java | 4 +- .../JobMasterTriggerSavepointITCase.java | 4 +- .../accumulators/AccumulatorLiveITCase.java | 3 +- .../test/cancelling/CancelingTestBase.java | 5 +- .../checkpointing/RegionFailoverITCase.java | 6 +- .../test/checkpointing/RescalingITCase.java | 25 ++---- .../ResumeCheckpointManuallyITCase.java | 4 +- .../test/checkpointing/SavepointITCase.java | 25 ++---- .../StreamFaultToleranceTestBase.java | 6 +- .../ZooKeeperHighAvailabilityITCase.java | 4 +- .../utils/SavepointMigrationTestBase.java | 7 +- .../example/client/JobRetrievalITCase.java | 4 +- .../failing/JobSubmissionFailsITCase.java | 11 ++- .../runtime/NetworkStackThroughputITCase.java | 5 +- .../AbstractOperatorRestoreTestBase.java | 15 ++-- .../BigUserProgramJobSubmitITCase.java | 4 +- .../flink/yarn/util/FakeClusterClient.java | 5 -- 37 files changed, 247 insertions(+), 335 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 e229b94e8e1f..176d58611d98 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 @@ -23,12 +23,16 @@ import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.ContextEnvironment; import org.apache.flink.client.program.ContextEnvironmentFactory; +import org.apache.flink.client.program.DetachedJobExecutionResult; import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.client.program.ProgramMissingJobException; import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.util.ExceptionUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,9 +42,12 @@ import java.net.URISyntaxException; import java.net.URL; import java.util.List; +import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicReference; import java.util.jar.JarFile; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * Utility functions for Flink client. */ @@ -95,15 +102,61 @@ public static ClassLoader buildUserCodeClassLoader(List jars, List cla return FlinkUserCodeClassLoaders.parentFirst(urls, parent); } + public static JobExecutionResult submitJob( + ClusterClient client, + JobGraph jobGraph) throws ProgramInvocationException { + checkNotNull(client); + checkNotNull(jobGraph); + try { + return client + .submitJob(jobGraph) + .thenApply(JobSubmissionResult::getJobID) + .thenApply(DetachedJobExecutionResult::new) + .get(); + } catch (InterruptedException | ExecutionException e) { + ExceptionUtils.checkInterrupted(e); + throw new ProgramInvocationException("Could not run job in detached mode.", jobGraph.getJobID(), e); + } + } + + public static JobExecutionResult submitJobAndWaitForResult( + ClusterClient client, + JobGraph jobGraph, + ClassLoader classLoader) throws ProgramInvocationException { + checkNotNull(client); + checkNotNull(jobGraph); + checkNotNull(classLoader); + + JobResult jobResult; + + try { + jobResult = client + .submitJob(jobGraph) + .thenApply(JobSubmissionResult::getJobID) + .thenCompose(client::requestJobResult) + .get(); + } catch (InterruptedException | ExecutionException e) { + ExceptionUtils.checkInterrupted(e); + throw new ProgramInvocationException("Could not run job", jobGraph.getJobID(), e); + } + + try { + return jobResult.toJobExecutionResult(classLoader); + } catch (JobExecutionException | IOException | ClassNotFoundException e) { + throw new ProgramInvocationException("Job failed", jobGraph.getJobID(), e); + } + } + public static JobSubmissionResult executeProgram( - ClusterClient client, - PackagedProgram program, - int parallelism) throws ProgramMissingJobException, ProgramInvocationException { + ClusterClient client, + PackagedProgram program, + int parallelism, + boolean detached) throws ProgramMissingJobException, ProgramInvocationException { final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader(); try { Thread.currentThread().setContextClassLoader(program.getUserCodeClassLoader()); - LOG.info("Starting program (detached: {})", client.isDetached()); + LOG.info("Starting program (detached: {})", detached); final List libraries = program.getAllLibraries(); @@ -115,7 +168,7 @@ public static JobSubmissionResult executeProgram( program.getClasspaths(), program.getUserCodeClassLoader(), parallelism, - client.isDetached(), + detached, program.getSavepointSettings(), jobExecutionResult); ContextEnvironment.setAsContext(factory); 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 91a5d3a930f4..b601367fe329 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 @@ -126,7 +126,7 @@ private JobExecutionResult executePlanWithJars(JobGraph jobGraph, ClassLoader cl try (ClusterClient client = new RestClusterClient<>( clientConfiguration, "RemoteExecutor")) { - return client.submitJob(jobGraph, classLoader).getJobExecutionResult(); + return ClientUtils.submitJobAndWaitForResult(client, jobGraph, classLoader).getJobExecutionResult(); } } } 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 33fb2533b307..cb38ed631357 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 @@ -273,17 +273,15 @@ private void runProgram( } try { - client.setDetached(executionParameters.getDetachedMode()); - int userParallelism = executionParameters.getParallelism(); LOG.debug("User parallelism is set to {}", userParallelism); if (ExecutionConfig.PARALLELISM_DEFAULT == userParallelism) { userParallelism = defaultParallelism; } - executeProgram(program, client, userParallelism); + executeProgram(program, client, userParallelism, executionParameters.getDetachedMode()); } finally { - if (clusterId == null && !client.isDetached()) { + if (clusterId == null && !executionParameters.getDetachedMode()) { // terminate the cluster only if we have started it before and if it's not detached try { client.shutDownCluster(); @@ -745,10 +743,14 @@ private void disposeSavepoint(ClusterClient clusterClient, String savepointPa // Interaction with programs and JobManager // -------------------------------------------------------------------------------------------- - protected void executeProgram(PackagedProgram program, ClusterClient client, int parallelism) throws ProgramMissingJobException, ProgramInvocationException { + protected void executeProgram( + PackagedProgram program, + ClusterClient client, + int parallelism, + boolean detached) throws ProgramMissingJobException, ProgramInvocationException { logAndSysout("Starting execution of program"); - JobSubmissionResult result = ClientUtils.executeProgram(client, program, parallelism); + JobSubmissionResult result = ClientUtils.executeProgram(client, program, parallelism, detached); if (result.isJobExecutionResult()) { logAndSysout("Program execution finished"); 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 e81bc5d6f302..eb3893539bbe 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 @@ -43,9 +43,6 @@ */ public abstract class ClusterClient implements AutoCloseable { - /** Switch for blocking/detached job submission of the client. */ - private boolean detachedJobSubmission = false; - /** * User overridable hook to close the client, possibly closes internal services. * @deprecated use the {@link #close()} instead. This method stays for backwards compatibility. @@ -153,36 +150,12 @@ public Map> getAccumulators(JobID jobID) throws */ public abstract T getClusterId(); - /** - * Set the mode of this client (detached or blocking job execution). - * @param isDetached If true, the client will submit programs detached via the {@code run} method - */ - public void setDetached(boolean isDetached) { - this.detachedJobSubmission = isDetached; - } - - /** - * A flag to indicate whether this clients submits jobs detached. - * @return True if the Client submits detached, false otherwise - */ - public boolean isDetached() { - return detachedJobSubmission; - } - /** * Return the Flink configuration object. * @return The Flink configuration object */ public abstract Configuration getFlinkConfiguration(); - /** - * Calls the subclasses' submitJob method. It may decide to simply call one of the run methods or it may perform - * some custom job submission logic. - * @param jobGraph The JobGraph to be submitted - * @return JobSubmissionResult - */ - public abstract JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException; - /** * Submit the given {@link JobGraph} to the cluster. * 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 06bfa714732f..8f9048c18ab7 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 @@ -87,9 +87,11 @@ public JobExecutionResult execute(String jobName) throws Exception { ClientUtils.addJarFiles(jobGraph, this.jarFilesToAttach); jobGraph.setClasspaths(this.classpathsToAttach); - lastJobExecutionResult = client - .submitJob(jobGraph, this.userCodeClassLoader) - .getJobExecutionResult(); + if (detached) { + lastJobExecutionResult = ClientUtils.submitJob(client, jobGraph); + } else { + lastJobExecutionResult = ClientUtils.submitJobAndWaitForResult(client, jobGraph, userCodeClassLoader).getJobExecutionResult(); + } setJobExecutionResult(lastJobExecutionResult); @@ -132,6 +134,10 @@ public SavepointRestoreSettings getSavepointRestoreSettings() { return savepointSettings; } + public boolean isDetached() { + return detached; + } + // -------------------------------------------------------------------------------------------- public static void setAsContext(ContextEnvironmentFactory factory) { 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 5e52740af9ab..8d9460407bae 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 @@ -21,7 +21,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobSubmissionResult; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.client.JobStatusMessage; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -29,19 +28,16 @@ import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.minicluster.MiniCluster; -import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.SerializedValue; import javax.annotation.Nonnull; import javax.annotation.Nullable; -import java.io.IOException; import java.util.Collection; import java.util.HashMap; import java.util.Map; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; /** * Client to interact with a {@link MiniCluster}. @@ -61,40 +57,6 @@ public Configuration getFlinkConfiguration() { return new Configuration(configuration); } - @Override - public JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException { - final CompletableFuture jobSubmissionResultFuture = submitJob(jobGraph); - - if (isDetached()) { - try { - final JobSubmissionResult jobSubmissionResult = jobSubmissionResultFuture.get(); - return new DetachedJobExecutionResult(jobSubmissionResult.getJobID()); - } catch (InterruptedException | ExecutionException e) { - ExceptionUtils.checkInterrupted(e); - - throw new ProgramInvocationException("Could not run job in detached mode.", jobGraph.getJobID(), e); - } - } else { - final CompletableFuture jobResultFuture = jobSubmissionResultFuture.thenCompose( - (JobSubmissionResult ignored) -> requestJobResult(jobGraph.getJobID())); - - final JobResult jobResult; - try { - jobResult = jobResultFuture.get(); - } catch (InterruptedException | ExecutionException e) { - ExceptionUtils.checkInterrupted(e); - - throw new ProgramInvocationException("Could not run job", jobGraph.getJobID(), e); - } - - try { - return jobResult.toJobExecutionResult(classLoader); - } catch (JobExecutionException | IOException | ClassNotFoundException e) { - throw new ProgramInvocationException("Job failed", jobGraph.getJobID(), e); - } - } - } - @Override public CompletableFuture submitJob(@Nonnull JobGraph jobGraph) { return miniCluster.submitJob(jobGraph); 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 8aa5b388291f..7352dd4c4071 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 @@ -26,14 +26,11 @@ 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.ProgramInvocationException; 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; import org.apache.flink.runtime.concurrent.FutureUtils; @@ -232,43 +229,6 @@ public void close() { } } - @Override - public JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException { - LOG.info("Submitting job {} (detached: {}).", jobGraph.getJobID(), isDetached()); - - final CompletableFuture jobSubmissionFuture = submitJob(jobGraph); - - if (isDetached()) { - try { - final JobSubmissionResult jobSubmissionResult = jobSubmissionFuture.get(); - - LOG.warn("Job was executed in detached mode, the results will be available on completion."); - - return new DetachedJobExecutionResult(jobSubmissionResult.getJobID()); - } catch (Exception e) { - throw new ProgramInvocationException("Could not submit job", - jobGraph.getJobID(), ExceptionUtils.stripExecutionException(e)); - } - } else { - final CompletableFuture jobResultFuture = jobSubmissionFuture.thenCompose( - ignored -> requestJobResult(jobGraph.getJobID())); - - final JobResult jobResult; - try { - jobResult = jobResultFuture.get(); - } catch (Exception e) { - throw new ProgramInvocationException("Could not retrieve the execution result.", - jobGraph.getJobID(), ExceptionUtils.stripExecutionException(e)); - } - - try { - return jobResult.toJobExecutionResult(classLoader); - } catch (JobExecutionException | IOException | ClassNotFoundException e) { - throw new ProgramInvocationException("Job failed.", jobGraph.getJobID(), e); - } - } - } - /** * Requests the job details. * diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java index 918b6dbd2b04..d2aff282e684 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java @@ -198,8 +198,8 @@ private RunTestingCliFrontend( } @Override - protected void executeProgram(PackagedProgram program, ClusterClient client, int parallelism) { - assertEquals(isDetached, client.isDetached()); + protected void executeProgram(PackagedProgram program, ClusterClient client, int parallelism, boolean detached) { + assertEquals(isDetached, detached); assertEquals(expectedParallelism, parallelism); } } 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 c4ded34728ce..6c6ec1dfc85a 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 @@ -98,11 +98,9 @@ public void setUp() throws Exception { @Test public void testDetachedMode() throws Exception{ final ClusterClient clusterClient = new MiniClusterClient(new Configuration(), MINI_CLUSTER_RESOURCE.getMiniCluster()); - clusterClient.setDetached(true); - try { PackagedProgram prg = new PackagedProgram(TestExecuteTwice.class); - ClientUtils.executeProgram(clusterClient, prg, 1); + ClientUtils.executeProgram(clusterClient, prg, 1, true); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { assertEquals( @@ -112,7 +110,7 @@ public void testDetachedMode() throws Exception{ try { PackagedProgram prg = new PackagedProgram(TestEager.class); - ClientUtils.executeProgram(clusterClient, prg, 1); + ClientUtils.executeProgram(clusterClient, prg, 1, true); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { assertEquals( @@ -122,7 +120,7 @@ public void testDetachedMode() throws Exception{ try { PackagedProgram prg = new PackagedProgram(TestGetRuntime.class); - ClientUtils.executeProgram(clusterClient, prg, 1); + ClientUtils.executeProgram(clusterClient, prg, 1, true); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { assertEquals( @@ -132,7 +130,7 @@ public void testDetachedMode() throws Exception{ try { PackagedProgram prg = new PackagedProgram(TestGetAccumulator.class); - ClientUtils.executeProgram(clusterClient, prg, 1); + ClientUtils.executeProgram(clusterClient, prg, 1, true); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { assertEquals( @@ -142,7 +140,7 @@ public void testDetachedMode() throws Exception{ try { PackagedProgram prg = new PackagedProgram(TestGetAllAccumulator.class); - ClientUtils.executeProgram(clusterClient, prg, 1); + ClientUtils.executeProgram(clusterClient, prg, 1, true); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { assertEquals( @@ -157,8 +155,6 @@ public void testDetachedMode() throws Exception{ @Test public void shouldSubmitToJobClient() throws Exception { final ClusterClient clusterClient = new MiniClusterClient(new Configuration(), MINI_CLUSTER_RESOURCE.getMiniCluster()); - clusterClient.setDetached(true); - JobGraph jobGraph = FlinkPipelineTranslationUtil.getJobGraph( plan, new Configuration(), @@ -167,8 +163,7 @@ public void shouldSubmitToJobClient() throws Exception { ClientUtils.addJarFiles(jobGraph, Collections.emptyList()); jobGraph.setClasspaths(Collections.emptyList()); - JobSubmissionResult result = clusterClient.submitJob(jobGraph, getClass().getClassLoader()); - + JobSubmissionResult result = ClientUtils.submitJob(clusterClient, jobGraph); assertNotNull(result); } @@ -189,8 +184,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { try { final ClusterClient client = new MiniClusterClient(new Configuration(), MINI_CLUSTER_RESOURCE.getMiniCluster()); - client.setDetached(true); - ClientUtils.executeProgram(client, packagedProgramMock, 1); + ClientUtils.executeProgram(client, packagedProgramMock, 1, true); fail("Creating the local execution environment should not be possible"); } catch (InvalidProgramException e) { diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java index d11b0466d099..691a3edb5496 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java @@ -21,6 +21,7 @@ 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.client.ClientUtils; import org.apache.flink.client.cli.DefaultCLI; import org.apache.flink.client.deployment.ClusterClientFactory; import org.apache.flink.client.deployment.ClusterClientServiceLoader; @@ -234,7 +235,7 @@ public void testJobSubmitCancel() throws Exception { try { Assert.assertFalse(submitHandler.jobSubmitted); - restClusterClient.submitJob(jobGraph, ClassLoader.getSystemClassLoader()); + ClientUtils.submitJobAndWaitForResult(restClusterClient, jobGraph, ClassLoader.getSystemClassLoader()); Assert.assertTrue(submitHandler.jobSubmitted); Assert.assertFalse(terminationHandler.jobCanceled); @@ -259,8 +260,7 @@ public void testDetachedJobSubmission() throws Exception { RestClusterClient restClusterClient = createRestClusterClient(restServerEndpoint.getServerAddress().getPort()); try { - restClusterClient.setDetached(true); - final JobSubmissionResult jobSubmissionResult = restClusterClient.submitJob(jobGraph, ClassLoader.getSystemClassLoader()); + final JobSubmissionResult jobSubmissionResult = ClientUtils.submitJob(restClusterClient, jobGraph); // if the detached mode didn't work, then we would not reach this point because the execution result // retrieval would have failed. @@ -367,9 +367,7 @@ public void testSubmitJobAndWaitForExecutionResult() throws Exception { try { JobExecutionResult jobExecutionResult; - jobExecutionResult = (JobExecutionResult) restClusterClient.submitJob( - jobGraph, - ClassLoader.getSystemClassLoader()); + jobExecutionResult = ClientUtils.submitJobAndWaitForResult(restClusterClient, jobGraph, ClassLoader.getSystemClassLoader()); assertThat(jobExecutionResult.getJobID(), equalTo(jobId)); assertThat(jobExecutionResult.getNetRuntime(), equalTo(Long.MAX_VALUE)); assertThat( @@ -377,7 +375,7 @@ public void testSubmitJobAndWaitForExecutionResult() throws Exception { equalTo(Collections.singletonMap("testName", 1.0))); try { - restClusterClient.submitJob(jobGraph, ClassLoader.getSystemClassLoader()); + ClientUtils.submitJobAndWaitForResult(restClusterClient, jobGraph, ClassLoader.getSystemClassLoader()); fail("Expected exception not thrown."); } catch (final ProgramInvocationException e) { final Optional cause = ExceptionUtils.findThrowable(e, RuntimeException.class); @@ -602,7 +600,7 @@ public void testJobSubmissionFailureThrowsProgramInvocationException() throws Ex RestClusterClient restClusterClient = createRestClusterClient(restServerEndpoint.getServerAddress().getPort()); try { - restClusterClient.submitJob(jobGraph, ClassLoader.getSystemClassLoader()); + ClientUtils.submitJobAndWaitForResult(restClusterClient, jobGraph, ClassLoader.getSystemClassLoader()); } catch (final ProgramInvocationException expected) { // expected } finally { diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java index d2a526c5642b..6c69eb90ff95 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java @@ -34,6 +34,7 @@ import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.configuration.Configuration; @@ -452,16 +453,12 @@ public void flatMap(Tuple2 value, Collector out) throw final JobID consumeJobId = jobGraph.getJobID(); final AtomicReference error = new AtomicReference<>(); - Thread consumeThread = new Thread(new Runnable() { - @Override - public void run() { - try { - client.setDetached(false); - client.submitJob(jobGraph, KafkaConsumerTestBase.class.getClassLoader()); - } catch (Throwable t) { - if (!ExceptionUtils.findThrowable(t, JobCancellationException.class).isPresent()) { - error.set(t); - } + Thread consumeThread = new Thread(() -> { + try { + ClientUtils.submitJobAndWaitForResult(client, jobGraph, KafkaConsumerTestBase.class.getClassLoader()); + } catch (Throwable t) { + if (!ExceptionUtils.findThrowable(t, JobCancellationException.class).isPresent()) { + error.set(t); } } }); @@ -1002,16 +999,11 @@ public void runCancelingOnFullInputTest() throws Exception { JobGraph jobGraph = StreamingJobGraphGenerator.createJobGraph(env.getStreamGraph()); final JobID jobId = jobGraph.getJobID(); - final Runnable jobRunner = new Runnable() { - @Override - public void run() { - try { - client.setDetached(false); - client.submitJob(jobGraph, KafkaConsumerTestBase.class.getClassLoader()); - } - catch (Throwable t) { - jobError.set(t); - } + final Runnable jobRunner = () -> { + try { + ClientUtils.submitJobAndWaitForResult(client, jobGraph, KafkaConsumerTestBase.class.getClassLoader()); + } catch (Throwable t) { + jobError.set(t); } }; @@ -1077,17 +1069,12 @@ public void runCancelingOnEmptyInputTest() throws Exception { JobGraph jobGraph = StreamingJobGraphGenerator.createJobGraph(env.getStreamGraph()); final JobID jobId = jobGraph.getJobID(); - final Runnable jobRunner = new Runnable() { - @Override - public void run() { - try { - client.setDetached(false); - client.submitJob(jobGraph, KafkaConsumerTestBase.class.getClassLoader()); - } - catch (Throwable t) { - LOG.error("Job Runner failed with exception", t); - error.set(t); - } + final Runnable jobRunner = () -> { + try { + ClientUtils.submitJobAndWaitForResult(client, jobGraph, KafkaConsumerTestBase.class.getClassLoader()); + } catch (Throwable t) { + LOG.error("Job Runner failed with exception", t); + error.set(t); } }; @@ -1604,21 +1591,16 @@ public void cancel() { JobGraph jobGraph = StreamingJobGraphGenerator.createJobGraph(env1.getStreamGraph()); final JobID jobId = jobGraph.getJobID(); - Runnable job = new Runnable() { - @Override - public void run() { - try { - client.setDetached(false); - client.submitJob(jobGraph, KafkaConsumerTestBase.class.getClassLoader()); - } catch (Throwable t) { - if (!ExceptionUtils.findThrowable(t, JobCancellationException.class).isPresent()) { - LOG.warn("Got exception during execution", t); - error.f0 = t; - } + Thread jobThread = new Thread(() -> { + try { + ClientUtils.submitJobAndWaitForResult(client, jobGraph, KafkaConsumerTestBase.class.getClassLoader()); + } catch (Throwable t) { + if (!ExceptionUtils.findThrowable(t, JobCancellationException.class).isPresent()) { + LOG.warn("Got exception during execution", t); + error.f0 = t; } } - }; - Thread jobThread = new Thread(job); + }); jobThread.start(); try { @@ -2049,20 +2031,16 @@ public Tuple2 map(Tuple2 value) throws Excep JobGraph jobGraph = StreamingJobGraphGenerator.createJobGraph(readEnv.getStreamGraph()); final JobID jobId = jobGraph.getJobID(); - Thread runner = new Thread() { - @Override - public void run() { - try { - client.setDetached(false); - client.submitJob(jobGraph, KafkaConsumerTestBase.class.getClassLoader()); - tryExecute(readEnv, "sequence validation"); - } catch (Throwable t) { - if (!ExceptionUtils.findThrowable(t, SuccessException.class).isPresent()) { - errorRef.set(t); - } + Thread runner = new Thread(() -> { + try { + ClientUtils.submitJobAndWaitForResult(client, jobGraph, KafkaConsumerTestBase.class.getClassLoader()); + tryExecute(readEnv, "sequence validation"); + } catch (Throwable t) { + if (!ExceptionUtils.findThrowable(t, SuccessException.class).isPresent()) { + errorRef.set(t); } } - }; + }); runner.start(); final long deadline = System.nanoTime() + 10_000_000_000L; diff --git a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointReaderITTestBase.java b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointReaderITTestBase.java index 805932cdc37d..06148435e291 100644 --- a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointReaderITTestBase.java +++ b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointReaderITTestBase.java @@ -27,6 +27,7 @@ import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -164,8 +165,6 @@ private String takeSavepoint(JobGraph jobGraph) throws Exception { SavepointSource.initializeForTest(); ClusterClient client = miniClusterResource.getClusterClient(); - client.setDetached(true); - JobID jobId = jobGraph.getJobID(); Deadline deadline = Deadline.fromNow(Duration.ofMinutes(5)); @@ -173,8 +172,7 @@ private String takeSavepoint(JobGraph jobGraph) throws Exception { String dirPath = getTempDirPath(new AbstractID().toHexString()); try { - client.setDetached(true); - JobSubmissionResult result = client.submitJob(jobGraph, SavepointReaderITCase.class.getClassLoader()); + JobSubmissionResult result = ClientUtils.submitJob(client, jobGraph); boolean finished = false; while (deadline.hasTimeLeft()) { diff --git a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointReaderKeyedStateITCase.java b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointReaderKeyedStateITCase.java index 915f9d2aef18..39825bd34207 100644 --- a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointReaderKeyedStateITCase.java +++ b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointReaderKeyedStateITCase.java @@ -25,6 +25,7 @@ import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.contrib.streaming.state.RocksDBStateBackend; @@ -105,7 +106,6 @@ private String takeSavepoint(JobGraph jobGraph) throws Exception { SavepointSource.initializeForTest(); ClusterClient client = miniClusterResource.getClusterClient(); - client.setDetached(true); JobID jobId = jobGraph.getJobID(); @@ -114,8 +114,7 @@ private String takeSavepoint(JobGraph jobGraph) throws Exception { String dirPath = getTempDirPath(new AbstractID().toHexString()); try { - client.setDetached(true); - JobSubmissionResult result = client.submitJob(jobGraph, getClass().getClassLoader()); + JobSubmissionResult result = ClientUtils.submitJob(client, jobGraph); boolean finished = false; while (deadline.hasTimeLeft()) { diff --git a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointWriterITCase.java b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointWriterITCase.java index 5621fac28239..d87a33d2b75f 100644 --- a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointWriterITCase.java +++ b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointWriterITCase.java @@ -28,6 +28,7 @@ import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.configuration.Configuration; @@ -169,7 +170,7 @@ private void validateBootstrap(String savepointPath) throws ProgramInvocationExc jobGraph.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(savepointPath, false)); ClusterClient client = miniClusterResource.getClusterClient(); - client.submitJob(jobGraph, SavepointWriterITCase.class.getClassLoader()); + ClientUtils.submitJobAndWaitForResult(client, jobGraph, SavepointWriterITCase.class.getClassLoader()); Assert.assertEquals("Unexpected output", 3, CollectSink.accountList.size()); } @@ -215,7 +216,7 @@ private void validateModification(String savepointPath) throws ProgramInvocation jobGraph.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(savepointPath, false)); ClusterClient client = miniClusterResource.getClusterClient(); - client.submitJob(jobGraph, SavepointWriterITCase.class.getClassLoader()); + ClientUtils.submitJobAndWaitForResult(client, jobGraph, SavepointWriterITCase.class.getClassLoader()); Assert.assertEquals("Unexpected output", 3, CollectSink.accountList.size()); } diff --git a/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java b/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java index 7156adb48090..8b641ef2bb43 100644 --- a/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java +++ b/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.time.Time; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -106,8 +107,7 @@ public void testJobManagerJMXMetricAccess() throws Exception { null)); ClusterClient client = MINI_CLUSTER_RESOURCE.getClusterClient(); - client.setDetached(true); - client.submitJob(jobGraph, JMXJobManagerMetricTest.class.getClassLoader()); + ClientUtils.submitJob(client, jobGraph); FutureUtils.retrySuccessfulWithDelay( () -> client.getJobStatus(jobGraph.getJobID()), diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java index ac4052eceec4..6791ab6fafdf 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java @@ -44,6 +44,7 @@ import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.configuration.Configuration; @@ -186,8 +187,7 @@ public Integer getKey(Tuple2 value) { final JobID jobId = autoCancellableJob.getJobId(); final JobGraph jobGraph = autoCancellableJob.getJobGraph(); - clusterClient.setDetached(true); - clusterClient.submitJob(jobGraph, AbstractQueryableStateTestBase.class.getClassLoader()); + ClientUtils.submitJob(clusterClient, jobGraph); final AtomicLongArray counts = new AtomicLongArray(numKeys); @@ -296,11 +296,9 @@ public Integer getKey(Tuple2 value) { // Submit the job graph final JobGraph jobGraph = env.getStreamGraph().getJobGraph(); - clusterClient.setDetached(false); - boolean caughtException = false; try { - clusterClient.submitJob(jobGraph, AbstractQueryableStateTestBase.class.getClassLoader()); + ClientUtils.submitJobAndWaitForResult(clusterClient, jobGraph, AbstractQueryableStateTestBase.class.getClassLoader()); } catch (ProgramInvocationException e) { String failureCause = ExceptionUtils.stringifyException(e); assertThat(failureCause, containsString("KvState with name '" + queryName + "' has already been registered by another operator")); @@ -348,9 +346,7 @@ public Integer getKey(Tuple2 value) { final JobID jobId = autoCancellableJob.getJobId(); final JobGraph jobGraph = autoCancellableJob.getJobGraph(); - clusterClient.setDetached(true); - clusterClient.submitJob(jobGraph, AbstractQueryableStateTestBase.class.getClassLoader()); - + ClientUtils.submitJob(clusterClient, jobGraph); executeValueQuery(deadline, client, jobId, "hakuna", valueState, numElements); } } @@ -384,9 +380,7 @@ public Integer getKey(Tuple2 value) { try (AutoCancellableJob closableJobGraph = new AutoCancellableJob(deadline, clusterClient, env)) { - clusterClient.setDetached(true); - clusterClient.submitJob( - closableJobGraph.getJobGraph(), AbstractQueryableStateTestBase.class.getClassLoader()); + ClientUtils.submitJob(clusterClient, closableJobGraph.getJobGraph()); CompletableFuture jobStatusFuture = clusterClient.getJobStatus(closableJobGraph.getJobId()); @@ -487,9 +481,7 @@ public Integer getKey(Tuple2 value) { BasicTypeInfo.INT_TYPE_INFO, valueState); - clusterClient.setDetached(true); - clusterClient.submitJob(jobGraph, AbstractQueryableStateTestBase.class.getClassLoader()); - + ClientUtils.submitJob(clusterClient, jobGraph); executeValueQuery(deadline, client, jobId, "hakuna", valueState, expected); } } @@ -536,8 +528,7 @@ public Integer getKey(Tuple2 value) { final JobID jobId = autoCancellableJob.getJobId(); final JobGraph jobGraph = autoCancellableJob.getJobGraph(); - clusterClient.setDetached(true); - clusterClient.submitJob(jobGraph, AbstractQueryableStateTestBase.class.getClassLoader()); + ClientUtils.submitJob(clusterClient, jobGraph); // Now query int key = 0; @@ -605,9 +596,7 @@ public Integer getKey(Tuple2 value) { final JobID jobId = autoCancellableJob.getJobId(); final JobGraph jobGraph = autoCancellableJob.getJobGraph(); - clusterClient.setDetached(true); - clusterClient.submitJob(jobGraph, AbstractQueryableStateTestBase.class.getClassLoader()); - + ClientUtils.submitJob(clusterClient, jobGraph); executeValueQuery(deadline, client, jobId, "matata", stateDesc, numElements); } } @@ -651,8 +640,7 @@ public Integer getKey(Tuple2 value) { final JobID jobId = autoCancellableJob.getJobId(); final JobGraph jobGraph = autoCancellableJob.getJobGraph(); - clusterClient.setDetached(true); - clusterClient.submitJob(jobGraph, AbstractQueryableStateTestBase.class.getClassLoader()); + ClientUtils.submitJob(clusterClient, jobGraph); final String expected = Integer.toString(numElements * (numElements + 1) / 2); @@ -724,8 +712,7 @@ public Integer getKey(Tuple2 value) { final JobID jobId = autoCancellableJob.getJobId(); final JobGraph jobGraph = autoCancellableJob.getJobGraph(); - clusterClient.setDetached(true); - clusterClient.submitJob(jobGraph, AbstractQueryableStateTestBase.class.getClassLoader()); + ClientUtils.submitJob(clusterClient, jobGraph); final long expected = numElements * (numElements + 1L) / 2L; @@ -817,8 +804,7 @@ public void processElement(Tuple2 value, Context ctx, Collector value, Context ctx, Collector> results = new HashMap<>(); @@ -994,8 +979,7 @@ public Integer getKey(Tuple2 value) { final JobID jobId = autoCancellableJob.getJobId(); final JobGraph jobGraph = autoCancellableJob.getJobGraph(); - clusterClient.setDetached(true); - clusterClient.submitJob(jobGraph, AbstractQueryableStateTestBase.class.getClassLoader()); + ClientUtils.submitJob(clusterClient, jobGraph); for (int key = 0; key < maxParallelism; key++) { boolean success = false; 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 1202499309c8..d484f5646136 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 @@ -19,6 +19,7 @@ package org.apache.flink.runtime.webmonitor; import org.apache.flink.api.common.JobID; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -247,8 +248,7 @@ public void testCancel() throws Exception { final JobID jid = jobGraph.getJobID(); ClusterClient clusterClient = CLUSTER.getClusterClient(); - clusterClient.setDetached(true); - clusterClient.submitJob(jobGraph, WebFrontendITCase.class.getClassLoader()); + ClientUtils.submitJob(clusterClient, jobGraph); // wait for job to show up while (getRunningJobs(CLUSTER.getClusterClient()).isEmpty()) { @@ -306,8 +306,7 @@ public void testCancelYarn() throws Exception { final JobID jid = jobGraph.getJobID(); ClusterClient clusterClient = CLUSTER.getClusterClient(); - clusterClient.setDetached(true); - clusterClient.submitJob(jobGraph, WebFrontendITCase.class.getClassLoader()); + ClientUtils.submitJob(clusterClient, jobGraph); // wait for job to show up while (getRunningJobs(CLUSTER.getClusterClient()).isEmpty()) { 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 f535cb42da14..278a75bd189f 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 @@ -57,9 +57,12 @@ public JobExecutionResult execute(StreamGraph streamGraph) throws Exception { // running from the CLI will override the savepoint restore settings jobGraph.setSavepointRestoreSettings(ctx.getSavepointRestoreSettings()); - JobExecutionResult jobExecutionResult = ctx.getClient() - .submitJob(jobGraph, ctx.getUserCodeClassLoader()) - .getJobExecutionResult(); + JobExecutionResult jobExecutionResult; + if (ctx.isDetached()) { + jobExecutionResult = ClientUtils.submitJob(ctx.getClient(), jobGraph); + } else { + jobExecutionResult = ClientUtils.submitJobAndWaitForResult(ctx.getClient(), jobGraph, ctx.getUserCodeClassLoader()); + } ctx.setJobExecutionResult(jobExecutionResult); 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 acfa6050be4c..319e981a6e85 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,11 +19,15 @@ package org.apache.flink.streaming.api.environment; 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.client.RemoteExecutor; import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.RestOptions; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.util.TestLogger; import org.junit.Assert; @@ -34,6 +38,10 @@ import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; +import java.util.concurrent.CompletableFuture; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.when; /** @@ -52,10 +60,16 @@ public void testPortForwarding() throws Exception { String host = "fakeHost"; int port = 99; - JobExecutionResult expectedResult = new JobExecutionResult(null, 0, null); + JobID jobID = new JobID(); + JobResult jobResult = (new JobResult.Builder()) + .jobId(jobID) + .netRuntime(0) + .applicationStatus(ApplicationStatus.SUCCEEDED) + .build(); RestClusterClient mockedClient = Mockito.mock(RestClusterClient.class); - when(mockedClient.submitJob(Mockito.any(), Mockito.any())).thenReturn(expectedResult); + when(mockedClient.submitJob(any())).thenReturn(CompletableFuture.completedFuture(new JobSubmissionResult(jobID))); + when(mockedClient.requestJobResult(any())).thenReturn(CompletableFuture.completedFuture(jobResult)); PowerMockito.whenNew(RestClusterClient.class).withAnyArguments().thenAnswer((invocation) -> { Object[] args = invocation.getArguments(); @@ -72,7 +86,7 @@ public void testPortForwarding() throws Exception { host, port, clientConfiguration); env.fromElements(1).map(x -> x * 2); JobExecutionResult actualResult = env.execute("fakeJobName"); - Assert.assertEquals(expectedResult, actualResult); + Assert.assertEquals(jobID, actualResult.getJobID()); } @Test @@ -82,13 +96,20 @@ public void testRemoteExecutionWithSavepoint() throws Exception { null, new String[]{}, null, restoreSettings); env.fromElements(1).map(x -> x * 2); + JobID jobID = new JobID(); + JobResult jobResult = (new JobResult.Builder()) + .jobId(jobID) + .netRuntime(0) + .applicationStatus(ApplicationStatus.SUCCEEDED) + .build(); + RestClusterClient mockedClient = Mockito.mock(RestClusterClient.class); - JobExecutionResult expectedResult = new JobExecutionResult(null, 0, null); PowerMockito.whenNew(RestClusterClient.class).withAnyArguments().thenReturn(mockedClient); - when(mockedClient.submitJob(Mockito.any(), Mockito.any())).thenReturn(expectedResult); + when(mockedClient.submitJob(any())).thenReturn(CompletableFuture.completedFuture(new JobSubmissionResult(jobID))); + when(mockedClient.requestJobResult(eq(jobID))).thenReturn(CompletableFuture.completedFuture(jobResult)); JobExecutionResult actualResult = env.execute("fakeJobName"); - Assert.assertEquals(expectedResult, actualResult); + Assert.assertEquals(jobID, actualResult.getJobID()); } } 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 61de25c54d7c..fcd83ca6a472 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.client.gateway.local; import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.deployment.ClusterDescriptor; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -159,14 +160,11 @@ private void deployJobOnExistingCluster( result.setClusterInformation(clusterClient.getClusterId(), webInterfaceUrl); // submit job (and get result) if (awaitJobResult) { - clusterClient.setDetached(false); - final JobExecutionResult jobResult = clusterClient - .submitJob(jobGraph, context.getClassLoader()) - .getJobExecutionResult(); // throws exception if job fails + // throws exception if job fails + final JobExecutionResult jobResult = ClientUtils.submitJobAndWaitForResult(clusterClient, jobGraph, context.getClassLoader()); executionResultBucket.add(jobResult); } else { - clusterClient.setDetached(true); - clusterClient.submitJob(jobGraph, context.getClassLoader()); + ClientUtils.submitJob(clusterClient, jobGraph); } } finally { try { diff --git a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointIT.java b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointIT.java index d65f48df8469..982643abe396 100644 --- a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointIT.java +++ b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointIT.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.time.Time; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.MiniClusterClient; import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.checkpoint.CheckpointMetaData; @@ -224,7 +225,6 @@ private void setUpJobGraph( miniClusterResource.getClusterClient() instanceof MiniClusterClient); clusterClient = (MiniClusterClient) miniClusterResource.getClusterClient(); - clusterClient.setDetached(true); jobGraph = new JobGraph(); @@ -252,7 +252,7 @@ private void setUpJobGraph( 0), null)); - clusterClient.submitJob(jobGraph, ClassLoader.getSystemClassLoader()); + ClientUtils.submitJob(clusterClient, jobGraph); invokeLatch.await(60, TimeUnit.SECONDS); waitForJob(); } diff --git a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointITCase.java index 96241f3b2069..1c5f270d00de 100644 --- a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointITCase.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.jobmaster; import org.apache.flink.api.common.time.Time; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.MiniClusterClient; import org.apache.flink.runtime.checkpoint.CheckpointException; import org.apache.flink.runtime.checkpoint.CheckpointMetaData; @@ -90,7 +91,6 @@ private void setUpWithCheckpointInterval(long checkpointInterval) throws Excepti miniClusterResource.getClusterClient() instanceof MiniClusterClient); clusterClient = (MiniClusterClient) miniClusterResource.getClusterClient(); - clusterClient.setDetached(true); jobGraph = new JobGraph(); @@ -113,7 +113,7 @@ private void setUpWithCheckpointInterval(long checkpointInterval) throws Excepti 0), null)); - clusterClient.submitJob(jobGraph, ClassLoader.getSystemClassLoader()); + ClientUtils.submitJob(clusterClient, jobGraph); invokeLatch.await(60, TimeUnit.SECONDS); waitForJob(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java index dadbe7455f99..98062555a731 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java @@ -26,6 +26,7 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.Configuration; @@ -147,7 +148,7 @@ private static void submitJobAndVerifyResults(JobGraph jobGraph) throws Exceptio final CheckedThread submissionThread = new CheckedThread() { @Override public void go() throws Exception { - client.submitJob(jobGraph, AccumulatorLiveITCase.class.getClassLoader()); + ClientUtils.submitJobAndWaitForResult(client, jobGraph, AccumulatorLiveITCase.class.getClassLoader()); } }; diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java index 8546a0bb9359..d6927221da22 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobSubmissionResult; import org.apache.flink.api.common.Plan; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.Configuration; @@ -95,9 +96,7 @@ protected void runAndCancelJob(Plan plan, final int msecsTillCanceling, int maxT final JobGraph jobGraph = getJobGraph(plan); ClusterClient client = CLUSTER.getClusterClient(); - client.setDetached(true); - - JobSubmissionResult jobSubmissionResult = client.submitJob(jobGraph, CancelingTestBase.class.getClassLoader()); + JobSubmissionResult jobSubmissionResult = ClientUtils.submitJob(client, jobGraph); Deadline submissionDeadLine = new FiniteDuration(2, TimeUnit.MINUTES).fromNow(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RegionFailoverITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RegionFailoverITCase.java index 181743a25d37..ac258992c08c 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RegionFailoverITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RegionFailoverITCase.java @@ -27,6 +27,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.HighAvailabilityOptions; @@ -148,10 +149,9 @@ public void testMultiRegionFailover() { try { JobGraph jobGraph = createJobGraph(); ClusterClient client = cluster.getClusterClient(); - client.submitJob(jobGraph, RegionFailoverITCase.class.getClassLoader()); + ClientUtils.submitJobAndWaitForResult(client, jobGraph, RegionFailoverITCase.class.getClassLoader()); verifyAfterJobExecuted(); - } - catch (Exception e) { + } catch (Exception e) { e.printStackTrace(); Assert.fail(e.getMessage()); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java index ba7704ec017f..cf1b00a5d2fe 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java @@ -30,6 +30,7 @@ import org.apache.flink.api.common.typeutils.base.IntSerializer; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; @@ -190,8 +191,7 @@ public void testSavepointRescalingKeyedState(boolean scaleOut, boolean deriveMax final JobID jobID = jobGraph.getJobID(); - client.setDetached(true); - client.submitJob(jobGraph, RescalingITCase.class.getClassLoader()); + ClientUtils.submitJob(client, jobGraph); // wait til the sources have emitted numberElements for each key and completed a checkpoint SubtaskIndexFlatMapper.workCompletedLatch.await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); @@ -229,8 +229,7 @@ public void testSavepointRescalingKeyedState(boolean scaleOut, boolean deriveMax scaledJobGraph.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(savepointPath)); - client.setDetached(false); - client.submitJob(scaledJobGraph, RescalingITCase.class.getClassLoader()); + ClientUtils.submitJobAndWaitForResult(client, scaledJobGraph, RescalingITCase.class.getClassLoader()); Set> actualResult2 = CollectionSink.getElementsSet(); @@ -271,8 +270,7 @@ public void testSavepointRescalingNonPartitionedStateCausesException() throws Ex final JobID jobID = jobGraph.getJobID(); - client.setDetached(true); - client.submitJob(jobGraph, RescalingITCase.class.getClassLoader()); + ClientUtils.submitJob(client, jobGraph); // wait until the operator is started StateSourceBase.workStartedLatch.await(); @@ -292,8 +290,7 @@ public void testSavepointRescalingNonPartitionedStateCausesException() throws Ex scaledJobGraph.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(savepointPath)); - client.setDetached(false); - client.submitJob(scaledJobGraph, RescalingITCase.class.getClassLoader()); + ClientUtils.submitJobAndWaitForResult(client, scaledJobGraph, RescalingITCase.class.getClassLoader()); } catch (JobExecutionException exception) { if (exception.getCause() instanceof IllegalStateException) { // we expect a IllegalStateException wrapped @@ -338,8 +335,7 @@ public void testSavepointRescalingWithKeyedAndNonPartitionedState() throws Excep final JobID jobID = jobGraph.getJobID(); - client.setDetached(true); - client.submitJob(jobGraph, RescalingITCase.class.getClassLoader()); + ClientUtils.submitJob(client, jobGraph); // wait til the sources have emitted numberElements for each key and completed a checkpoint SubtaskIndexFlatMapper.workCompletedLatch.await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); @@ -382,8 +378,7 @@ public void testSavepointRescalingWithKeyedAndNonPartitionedState() throws Excep scaledJobGraph.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(savepointPath)); - client.setDetached(false); - client.submitJob(scaledJobGraph, RescalingITCase.class.getClassLoader()); + ClientUtils.submitJobAndWaitForResult(client, scaledJobGraph, RescalingITCase.class.getClassLoader()); Set> actualResult2 = CollectionSink.getElementsSet(); @@ -462,8 +457,7 @@ public void testSavepointRescalingPartitionedOperatorState(boolean scaleOut, Ope final JobID jobID = jobGraph.getJobID(); - client.setDetached(true); - client.submitJob(jobGraph, RescalingITCase.class.getClassLoader()); + ClientUtils.submitJob(client, jobGraph); // wait until the operator is started StateSourceBase.workStartedLatch.await(); @@ -494,8 +488,7 @@ public void testSavepointRescalingPartitionedOperatorState(boolean scaleOut, Ope scaledJobGraph.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(savepointPath)); - client.setDetached(false); - client.submitJob(scaledJobGraph, RescalingITCase.class.getClassLoader()); + ClientUtils.submitJobAndWaitForResult(client, scaledJobGraph, RescalingITCase.class.getClassLoader()); int sumExp = 0; int sumAct = 0; diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java index 45b51385a63e..e7df79792d2e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java @@ -20,6 +20,7 @@ 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.program.ClusterClient; import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; @@ -277,7 +278,6 @@ private void testExternalizedCheckpoints( cluster.before(); ClusterClient client = cluster.getClusterClient(); - client.setDetached(true); try { // main test sequence: start job -> eCP -> restore job -> eCP -> restore job @@ -298,7 +298,7 @@ private static String runJobAndGetExternalizedCheckpoint(StateBackend backend, F JobGraph initialJobGraph = getJobGraph(backend, externalCheckpoint); NotifyingInfiniteTupleSource.countDownLatch = new CountDownLatch(PARALLELISM); - client.submitJob(initialJobGraph, ResumeCheckpointManuallyITCase.class.getClassLoader()); + ClientUtils.submitJob(client, initialJobGraph); // wait until all sources have been started NotifyingInfiniteTupleSource.countDownLatch.await(); 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 027f1135e6fe..032db60bbf8a 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 @@ -29,6 +29,7 @@ import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; @@ -190,8 +191,7 @@ private String submitJobAndTakeSavepoint(MiniClusterResourceFactory clusterFacto ClusterClient client = cluster.getClusterClient(); try { - client.setDetached(true); - client.submitJob(jobGraph, SavepointITCase.class.getClassLoader()); + ClientUtils.submitJob(client, jobGraph); StatefulCounter.getProgressLatch().await(); @@ -232,8 +232,7 @@ private void restoreJobAndVerifyState(String savepointPath, MiniClusterResourceF ClusterClient client = cluster.getClusterClient(); try { - client.setDetached(true); - client.submitJob(jobGraph, SavepointITCase.class.getClassLoader()); + ClientUtils.submitJob(client, jobGraph); // Await state is restored StatefulCounter.getRestoreLatch().await(); @@ -317,8 +316,7 @@ public void testTriggerSavepointWithCheckpointingDisabled() throws Exception { final JobGraph graph = new JobGraph(vertex); try { - client.setDetached(true); - client.submitJob(graph, SavepointITCase.class.getClassLoader()); + ClientUtils.submitJob(client, graph); client.triggerSavepoint(graph.getJobID(), null).get(); @@ -367,8 +365,7 @@ public void testSubmitWithUnknownSavepointPath() throws Exception { LOG.info("Submitting job " + jobGraph.getJobID() + " in detached mode."); try { - client.setDetached(false); - client.submitJob(jobGraph, SavepointITCase.class.getClassLoader()); + ClientUtils.submitJobAndWaitForResult(client, jobGraph, SavepointITCase.class.getClassLoader()); } catch (Exception e) { Optional expectedJobExecutionException = ExceptionUtils.findThrowable(e, JobExecutionException.class); Optional expectedFileNotFoundException = ExceptionUtils.findThrowable(e, FileNotFoundException.class); @@ -434,8 +431,7 @@ public void testCanRestoreWithModifiedStatelessOperators() throws Exception { JobGraph originalJobGraph = env.getStreamGraph().getJobGraph(); - client.setDetached(true); - JobSubmissionResult submissionResult = client.submitJob(originalJobGraph, SavepointITCase.class.getClassLoader()); + JobSubmissionResult submissionResult = ClientUtils.submitJob(client, originalJobGraph); JobID jobID = submissionResult.getJobID(); // wait for the Tasks to be ready @@ -485,8 +481,7 @@ public void testCanRestoreWithModifiedStatelessOperators() throws Exception { "savepoint path " + savepointPath + " in detached mode."); // Submit the job - client.setDetached(true); - client.submitJob(modifiedJobGraph, SavepointITCase.class.getClassLoader()); + ClientUtils.submitJob(client, modifiedJobGraph); // Await state is restored StatefulCounter.getRestoreLatch().await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); @@ -680,8 +675,7 @@ public Integer map(Integer value) throws Exception { String savepointPath = null; try { - client.setDetached(true); - client.submitJob(jobGraph, SavepointITCase.class.getClassLoader()); + ClientUtils.submitJob(client, jobGraph); for (OneShotLatch latch : iterTestSnapshotWait) { latch.await(); } @@ -695,8 +689,7 @@ public Integer map(Integer value) throws Exception { jobGraph = streamGraph.getJobGraph(); jobGraph.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(savepointPath)); - client.setDetached(true); - client.submitJob(jobGraph, SavepointITCase.class.getClassLoader()); + ClientUtils.submitJob(client, jobGraph); for (OneShotLatch latch : iterTestRestoreWait) { latch.await(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java index ad949de380b0..06c0371a0077 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java @@ -19,6 +19,7 @@ package org.apache.flink.test.checkpointing; import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; @@ -128,9 +129,8 @@ public void runCheckpointedProgram() throws Exception { JobGraph jobGraph = env.getStreamGraph().getJobGraph(); try { - cluster.getClusterClient().submitJob(jobGraph, getClass().getClassLoader()).getJobExecutionResult(); - } - catch (ProgramInvocationException root) { + ClientUtils.submitJobAndWaitForResult(cluster.getClusterClient(), jobGraph, getClass().getClassLoader()).getJobExecutionResult(); + } catch (ProgramInvocationException root) { Throwable cause = root.getCause(); // search for nested SuccessExceptions diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java index 5fbe503681ba..4a68c914a06e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java @@ -25,6 +25,7 @@ import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -207,8 +208,7 @@ public void testRestoreBehaviourWithFaultyStateHandles() throws Exception { JobGraph jobGraph = env.getStreamGraph().getJobGraph(); JobID jobID = Preconditions.checkNotNull(jobGraph.getJobID()); - clusterClient.setDetached(true); - clusterClient.submitJob(jobGraph, ZooKeeperHighAvailabilityITCase.class.getClassLoader()); + ClientUtils.submitJob(clusterClient, jobGraph); // wait until we did some checkpoints waitForCheckpointLatch.await(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java index 5fbfcfb7564b..7a43704e5e9d 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.JobSubmissionResult; import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.ConfigConstants; @@ -134,12 +135,11 @@ protected final void executeAndSavepoint( final Deadline deadLine = Deadline.fromNow(Duration.ofMinutes(5)); ClusterClient client = miniClusterResource.getClusterClient(); - client.setDetached(true); // Submit the job JobGraph jobGraph = env.getStreamGraph().getJobGraph(); - JobSubmissionResult jobSubmissionResult = client.submitJob(jobGraph, SavepointMigrationTestBase.class.getClassLoader()); + JobSubmissionResult jobSubmissionResult = ClientUtils.submitJob(client, jobGraph); LOG.info("Submitted job {} and waiting...", jobSubmissionResult.getJobID()); @@ -200,14 +200,13 @@ protected final void restoreAndExecute( final Deadline deadLine = Deadline.fromNow(Duration.ofMinutes(5)); ClusterClient client = miniClusterResource.getClusterClient(); - client.setDetached(true); // Submit the job JobGraph jobGraph = env.getStreamGraph().getJobGraph(); jobGraph.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(savepointPath)); - JobSubmissionResult jobSubmissionResult = client.submitJob(jobGraph, SavepointMigrationTestBase.class.getClassLoader()); + JobSubmissionResult jobSubmissionResult = ClientUtils.submitJob(client, jobGraph); boolean done = false; while (deadLine.hasTimeLeft()) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java index bbb80ba38c73..658be996311f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java @@ -20,6 +20,7 @@ package org.apache.flink.test.example.client; import org.apache.flink.api.common.JobID; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.deployment.StandaloneClusterId; import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.Configuration; @@ -97,8 +98,7 @@ public void testJobRetrieval() throws Exception { // has been attached in resumingThread lock.acquire(); - client.setDetached(true); - client.submitJob(jobGraph, JobRetrievalITCase.class.getClassLoader()); + ClientUtils.submitJob(client, jobGraph); final CheckedThread resumingThread = new CheckedThread("Flink-Job-Retriever") { @Override 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 e11320857b78..22d291db7176 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 @@ -19,6 +19,7 @@ package org.apache.flink.test.example.failing; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.TaskManagerOptions; @@ -126,10 +127,13 @@ public void testMissingJarBlob() throws Exception { private void runJobSubmissionTest(JobGraph jobGraph, Predicate failurePredicate) throws org.apache.flink.client.program.ProgramInvocationException { ClusterClient client = MINI_CLUSTER_RESOURCE.getClusterClient(); - client.setDetached(detached); try { - client.submitJob(jobGraph, JobSubmissionFailsITCase.class.getClassLoader()); + if (detached) { + ClientUtils.submitJob(client, jobGraph); + } else { + ClientUtils.submitJobAndWaitForResult(client, jobGraph, JobSubmissionFailsITCase.class.getClassLoader()); + } fail("Job submission should have thrown an exception."); } catch (Exception e) { if (!failurePredicate.test(e)) { @@ -137,8 +141,7 @@ private void runJobSubmissionTest(JobGraph jobGraph, Predicate failur } } - client.setDetached(false); - client.submitJob(getWorkingJobGraph(), JobSubmissionFailsITCase.class.getClassLoader()); + ClientUtils.submitJobAndWaitForResult(client, getWorkingJobGraph(), JobSubmissionFailsITCase.class.getClassLoader()); } @Nonnull 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 e78d022ae937..8255584de8eb 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 @@ -19,6 +19,7 @@ package org.apache.flink.test.runtime; import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.core.io.IOReadableWritable; import org.apache.flink.core.memory.DataInputView; @@ -269,9 +270,9 @@ private void testProgram( final boolean isSlowReceiver, final int parallelism) throws Exception { ClusterClient client = cluster.getClusterClient(); - client.setDetached(false); - JobExecutionResult jer = (JobExecutionResult) client.submitJob( + JobExecutionResult jer = ClientUtils.submitJobAndWaitForResult( + client, createJobGraph( dataVolumeGb, useForwarder, diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java index f598a4da76c4..ae1484c27ae0 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.time.Time; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.runtime.checkpoint.CheckpointFailureReason; import org.apache.flink.runtime.checkpoint.savepoint.SavepointSerializers; @@ -109,18 +110,16 @@ public static void beforeClass() { @Test public void testMigrationAndRestore() throws Throwable { - ClassLoader classLoader = this.getClass().getClassLoader(); ClusterClient clusterClient = cluster.getClusterClient(); - clusterClient.setDetached(true); final Deadline deadline = Deadline.now().plus(TEST_TIMEOUT); // submit job with old version savepoint and create a migrated savepoint in the new version - String savepointPath = migrateJob(classLoader, clusterClient, deadline); + String savepointPath = migrateJob(clusterClient, deadline); // restore from migrated new version savepoint - restoreJob(classLoader, clusterClient, deadline, savepointPath); + restoreJob(clusterClient, deadline, savepointPath); } - private String migrateJob(ClassLoader classLoader, ClusterClient clusterClient, Deadline deadline) throws Throwable { + private String migrateJob(ClusterClient clusterClient, Deadline deadline) throws Throwable { URL savepointResource = AbstractOperatorRestoreTestBase.class.getClassLoader().getResource("operatorstate/" + getMigrationSavepointName()); if (savepointResource == null) { @@ -131,7 +130,7 @@ private String migrateJob(ClassLoader classLoader, ClusterClient clusterClien assertNotNull(jobToMigrate.getJobID()); - clusterClient.submitJob(jobToMigrate, classLoader); + ClientUtils.submitJob(clusterClient, jobToMigrate); CompletableFuture jobRunningFuture = FutureUtils.retrySuccessfulWithDelay( () -> clusterClient.getJobStatus(jobToMigrate.getJobID()), @@ -177,13 +176,13 @@ private String migrateJob(ClassLoader classLoader, ClusterClient clusterClien return savepointPath; } - private void restoreJob(ClassLoader classLoader, ClusterClient clusterClient, Deadline deadline, String savepointPath) throws Exception { + private void restoreJob(ClusterClient clusterClient, Deadline deadline, String savepointPath) throws Exception { JobGraph jobToRestore = createJobGraph(ExecutionMode.RESTORE); jobToRestore.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(savepointPath, allowNonRestoredState)); assertNotNull("Job doesn't have a JobID.", jobToRestore.getJobID()); - clusterClient.submitJob(jobToRestore, classLoader); + ClientUtils.submitJob(clusterClient, jobToRestore); CompletableFuture jobStatusFuture = FutureUtils.retrySuccessfulWithDelay( () -> clusterClient.getJobStatus(jobToRestore.getJobID()), diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/BigUserProgramJobSubmitITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/BigUserProgramJobSubmitITCase.java index 43834fd0939d..bed8330bdf4d 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/BigUserProgramJobSubmitITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/BigUserProgramJobSubmitITCase.java @@ -18,6 +18,7 @@ package org.apache.flink.test.streaming.runtime; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.client.ClientUtils; import org.apache.flink.client.deployment.StandaloneClusterId; import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -94,8 +95,7 @@ public String map(Integer value) throws Exception { StandaloneClusterId.getInstance()); try { - restClusterClient.setDetached(false); - restClusterClient.submitJob(jobGraph, BigUserProgramJobSubmitITCase.class.getClassLoader()); + ClientUtils.submitJobAndWaitForResult(restClusterClient, jobGraph, BigUserProgramJobSubmitITCase.class.getClassLoader()); List expected = Arrays.asList("x 1 0", "x 3 0", "x 5 0"); 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 9d7ce7260221..97db45df4f1e 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 @@ -59,11 +59,6 @@ public ApplicationId getClusterId() { throw new UnsupportedOperationException("Not needed in test."); } - @Override - public JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) { - throw new UnsupportedOperationException("Not needed in test."); - } - @Override public CompletableFuture getJobStatus(JobID jobId) { throw new UnsupportedOperationException("Not needed in test."); From b090d66e2c18f4f65ad94f5d833643280181246b Mon Sep 17 00:00:00 2001 From: Jeff Yang <54518670+yangjf2019@users.noreply.github.com> Date: Fri, 1 Nov 2019 17:20:54 +0800 Subject: [PATCH 403/746] [hotfix][docs] Fix paragraph tags --- docs/dev/stream/operators/index.md | 2 +- docs/dev/stream/operators/index.zh.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/dev/stream/operators/index.md b/docs/dev/stream/operators/index.md index b89fa898fe79..31d87b93eb2e 100644 --- a/docs/dev/stream/operators/index.md +++ b/docs/dev/stream/operators/index.md @@ -1016,7 +1016,7 @@ dataStream.rebalance() downstream operations will have a differing number of inputs from upstream operations.

    -

    +

    Please see this figure for a visualization of the connection pattern in the above example:

    diff --git a/docs/dev/stream/operators/index.zh.md b/docs/dev/stream/operators/index.zh.md index 1aca1dabb111..3913849eba39 100644 --- a/docs/dev/stream/operators/index.zh.md +++ b/docs/dev/stream/operators/index.zh.md @@ -1016,7 +1016,7 @@ dataStream.rebalance() downstream operations will have a differing number of inputs from upstream operations.

    -

    +

    Please see this figure for a visualization of the connection pattern in the above example:

    From 1ffbcfbf9fa62fdbccc7ef1087ddfdcf6da3a5e8 Mon Sep 17 00:00:00 2001 From: Frey Gao Date: Fri, 1 Nov 2019 20:34:58 +0800 Subject: [PATCH 404/746] [FLINK-14115][docs-zh] Translate DataStream Code Walkthrough into chinese (#9749) --- .../walkthroughs/datastream_api.zh.md | 233 +++++++++--------- 1 file changed, 117 insertions(+), 116 deletions(-) diff --git a/docs/getting-started/walkthroughs/datastream_api.zh.md b/docs/getting-started/walkthroughs/datastream_api.zh.md index 8676ae47d100..a2660a976e76 100644 --- a/docs/getting-started/walkthroughs/datastream_api.zh.md +++ b/docs/getting-started/walkthroughs/datastream_api.zh.md @@ -24,43 +24,43 @@ specific language governing permissions and limitations under the License. --> -Apache Flink offers a DataStream API for building robust, stateful streaming applications. -It provides fine-grained control over state and time, which allows for the implementation of advanced event-driven systems. -In this step-by-step guide you'll learn how to build a stateful streaming application with Flink's DataStream API. +Apache Flink 提供了 DataStream API 来实现稳定可靠的、有状态的流处理应用程序。 +Flink 支持对状态和时间的细粒度控制,以此来实现复杂的事件驱动数据处理系统。 +这个入门指导手册讲述了如何通过 Flink DataStream API 来实现一个有状态流处理程序。 * This will be replaced by the TOC {:toc} -## What Are You Building? +## 你要搭建一个什么系统 -Credit card fraud is a growing concern in the digital age. -Criminals steal credit card numbers by running scams or hacking into insecure systems. -Stolen numbers are tested by making one or more small purchases, often for a dollar or less. -If that works, they then make more significant purchases to get items they can sell or keep for themselves. +在当今数字时代,信用卡欺诈行为越来越被重视。 +罪犯可以通过诈骗或者入侵安全级别较低系统来盗窃信用卡卡号。 +用盗得的信用卡进行很小额度的例如一美元或者更小额度的消费进行测试。 +如果测试消费成功,那么他们就会用这个信用卡进行大笔消费,来购买一些他们希望得到的,或者可以倒卖的财物。 -In this tutorial, you will build a fraud detection system for alerting on suspicious credit card transactions. -Using a simple set of rules, you will see how Flink allows us to implement advanced business logic and act in real-time. +在这个教程中,你将会建立一个针对可疑信用卡交易行为的反欺诈检测系统。 +通过使用一组简单的规则,你将了解到 Flink 如何为我们实现复杂业务逻辑并实时执行。 -## Prerequisites +## 准备条件 -This walkthrough assumes that you have some familiarity with Java or Scala, but you should be able to follow along even if you are coming from a different programming language. +这个代码练习假定你对 Java 或 Scala 有一定的了解,当然,如果你之前使用的是其他开发语言,你也应该能够跟随本教程进行学习。 -## Help, I’m Stuck! +## 困难求助 -If you get stuck, check out the [community support resources](https://flink.apache.org/gettinghelp.html). -In particular, Apache Flink's [user mailing list](https://flink.apache.org/community.html#mailing-lists) is consistently ranked as one of the most active of any Apache project and a great way to get help quickly. +如果遇到困难,可以参考 [社区支持资源](https://flink.apache.org/zh/gettinghelp.html)。 +当然也可以在邮件列表提问,Flink 的 [用户邮件列表](https://flink.apache.org/zh/community.html#mailing-lists) 一直被评为所有Apache项目中最活跃的一个,这也是快速获得帮助的好方法。 -## How to Follow Along +## 怎样跟着教程练习 -If you want to follow along, you will require a computer with: +首先,你需要在你的电脑上准备以下环境: -* Java 8 -* Maven +* Java 8 +* Maven -A provided Flink Maven Archetype will create a skeleton project with all the necessary dependencies quickly, so you only need to focus on filling out the business logic. -These dependencies include `flink-streaming-java` which is the core dependency for all Flink streaming applications and `flink-walkthrough-common` that has data generators and other classes specific to this walkthrough. +一个准备好的 Flink Maven Archetype 能够快速创建一个包含了必要依赖的 Flink 程序骨架,基于此,你可以把精力集中在编写业务逻辑上即可。 +这些已包含的依赖包括 `flink-streaming-java`、`flink-walkthrough-common` 等,他们分别是 Flink 应用程序的核心依赖项和这个代码练习需要的数据生成器,当然还包括其他本代码练习所依赖的类。 -{% panel **Note:** Each code block within this walkthrough may not contain the full surrounding class for brevity. The full code is available [at the bottom of the page](#final-application). %} +{% panel **说明:** 为简洁起见,本练习中的代码块中可能不包含完整的类路径。完整的类路径可以在文档底部 [链接](#完整的程序) 中找到。 %}
    @@ -95,14 +95,14 @@ $ mvn archetype:generate \ {% unless site.is_stable %}

    - Note: For Maven 3.0 or higher, it is no longer possible to specify the repository (-DarchetypeCatalog) via the commandline. If you wish to use the snapshot repository, you need to add a repository entry to your settings.xml. For details about this change, please refer to Maven official document + 说明: 对于 Maven 3.0 或者更高的版本,已经不再支持通过命令行参数 -DarchetypeCatalog 来指定 repository。如果你使用的是 snapshot repository,你需要添加一个 repository 地址在你的 settings.xml 配置文件中。具体细节参考:Maven 官方文档

    {% endunless %} -You can edit the `groupId`, `artifactId` and `package` if you like. With the above parameters, -Maven will create a folder named `frauddetection` that contains a project with all the dependencies to complete this tutorial. -After importing the project into your editor, you can find a file `FraudDetectionJob.java` (or `FraudDetectionJob.scala`) with the following code which you can run directly inside your IDE. -Try setting break points through out the data stream and run the code in DEBUG mode to get a feeling for how everything works. +你可以根据自己的情况修改 `groupId`、 `artifactId` 和 `package`。通过这三个参数, +Maven 将会创建一个名为 `frauddetection` 的文件夹,包含了所有依赖的整个工程项目将会位于该文件夹下。 +将工程目录导入到你的开发环境之后,你可以找到 `FraudDetectionJob.java` (或 `FraudDetectionJob.scala`) 代码文件,文件中的代码如下所示。你可以在 IDE 中直接运行这个文件。 +同时,你可以试着在数据流中设置一些断点或者以 DEBUG 模式来运行程序,体验 Flink 是如何运行的。
    @@ -126,7 +126,7 @@ public class FraudDetectionJob { DataStream transactions = env .addSource(new TransactionSource()) .name("transactions"); - + DataStream alerts = transactions .keyBy(Transaction::getAccountId) .process(new FraudDetector()) @@ -245,16 +245,16 @@ class FraudDetector extends KeyedProcessFunction[Long, Transaction, Alert] {
    -## Breaking Down the Code +## 代码分析 -Let's walk step-by-step through the code of these two files. The `FraudDetectionJob` class defines the data flow of the application and the `FraudDetector` class defines the business logic of the function that detects fraudulent transactions. +让我们一步步地来分析一下这两个代码文件。`FraudDetectionJob` 类定义了程序的数据流,而 `FraudDetector` 类定义了欺诈交易检测的业务逻辑。 -We start describing how the Job is assembled in the `main` method of the `FraudDetectionJob` class. +下面我们开始讲解整个 Job 是如何组装到 `FraudDetectionJob` 类的 `main` 函数中的。 -#### The Execution Environment +#### 执行环境 -The first line sets up your `StreamExecutionEnvironment`. -The execution environment is how you set properties for your Job, create your sources, and finally trigger the execution of the Job. +第一行的 `StreamExecutionEnvironment` 用于设置你的执行环境。 +任务执行环境用于定义任务的属性、创建数据源以及最终启动任务的执行。
    @@ -270,12 +270,12 @@ val env: StreamExecutionEnvironment = StreamExecutionEnvironment.getExecutionEnv
    -#### Creating a Source +#### 创建数据源 -Sources ingest data from external systems, such as Apache Kafka, Rabbit MQ, or Apache Pulsar, into Flink Jobs. -This walkthrough uses a source that generates an infinite stream of credit card transactions for you to process. -Each transaction contains an account ID (`accountId`), timestamp (`timestamp`) of when the transaction occurred, and US$ amount (`amount`). -The `name` attached to the source is just for debugging purposes, so if something goes wrong, we will know where the error originated. +数据源从外部系统例如 Apache Kafka、Rabbit MQ 或者 Apache Pulsar 接收数据,然后将数据送到 Flink 程序中。 +这个代码练习使用的是一个能够无限循环生成信用卡模拟交易数据的数据源。 +每条交易数据包括了信用卡 ID (`accountId`),交易发生的时间 (`timestamp`) 以及交易的金额(`amount`)。 +绑定到数据源上的 `name` 属性是为了调试方便,如果发生一些异常,我们能够通过它快速定位问题发生在哪里。
    @@ -296,13 +296,13 @@ val transactions: DataStream[Transaction] = env
    -#### Partitioning Events & Detecting Fraud +#### 对事件分区 & 欺诈检测 -The `transactions` stream contains a lot of transactions from a large number of users, such that it needs to be processed in parallel my multiple fraud detection tasks. Since fraud occurs on a per-account basis, you must ensure that all transactions for the same account are processed by the same parallel task of the fraud detector operator. +`transactions` 这个数据流包含了大量的用户交易数据,需要被划分到多个并发上进行欺诈检测处理。由于欺诈行为的发生是基于某一个账户的,所以,必须要要保证同一个账户的所有交易行为数据要被同一个并发的 task 进行处理。 -To ensure that the same physical task processes all records for a particular key, you can partition a stream using `DataStream#keyBy`. -The `process()` call adds an operator that applies a function to each partitioned element in the stream. -It is common to say the operator immediately after a `keyBy`, in this case `FraudDetector`, is executed within a _keyed context_. +为了保证同一个 task 处理同一个 key 的所有数据,你可以使用 `DataStream#keyBy` 对流进行分区。 +`process()` 函数对流绑定了一个操作,这个操作将会对流上的每一个消息调用所定义好的函数。 +通常,一个操作会紧跟着 `keyBy` 被调用,在这个例子中,这个操作是`FraudDetector`,该操作是在一个 _keyed context_ 上执行的。
    @@ -325,10 +325,10 @@ val alerts: DataStream[Alert] = transactions
    -#### Outputting Results - -A sink writes a `DataStream` to an external system; such as Apache Kafka, Cassandra, and AWS Kinesis. -The `AlertSink` logs each `Alert` record with log level **INFO**, instead of writing it to persistent storage, so you can easily see your results. +#### 输出结果 + +sink 会将 `DataStream` 写出到外部系统,例如 Apache Kafka、Cassandra 或者 AWS Kinesis 等。 +`AlertSink` 使用 **INFO** 的日志级别打印每一个 `Alert` 的数据记录,而不是将其写入持久存储,以便你可以方便地查看结果。
    @@ -344,10 +344,10 @@ alerts.addSink(new AlertSink)
    -#### Executing the Job +#### 运行作业 -Flink applications are built lazily and shipped to the cluster for execution only once fully formed. -Call `StreamExecutionEnvironment#execute` to begin the execution of our Job and give it a name. +Flink 程序是懒加载的,并且只有在完全搭建好之后,才能够发布到集群上执行。 +调用 `StreamExecutionEnvironment#execute` 时给任务传递一个任务名参数,就可以开始运行任务。
    @@ -363,13 +363,13 @@ env.execute("Fraud Detection")
    -#### The Fraud Detector +#### 欺诈检测器 -The fraud detector is implemented as a `KeyedProcessFunction`. -Its method `KeyedProcessFunction#processElement` is called for every transaction event. -This first version produces an alert on every transaction, which some may say is overly conservative. +欺诈检查类 `FraudDetector` 是 `KeyedProcessFunction` 接口的一个实现。 +他的方法 `KeyedProcessFunction#processElement` 将会在每个交易事件上被调用。 +这个程序里边会对每笔交易发出警报,有人可能会说这做报过于保守了。 -The next steps of this tutorial will guide you to expand the fraud detector with more meaningful business logic. +本教程的后续步骤将指导你对这个欺诈检测器进行更有意义的业务逻辑扩展。
    @@ -385,7 +385,7 @@ public class FraudDetector extends KeyedProcessFunction collector) throws Exception { - + Alert alert = new Alert(); alert.setId(transaction.getAccountId()); @@ -422,38 +422,39 @@ class FraudDetector extends KeyedProcessFunction[Long, Transaction, Alert] {
    -## Writing a Real Application (v1) +## 实现一个真正的应用程序 -For the first version, the fraud detector should output an alert for any account that makes a small transaction immediately followed by a large one. Where small is anything less than $1.00 and large is more than $500. -Imagine your fraud detector processes the following stream of transactions for a particular account. +我们先实现第一版报警程序,对于一个账户,如果出现小于 $1 美元的交易后紧跟着一个大于 $500 的交易,就输出一个报警信息。 + +假设你的欺诈检测器所处理的交易数据如下:

    Transactions

    -Transactions 3 and 4 should be marked as fraudulent because it is a small transaction, $0.09, followed by a large one, $510. -Alternatively, transactions 7, 8, and 9 are not fraud because the small amount of $0.02 is not immediately followed by the large one; instead, there is an intermediate transaction that breaks the pattern. +交易 3 和交易 4 应该被标记为欺诈行为,因为交易 3 是一个 $0.09 的小额交易,而紧随着的交易 4 是一个 $510 的大额交易。 +另外,交易 7、8 和 交易 9 就不属于欺诈交易了,因为在交易 7 这个 $0.02 的小额交易之后,并没有跟随一个大额交易,而是一个金额适中的交易,这使得交易 7 到 交易 9 不属于欺诈行为。 -To do this, the fraud detector must _remember_ information across events; a large transaction is only fraudulent if the previous one was small. -Remembering information across events requires [state]({{ site.baseurl }}/concepts/glossary.html#managed-state), and that is why we decided to use a [KeyedProcessFunction]({{ site.baseurl }}/dev/stream/operators/process_function.html). -It provides fine-grained control over both state and time, which will allow us to evolve our algorithm with more complex requirements throughout this walkthrough. +欺诈检测器需要在多个交易事件之间记住一些信息。仅当一个大额的交易紧随一个小额交易的情况发生时,这个大额交易才被认为是欺诈交易。 +在多个事件之间存储信息就需要使用到 [状态]({{ site.baseurl }}/zh/concepts/glossary.html#managed-state),这也是我们选择使用 [KeyedProcessFunction]({{ site.baseurl }}/zh/dev/stream/operators/process_function.html) 的原因。 +它能够同时提供对状态和时间的细粒度操作,这使得我们能够在接下来的代码练习中实现更复杂的算法。 -The most straightforward implementation is a boolean flag that is set whenever a small transaction is processed. -When a large transaction comes through, you can simply check if the flag is set for that account. +最直接的实现方式是使用一个 boolean 型的标记状态来表示是否刚处理过一个小额交易。 +当处理到该账户的一个大额交易时,你只需要检查这个标记状态来确认上一个交易是是否小额交易即可。 -However, merely implementing the flag as a member variable in the `FraudDetector` class will not work. -Flink processes the transactions of multiple accounts with the same object instance of `FraudDetector`, which means if accounts A and B are routed through the same instance of `FraudDetector`, a transaction for account A could set the flag to true and then a transaction for account B could set off a false alert. -We could of course use a data structure like a `Map` to keep track of the flags for individual keys, however, a simple member variable would not be fault-tolerant and all its information be lost in case of a failure. -Hence, the fraud detector would possibly miss alerts if the application ever had to restart to recover from a failure. +然而,仅使用一个标记作为 `FraudDetector` 的类成员来记录账户的上一个交易状态是不准确的。 +Flink 会在同一个 `FraudDetector` 的并发实例中处理多个账户的交易数据,假设,当账户 A 和账户 B 的数据被分发的同一个并发实例上处理时,账户 A 的小额交易行为可能会将标记状态设置为真,随后账户 B 的大额交易可能会被误判为欺诈交易。 +当然,我们可以使用如 `Map` 这样的数据结构来保存每一个账户的状态,但是常规的类成员变量是无法做到容错处理的,当任务失败重启后,之前的状态信息将会丢失。 +这样的话,如果程序曾出现过失败重启的情况,将会漏掉一些欺诈报警。 -To address these challenges, Flink provides primitives for fault-tolerant state that are almost as easy to use as regular member variables. +为了应对这个问题,Flink 提供了一套支持容错状态的原语,这些原语几乎与常规成员变量一样易于使用。 -The most basic type of state in Flink is [ValueState]({{ site.baseurl }}/dev/stream/state/state.html#using-managed-keyed-state), a data type that adds fault tolerance to any variable it wraps. -`ValueState` is a form of _keyed state_, meaning it is only available in operators that are applied in a _keyed context_; any operator immediately following `DataStream#keyBy`. -A _keyed state_ of an operator is automatically scoped to the key of the record that is currently processed. -In this example, the key is the account id for the current transaction (as declared by `keyBy()`), and `FraudDetector` maintains an independent state for each account. -`ValueState` is created using a `ValueStateDescriptor` which contains metadata about how Flink should manage the variable. The state should be registered before the function starts processing data. -The right hook for this is the `open()` method. +Flink 中最基础的状态类型是 [ValueState]({{ site.baseurl }}/zh/dev/stream/state/state.html#using-managed-keyed-state),这是一种能够为被其封装的变量添加容错能力的类型。 +`ValueState` 是一种 _keyed state_,也就是说它只能被用于 _keyed context_ 提供的 operator 中,即所有能够紧随 `DataStream#keyBy` 之后被调用的operator。 +一个 operator 中的 _keyed state_ 的作用域默认是属于它所属的 key 的。 +这个例子中,key 就是当前正在处理的交易行为所属的信用卡账户(key 传入 keyBy() 函数调用),而 `FraudDetector` 维护了每个帐户的标记状态。 +`ValueState` 需要使用 `ValueStateDescriptor` 来创建,`ValueStateDescriptor` 包含了 Flink 如何管理变量的一些元数据信息。状态在使用之前需要先被注册。 +状态需要使用 `open()` 函数来注册状态。
    @@ -490,13 +491,13 @@ class FraudDetector extends KeyedProcessFunction[Long, Transaction, Alert] {
    -`ValueState` is a wrapper class, similar to `AtomicReference` or `AtomicLong` in the Java standard library. -It provides three methods for interacting with its contents; `update` sets the state, `value` gets the current value, and `clear` deletes its contents. -If the state for a particular key is empty, such as at the beginning of an application or after calling `ValueState#clear`, then `ValueState#value` will return `null`. -Modifications to the object returned by `ValueState#value` are not guaranteed to be recognized by the system, and so all changes must be performed with `ValueState#update`. -Otherwise, fault tolerance is managed automatically by Flink under the hood, and so you can interact with it like with any standard variable. +`ValueState` 是一个包装类,类似于 Java 标准库里边的 `AtomicReference` 和 `AtomicLong`。 +它提供了三个用于交互的方法。`update` 用于更新状态,`value` 用于获取状态值,还有 `clear` 用于清空状态。 +如果一个 key 还没有状态,例如当程序刚启动或者调用过 `ValueState#clear` 方法时,`ValueState#value` 将会返回 `null`。 +如果需要更新状态,需要调用 `ValueState#update` 方法,直接更改 `ValueState#value` 的返回值可能不会被系统识别。 +容错处理将在 Flink 后台自动管理,你可以像与常规变量那样与状态变量进行交互。 -Below, you can see an example of how you can use a flag state to track potential fraudulent transactions. +下边的示例,说明了如何使用标记状态来追踪可能的欺诈交易行为。
    @@ -517,7 +518,7 @@ Below, you can see an example of how you can use a flag state to track potential Alert alert = new Alert(); alert.setId(transaction.getAccountId()); - collector.collect(alert); + collector.collect(alert); } // Clean up our state @@ -564,31 +565,31 @@ Below, you can see an example of how you can use a flag state to track potential
    -For every transaction, the fraud detector checks the state of the flag for that account. -Remember, `ValueState` is always scoped to the current key, i.e., account. -If the flag is non-null, then the last transaction seen for that account was small, and so if the amount for this transaction is large, then the detector outputs a fraud alert. +对于每笔交易,欺诈检测器都会检查该帐户的标记状态。 +请记住,`ValueState` 的作用域始终限于当前的 key,即信用卡帐户。 +如果标记状态不为空,则该帐户的上一笔交易是小额的,因此,如果当前这笔交易的金额很大,那么检测程序将输出报警信息。 -After that check, the flag state is unconditionally cleared. -Either the current transaction caused a fraud alert, and the pattern is over, or the current transaction did not cause an alert, and the pattern is broken and needs to be restarted. +在检查之后,不论是什么状态,都需要被清空。 +不管是当前交易触发了欺诈报警而造成模式的结束,还是当前交易没有触发报警而造成模式的中断,都需要重新开始新的模式检测。 -Finally, the transaction amount is checked to see if it is small. -If so, then the flag is set so that it can be checked by the next event. -Notice that `ValueState` actually has three states, unset ( `null`), `true`, and `false`, because all `ValueState`'s are nullable. -This job only makes use of unset ( `null`) and `true` to check whether the flag is set or not. +最后,检查当前交易的金额是否属于小额交易。 +如果是,那么需要设置标记状态,以便可以在下一个事件中对其进行检查。 +注意,`ValueState` 实际上有 3 种状态:unset (`null`),`true`,和 `false`,`ValueState` 是允许空值的。 +我们的程序只使用了 unset (`null`) 和 `true` 两种来判断标记状态被设置了与否。 -## Fraud Detector v2: State + Time = ❤️ +## 欺诈检测器 v2:状态 + 时间 = ❤️ -Scammers don't wait long to make their large purchase to reduce the chances their test transaction is noticed. -For example, suppose you wanted to set a 1 minute timeout to your fraud detector; i.e., in the previous example transactions 3 and 4 would only be considered fraud if they occurred within 1 minute of each other. -Flink's `KeyedProcessFunction` allows you to set timers which invoke a callback method at some point in time in the future. +骗子们在小额交易后不会等很久就进行大额消费,这样可以降低小额测试交易被发现的几率。 +比如,假设你为欺诈检测器设置了一分钟的超时,对于上边的例子,交易 3 和 交易 4 只有间隔在一分钟之内才被认为是欺诈交易。 +Flink 中的 `KeyedProcessFunction` 允许您设置计时器,该计时器在将来的某个时间点执行回调函数。 -Let's see how we can modify our Job to comply with our new requirements: +让我们看看如何修改程序以符合我们的新要求: -* Whenever the flag is set to `true`, also set a timer for 1 minute in the future. -* When the timer fires, reset the flag by clearing its state. -* If the flag is ever cleared the timer should be canceled. +* 当标记状态被设置为 `true` 时,设置一个在当前时间一分钟后触发的定时器。 +* 当定时器被触发时,重置标记状态。 +* 当标记状态被重置时,删除定时器。 -To cancel a timer, you have to remember what time it is set for, and remembering implies state, so you will begin by creating a timer state along with your flag state. +要删除一个定时器,你需要记录这个定时器的触发时间,这同样需要状态来实现,所以你需要在标记状态后也创建一个记录定时器时间的状态。
    @@ -631,9 +632,9 @@ class FraudDetector extends KeyedProcessFunction[Long, Transaction, Alert] {
    -`KeyedProcessFunction#processElement` is called with a `Context` that contains a timer service. -The timer service can be used to query the current time, register timers, and delete timers. -With this, you can set a timer for 1 minute in the future every time the flag is set and store the timestamp in `timerState`. +`KeyedProcessFunction#processElement` 需要使用提供了定时器服务的 `Context` 来调用。 +定时器服务可以用于查询当前时间、注册定时器和删除定时器。 +使用它,你可以在标记状态被设置时,也设置一个当前时间一分钟后触发的定时器,同时,将触发时间保存到 `timerState` 状态中。
    @@ -664,10 +665,10 @@ if (transaction.getAmount < FraudDetector.SMALL_AMOUNT) {
    -Processing time is wall clock time, and is determined by the system clock of the machine running the operator. +处理时间是本地时钟时间,这是由运行任务的服务器的系统时间来决定的。 -When a timer fires, it calls `KeyedProcessFunction#onTimer`. -Overriding this method is how you can implement your callback to reset the flag. +当定时器触发时,将会调用 `KeyedProcessFunction#onTimer` 方法。 +通过重写这个方法来实现一个你自己的重置状态的回调逻辑。
    @@ -694,8 +695,8 @@ override def onTimer(
    -Finally, to cancel the timer, you need to delete the registered timer and delete the timer state. -You can wrap this in a helper method and call this method instead of `flagState.clear()`. +最后,如果要取消定时器,你需要删除已经注册的定时器,并同时清空保存定时器的状态。 +你可以把这些逻辑封装到一个助手函数中,而不是直接调用 `flagState.clear()`。
    @@ -728,9 +729,9 @@ private def cleanUp(ctx: KeyedProcessFunction[Long, Transaction, Alert]#Context)
    -And that's it, a fully functional, stateful, distributed streaming application! +这就是一个功能完备的,有状态的分布式流处理程序了。 -## Final Application +## 完整的程序
    @@ -911,10 +912,10 @@ class FraudDetector extends KeyedProcessFunction[Long, Transaction, Alert] {
    -### Expected Output +### 期望的结果 -Running this code with the provided `TransactionSource` will emit fraud alerts for account 3. -You should see the following output in your task manager logs: +使用已准备好的 `TransactionSource` 数据源运行这个代码,将会检测到账户 3 的欺诈行为,并输出报警信息。 +你将能够在你的 task manager 的日志中看到下边输出: {% highlight bash %} 2019-08-19 14:22:06,220 INFO org.apache.flink.walkthrough.common.sink.AlertSink - Alert{id=3} From 3011bffd51912b181471235af81a838cf3d180c0 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Fri, 1 Nov 2019 20:39:04 +0800 Subject: [PATCH 405/746] [FLINK-14322][table-api] Add watermark information in TableSchema (#9994) --- .../apache/flink/table/api/TableSchema.java | 152 ++++++++++---- .../apache/flink/table/api/WatermarkSpec.java | 92 +++++++++ .../descriptors/DescriptorProperties.java | 48 +++++ .../flink/table/api/TableSchemaTest.java | 187 ++++++++++++++++++ .../descriptors/DescriptorPropertiesTest.java | 37 ++++ .../table/api/TableEnvironmentTest.scala | 33 ++++ .../flink/table/api/TableSchemaTest.scala | 105 ---------- .../TableSchemaValidationTest.scala | 12 +- .../validation/TableSinksValidationTest.scala | 2 +- 9 files changed, 521 insertions(+), 147 deletions(-) create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/WatermarkSpec.java create mode 100644 flink-table/flink-table-common/src/test/java/org/apache/flink/table/api/TableSchemaTest.java delete mode 100644 flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableSchemaTest.scala diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/TableSchema.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/TableSchema.java index 7c72df99af2c..7cfac1d776ab 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/TableSchema.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/TableSchema.java @@ -22,23 +22,25 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.CompositeType; import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.FieldsDataType; +import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.utils.TypeConversions; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.Set; import java.util.stream.IntStream; import static org.apache.flink.table.api.DataTypes.FIELD; import static org.apache.flink.table.api.DataTypes.Field; import static org.apache.flink.table.api.DataTypes.ROW; +import static org.apache.flink.table.types.logical.LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE; import static org.apache.flink.table.types.utils.TypeConversions.fromDataTypeToLegacyInfo; import static org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType; @@ -54,44 +56,51 @@ public class TableSchema { private final DataType[] fieldDataTypes; - private final Map fieldNameToIndex; + /** Mapping from qualified field name to (nested) field type. */ + private final Map fieldNameToType; - private TableSchema(String[] fieldNames, DataType[] fieldDataTypes) { + private final List watermarkSpecs; + + private TableSchema(String[] fieldNames, DataType[] fieldDataTypes, List watermarkSpecs) { this.fieldNames = Preconditions.checkNotNull(fieldNames); this.fieldDataTypes = Preconditions.checkNotNull(fieldDataTypes); + this.watermarkSpecs = Preconditions.checkNotNull(watermarkSpecs); if (fieldNames.length != fieldDataTypes.length) { - throw new TableException( + throw new ValidationException( "Number of field names and field data types must be equal.\n" + "Number of names is " + fieldNames.length + ", number of data types is " + fieldDataTypes.length + ".\n" + "List of field names: " + Arrays.toString(fieldNames) + "\n" + "List of field data types: " + Arrays.toString(fieldDataTypes)); } - // validate and create name to index mapping - fieldNameToIndex = new HashMap<>(); - final Set duplicateNames = new HashSet<>(); - final Set uniqueNames = new HashSet<>(); + // validate and create name to type mapping + fieldNameToType = new HashMap<>(); for (int i = 0; i < fieldNames.length; i++) { // check for null - Preconditions.checkNotNull(fieldDataTypes[i]); - final String fieldName = Preconditions.checkNotNull(fieldNames[i]); - - // collect indices - fieldNameToIndex.put(fieldName, i); + DataType fieldType = Preconditions.checkNotNull(fieldDataTypes[i]); + String fieldName = Preconditions.checkNotNull(fieldNames[i]); + validateAndCreateNameToTypeMapping(fieldName, fieldType, ""); + } - // check uniqueness of field names - if (uniqueNames.contains(fieldName)) { - duplicateNames.add(fieldName); - } else { - uniqueNames.add(fieldName); + // validate watermark and rowtime attribute + for (WatermarkSpec watermark : watermarkSpecs) { + String rowtimeAttribute = watermark.getRowtimeAttribute(); + DataType rowtimeType = getFieldDataType(rowtimeAttribute) + .orElseThrow(() -> new ValidationException(String.format( + "Rowtime attribute '%s' is not defined in schema.", rowtimeAttribute))); + if (rowtimeType.getLogicalType().getTypeRoot() != TIMESTAMP_WITHOUT_TIME_ZONE) { + throw new ValidationException(String.format( + "Rowtime attribute '%s' must be of type TIMESTAMP but is of type '%s'.", + rowtimeAttribute, rowtimeType)); + } + LogicalType watermarkOutputType = watermark.getWatermarkExprOutputType().getLogicalType(); + if (watermarkOutputType.getTypeRoot() != TIMESTAMP_WITHOUT_TIME_ZONE) { + throw new ValidationException(String.format( + "Watermark strategy '%s' must be of type TIMESTAMP but is of type '%s'.", + watermark.getWatermarkExpressionString(), + watermarkOutputType.asSerializableString())); } - } - if (!duplicateNames.isEmpty()) { - throw new TableException( - "Field names must be unique.\n" + - "List of duplicate fields: " + duplicateNames.toString() + "\n" + - "List of all fields: " + Arrays.toString(fieldNames)); } } @@ -100,14 +109,14 @@ private TableSchema(String[] fieldNames, DataType[] fieldDataTypes) { */ @Deprecated public TableSchema(String[] fieldNames, TypeInformation[] fieldTypes) { - this(fieldNames, fromLegacyInfoToDataType(fieldTypes)); + this(fieldNames, fromLegacyInfoToDataType(fieldTypes), Collections.emptyList()); } /** * Returns a deep copy of the table schema. */ public TableSchema copy() { - return new TableSchema(fieldNames.clone(), fieldDataTypes.clone()); + return new TableSchema(fieldNames.clone(), fieldDataTypes.clone(), new ArrayList<>(watermarkSpecs)); } /** @@ -157,11 +166,12 @@ public Optional> getFieldType(int fieldIndex) { /** * Returns the specified data type for the given field name. * - * @param fieldName the name of the field + * @param fieldName the name of the field. the field name can be a nested field using a dot separator, + * e.g. "field1.innerField2" */ public Optional getFieldDataType(String fieldName) { - if (fieldNameToIndex.containsKey(fieldName)) { - return Optional.of(fieldDataTypes[fieldNameToIndex.get(fieldName)]); + if (fieldNameToType.containsKey(fieldName)) { + return Optional.of(fieldNameToType.get(fieldName)); } return Optional.empty(); } @@ -224,6 +234,17 @@ public TypeInformation toRowType() { return (TypeInformation) fromDataTypeToLegacyInfo(toRowDataType()); } + /** + * Returns a list of the watermark specification which contains rowtime attribute + * and watermark strategy expression. + * + *

    NOTE: Currently, there is at most one {@link WatermarkSpec} in the list, because we don't + * support multiple watermarks definition yet. But in the future, we may support multiple watermarks. + */ + public List getWatermarkSpecs() { + return watermarkSpecs; + } + @Override public String toString() { final StringBuilder sb = new StringBuilder(); @@ -231,6 +252,13 @@ public String toString() { for (int i = 0; i < fieldNames.length; i++) { sb.append(" |-- ").append(fieldNames[i]).append(": ").append(fieldDataTypes[i]).append('\n'); } + if (!watermarkSpecs.isEmpty()) { + for (WatermarkSpec watermark : watermarkSpecs) { + sb.append(" |-- ").append("WATERMARK FOR ") + .append(watermark.getRowtimeAttribute()).append(" AS ") + .append(watermark.getWatermarkExpressionString()); + } + } return sb.toString(); } @@ -244,16 +272,46 @@ public boolean equals(Object o) { } TableSchema schema = (TableSchema) o; return Arrays.equals(fieldNames, schema.fieldNames) && - Arrays.equals(fieldDataTypes, schema.fieldDataTypes); + Arrays.equals(fieldDataTypes, schema.fieldDataTypes) && + watermarkSpecs.equals(schema.getWatermarkSpecs()); } @Override public int hashCode() { int result = Arrays.hashCode(fieldNames); result = 31 * result + Arrays.hashCode(fieldDataTypes); + result = 31 * result + watermarkSpecs.hashCode(); return result; } + /** + * Creates a mapping from field name to data type, the field name can be a nested field. + * This is mainly used for validating whether the rowtime attribute (might be nested) exists + * in the schema. During creating, it also validates whether there is duplicate field names. + * + *

    For example, a "f0" field of ROW type has two nested fields "q1" and "q2". Then the + * mapping will be ["f0" -> ROW, "f0.q1" -> INT, "f0.q2" -> STRING]. + *

    +	 * {@code
    +	 *     f0 ROW
    +	 * }
    +	 * 
    + * @param fieldName name of this field, e.g. "q1" or "q2" in the above example. + * @param fieldType data type of this field + * @param parentFieldName the field name of parent type, e.g. "f0" in the above example. + */ + private void validateAndCreateNameToTypeMapping(String fieldName, DataType fieldType, String parentFieldName) { + String fullFieldName = parentFieldName.isEmpty() ? fieldName : parentFieldName + "." + fieldName; + DataType oldType = fieldNameToType.put(fullFieldName, fieldType); + if (oldType != null) { + throw new ValidationException("Field names must be unique. Duplicate field: '" + fullFieldName + "'"); + } + if (fieldType instanceof FieldsDataType) { + Map fieldDataTypes = ((FieldsDataType) fieldType).getFieldDataTypes(); + fieldDataTypes.forEach((key, value) -> validateAndCreateNameToTypeMapping(key, value, fullFieldName)); + } + } + /** * Creates a table schema from a {@link TypeInformation} instance. If the type information is * a {@link CompositeType}, the field names and types for the composite type are used to @@ -295,13 +353,16 @@ public static Builder builder() { */ public static class Builder { - private List fieldNames; + private final List fieldNames; + + private final List fieldDataTypes; - private List fieldDataTypes; + private final List watermarkSpecs; public Builder() { fieldNames = new ArrayList<>(); fieldDataTypes = new ArrayList<>(); + watermarkSpecs = new ArrayList<>(); } /** @@ -343,13 +404,36 @@ public Builder field(String name, TypeInformation typeInfo) { return field(name, fromLegacyInfoToDataType(typeInfo)); } + /** + * Specifies the previously defined field as an event-time attribute and specifies the watermark strategy. + * + * @param rowtimeAttribute the field name as a rowtime attribute, can be a nested field using dot separator. + * @param watermarkExpressionString the string representation of watermark generation expression, + * e.g. "ts - INTERVAL '5' SECOND". The string is a qualified SQL expression + * string (UDFs are expanded) but will not be validated by {@link TableSchema}. + * @param watermarkExprOutputType the data type of the computation result of watermark generation expression. + * Whether the data type equals to the output type of expression will also + * not be validated by {@link TableSchema}. + */ + public Builder watermark(String rowtimeAttribute, String watermarkExpressionString, DataType watermarkExprOutputType) { + Preconditions.checkNotNull(rowtimeAttribute); + Preconditions.checkNotNull(watermarkExpressionString); + Preconditions.checkNotNull(watermarkExprOutputType); + if (!this.watermarkSpecs.isEmpty()) { + throw new IllegalStateException("Multiple watermark definition is not supported yet."); + } + this.watermarkSpecs.add(new WatermarkSpec(rowtimeAttribute, watermarkExpressionString, watermarkExprOutputType)); + return this; + } + /** * Returns a {@link TableSchema} instance. */ public TableSchema build() { return new TableSchema( fieldNames.toArray(new String[0]), - fieldDataTypes.toArray(new DataType[0])); + fieldDataTypes.toArray(new DataType[0]), + watermarkSpecs); } } } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/WatermarkSpec.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/WatermarkSpec.java new file mode 100644 index 000000000000..f997badbfbe0 --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/WatermarkSpec.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.table.api; + +import org.apache.flink.table.types.DataType; + +import java.util.Objects; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Watermark metadata defined in {@link TableSchema}. It mainly contains 3 parts: + * + *
      + *
    1. the rowtime attribute.
    2. + *
    3. the string representation of watermark generation expression.
    4. + *
    5. the data type of the computation result of watermark generation expression.
    6. + *
    + */ +public class WatermarkSpec { + + private final String rowtimeAttribute; + + private final String watermarkExpressionString; + + private final DataType watermarkExprOutputType; + + public WatermarkSpec(String rowtimeAttribute, String watermarkExpressionString, DataType watermarkExprOutputType) { + this.rowtimeAttribute = checkNotNull(rowtimeAttribute); + this.watermarkExpressionString = checkNotNull(watermarkExpressionString); + this.watermarkExprOutputType = checkNotNull(watermarkExprOutputType); + } + + /** + * Returns the name of rowtime attribute, it can be a nested field using dot separator. + * The referenced attribute must be present in the {@link TableSchema} and of + * type {@link org.apache.flink.table.types.logical.LogicalTypeRoot#TIMESTAMP_WITHOUT_TIME_ZONE}. + */ + public String getRowtimeAttribute() { + return rowtimeAttribute; + } + + /** + * Returns the string representation of watermark generation expression. + * The string representation is a qualified SQL expression string (UDFs are expanded). + */ + public String getWatermarkExpressionString() { + return watermarkExpressionString; + } + + /** + * Returns the data type of the computation result of watermark generation expression. + */ + public DataType getWatermarkExprOutputType() { + return watermarkExprOutputType; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + WatermarkSpec that = (WatermarkSpec) o; + return Objects.equals(rowtimeAttribute, that.rowtimeAttribute) && + Objects.equals(watermarkExpressionString, that.watermarkExpressionString) && + Objects.equals(watermarkExprOutputType, that.watermarkExprOutputType); + } + + @Override + public int hashCode() { + return Objects.hash(rowtimeAttribute, watermarkExpressionString, watermarkExprOutputType); + } +} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/DescriptorProperties.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/DescriptorProperties.java index 3606504821a5..526907979972 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/DescriptorProperties.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/DescriptorProperties.java @@ -25,6 +25,10 @@ import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.api.WatermarkSpec; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.utils.LogicalTypeParser; +import org.apache.flink.table.types.utils.TypeConversions; import org.apache.flink.table.utils.EncodingUtils; import org.apache.flink.table.utils.TypeStringUtils; import org.apache.flink.util.InstantiationUtil; @@ -72,6 +76,14 @@ public class DescriptorProperties { public static final String TABLE_SCHEMA_TYPE = "type"; + public static final String WATERMARK = "watermark"; + + public static final String WATERMARK_ROWTIME = "rowtime"; + + public static final String WATERMARK_STRATEGY_EXPRESSION = "strategy.expression"; + + public static final String WATERMARK_STRATEGY_DATATYPE = "strategy.datatype"; + private static final Consumer EMPTY_CONSUMER = (value) -> {}; private final boolean normalizeKeys; @@ -193,6 +205,20 @@ public void putTableSchema(String key, TableSchema schema) { key, Arrays.asList(TABLE_SCHEMA_NAME, TABLE_SCHEMA_TYPE), values); + + if (!schema.getWatermarkSpecs().isEmpty()) { + final List> watermarkValues = new ArrayList<>(); + for (WatermarkSpec spec : schema.getWatermarkSpecs()) { + watermarkValues.add(Arrays.asList( + spec.getRowtimeAttribute(), + spec.getWatermarkExpressionString(), + spec.getWatermarkExprOutputType().getLogicalType().asSerializableString())); + } + putIndexedFixedProperties( + key + '.' + WATERMARK, + Arrays.asList(WATERMARK_ROWTIME, WATERMARK_STRATEGY_EXPRESSION, WATERMARK_STRATEGY_DATATYPE), + watermarkValues); + } } /** @@ -519,6 +545,28 @@ public Optional getOptionalTableSchema(String key) { schemaBuilder.field(name, type); } + + // extract watermark information + + // filter for number of fields + String watermarkPrefixKey = key + '.' + WATERMARK; + final int watermarkCount = properties.keySet().stream() + .filter((k) -> k.startsWith(watermarkPrefixKey) && k.endsWith('.' + WATERMARK_ROWTIME)) + .mapToInt((k) -> 1) + .sum(); + if (watermarkCount > 0) { + for (int i = 0; i < watermarkCount; i++) { + final String rowtimeKey = watermarkPrefixKey + '.' + i + '.' + WATERMARK_ROWTIME; + final String exprKey = watermarkPrefixKey + '.' + i + '.' + WATERMARK_STRATEGY_EXPRESSION; + final String typeKey = watermarkPrefixKey + '.' + i + '.' + WATERMARK_STRATEGY_DATATYPE; + final String rowtime = optionalGet(rowtimeKey).orElseThrow(exceptionSupplier(rowtimeKey)); + final String exprString = optionalGet(exprKey).orElseThrow(exceptionSupplier(exprKey)); + final String typeString = optionalGet(typeKey).orElseThrow(exceptionSupplier(typeKey)); + final DataType exprType = TypeConversions.fromLogicalToDataType(LogicalTypeParser.parse(typeString)); + schemaBuilder.watermark(rowtime, exprString, exprType); + } + } + return Optional.of(schemaBuilder.build()); } diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/api/TableSchemaTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/api/TableSchemaTest.java new file mode 100644 index 000000000000..15d9132ef17d --- /dev/null +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/api/TableSchemaTest.java @@ -0,0 +1,187 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.api; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.table.types.DataType; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * Tests for {@link TableSchema}. + */ +public class TableSchemaTest { + + private static final String WATERMARK_EXPRESSION = "now()"; + private static final DataType WATERMARK_DATATYPE = DataTypes.TIMESTAMP(3); + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Test + public void testTableSchema() { + TableSchema schema = TableSchema.builder() + .field("f0", DataTypes.BIGINT()) + .field("f1", DataTypes.ROW( + DataTypes.FIELD("q1", DataTypes.STRING()), + DataTypes.FIELD("q2", DataTypes.TIMESTAMP(3)))) + .field("f2", DataTypes.STRING()) + .watermark("f1.q2", WATERMARK_EXPRESSION, WATERMARK_DATATYPE) + .build(); + + // test toString() + String expected = "root\n" + + " |-- f0: BIGINT\n" + + " |-- f1: ROW<`q1` STRING, `q2` TIMESTAMP(3)>\n" + + " |-- f2: STRING\n" + + " |-- WATERMARK FOR f1.q2 AS now()"; + assertEquals(expected, schema.toString()); + + // test getFieldNames and getFieldDataType + assertEquals(Optional.of("f2"), schema.getFieldName(2)); + assertEquals(Optional.of(DataTypes.STRING()), schema.getFieldDataType("f2")); + assertEquals(Optional.of(DataTypes.STRING()), schema.getFieldDataType("f1.q1")); + assertFalse(schema.getFieldName(3).isPresent()); + assertFalse(schema.getFieldType(-1).isPresent()); + assertFalse(schema.getFieldType("c").isPresent()); + assertFalse(schema.getFieldDataType("f1.q3").isPresent()); + + // test copy() and equals() + assertEquals(schema, schema.copy()); + assertEquals(schema.hashCode(), schema.copy().hashCode()); + } + + @Test + public void testWatermarkOnDifferentFields() { + // column_name, column_type, exception_msg + List> testData = new ArrayList<>(); + testData.add(Tuple3.of("a", DataTypes.BIGINT(), "but is of type 'BIGINT'")); + testData.add(Tuple3.of("b", DataTypes.STRING(), "but is of type 'STRING'")); + testData.add(Tuple3.of("c", DataTypes.INT(), "but is of type 'INT'")); + testData.add(Tuple3.of("d", DataTypes.TIMESTAMP(), "PASS")); + testData.add(Tuple3.of("e", DataTypes.TIMESTAMP(0), "PASS")); + testData.add(Tuple3.of("f", DataTypes.TIMESTAMP(3), "PASS")); + testData.add(Tuple3.of("g", DataTypes.TIMESTAMP(9), "PASS")); + testData.add(Tuple3.of("h", DataTypes.TIMESTAMP_WITH_TIME_ZONE(3), "but is of type 'TIMESTAMP(3) WITH TIME ZONE'")); + + testData.forEach(t -> { + TableSchema.Builder builder = TableSchema.builder(); + testData.forEach(e -> builder.field(e.f0, e.f1)); + builder.watermark(t.f0, WATERMARK_EXPRESSION, WATERMARK_DATATYPE); + if (t.f2.equals("PASS")) { + TableSchema schema = builder.build(); + assertEquals(1, schema.getWatermarkSpecs().size()); + assertEquals(t.f0, schema.getWatermarkSpecs().get(0).getRowtimeAttribute()); + } else { + try { + builder.build(); + } catch (Exception e) { + assertTrue(e.getMessage().contains(t.f2)); + } + } + }); + } + + @Test + public void testWatermarkOnNestedField() { + TableSchema schema = TableSchema.builder() + .field("f0", DataTypes.BIGINT()) + .field("f1", DataTypes.ROW( + DataTypes.FIELD("q1", DataTypes.STRING()), + DataTypes.FIELD("q2", DataTypes.TIMESTAMP(3)), + DataTypes.FIELD("q3", DataTypes.ROW( + DataTypes.FIELD("t1", DataTypes.TIMESTAMP(3)), + DataTypes.FIELD("t2", DataTypes.STRING()) + ))) + ) + .watermark("f1.q3.t1", WATERMARK_EXPRESSION, WATERMARK_DATATYPE) + .build(); + + assertEquals(1, schema.getWatermarkSpecs().size()); + assertEquals("f1.q3.t1", schema.getWatermarkSpecs().get(0).getRowtimeAttribute()); + } + + @Test + public void testWatermarkOnNonExistedField() { + thrown.expectMessage("Rowtime attribute 'f1.q0' is not defined in schema"); + + TableSchema.builder() + .field("f0", DataTypes.BIGINT()) + .field("f1", DataTypes.ROW( + DataTypes.FIELD("q1", DataTypes.STRING()), + DataTypes.FIELD("q2", DataTypes.TIMESTAMP(3)))) + .watermark("f1.q0", WATERMARK_EXPRESSION, WATERMARK_DATATYPE) + .build(); + } + + @Test + public void testMultipleWatermarks() { + thrown.expectMessage("Multiple watermark definition is not supported yet."); + + TableSchema.builder() + .field("f0", DataTypes.TIMESTAMP()) + .field("f1", DataTypes.ROW( + DataTypes.FIELD("q1", DataTypes.STRING()), + DataTypes.FIELD("q2", DataTypes.TIMESTAMP(3)))) + .watermark("f1.q2", WATERMARK_EXPRESSION, WATERMARK_DATATYPE) + .watermark("f0", WATERMARK_EXPRESSION, WATERMARK_DATATYPE) + .build(); + } + + @Test + public void testDifferentWatermarkStrategyOutputTypes() { + List> testData = new ArrayList<>(); + testData.add(Tuple2.of(DataTypes.BIGINT(), "but is of type 'BIGINT'")); + testData.add(Tuple2.of(DataTypes.STRING(), "but is of type 'VARCHAR(2147483647)'")); + testData.add(Tuple2.of(DataTypes.INT(), "but is of type 'INT'")); + testData.add(Tuple2.of(DataTypes.TIMESTAMP(), "PASS")); + testData.add(Tuple2.of(DataTypes.TIMESTAMP(0), "PASS")); + testData.add(Tuple2.of(DataTypes.TIMESTAMP(3), "PASS")); + testData.add(Tuple2.of(DataTypes.TIMESTAMP(9), "PASS")); + testData.add(Tuple2.of(DataTypes.TIMESTAMP_WITH_TIME_ZONE(3), "but is of type 'TIMESTAMP(3) WITH TIME ZONE'")); + + testData.forEach(t -> { + TableSchema.Builder builder = TableSchema.builder() + .field("f0", DataTypes.TIMESTAMP()) + .watermark("f0", "f0 - INTERVAL '5' SECOND", t.f0); + if (t.f1.equals("PASS")) { + TableSchema schema = builder.build(); + assertEquals(1, schema.getWatermarkSpecs().size()); + } else { + try { + builder.build(); + } catch (Exception e) { + assertTrue(e.getMessage().contains(t.f1)); + } + } + }); + } + +} diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/descriptors/DescriptorPropertiesTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/descriptors/DescriptorPropertiesTest.java index 4859fc51f11f..38e39960f167 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/descriptors/DescriptorPropertiesTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/descriptors/DescriptorPropertiesTest.java @@ -18,10 +18,13 @@ package org.apache.flink.table.descriptors; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; import org.junit.Test; +import java.sql.Timestamp; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -168,6 +171,40 @@ public void testPrefixedMap() { assertEquals(expected.asMap(), actual); } + @Test + public void testTableSchema() { + TableSchema schema = TableSchema.builder() + .field("f0", DataTypes.BIGINT()) + .field("f1", DataTypes.ROW( + DataTypes.FIELD("q1", DataTypes.STRING()), + // the "bridgedTo" is a temporary solution because the type string format is based on TypeInformation. + DataTypes.FIELD("q2", DataTypes.TIMESTAMP(3).bridgedTo(Timestamp.class)))) + .field("f2", DataTypes.STRING()) + .watermark( + "f1.q2", + "`f1`.`q2` - INTERVAL '5' SECOND", + DataTypes.TIMESTAMP(3)) + .build(); + + DescriptorProperties properties = new DescriptorProperties(); + properties.putTableSchema("schema", schema); + Map actual = properties.asMap(); + Map expected = new HashMap<>(); + expected.put("schema.0.name", "f0"); + expected.put("schema.0.type", "BIGINT"); + expected.put("schema.1.name", "f1"); + expected.put("schema.1.type", "ROW"); + expected.put("schema.2.name", "f2"); + expected.put("schema.2.type", "VARCHAR"); + expected.put("schema.watermark.0.rowtime", "f1.q2"); + expected.put("schema.watermark.0.strategy.expression", "`f1`.`q2` - INTERVAL '5' SECOND"); + expected.put("schema.watermark.0.strategy.datatype", "TIMESTAMP(3)"); + assertEquals(expected, actual); + + TableSchema restored = properties.getTableSchema("schema"); + assertEquals(schema, restored); + } + private void testArrayValidation( DescriptorProperties properties, int minLength, diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala index 91077269e83b..ae6b6afe8b3a 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala @@ -30,6 +30,7 @@ import org.apache.flink.types.Row import org.apache.flink.api.java.tuple.{Tuple3 => JTuple3} import org.apache.flink.api.java.typeutils.GenericTypeInfo import org.apache.flink.api.scala.typeutils.UnitTypeInfo +import org.junit.Assert.assertEquals import org.junit.Test class TableEnvironmentTest extends TableTestBase { @@ -544,6 +545,38 @@ class TableEnvironmentTest extends TableTestBase { util.addTable[JTuple3[Int, Long, String]]('f0, 'f1.rowtime as 'new, 'f2), Seq("f0" -> INT, "new" -> ROWTIME, "f2" -> STRING)) } + + @Test + def testTableSchemaWithDifferentRowTypes(): Unit = { + + def createInnerRow(innerFieldName: String): TypeInformation[_] = { + Types.ROW( + Array[String](innerFieldName), + Array[TypeInformation[_]](Types.INT())) + } + + def createRow(innerFieldName: String): TypeInformation[_] = { + Types.ROW( + Array[String]("field"), + Array[TypeInformation[_]](createInnerRow(innerFieldName)) + ) + } + + val util = streamTestUtil() + util.addTable("MyTableA", 'field)(createRow("A")) + util.addTable("MyTableB", 'field)(createRow("B")) + + val actualSchema = util.tableEnv + .sqlQuery("SELECT MyTableA.field AS a, MyTableB.field AS b FROM MyTableA, MyTableB") + .getSchema + + val expectedSchema = TableSchema.builder() + .field("a", createInnerRow("A")) + .field("b", createInnerRow("B")) + .build() + + assertEquals(expectedSchema, actualSchema) + } } object TableEnvironmentTest { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableSchemaTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableSchemaTest.scala deleted file mode 100644 index a4a05d768925..000000000000 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableSchemaTest.scala +++ /dev/null @@ -1,105 +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.api - -import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.scala._ -import org.apache.flink.table.api.scala._ -import org.apache.flink.table.utils.TableTestBase -import org.junit.Assert.{assertEquals, assertTrue} -import org.junit.Test - -class TableSchemaTest extends TableTestBase { - - @Test - def testBatchTableSchema(): Unit = { - val util = batchTestUtil() - val table = util.addTable[(Int, String)]("MyTable", 'a, 'b) - val schema = table.getSchema - - assertEquals("a", schema.getFieldNames.apply(0)) - assertEquals("b", schema.getFieldNames.apply(1)) - - assertEquals(Types.INT, schema.getFieldTypes.apply(0)) - assertEquals(Types.STRING, schema.getFieldTypes.apply(1)) - - val expectedString = "root\n" + - " |-- a: INT\n" + - " |-- b: STRING\n" - assertEquals(expectedString, schema.toString) - - assertTrue(!schema.getFieldName(3).isPresent) - assertTrue(!schema.getFieldType(-1).isPresent) - assertTrue(!schema.getFieldType("c").isPresent) - } - - @Test - def testStreamTableSchema(): Unit = { - val util = streamTestUtil() - val table = util.addTable[(Int, String)]("MyTable", 'a, 'b) - val schema = table.getSchema - - assertEquals("a", schema.getFieldNames.apply(0)) - assertEquals("b", schema.getFieldNames.apply(1)) - - assertEquals(Types.INT, schema.getFieldTypes.apply(0)) - assertEquals(Types.STRING, schema.getFieldTypes.apply(1)) - - val expectedString = "root\n" + - " |-- a: INT\n" + - " |-- b: STRING\n" - assertEquals(expectedString, schema.toString) - - assertTrue(!schema.getFieldName(3).isPresent) - assertTrue(!schema.getFieldType(-1).isPresent) - assertTrue(!schema.getFieldType("c").isPresent) - } - - @Test - def testTableSchemaWithDifferentRowTypes(): Unit = { - - def createInnerRow(innerFieldName: String): TypeInformation[_] = { - Types.ROW( - Array[String](innerFieldName), - Array[TypeInformation[_]](Types.INT())) - } - - def createRow(innerFieldName: String): TypeInformation[_] = { - Types.ROW( - Array[String]("field"), - Array[TypeInformation[_]](createInnerRow(innerFieldName)) - ) - } - - val util = streamTestUtil() - util.addTable("MyTableA", 'field)(createRow("A")) - util.addTable("MyTableB", 'field)(createRow("B")) - - val actualSchema = util.tableEnv - .sqlQuery("SELECT MyTableA.field AS a, MyTableB.field AS b FROM MyTableA, MyTableB") - .getSchema - - val expectedSchema = TableSchema.builder() - .field("a", createInnerRow("A")) - .field("b", createInnerRow("B")) - .build() - - assertEquals(expectedSchema, actualSchema) - } -} diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/validation/TableSchemaValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/validation/TableSchemaValidationTest.scala index ad5b3a12a6c6..e2296b78e687 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/validation/TableSchemaValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/validation/TableSchemaValidationTest.scala @@ -18,15 +18,16 @@ package org.apache.flink.table.api.validation import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} -import org.apache.flink.table.api.{TableException, TableSchema} +import org.apache.flink.table.api.{TableSchema, ValidationException} import org.apache.flink.table.utils.TableTestBase import org.junit.Test + class TableSchemaValidationTest extends TableTestBase { @Test def testColumnNameAndColumnTypeNotEqual() { - thrown.expect(classOf[TableException]) + thrown.expect(classOf[ValidationException]) thrown.expectMessage( "Number of field names and field data types must be equal.\n" + "Number of names is 3, number of data types is 2.\n" + @@ -42,11 +43,8 @@ class TableSchemaValidationTest extends TableTestBase { @Test def testColumnNamesDuplicate() { - thrown.expect(classOf[TableException]) - thrown.expectMessage( - "Field names must be unique.\n" + - "List of duplicate fields: [a]\n" + - "List of all fields: [a, a, c]") + thrown.expect(classOf[ValidationException]) + thrown.expectMessage("Field names must be unique. Duplicate field: 'a'") val fieldNames = Array("a", "a", "c") val typeInfos: Array[TypeInformation[_]] = Array( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/validation/TableSinksValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/validation/TableSinksValidationTest.scala index c9725b920096..864e7e1b424c 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/validation/TableSinksValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/validation/TableSinksValidationTest.scala @@ -55,7 +55,7 @@ class TableSinksValidationTest extends TableTestBase { new UnsafeMemoryAppendTableSink().configure(fieldNames, fieldTypes)) } - @Test(expected = classOf[TableException]) + @Test(expected = classOf[ValidationException]) def testRegistrationWithInconsistentFieldNamesAndTypesLength(): Unit = { val util = streamTestUtil() From a6afeb45c5f219ae6f6656442962f7aa36ee225f Mon Sep 17 00:00:00 2001 From: yangjf2019 Date: Wed, 25 Sep 2019 10:54:12 +0800 Subject: [PATCH 406/746] [FLINK-12939][docs-zh] Translate "Apache Kafka Connector" page into Chinese This closes #9764 --- docs/dev/connectors/kafka.zh.md | 690 +++++++++++--------------------- 1 file changed, 241 insertions(+), 449 deletions(-) diff --git a/docs/dev/connectors/kafka.zh.md b/docs/dev/connectors/kafka.zh.md index 40c03f10c651..7d917d37946e 100644 --- a/docs/dev/connectors/kafka.zh.md +++ b/docs/dev/connectors/kafka.zh.md @@ -1,5 +1,5 @@ --- -title: "Apache Kafka Connector" +title: "Apache Kafka 连接器" nav-title: Kafka nav-parent_id: connectors nav-pos: 1 @@ -26,25 +26,20 @@ under the License. * This will be replaced by the TOC {:toc} -This connector provides access to event streams served by [Apache Kafka](https://kafka.apache.org/). +此连接器提供了访问 [Apache Kafka](https://kafka.apache.org/) 事件流的服务。 -Flink provides special Kafka Connectors for reading and writing data from/to Kafka topics. -The Flink Kafka Consumer integrates with Flink's checkpointing mechanism to provide -exactly-once processing semantics. To achieve that, Flink does not purely rely on Kafka's consumer group -offset tracking, but tracks and checkpoints these offsets internally as well. +Flink 提供了专门的 Kafka 连接器,向 Kafka topic 中读取或者写入数据。Flink Kafka Consumer 集成了 Flink 的 Checkpoint 机制,可提供 exactly-once 的处理语义。为此,Flink 并不完全依赖于跟踪 Kafka 消费组的偏移量,而是在内部跟踪和检查偏移量。 -Please pick a package (maven artifact id) and class name for your use-case and environment. -For most users, the `FlinkKafkaConsumer08` (part of `flink-connector-kafka`) is appropriate. +根据你的用例和环境选择相应的包(maven artifact id)和类名。对于大多数用户来说,使用 `FlinkKafkaConsumer08`( `flink-connector-kafka` 的一部分)是比较合适的。 - - - - - + + + + + @@ -54,7 +49,8 @@ For most users, the `FlinkKafkaConsumer08` (part of `flink-connector-kafka`) is - + @@ -62,7 +58,7 @@ For most users, the `FlinkKafkaConsumer08` (part of `flink-connector-kafka`) is - + @@ -70,7 +66,7 @@ For most users, the `FlinkKafkaConsumer08` (part of `flink-connector-kafka`) is - + @@ -78,7 +74,7 @@ For most users, the `FlinkKafkaConsumer08` (part of `flink-connector-kafka`) is - + @@ -87,17 +83,14 @@ For most users, the `FlinkKafkaConsumer08` (part of `flink-connector-kafka`) is FlinkKafkaProducer
    Maven DependencySupported sinceConsumer and
    - Producer Class name
    Kafka versionNotesMaven 依赖自从哪个版本
    开始支持
    消费者和
    生产者的类名称
    Kafka 版本注意
    FlinkKafkaConsumer08
    FlinkKafkaProducer08
    0.8.xUses the SimpleConsumer API of Kafka internally. Offsets are committed to ZK by Flink.这个连接器在内部使用 Kafka 的 SimpleConsumer API。偏移量由 Flink 提交给 ZK。 +
    flink-connector-kafka-0.9{{ site.scala_version_suffix }}FlinkKafkaConsumer09
    FlinkKafkaProducer09
    0.9.xUses the new Consumer API Kafka.这个连接器使用新的 Kafka Consumer API
    flink-connector-kafka-0.10{{ site.scala_version_suffix }}FlinkKafkaConsumer010
    FlinkKafkaProducer010
    0.10.xThis connector supports Kafka messages with timestamps both for producing and consuming.这个连接器支持 带有时间戳的 Kafka 消息,用于生产和消费。
    flink-connector-kafka-0.11{{ site.scala_version_suffix }}FlinkKafkaConsumer011
    FlinkKafkaProducer011
    0.11.xSince 0.11.x Kafka does not support scala 2.10. This connector supports Kafka transactional messaging to provide exactly once semantic for the producer.Kafka 从 0.11.x 版本开始不支持 Scala 2.10。此连接器支持了 Kafka 事务性的消息传递来为生产者提供 Exactly once 语义。
    flink-connector-kafka{{ site.scala_version_suffix }} >= 1.0.0 - This universal Kafka connector attempts to track the latest version of the Kafka client. - The version of the client it uses may change between Flink releases. Starting with Flink 1.9 release, it uses the Kafka 2.2.0 client. - Modern Kafka clients are backwards compatible with broker versions 0.10.0 or later. - However for Kafka 0.11.x and 0.10.x versions, we recommend using dedicated - flink-connector-kafka-0.11{{ site.scala_version_suffix }} and flink-connector-kafka-0.10{{ site.scala_version_suffix }} respectively. + 这个通用的 Kafka 连接器尽力与 Kafka client 的最新版本保持同步。该连接器使用的 Kafka client 版本可能会在 Flink 版本之间发生变化。从 Flink 1.9 版本开始,它使用 Kafka 2.2.0 client。当前 Kafka 客户端向后兼容 0.10.0 或更高版本的 Kafka broker。 + 但是对于 Kafka 0.11.x 和 0.10.x 版本,我们建议你分别使用专用的 flink-connector-kafka-0.11{{ site.scala_version_suffix }} 和 flink-connector-kafka-0.10{{ site.scala_version_suffix }} 连接器。
    -Then, import the connector in your maven project: +接着,在你的 maven 项目中导入连接器: {% highlight xml %} @@ -107,40 +100,36 @@ Then, import the connector in your maven project: {% endhighlight %} -Note that the streaming connectors are currently not part of the binary distribution. -See how to link with them for cluster execution [here]({{ site.baseurl}}/dev/projectsetup/dependencies.html). +请注意:目前流连接器还不是二进制分发的一部分。 +[在此处]({{ site.baseurl }}/zh/dev/projectsetup/dependencies.html)可以了解到如何链接它们以实现在集群中执行。 -## Installing Apache Kafka +## 安装 Apache Kafka -* Follow the instructions from [Kafka's quickstart](https://kafka.apache.org/documentation.html#quickstart) to download the code and launch a server (launching a Zookeeper and a Kafka server is required every time before starting the application). -* If the Kafka and Zookeeper servers are running on a remote machine, then the `advertised.host.name` setting in the `config/server.properties` file must be set to the machine's IP address. +* 按照 [ Kafka 快速入门](https://kafka.apache.org/documentation.html#quickstart)的说明下载代码并启动 Kafka 服务器(每次启动应用程序之前都需要启动 Zookeeper 和 Kafka server)。 +* 如果 Kafka 和 Zookeeper 服务器运行在远端机器上,那么必须要将 `config/server.properties` 文件中的 `advertised.host.name`属性设置为远端设备的 IP 地址。 -## Kafka 1.0.0+ Connector +## Kafka 1.0.0+ 连接器 -Starting with Flink 1.7, there is a new universal Kafka connector that does not track a specific Kafka major version. -Rather, it tracks the latest version of Kafka at the time of the Flink release. +从 Flink 1.7 开始,有一个新的通用 Kafka 连接器,它不跟踪特定的 Kafka 主版本。相反,它是在 Flink 发布时跟踪最新版本的 Kafka。 +如果你的 Kafka broker 版本是 1.0.0 或 更新的版本,你应该使用这个 Kafka 连接器。 +如果你使用的是 Kafka 的旧版本( 0.11、0.10、0.9 或 0.8 ),那么你应该使用与 Kafka broker 版本相对应的连接器。 -If your Kafka broker version is 1.0.0 or newer, you should use this Kafka connector. -If you use an older version of Kafka (0.11, 0.10, 0.9, or 0.8), you should use the connector corresponding to the broker version. +### 兼容性 -### Compatibility +通过 Kafka client API 和 broker 的兼容性保证,通用的 Kafka 连接器兼容较旧和较新的 Kafka broker。 +它兼容 Kafka broker 0.11.0 或者更高版本,具体兼容性取决于所使用的功能。有关 Kafka 兼容性的详细信息,请参考 [Kafka 文档](https://kafka.apache.org/protocol.html#protocol_compatibility)。 -The universal Kafka connector is compatible with older and newer Kafka brokers through the compatibility guarantees of the Kafka client API and broker. -It is compatible with broker versions 0.11.0 or newer, depending on the features used. -For details on Kafka compatibility, please refer to the [Kafka documentation](https://kafka.apache.org/protocol.html#protocol_compatibility). +### 将 Kafka Connector 从 0.11 迁移到通用版本 -### Migrating Kafka Connector from 0.11 to universal +以便执行迁移,请参考 [升级 Jobs 和 Flink 版本指南]({{ site.baseurl }}/zh/ops/upgrading.html): +* 在全程中使用 Flink 1.9 或更新版本。 +* 不要同时升级 Flink 和 Operator。 +* 确保你的 Job 中所使用的 Kafka Consumer 和 Kafka Producer 分配了唯一的标识符(uid)。 +* 使用 stop with savepoint 的特性来执行 savepoint(例如,使用 `stop --withSavepoint`)[CLI 命令]({{ site.baseurl }}/zh/ops/cli.html)。 -In order to perform the migration, see the [upgrading jobs and Flink versions guide]({{ site.baseurl }}/ops/upgrading.html) -and: -* Use Flink 1.9 or newer for the whole process. -* Do not upgrade the Flink and operators at the same time. -* Make sure that Kafka Consumer and/or Kafka Producer used in your job have assigned unique identifiers (`uid`): -* Use stop with savepoint feature to take the savepoint (for example by using `stop --withSavepoint`)[CLI command]({{ site.baseurl }}/ops/cli.html). +### 用法 -### Usage - -To use the universal Kafka connector add a dependency to it: +要使用通用的 Kafka 连接器,请为它添加依赖关系: {% highlight xml %} @@ -150,44 +139,40 @@ To use the universal Kafka connector add a dependency to it: {% endhighlight %} -Then instantiate the new source (`FlinkKafkaConsumer`) and sink (`FlinkKafkaProducer`). -The API is backward compatible with the Kafka 0.11 connector, -except of dropping specific Kafka version from the module and class names. +然后,实例化 source( `FlinkKafkaConsumer`)和 sink( `FlinkKafkaProducer`)。除了从模块和类名中删除了特定的 Kafka 版本外,这个 API 向后兼容 Kafka 0.11 版本的 connector。 ## Kafka Consumer -Flink's Kafka consumer is called `FlinkKafkaConsumer08` (or 09 for Kafka 0.9.0.x versions, etc. -or just `FlinkKafkaConsumer` for Kafka >= 1.0.0 versions). It provides access to one or more Kafka topics. +Flink 的 Kafka consumer 称为 `FlinkKafkaConsumer08`(或适用于 Kafka 0.9.0.x 版本的 `FlinkKafkaConsumer09`,或适用于 Kafka >= 1.0.0 的版本的 `FlinkKafkaConsumer`)。它提供对一个或多个 Kafka topics 的访问。 -The constructor accepts the following arguments: +构造函数接受以下参数: -1. The topic name / list of topic names -2. A DeserializationSchema / KafkaDeserializationSchema for deserializing the data from Kafka -3. Properties for the Kafka consumer. - The following properties are required: - - "bootstrap.servers" (comma separated list of Kafka brokers) - - "zookeeper.connect" (comma separated list of Zookeeper servers) (**only required for Kafka 0.8**) - - "group.id" the id of the consumer group +1. Topic 名称或者名称列表 +2. 用于反序列化 Kafka 数据的 DeserializationSchema 或者 KafkaDeserializationSchema +3. Kafka 消费者的属性。需要以下属性: + - "bootstrap.servers"(以逗号分隔的 Kafka broker 列表) + - "zookeeper.connect"(以逗号分割的 Zookeeper servers 列表) (**仅 Kafka 0.8 需要**) + - "group.id" 消费组 ID -Example: +示例:
    {% highlight java %} Properties properties = new Properties(); properties.setProperty("bootstrap.servers", "localhost:9092"); -// only required for Kafka 0.8 +// 仅 Kafka 0.8 需要 properties.setProperty("zookeeper.connect", "localhost:2181"); properties.setProperty("group.id", "test"); DataStream stream = env - .addSource(new FlinkKafkaConsumer08<>("topic", new SimpleStringSchema(), properties)); + .addSource(new FlinkKafkaConsumer08<>("topic", new SimpleStringSchema(), properties)); {% endhighlight %}
    {% highlight scala %} val properties = new Properties() properties.setProperty("bootstrap.servers", "localhost:9092") -// only required for Kafka 0.8 +// 仅 Kafka 0.8 需要 properties.setProperty("zookeeper.connect", "localhost:2181") properties.setProperty("group.id", "test") stream = env @@ -197,41 +182,31 @@ stream = env
    -### The `DeserializationSchema` +### `DeserializationSchema` -The Flink Kafka Consumer needs to know how to turn the binary data in Kafka into Java/Scala objects. The -`DeserializationSchema` allows users to specify such a schema. The `T deserialize(byte[] message)` -method gets called for each Kafka message, passing the value from Kafka. +Flink Kafka Consumer 需要知道如何将 Kafka 中的二进制数据转换为 Java 或者 Scala 对象。`DeserializationSchema` 允许用户指定这样的 schema,为每条 Kafka 消息调用 `T deserialize(byte[] message)` 方法,传递来自 Kafka 的值。 -It is usually helpful to start from the `AbstractDeserializationSchema`, which takes care of describing the -produced Java/Scala type to Flink's type system. Users that implement a vanilla `DeserializationSchema` need -to implement the `getProducedType(...)` method themselves. +从 `AbstractDeserializationSchema` 开始通常很有帮助,它负责将生成的 Java 或 Scala 类型描述为 Flink 的类型系统。 +用户如果要自己去实现一个`DeserializationSchema`,需要自己去实现 `getProducedType(...)`方法。 -For accessing the key, value and metadata of the Kafka message, the `KafkaDeserializationSchema` has -the following deserialize method `T deserialize(ConsumerRecord record)`. +为了访问 Kafka 消息的 key、value 和元数据,`KafkaDeserializationSchema` 具有以下反序列化方法 `T deserialize(ConsumerRecord record)`。 -For convenience, Flink provides the following schemas: +为了方便使用,Flink 提供了以下几种 schemas: -1. `TypeInformationSerializationSchema` (and `TypeInformationKeyValueSerializationSchema`) which creates - a schema based on a Flink's `TypeInformation`. This is useful if the data is both written and read by Flink. - This schema is a performant Flink-specific alternative to other generic serialization approaches. +1. `TypeInformationSerializationSchema`(和 `TypeInformationKeyValueSerializationSchema`) 基于 Flink 的 `TypeInformation` 创建 `schema`。 + 如果该数据的读和写都发生在 Flink 中,那么这将是非常有用的。此 schema 是其他通用序列化方法的高性能 Flink 替代方案。 -2. `JsonDeserializationSchema` (and `JSONKeyValueDeserializationSchema`) which turns the serialized JSON - into an ObjectNode object, from which fields can be accessed using `objectNode.get("field").as(Int/String/...)()`. - The KeyValue objectNode contains a "key" and "value" field which contain all fields, as well as - an optional "metadata" field that exposes the offset/partition/topic for this message. +2. `JsonDeserializationSchema`(和 `JSONKeyValueDeserializationSchema`)将序列化的 JSON 转化为 ObjectNode 对象,可以使用 `objectNode.get("field").as(Int/String/...)()` 来访问某个字段。 + KeyValue objectNode 包含一个含所有字段的 key 和 values 字段,以及一个可选的"metadata"字段,可以访问到消息的 offset、partition、topic 等信息。 -3. `AvroDeserializationSchema` which reads data serialized with Avro format using a statically provided schema. It can - infer the schema from Avro generated classes (`AvroDeserializationSchema.forSpecific(...)`) or it can work with `GenericRecords` - with a manually provided schema (with `AvroDeserializationSchema.forGeneric(...)`). This deserialization schema expects that - the serialized records DO NOT contain embedded schema. +3. `AvroDeserializationSchema` 使用静态提供的 schema 读取 Avro 格式的序列化数据。 + 它能够从 Avro 生成的类(`AvroDeserializationSchema.forSpecific(...)`)中推断出 schema,或者可以与 `GenericRecords` + 一起使用手动提供的 schema(用 `AvroDeserializationSchema.forGeneric(...)`)。此反序列化 schema 要求序列化记录不能包含嵌入式架构! - - There is also a version of this schema available that can lookup the writer's schema (schema which was used to write the record) in - [Confluent Schema Registry](https://docs.confluent.io/current/schema-registry/docs/index.html). Using these deserialization schema - record will be read with the schema that was retrieved from Schema Registry and transformed to a statically provided( either through - `ConfluentRegistryAvroDeserializationSchema.forGeneric(...)` or `ConfluentRegistryAvroDeserializationSchema.forSpecific(...)`). + - 此模式还有一个版本,可以在 [Confluent Schema Registry](https://docs.confluent.io/current/schema-registry/docs/index.html) 中查找编写器的 schema(用于编写记录的 schema)。 + - 使用这些反序列化 schema 记录将读取从 schema 注册表检索到的 schema 转换为静态提供的 schema(或者通过 `ConfluentRegistryAvroDeserializationSchema.forGeneric(...)` 或 `ConfluentRegistryAvroDeserializationSchema.forSpecific(...)`)。 -
    To use this deserialization schema one has to add the following additional dependency: +
    要使用此反序列化 schema 必须添加以下依赖:
    @@ -254,22 +229,13 @@ For convenience, Flink provides the following schemas:
    -When encountering a corrupted message that cannot be deserialized for any reason, there -are two options - either throwing an exception from the `deserialize(...)` method -which will cause the job to fail and be restarted, or returning `null` to allow -the Flink Kafka consumer to silently skip the corrupted message. Note that -due to the consumer's fault tolerance (see below sections for more details), -failing the job on the corrupted message will let the consumer attempt -to deserialize the message again. Therefore, if deserialization still fails, the -consumer will fall into a non-stop restart and fail loop on that corrupted -message. +当遇到因一些原因而无法反序列化的损坏消息时,这里有两个选项 - 从 `deserialize(...)` 方法抛出异常会导致作业失败并重新启动,或返回 `null`,以允许 Flink Kafka 消费者悄悄地跳过损坏的消息。请注意,由于 Consumer 的容错能力(请参阅下面的部分以获取更多详细信息),在损坏的消息上失败作业将使 consumer 尝试再次反序列化消息。因此,如果反序列化仍然失败,则 consumer 将在该损坏的消息上进入不间断重启和失败的循环。 -### Kafka Consumers Start Position Configuration +### 配置 Kafka Consumer 开始消费的位置 -The Flink Kafka Consumer allows configuring how the start position for Kafka -partitions are determined. +Flink Kafka Consumer 允许通过配置来确定 Kafka 分区的起始位置。 -Example: +例如:
    @@ -277,10 +243,10 @@ Example: final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); FlinkKafkaConsumer08 myConsumer = new FlinkKafkaConsumer08<>(...); -myConsumer.setStartFromEarliest(); // start from the earliest record possible -myConsumer.setStartFromLatest(); // start from the latest record -myConsumer.setStartFromTimestamp(...); // start from specified epoch timestamp (milliseconds) -myConsumer.setStartFromGroupOffsets(); // the default behaviour +myConsumer.setStartFromEarliest(); // 尽可能从最早的记录开始 +myConsumer.setStartFromLatest(); // 从最新的记录开始 +myConsumer.setStartFromTimestamp(...); // 从指定的时间开始(毫秒) +myConsumer.setStartFromGroupOffsets(); // 默认的方法 DataStream stream = env.addSource(myConsumer); ... @@ -291,10 +257,10 @@ DataStream stream = env.addSource(myConsumer); val env = StreamExecutionEnvironment.getExecutionEnvironment() val myConsumer = new FlinkKafkaConsumer08[String](...) -myConsumer.setStartFromEarliest() // start from the earliest record possible -myConsumer.setStartFromLatest() // start from the latest record -myConsumer.setStartFromTimestamp(...) // start from specified epoch timestamp (milliseconds) -myConsumer.setStartFromGroupOffsets() // the default behaviour +myConsumer.setStartFromEarliest() // 尽可能从最早的记录开始 +myConsumer.setStartFromLatest() // 从最新的记录开始 +myConsumer.setStartFromTimestamp(...) // 从指定的时间开始(毫秒) +myConsumer.setStartFromGroupOffsets() // 默认的方法 val stream = env.addSource(myConsumer) ... @@ -302,22 +268,14 @@ val stream = env.addSource(myConsumer)
    -All versions of the Flink Kafka Consumer have the above explicit configuration methods for start position. +Flink Kafka Consumer 的所有版本都具有上述明确的起始位置配置方法。 - * `setStartFromGroupOffsets` (default behaviour): Start reading partitions from - the consumer group's (`group.id` setting in the consumer properties) committed - offsets in Kafka brokers (or Zookeeper for Kafka 0.8). If offsets could not be - found for a partition, the `auto.offset.reset` setting in the properties will be used. - * `setStartFromEarliest()` / `setStartFromLatest()`: Start from the earliest / latest - record. Under these modes, committed offsets in Kafka will be ignored and - not used as starting positions. - * `setStartFromTimestamp(long)`: Start from the specified timestamp. For each partition, the record - whose timestamp is larger than or equal to the specified timestamp will be used as the start position. - If a partition's latest record is earlier than the timestamp, the partition will simply be read - from the latest record. Under this mode, committed offsets in Kafka will be ignored and not used as - starting positions. + * `setStartFromGroupOffsets`(默认方法):从 Kafka brokers(或者从 Kafka 0.8 版本的 Zookeeper 中)中的 consumer 组(consumer 属性中的 `group.id` 设置)提交的偏移量中开始读取分区。 + 如果找不到分区的偏移量,那么将会使用配置中的 `auto.offset.reset` 设置。 + * `setStartFromEarliest()` 或者 `setStartFromLatest()`:从最早或者最新的记录开始消费,在这些模式下,Kafka 中的 committed offset 将被忽略,不会用作起始位置。 + * `setStartFromTimestamp(long)`:从指定的时间戳开始。对于每个分区,其时间戳大于或等于指定时间戳的记录将用作起始位置。如果一个分区的最新记录早于指定的时间戳,则只从最新记录读取该分区数据。在这种模式下,Kafka 中的已提交 offset 将被忽略,不会用作起始位置。 -You can also specify the exact offsets the consumer should start from for each partition: +你也可以为每个分区指定 consumer 应该开始消费的具体 offset:
    @@ -342,74 +300,51 @@ myConsumer.setStartFromSpecificOffsets(specificStartOffsets)
    -The above example configures the consumer to start from the specified offsets for -partitions 0, 1, and 2 of topic `myTopic`. The offset values should be the -next record that the consumer should read for each partition. Note that -if the consumer needs to read a partition which does not have a specified -offset within the provided offsets map, it will fallback to the default -group offsets behaviour (i.e. `setStartFromGroupOffsets()`) for that -particular partition. +上面的例子中使用的配置是指定从 `myTopic` 主题的 0 、1 和 2 分区的指定偏移量开始消费。offset 值是 consumer 应该为每个分区读取的下一条消息。请注意:如果 consumer 需要读取在提供的 offset 映射中没有指定 offset 的分区,那么它将回退到该特定分区的默认组偏移行为(即 `setStartFromGroupOffsets()`)。 + -Note that these start position configuration methods do not affect the start position when the job is -automatically restored from a failure or manually restored using a savepoint. -On restore, the start position of each Kafka partition is determined by the -offsets stored in the savepoint or checkpoint -(please see the next section for information about checkpointing to enable -fault tolerance for the consumer). +请注意:当 Job 从故障中自动恢复或使用 savepoint 手动恢复时,这些起始位置配置方法不会影响消费的起始位置。在恢复时,每个 Kafka 分区的起始位置由存储在 savepoint 或 checkpoint 中的 offset 确定(有关 checkpointing 的信息,请参阅下一节,以便为 consumer 启用容错功能)。 -### Kafka Consumers and Fault Tolerance +### Kafka Consumer 和容错 -With Flink's checkpointing enabled, the Flink Kafka Consumer will consume records from a topic and periodically checkpoint all -its Kafka offsets, together with the state of other operations, in a consistent manner. In case of a job failure, Flink will restore -the streaming program to the state of the latest checkpoint and re-consume the records from Kafka, starting from the offsets that were -stored in the checkpoint. +伴随着启用 Flink 的 checkpointing 后,Flink Kafka Consumer 将使用 topic 中的记录,并以一致的方式定期检查其所有 Kafka offset 和其他算子的状态。如果 Job 失败,Flink 会将流式程序恢复到最新 checkpoint 的状态,并从存储在 checkpoint 中的 offset 开始重新消费 Kafka 中的消息。 -The interval of drawing checkpoints therefore defines how much the program may have to go back at most, in case of a failure. +因此,设置 checkpoint 的间隔定义了程序在发生故障时最多需要返回多少。 -To use fault tolerant Kafka Consumers, checkpointing of the topology needs to be enabled at the execution environment: +要使用容错的 Kafka Consumer,需要在执行环境中启用拓扑的 checkpointing。
    {% highlight java %} final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); -env.enableCheckpointing(5000); // checkpoint every 5000 msecs +env.enableCheckpointing(5000); // 每隔 5000 毫秒 执行一次 checkpoint {% endhighlight %}
    {% highlight scala %} val env = StreamExecutionEnvironment.getExecutionEnvironment() -env.enableCheckpointing(5000) // checkpoint every 5000 msecs +env.enableCheckpointing(5000) // 每隔 5000 毫秒 执行一次 checkpoint {% endhighlight %}
    -Also note that Flink can only restart the topology if enough processing slots are available to restart the topology. -So if the topology fails due to loss of a TaskManager, there must still be enough slots available afterwards. -Flink on YARN supports automatic restart of lost YARN containers. +另请注意,只有当可用的 slots 足够时,Flink 才能重新启动。因此,如果拓扑由于丢失了 TaskManager 而失败,那么之后必须要一直有足够可用的 solt。Flink on YARN 支持自动重启丢失的 YARN 容器。 -If checkpointing is not enabled, the Kafka consumer will periodically commit the offsets to Zookeeper. +如果未启用 checkpoint,那么 Kafka consumer 将定期向 Zookeeper 提交 offset。 -### Kafka Consumers Topic and Partition Discovery +### Kafka Consumer Topic 和分区发现 -#### Partition discovery +#### 分区发现 -The Flink Kafka Consumer supports discovering dynamically created Kafka partitions, and consumes them with -exactly-once guarantees. All partitions discovered after the initial retrieval of partition metadata (i.e., when the -job starts running) will be consumed from the earliest possible offset. +Flink Kafka Consumer 支持发现动态创建的 Kafka 分区,并使用精准一次的语义保证去消耗它们。在初始检索分区元数据之后(即,当 Job 开始运行时)发现的所有分区将从最早可能的 offset 中消费。 -By default, partition discovery is disabled. To enable it, set a non-negative value -for `flink.partition-discovery.interval-millis` in the provided properties config, -representing the discovery interval in milliseconds. +默认情况下,是禁用了分区发现的。若要启用它,请在提供的属性配置中为 `flink.partition-discovery.interval-millis` 设置大于 0 的值,表示发现分区的间隔是以毫秒为单位的。 -Limitation When the consumer is restored from a savepoint from Flink versions -prior to Flink 1.3.x, partition discovery cannot be enabled on the restore run. If enabled, the restore would fail -with an exception. In this case, in order to use partition discovery, please first take a savepoint in Flink 1.3.x and -then restore again from that. +局限性 当从 Flink 1.3.x 之前的 Flink 版本的 savepoint 恢复 consumer 时,分区发现无法在恢复运行时启用。如果启用了,那么还原将会失败并且出现异常。在这种情况下,为了使用分区发现,请首先在 Flink 1.3.x 中使用 savepoint,然后再从 savepoint 中恢复。 -#### Topic discovery +#### Topic 发现 -At a higher-level, the Flink Kafka Consumer is also capable of discovering topics, based on pattern matching on the -topic names using regular expressions. See the below for an example: +在更高的级别上,Flink Kafka Consumer 还能够使用正则表达式基于 Topic 名称的模式匹配来发现 Topic。请看下面的例子:
    @@ -448,61 +383,36 @@ val stream = env.addSource(myConsumer)
    -In the above example, all topics with names that match the specified regular expression -(starting with `test-topic-` and ending with a single digit) will be subscribed by the consumer -when the job starts running. - -To allow the consumer to discover dynamically created topics after the job started running, -set a non-negative value for `flink.partition-discovery.interval-millis`. This allows -the consumer to discover partitions of new topics with names that also match the specified -pattern. - -### Kafka Consumers Offset Committing Behaviour Configuration - -The Flink Kafka Consumer allows configuring the behaviour of how offsets -are committed back to Kafka brokers (or Zookeeper in 0.8). Note that the -Flink Kafka Consumer does not rely on the committed offsets for fault -tolerance guarantees. The committed offsets are only a means to expose -the consumer's progress for monitoring purposes. - -The way to configure offset commit behaviour is different, depending on -whether or not checkpointing is enabled for the job. - - - *Checkpointing disabled:* if checkpointing is disabled, the Flink Kafka - Consumer relies on the automatic periodic offset committing capability - of the internally used Kafka clients. Therefore, to disable or enable offset - committing, simply set the `enable.auto.commit` (or `auto.commit.enable` - for Kafka 0.8) / `auto.commit.interval.ms` keys to appropriate values - in the provided `Properties` configuration. - - - *Checkpointing enabled:* if checkpointing is enabled, the Flink Kafka - Consumer will commit the offsets stored in the checkpointed states when - the checkpoints are completed. This ensures that the committed offsets - in Kafka brokers is consistent with the offsets in the checkpointed states. - Users can choose to disable or enable offset committing by calling the - `setCommitOffsetsOnCheckpoints(boolean)` method on the consumer (by default, - the behaviour is `true`). - Note that in this scenario, the automatic periodic offset committing - settings in `Properties` is completely ignored. - -### Kafka Consumers and Timestamp Extraction/Watermark Emission - -In many scenarios, the timestamp of a record is embedded (explicitly or implicitly) in the record itself. -In addition, the user may want to emit watermarks either periodically, or in an irregular fashion, e.g. based on -special records in the Kafka stream that contain the current event-time watermark. For these cases, the Flink Kafka -Consumer allows the specification of an `AssignerWithPeriodicWatermarks` or an `AssignerWithPunctuatedWatermarks`. - -You can specify your custom timestamp extractor/watermark emitter as described -[here]({{ site.baseurl }}/dev/event_timestamps_watermarks.html), or use one from the -[predefined ones]({{ site.baseurl }}/dev/event_timestamp_extractors.html). After doing so, you -can pass it to your consumer in the following way: +在上面的例子中,当 Job 开始运行时,Consumer 将订阅名称与指定正则表达式匹配的所有主题(以 `test-topic` 开头并以单个数字结尾)。 + +要允许 consumer 在作业开始运行后发现动态创建的主题,那么请为 `flink.partition-discovery.interval-millis` 设置非负值。这允许 consumer 发现名称与指定模式匹配的新主题的分区。 + +### Kafka Consumer 提交 Offset 的行为配置 + +Flink Kafka Consumer 允许有配置如何将 offset 提交回 Kafka broker(或 0.8 版本的 Zookeeper)的行为。请注意:Flink Kafka Consumer 不依赖于提交的 offset 来实现容错保证。提交的 offset 只是一种方法,用于公开 consumer 的进度以便进行监控。 + +配置 offset 提交行为的方法是否相同,取决于是否为 job 启用了 checkpointing。 + + - *禁用 Checkpointing:* 如果禁用了 checkpointing,则 Flink Kafka Consumer 依赖于内部使用的 Kafka client 自动定期 offset 提交功能。 + 因此,要禁用或启用 offset 的提交,只需将 `enable.auto.commit`(或 Kafka 0.8 的 `auto.commit.enable`)或者 `auto.commit.interval.ms` 的Key 值设置为提供的 `Properties` 配置中的适当值。 + + - *启用 Checkpointing:* 如果启用了 checkpointing,那么当 checkpointing 完成时,Flink Kafka Consumer 将提交的 offset 存储在 checkpoint 状态中。 + 这确保 Kafka broker 中提交的 offset 与 checkpoint 状态中的 offset 一致。 + 用户可以通过调用 consumer 上的 `setCommitOffsetsOnCheckpoints(boolean)` 方法来禁用或启用 offset 的提交(默认情况下,这个值是 true )。 + 注意,在这个场景中,`Properties` 中的自动定期 offset 提交设置会被完全忽略。 + +### Kafka Consumer 和 时间戳抽取以及 watermark 发送 + +在许多场景中,记录的时间戳是(显式或隐式)嵌入到记录本身中。此外,用户可能希望定期或以不规则的方式 Watermark,例如基于 Kafka 流中包含当前事件时间的 watermark 的特殊记录。对于这些情况,Flink Kafka Consumer 允许指定 `AssignerWithPeriodicWatermarks` 或 `AssignerWithPunctuatedWatermarks`。 + +你可以按照[此处]({{ site.baseurl }}/zh/dev/event_timestamps_watermarks.html)的说明指定自定义时间戳抽取器或者 Watermark 发送器,或者使用 [内置的]({{ site.baseurl }}/zh/dev/event_timestamp_extractors.html)。你也可以通过以下方式将其传递给你的 consumer:
    {% highlight java %} Properties properties = new Properties(); properties.setProperty("bootstrap.servers", "localhost:9092"); -// only required for Kafka 0.8 +// 仅 Kafka 0.8 需要 properties.setProperty("zookeeper.connect", "localhost:2181"); properties.setProperty("group.id", "test"); @@ -511,15 +421,15 @@ FlinkKafkaConsumer08 myConsumer = myConsumer.assignTimestampsAndWatermarks(new CustomWatermarkEmitter()); DataStream stream = env - .addSource(myConsumer) - .print(); + .addSource(myConsumer) + .print(); {% endhighlight %}
    {% highlight scala %} val properties = new Properties() properties.setProperty("bootstrap.servers", "localhost:9092") -// only required for Kafka 0.8 +// 仅 Kafka 0.8 需要 properties.setProperty("zookeeper.connect", "localhost:2181") properties.setProperty("group.id", "test") @@ -532,28 +442,15 @@ stream = env
    -Internally, an instance of the assigner is executed per Kafka partition. -When such an assigner is specified, for each record read from Kafka, the -`extractTimestamp(T element, long previousElementTimestamp)` is called to assign a timestamp to the record and -the `Watermark getCurrentWatermark()` (for periodic) or the -`Watermark checkAndGetNextWatermark(T lastElement, long extractedTimestamp)` (for punctuated) is called to determine -if a new watermark should be emitted and with which timestamp. - -**Note**: If a watermark assigner depends on records read from Kafka to advance its watermarks -(which is commonly the case), all topics and partitions need to have a continuous stream of records. -Otherwise, the watermarks of the whole application cannot advance and all time-based operations, -such as time windows or functions with timers, cannot make progress. A single idle Kafka partition causes this behavior. -A Flink improvement is planned to prevent this from happening -(see [FLINK-5479: Per-partition watermarks in FlinkKafkaConsumer should consider idle partitions]( -https://issues.apache.org/jira/browse/FLINK-5479)). -In the meanwhile, a possible workaround is to send *heartbeat messages* to all consumed partitions that advance the watermarks of idle partitions. +在内部,每个 Kafka 分区执行一个 assigner 实例。当指定了这样的 assigner 时,对于从 Kafka 读取的每条消息,调用 `extractTimestamp(T element, long previousElementTimestamp)` 来为记录分配时间戳,并为 `Watermark getCurrentWatermark()`(定期形式)或 `Watermark checkAndGetNextWatermark(T lastElement, long extractedTimestamp)`(打点形式)以确定是否应该发出新的 watermark 以及使用哪个时间戳。 + +**请注意**:如果 watermark assigner 依赖于从 Kafka 读取的消息来上涨其 watermark (通常就是这种情况),那么所有主题和分区都需要有连续的消息流。否则,整个应用程序的 watermark 将无法上涨,所有基于时间的算子(例如时间窗口或带有计时器的函数)也无法运行。单个的 Kafka 分区也会导致这种反应。这是一个已在计划中的 Flink 改进,目的是为了防止这种情况发生(请见[FLINK-5479: Per-partition watermarks in FlinkKafkaConsumer should consider idle partitions](https://issues.apache.org/jira/browse/FLINK-5479))。同时,可能的解决方法是将*心跳消息*发送到所有 consumer 的分区里,从而上涨空闲分区的 watermark。 ## Kafka Producer -Flink’s Kafka Producer is called `FlinkKafkaProducer011` (or `010` for Kafka 0.10.0.x versions, etc. or just `FlinkKafkaProducer` for Kafka >= 1.0.0 versions). -It allows writing a stream of records to one or more Kafka topics. +Flink Kafka Producer 被称为 `FlinkKafkaProducer011`(或适用于 Kafka 0.10.0.x 版本的 `FlinkKafkaProducer010`,或适用于 Kafka >= 1.0.0 版本的 `FlinkKafkaProducer`)。它允许将消息流写入一个或多个 Kafka topic。 -Example: +示例:
    @@ -561,12 +458,12 @@ Example: DataStream stream = ...; FlinkKafkaProducer011 myProducer = new FlinkKafkaProducer011( - "localhost:9092", // broker list - "my-topic", // target topic - new SimpleStringSchema()); // serialization schema + "localhost:9092", // broker 列表 + "my-topic", // 目标 topic + new SimpleStringSchema()); // 序列化 schema -// versions 0.10+ allow attaching the records' event timestamp when writing them to Kafka; -// this method is not available for earlier Kafka versions +// 0.10+ 版本的 Kafka 允许在将记录写入 Kafka 时附加记录的事件时间戳; +// 此方法不适用于早期版本的 Kafka myProducer.setWriteTimestampToKafka(true); stream.addSink(myProducer); @@ -577,12 +474,12 @@ stream.addSink(myProducer); val stream: DataStream[String] = ... val myProducer = new FlinkKafkaProducer011[String]( - "localhost:9092", // broker list - "my-topic", // target topic - new SimpleStringSchema) // serialization schema + "localhost:9092", // broker 列表 + "my-topic", // 目标 topic + new SimpleStringSchema) // 序列化 schema -// versions 0.10+ allow attaching the records' event timestamp when writing them to Kafka; -// this method is not available for earlier Kafka versions +// 0.10+ 版本的 Kafka 允许在将记录写入 Kafka 时附加记录的事件时间戳; +// 此方法不适用于早期版本的 Kafka myProducer.setWriteTimestampToKafka(true) stream.addSink(myProducer) @@ -590,158 +487,88 @@ stream.addSink(myProducer)
    -The above examples demonstrate the basic usage of creating a Flink Kafka Producer -to write streams to a single Kafka target topic. For more advanced usages, there -are other constructor variants that allow providing the following: - - * *Providing custom properties*: - The producer allows providing a custom properties configuration for the internal `KafkaProducer`. - Please refer to the [Apache Kafka documentation](https://kafka.apache.org/documentation.html) for - details on how to configure Kafka Producers. - * *Custom partitioner*: To assign records to specific - partitions, you can provide an implementation of a `FlinkKafkaPartitioner` to the - constructor. This partitioner will be called for each record in the stream - to determine which exact partition of the target topic the record should be sent to. - Please see [Kafka Producer Partitioning Scheme](#kafka-producer-partitioning-scheme) for more details. - * *Advanced serialization schema*: Similar to the consumer, - the producer also allows using an advanced serialization schema called `KeyedSerializationSchema`, - which allows serializing the key and value separately. It also allows to override the target topic, - so that one producer instance can send data to multiple topics. - -### Kafka Producer Partitioning Scheme - -By default, if a custom partitioner is not specified for the Flink Kafka Producer, the producer will use -a `FlinkFixedPartitioner` that maps each Flink Kafka Producer parallel subtask to a single Kafka partition -(i.e., all records received by a sink subtask will end up in the same Kafka partition). - -A custom partitioner can be implemented by extending the `FlinkKafkaPartitioner` class. All -Kafka versions' constructors allow providing a custom partitioner when instantiating the producer. -Note that the partitioner implementation must be serializable, as they will be transferred across Flink nodes. -Also, keep in mind that any state in the partitioner will be lost on job failures since the partitioner -is not part of the producer's checkpointed state. - -It is also possible to completely avoid using and kind of partitioner, and simply let Kafka partition -the written records by their attached key (as determined for each record using the provided serialization schema). -To do this, provide a `null` custom partitioner when instantiating the producer. It is important -to provide `null` as the custom partitioner; as explained above, if a custom partitioner is not specified -the `FlinkFixedPartitioner` is used instead. - -### Kafka Producers and Fault Tolerance +上面的例子演示了创建 Flink Kafka Producer 来将流消息写入单个 Kafka 目标 topic 的基本用法。 +对于更高级的用法,这还有其他构造函数变体允许提供以下内容: + + * *提供自定义属性*:producer 允许为内部 `KafkaProducer` 提供自定义属性配置。有关如何配置 Kafka Producer 的详细信息,请参阅 [Apache Kafka 文档](https://kafka.apache.org/documentation.html)。 + * *自定义分区器*:要将消息分配给特定的分区,可以向构造函数提供一个 `FlinkKafkaPartitioner` 的实现。这个分区器将被流中的每条记录调用,以确定消息应该发送到目标 topic 的哪个具体分区里。有关详细信息,请参阅 [Kafka Producer 分区方案](#kafka-producer-分区方案)。 + * *高级的序列化 schema*:与 consumer 类似,producer 还允许使用名为 `KeyedSerializationSchema` 的高级序列化 schema,该 schema 允许单独序列化 key 和 value。它还允许覆盖目标 topic,以便 producer 实例可以将数据发送到多个 topic。 + +### Kafka Producer 分区方案 + +默认情况下,如果没有为 Flink Kafka Producer 指定自定义分区程序,则 producer 将使用 `FlinkFixedPartitioner` 为每个 Flink Kafka Producer 并行子任务映射到单个 Kafka 分区(即,接收子任务接收到的所有消息都将位于同一个 Kafka 分区中)。 + +可以通过扩展 `FlinkKafkaPartitioner` 类来实现自定义分区程序。所有 Kafka 版本的构造函数都允许在实例化 producer 时提供自定义分区程序。 +注意:分区器实现必须是可序列化的,因为它们将在 Flink 节点之间传输。此外,请记住分区器中的任何状态都将在作业失败时丢失,因为分区器不是 producer 的 checkpoint 状态的一部分。 + +也可以完全避免使用分区器,并简单地让 Kafka 通过其附加 key 写入的消息进行分区(使用提供的序列化 schema 为每条记录确定分区)。 +为此,在实例化 producer 时提供 `null` 自定义分区程序,提供 `null` 作为自定义分区器是很重要的; 如上所述,如果未指定自定义分区程序,则默认使用 `FlinkFixedPartitioner`。 + +### Kafka Producer 和容错 #### Kafka 0.8 -Before 0.9 Kafka did not provide any mechanisms to guarantee at-least-once or exactly-once semantics. +在 0.9 版本之前,Kafka 没有提供任何机制来保证至少一次或精准一次的语义。 #### Kafka 0.9 and 0.10 -With Flink's checkpointing enabled, the `FlinkKafkaProducer09` and `FlinkKafkaProducer010` -can provide at-least-once delivery guarantees. - -Besides enabling Flink's checkpointing, you should also configure the setter -methods `setLogFailuresOnly(boolean)` and `setFlushOnCheckpoint(boolean)` appropriately. - - * `setLogFailuresOnly(boolean)`: by default, this is set to `false`. - Enabling this will let the producer only log failures - instead of catching and rethrowing them. This essentially accounts the record - to have succeeded, even if it was never written to the target Kafka topic. This - must be disabled for at-least-once. - * `setFlushOnCheckpoint(boolean)`: by default, this is set to `true`. - With this enabled, Flink's checkpoints will wait for any - on-the-fly records at the time of the checkpoint to be acknowledged by Kafka before - succeeding the checkpoint. This ensures that all records before the checkpoint have - been written to Kafka. This must be enabled for at-least-once. - -In conclusion, the Kafka producer by default has at-least-once guarantees for versions -0.9 and 0.10, with `setLogFailureOnly` set to `false` and `setFlushOnCheckpoint` set -to `true`. - -**Note**: By default, the number of retries is set to "0". This means that when `setLogFailuresOnly` is set to `false`, -the producer fails immediately on errors, including leader changes. The value is set to "0" by default to avoid -duplicate messages in the target topic that are caused by retries. For most production environments with frequent broker changes, -we recommend setting the number of retries to a higher value. - -**Note**: There is currently no transactional producer for Kafka, so Flink can not guarantee exactly-once delivery -into a Kafka topic. - -#### Kafka 0.11 and newer - -With Flink's checkpointing enabled, the `FlinkKafkaProducer011` (`FlinkKafkaProducer` for Kafka >= 1.0.0 versions) can provide -exactly-once delivery guarantees. - -Besides enabling Flink's checkpointing, you can also choose three different modes of operating -chosen by passing appropriate `semantic` parameter to the `FlinkKafkaProducer011` (`FlinkKafkaProducer` for Kafka >= 1.0.0 versions): - - * `Semantic.NONE`: Flink will not guarantee anything. Produced records can be lost or they can - be duplicated. - * `Semantic.AT_LEAST_ONCE` (default setting): similar to `setFlushOnCheckpoint(true)` in - `FlinkKafkaProducer010`. This guarantees that no records will be lost (although they can be duplicated). - * `Semantic.EXACTLY_ONCE`: uses Kafka transactions to provide exactly-once semantic. Whenever you write - to Kafka using transactions, do not forget about setting desired `isolation.level` (`read_committed` - or `read_uncommitted` - the latter one is the default value) for any application consuming records - from Kafka. - -##### Caveats - -`Semantic.EXACTLY_ONCE` mode relies on the ability to commit transactions -that were started before taking a checkpoint, after recovering from the said checkpoint. If the time -between Flink application crash and completed restart is larger than Kafka's transaction timeout -there will be data loss (Kafka will automatically abort transactions that exceeded timeout time). -Having this in mind, please configure your transaction timeout appropriately to your expected down -times. - -Kafka brokers by default have `transaction.max.timeout.ms` set to 15 minutes. This property will -not allow to set transaction timeouts for the producers larger than it's value. -`FlinkKafkaProducer011` by default sets the `transaction.timeout.ms` property in producer config to -1 hour, thus `transaction.max.timeout.ms` should be increased before using the -`Semantic.EXACTLY_ONCE` mode. - -In `read_committed` mode of `KafkaConsumer`, any transactions that were not finished -(neither aborted nor completed) will block all reads from the given Kafka topic past any -un-finished transaction. In other words after following sequence of events: - -1. User started `transaction1` and written some records using it -2. User started `transaction2` and written some further records using it -3. User committed `transaction2` - -Even if records from `transaction2` are already committed, they will not be visible to -the consumers until `transaction1` is committed or aborted. This has two implications: - - * First of all, during normal working of Flink applications, user can expect a delay in visibility - of the records produced into Kafka topics, equal to average time between completed checkpoints. - * Secondly in case of Flink application failure, topics into which this application was writing, - will be blocked for the readers until the application restarts or the configured transaction - timeout time will pass. This remark only applies for the cases when there are multiple - agents/applications writing to the same Kafka topic. - -**Note**: `Semantic.EXACTLY_ONCE` mode uses a fixed size pool of KafkaProducers -per each `FlinkKafkaProducer011` instance. One of each of those producers is used per one -checkpoint. If the number of concurrent checkpoints exceeds the pool size, `FlinkKafkaProducer011` -will throw an exception and will fail the whole application. Please configure max pool size and max -number of concurrent checkpoints accordingly. - -**Note**: `Semantic.EXACTLY_ONCE` takes all possible measures to not leave any lingering transactions -that would block the consumers from reading from Kafka topic more then it is necessary. However in the -event of failure of Flink application before first checkpoint, after restarting such application there -is no information in the system about previous pool sizes. Thus it is unsafe to scale down Flink -application before first checkpoint completes, by factor larger than `FlinkKafkaProducer011.SAFE_SCALE_DOWN_FACTOR`. - -## Using Kafka timestamps and Flink event time in Kafka 0.10 - -Since Apache Kafka 0.10+, Kafka's messages can carry -[timestamps](https://cwiki.apache.org/confluence/display/KAFKA/KIP-32+-+Add+timestamps+to+Kafka+message), indicating -the time the event has occurred (see ["event time" in Apache Flink](../event_time.html)) or the time when the message -has been written to the Kafka broker. - -The `FlinkKafkaConsumer010` will emit records with the timestamp attached, if the time characteristic in Flink is -set to `TimeCharacteristic.EventTime` (`StreamExecutionEnvironment.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)`). - -The Kafka consumer does not emit watermarks. To emit watermarks, the same mechanisms as described above in -"Kafka Consumers and Timestamp Extraction/Watermark Emission" using the `assignTimestampsAndWatermarks` method are applicable. - -There is no need to define a timestamp extractor when using the timestamps from Kafka. The `previousElementTimestamp` argument of -the `extractTimestamp()` method contains the timestamp carried by the Kafka message. - -A timestamp extractor for a Kafka consumer would look like this: +启用 Flink 的 checkpointing 后,`FlinkKafkaProducer09` 和 `FlinkKafkaProducer010` 可以提供至少一次的语义。 + +除了启用 Flink 的 checkpointing 之外,还应该适当地配置 setter 方法,`setLogFailuresOnly(boolean)` 和 `setFlushOnCheckpoint(boolean)`。 + + * `setLogFailuresOnly(boolean)`:默认情况下,此值设置为 `false`。启用这个选项将使 producer 仅记录失败而不是捕获和重新抛出它们。这基本上是记录了成功的记录,即使它从未写入目标 Kafka topic。对 at-least-once 的语义,这个方法必须禁用。 + * `setFlushOnCheckpoint(boolean)`:默认情况下,此值设置为 `true`。启用此功能后,Flink 的 checkpoint 将在 checkpoint 成功之前等待 Kafka 确认 checkpoint 时的任意即时记录。这样可确保 checkpoint 之前的所有记录都已写入 Kafka。对 at-least-once 的语义,这个方法必须启用。 + +总之,默认情况下,Kafka producer 中,`setLogFailureOnly` 设置为 `false` 及 `setFlushOnCheckpoint` 设置为 `true` 会为 0.9 和 0.10 版本提供 at-least-once 语义。 + +**注意**:默认情况下,重试次数设置为 0。这意味着当 `setLogFailuresOnly` 设置为 `false` 时,producer 会立即失败,包括 leader 更改。该值默认设置为 0,以避免重试导致目标 topic 中出现重复的消息。对于大多数频繁更改 broker 的生产环境,我们建议将重试次数设置为更高的值。 + +**注意**:目前还没有 Kafka 的事务 producer,所以 Flink 不能保证写入 Kafka topic 的精准一次语义。 + +#### Kafka 0.11 和更新的版本 + +启用 Flink 的 checkpointing 后,`FlinkKafkaProducer011`(适用于 Kafka >= 1.0.0 版本的 `FlinkKafkaProducer`)可以提供精准一次的语义保证。 + +除了启用 Flink 的 checkpointing,还可以通过将适当的 `semantic` 参数传递给 `FlinkKafkaProducer011`(适用于 Kafka >= 1.0.0 版本的 `FlinkKafkaProducer`)来选择三种不同的操作模式: + + * `Semantic.NONE`:Flink 不会有任何语义的保证,产生的记录可能会丢失或重复。 + * `Semantic.AT_LEAST_ONCE`(默认设置):类似 `FlinkKafkaProducer010` 中的 `setFlushOnCheckpoint(true)`,这可以保证不会丢失任何记录(虽然记录可能会重复)。 + * `Semantic.EXACTLY_ONCE`:使用 Kafka 事务提供精准一次的语义。无论何时,在使用事务写入 Kafka 时,都要记得为所有消费 Kafka 消息的应用程序设置所需的 `isolation.level`( `read_committed` 或 `read_uncommitted` - 后者是默认值)。 + +##### 注意事项 + +`Semantic.EXACTLY_ONCE` 模式依赖于事务提交的能力。事务提交发生于触发 checkpoint 之前,以及从 checkpoint 恢复之后。如果从 Flink 应用程序崩溃到完全重启的时间超过了 Kafka 的事务超时时间,那么将会有数据丢失(Kafka 会自动丢弃超出超时时间的事务)。考虑到这一点,请根据预期的宕机时间来合理地配置事务超时时间。 + +默认情况下,Kafka broker 将 `transaction.max.timeout.ms` 设置为 15 分钟。此属性不允许为大于其值的 producer 设置事务超时时间。 +默认情况下,`FlinkKafkaProducer011` 将 producer config 中的 `transaction.timeout.ms` 属性设置为 1 小时,因此在使用 `Semantic.EXACTLY_ONCE` 模式之前应该增加 `transaction.max.timeout.ms` 的值。 + +在 `KafkaConsumer` 的 `read_committed` 模式中,任何未结束(既未中止也未完成)的事务将阻塞来自给定 Kafka topic 的未结束事务之后的所有读取数据。 +换句话说,在遵循如下一系列事件之后: + +1. 用户启动了 `transaction1` 并使用它写了一些记录 +2. 用户启动了 `transaction2` 并使用它编写了一些其他记录 +3. 用户提交了 `transaction2` + +即使 `transaction2` 中的记录已提交,在提交或中止 `transaction1` 之前,消费者也不会看到这些记录。这有 2 层含义: + + * 首先,在 Flink 应用程序的正常工作期间,用户可以预料 Kafka 主题中生成的记录的可见性会延迟,相当于已完成 checkpoint 之间的平均时间。 + * 其次,在 Flink 应用程序失败的情况下,此应用程序正在写入的供消费者读取的主题将被阻塞,直到应用程序重新启动或配置的事务超时时间过去后,才恢复正常。此标注仅适用于有多个 agent 或者应用程序写入同一 Kafka 主题的情况。 + +**注意**:`Semantic.EXACTLY_ONCE` 模式为每个 `FlinkKafkaProducer011` 实例使用固定大小的 KafkaProducer 池。每个 checkpoint 使用其中一个 producer。如果并发 checkpoint 的数量超过池的大小,`FlinkKafkaProducer011` 将抛出异常,并导致整个应用程序失败。请合理地配置最大池大小和最大并发 checkpoint 数量。 + +**注意**:`Semantic.EXACTLY_ONCE` 会尽一切可能不留下任何逗留的事务,否则会阻塞其他消费者从这个 Kafka topic 中读取数据。但是,如果 Flink 应用程序在第一次 checkpoint 之前就失败了,那么在重新启动此类应用程序后,系统中不会有先前池大小(pool size)相关的信息。因此,在第一次 checkpoint 完成前对 Flink 应用程序进行缩容,且并发数缩容倍数大于安全系数 `FlinkKafkaProducer011.SAFE_SCALE_DOWN_FACTOR` 的值的话,是不安全的。 + +## 在 Kafka 0.10.x 中使用 Kafka 时间戳和 Flink 事件时间 + +自 Apache Kafka 0.10+ 以来,Kafka 的消息可以携带[时间戳](https://cwiki.apache.org/confluence/display/KAFKA/KIP-32+-+Add+timestamps+to+Kafka+message),指示事件发生的时间(请参阅 [Apache Flink 中的"事件时间"](../event_time.html))或消息写入 Kafka broker 的时间。 + +如果 Flink 中的时间特性设置为 `TimeCharacteristic.EventTime`( `StreamExecutionEnvironment.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)`),则 `FlinkKafkaConsumer010` 将发出附加时间戳的记录。 + +Kafka consumer 不会发出 watermark。为了发出 watermark,可采用上文 “Kafka Consumer 和时间戳抽取以及 watermark 发送” 章节中描述的 `assignTimestampsAndWatermarks` 方法。 + +使用 Kafka 的时间戳时,无需定义时间戳提取器。`extractTimestamp()` 方法的 `previousElementTimestamp` 参数包含 `Kafka` 消息携带的时间戳。 + +Kafka consumer 的时间戳提取器应该是这样的: {% highlight java %} public long extractTimestamp(Long element, long previousElementTimestamp) { return previousElementTimestamp; @@ -750,7 +577,7 @@ public long extractTimestamp(Long element, long previousElementTimestamp) { -The `FlinkKafkaProducer010` only emits the record timestamp, if `setWriteTimestampToKafka(true)` is set. +只有设置了 `setWriteTimestampToKafka(true)`,则 `FlinkKafkaProducer010` 才会发出记录时间戳。 {% highlight java %} FlinkKafkaProducer010.FlinkKafkaProducer010Configuration config = FlinkKafkaProducer010.writeToKafkaWithTimestamps(streamWithTimestamps, topic, new SimpleStringSchema(), standardProps); @@ -759,85 +586,50 @@ config.setWriteTimestampToKafka(true); -## Kafka Connector metrics - -Flink's Kafka connectors provide some metrics through Flink's [metrics system]({{ site.baseurl }}/monitoring/metrics.html) to analyze -the behavior of the connector. -The producers export Kafka's internal metrics through Flink's metric system for all supported versions. The consumers export -all metrics starting from Kafka version 0.9. The Kafka documentation lists all exported metrics -in its [documentation](http://kafka.apache.org/documentation/#selector_monitoring). +## Kafka 连接器指标 -In addition to these metrics, all consumers expose the `current-offsets` and `committed-offsets` for each topic partition. -The `current-offsets` refers to the current offset in the partition. This refers to the offset of the last element that -we retrieved and emitted successfully. The `committed-offsets` is the last committed offset. +Flink 的 Kafka 连接器通过 Flink 的 [metric 系统]({{ site.baseurl }}/zh/monitoring/metrics.html) 提供一些指标来分析 Kafka Connector 的状况。Producer 通过 Flink 的 metrics 系统为所有支持的版本导出 Kafka 的内部指标。consumer 从 Kafka 0.9 版本开始导出所有指标。Kafka 文档在其[文档](http://kafka.apache.org/documentation/#selector_monitoring)中列出了所有导出的指标。 -The Kafka Consumers in Flink commit the offsets back to Zookeeper (Kafka 0.8) or the Kafka brokers (Kafka 0.9+). If checkpointing -is disabled, offsets are committed periodically. -With checkpointing, the commit happens once all operators in the streaming topology have confirmed that they've created a checkpoint of their state. -This provides users with at-least-once semantics for the offsets committed to Zookeeper or the broker. For offsets checkpointed to Flink, the system -provides exactly once guarantees. +除了这些指标之外,所有 consumer 都暴露了每个主题分区的 `current-offsets` 和 `committed-offsets`。`current-offsets` 是指分区中的当前偏移量。指的是我们成功检索和发出的最后一个元素的偏移量。`committed-offsets` 是最后提交的偏移量。这为用户提供了 at-least-once 语义,用于提交给 Zookeeper 或 broker 的偏移量。对于 Flink 的偏移检查点,系统提供精准一次语义。 -The offsets committed to ZK or the broker can also be used to track the read progress of the Kafka consumer. The difference between -the committed offset and the most recent offset in each partition is called the *consumer lag*. If the Flink topology is consuming -the data slower from the topic than new data is added, the lag will increase and the consumer will fall behind. -For large production deployments we recommend monitoring that metric to avoid increasing latency. +提交给 ZK 或 broker 的偏移量也可以用来跟踪 Kafka consumer 的读取进度。每个分区中提交的偏移量和最近偏移量之间的差异称为 *consumer lag*。如果 Flink 拓扑消耗来自 topic 的数据的速度比添加新数据的速度慢,那么延迟将会增加,consumer 将会滞后。对于大型生产部署,我们建议监视该指标,以避免增加延迟。 -## Enabling Kerberos Authentication (for versions 0.9+ and above only) +## 启用 Kerberos 身份验证(仅适用于 0.9 以上版本) -Flink provides first-class support through the Kafka connector to authenticate to a Kafka installation -configured for Kerberos. Simply configure Flink in `flink-conf.yaml` to enable Kerberos authentication for Kafka like so: +Flink 通过 Kafka 连接器提供了一流的支持,可以对 Kerberos 配置的 Kafka 安装进行身份验证。只需在 `flink-conf.yaml` 中配置 Flink。像这样为 Kafka 启用 Kerberos 身份验证: -1. Configure Kerberos credentials by setting the following - - - `security.kerberos.login.use-ticket-cache`: By default, this is `true` and Flink will attempt to use Kerberos credentials in ticket caches managed by `kinit`. - Note that when using the Kafka connector in Flink jobs deployed on YARN, Kerberos authorization using ticket caches will not work. - This is also the case when deploying using Mesos, as authorization using ticket cache is not supported for Mesos deployments. - - `security.kerberos.login.keytab` and `security.kerberos.login.principal`: To use Kerberos keytabs instead, set values for both of these properties. +1. 通过设置以下内容配置 Kerberos 票据 + - `security.kerberos.login.use-ticket-cache`:默认情况下,这个值是 `true`,Flink 将尝试在 `kinit` 管理的票据缓存中使用 Kerberos 票据。注意!在 YARN 上部署的 Flink jobs 中使用 Kafka 连接器时,使用票据缓存的 Kerberos 授权将不起作用。使用 Mesos 进行部署时也是如此,因为 Mesos 部署不支持使用票据缓存进行授权。 + - `security.kerberos.login.keytab` 和 `security.kerberos.login.principal`:要使用 Kerberos keytabs,需为这两个属性设置值。 -2. Append `KafkaClient` to `security.kerberos.login.contexts`: This tells Flink to provide the configured Kerberos credentials to the Kafka login context to be used for Kafka authentication. +2. 将 `KafkaClient` 追加到 `security.kerberos.login.contexts`:这告诉 Flink 将配置的 Kerberos 票据提供给 Kafka 登录上下文以用于 Kafka 身份验证。 -Once Kerberos-based Flink security is enabled, you can authenticate to Kafka with either the Flink Kafka Consumer or Producer -by simply including the following two settings in the provided properties configuration that is passed to the internal Kafka client: +一旦启用了基于 Kerberos 的 Flink 安全性后,只需在提供的属性配置中包含以下两个设置(通过传递给内部 Kafka 客户端),即可使用 Flink Kafka Consumer 或 Producer 向 Kafk a进行身份验证: -- Set `security.protocol` to `SASL_PLAINTEXT` (default `NONE`): The protocol used to communicate to Kafka brokers. -When using standalone Flink deployment, you can also use `SASL_SSL`; please see how to configure the Kafka client for SSL [here](https://kafka.apache.org/documentation/#security_configclients). -- Set `sasl.kerberos.service.name` to `kafka` (default `kafka`): The value for this should match the `sasl.kerberos.service.name` used for Kafka broker configurations. -A mismatch in service name between client and server configuration will cause the authentication to fail. +- 将 `security.protocol` 设置为 `SASL_PLAINTEXT`(默认为 `NONE`):用于与 Kafka broker 进行通信的协议。使用独立 Flink 部署时,也可以使用 `SASL_SSL`;请在[此处](https://kafka.apache.org/documentation/#security_configclients)查看如何为 SSL 配置 Kafka 客户端。 +- 将 `sasl.kerberos.service.name` 设置为 `kafka`(默认为 `kafka`):此值应与用于 Kafka broker 配置的 `sasl.kerberos.service.name` 相匹配。客户端和服务器配置之间的服务名称不匹配将导致身份验证失败。 -For more information on Flink configuration for Kerberos security, please see [here]({{ site.baseurl}}/ops/config.html). -You can also find [here]({{ site.baseurl}}/ops/security-kerberos.html) further details on how Flink internally setups Kerberos-based security. +有关 Kerberos 安全性 Flink 配置的更多信息,请参见[这里]({{ site.baseurl }}/zh/ops/config.html)。你也可以在[这里]({{ site.baseurl }}/zh/ops/security-kerberos.html)进一步了解 Flink 如何在内部设置基于 kerberos 的安全性。 -## Troubleshooting +## 问题排查
    -If you have a problem with Kafka when using Flink, keep in mind that Flink only wraps -KafkaConsumer or -KafkaProducer -and your problem might be independent of Flink and sometimes can be solved by upgrading Kafka brokers, -reconfiguring Kafka brokers or reconfiguring KafkaConsumer or KafkaProducer in Flink. -Some examples of common problems are listed below. +如果你在使用 Flink 时对 Kafka 有问题,请记住,Flink 只封装 KafkaConsumerKafkaProducer,你的问题可能独立于 Flink,有时可以通过升级 Kafka broker 程序、重新配置 Kafka broker 程序或在 Flink 中重新配置 KafkaConsumerKafkaProducer 来解决。下面列出了一些常见问题的示例。
    -### Data loss +### 数据丢失 -Depending on your Kafka configuration, even after Kafka acknowledges -writes you can still experience data loss. In particular keep in mind about the following properties -in Kafka config: +根据你的 Kafka 配置,即使在 Kafka 确认写入后,你仍然可能会遇到数据丢失。特别要记住在 Kafka 的配置中设置以下属性: - `acks` - `log.flush.interval.messages` - `log.flush.interval.ms` - `log.flush.*` -Default values for the above options can easily lead to data loss. -Please refer to the Kafka documentation for more explanation. +上述选项的默认值是很容易导致数据丢失的。请参考 Kafka 文档以获得更多的解释。 ### UnknownTopicOrPartitionException -One possible cause of this error is when a new leader election is taking place, -for example after or during restarting a Kafka broker. -This is a retriable exception, so Flink job should be able to restart and resume normal operation. -It also can be circumvented by changing `retries` property in the producer settings. -However this might cause reordering of messages, -which in turn if undesired can be circumvented by setting `max.in.flight.requests.per.connection` to 1. +导致此错误的一个可能原因是正在进行新的 leader 选举,例如在重新启动 Kafka broker 之后或期间。这是一个可重试的异常,因此 Flink job 应该能够重启并恢复正常运行。也可以通过更改 producer 设置中的 `retries` 属性来规避。但是,这可能会导致重新排序消息,反过来可以通过将 `max.in.flight.requests.per.connection` 设置为 1 来避免不需要的消息。 {% top %} From 7009ddb4884502079e31ace724942e33909fd531 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Fri, 1 Nov 2019 14:17:43 +0800 Subject: [PATCH 407/746] [hotfix][runtime] Fix code style violations in JobVertexD and IntermediateDataSetID --- .../runtime/jobgraph/IntermediateDataSetID.java | 15 +++++++++------ .../flink/runtime/jobgraph/JobVertexID.java | 5 +++-- 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSetID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSetID.java index af3dfd03c671..530462f1256d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSetID.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSetID.java @@ -18,10 +18,13 @@ package org.apache.flink.runtime.jobgraph; -import java.util.UUID; - import org.apache.flink.util.AbstractID; +import java.util.UUID; + +/** + * Id identifying {@link IntermediateDataSet}. + */ public class IntermediateDataSetID extends AbstractID { private static final long serialVersionUID = 1L; @@ -32,19 +35,19 @@ public class IntermediateDataSetID extends AbstractID { public IntermediateDataSetID() { super(); } - + /** * Creates a new intermediate data set ID with the bytes of the given ID. - * + * * @param from The ID to create this ID from. */ public IntermediateDataSetID(AbstractID from) { super(from); } - + /** * Creates a new intermediate data set ID with the bytes of the given UUID. - * + * * @param from The UUID to create this ID from. */ public IntermediateDataSetID(UUID from) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java index 6c570e965752..8c5167cc6a97 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java @@ -25,12 +25,13 @@ * A class for statistically unique job vertex IDs. */ public class JobVertexID extends AbstractID { - + private static final long serialVersionUID = 1L; - + public JobVertexID() { super(); } + public JobVertexID(byte[] bytes) { super(bytes); } From dbbbcdc444816711c10f03c3b62da138cff3dfd8 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Fri, 1 Nov 2019 14:18:15 +0800 Subject: [PATCH 408/746] [FLINK-14312][runtime] Introduce LogicalTopology --- .../jobgraph/IntermediateDataSetID.java | 3 +- .../flink/runtime/jobgraph/JobVertexID.java | 3 +- .../jobgraph/topology/LogicalResult.java | 30 +++++++++++++++++++ .../jobgraph/topology/LogicalTopology.java | 30 +++++++++++++++++++ .../jobgraph/topology/LogicalVertex.java | 30 +++++++++++++++++++ 5 files changed, 94 insertions(+), 2 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalResult.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalTopology.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalVertex.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSetID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSetID.java index 530462f1256d..c22ca1ee4807 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSetID.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSetID.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.jobgraph; +import org.apache.flink.runtime.topology.ResultID; import org.apache.flink.util.AbstractID; import java.util.UUID; @@ -25,7 +26,7 @@ /** * Id identifying {@link IntermediateDataSet}. */ -public class IntermediateDataSetID extends AbstractID { +public class IntermediateDataSetID extends AbstractID implements ResultID { private static final long serialVersionUID = 1L; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java index 8c5167cc6a97..209f79980537 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java @@ -18,13 +18,14 @@ package org.apache.flink.runtime.jobgraph; +import org.apache.flink.runtime.topology.VertexID; import org.apache.flink.util.AbstractID; import org.apache.flink.util.StringUtils; /** * A class for statistically unique job vertex IDs. */ -public class JobVertexID extends AbstractID { +public class JobVertexID extends AbstractID implements VertexID { private static final long serialVersionUID = 1L; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalResult.java new file mode 100644 index 000000000000..a43c6aff4cf8 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalResult.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobgraph.topology; + +import org.apache.flink.runtime.jobgraph.IntermediateDataSet; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.topology.Result; + +/** + * Represents a data set produced by a {@link LogicalVertex}, i.e. {@link IntermediateDataSet}. + */ +public interface LogicalResult, R extends LogicalResult> + extends Result { +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalTopology.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalTopology.java new file mode 100644 index 000000000000..3c195cf46acc --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalTopology.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobgraph.topology; + +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.topology.Topology; + +/** + * Represents a logical topology, i.e. {@link JobGraph}. + */ +public interface LogicalTopology, R extends LogicalResult> + extends Topology { +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalVertex.java new file mode 100644 index 000000000000..5e20bfbd4924 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalVertex.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobgraph.topology; + +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.topology.Vertex; + +/** + * Represents a vertex in {@link LogicalTopology}, i.e. {@link JobVertex}. + */ +public interface LogicalVertex, R extends LogicalResult> + extends Vertex { +} From ee2fb1011812e434a58144ebe55f505c0e677855 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Fri, 1 Nov 2019 14:18:51 +0800 Subject: [PATCH 409/746] [FLINK-14312][runtime] Introduce LogicalPipelinedRegion --- .../topology/LogicalPipelinedRegion.java | 54 +++++++++++++++++++ 1 file changed, 54 insertions(+) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalPipelinedRegion.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalPipelinedRegion.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalPipelinedRegion.java new file mode 100644 index 000000000000..9b7cc8d6ff31 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/LogicalPipelinedRegion.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.jobgraph.topology; + +import org.apache.flink.runtime.jobgraph.JobVertexID; + +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Set of {@link LogicalVertex} that are connected through pipelined {@link LogicalResult}. + */ +public class LogicalPipelinedRegion { + + private final Set vertexIDs; + + public LogicalPipelinedRegion(final Set> logicalVertices) { + checkNotNull(logicalVertices); + + this.vertexIDs = logicalVertices.stream() + .map(LogicalVertex::getId) + .collect(Collectors.toSet()); + } + + public Set getVertexIDs() { + return vertexIDs; + } + + @Override + public String toString() { + return "LogicalPipelinedRegion{" + + "vertexIDs=" + vertexIDs + + '}'; + } +} From e6270375e239f35f857498631c2c38fc4d7eb515 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Fri, 1 Nov 2019 14:19:19 +0800 Subject: [PATCH 410/746] [FLINK-14312][runtime] Introduce DefaultLogicalTopology It is an adapter of JobGraph to LogicalTopology. With it we can get logical pipelined regions of the underlying JobGraph. This closes #10006. --- .../topology/DefaultLogicalResult.java | 74 ++++++++++ .../topology/DefaultLogicalTopology.java | 114 +++++++++++++++ .../topology/DefaultLogicalVertex.java | 71 ++++++++++ .../topology/DefaultLogicalResultTest.java | 129 +++++++++++++++++ .../topology/DefaultLogicalTopologyTest.java | 134 ++++++++++++++++++ .../topology/DefaultLogicalVertexTest.java | 133 +++++++++++++++++ 6 files changed, 655 insertions(+) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalResult.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalTopology.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalVertex.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalResultTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalTopologyTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalVertexTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalResult.java new file mode 100644 index 000000000000..771bdf7b9824 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalResult.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.runtime.jobgraph.topology; + +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; +import org.apache.flink.runtime.jobgraph.IntermediateDataSet; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.jobgraph.JobEdge; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobgraph.JobVertexID; + +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Default implementation of {@link LogicalResult}. + * It is an adapter of {@link IntermediateDataSet}. + */ +public class DefaultLogicalResult implements LogicalResult { + + private final IntermediateDataSet intermediateDataSet; + + private final Function vertexRetriever; + + DefaultLogicalResult( + final IntermediateDataSet intermediateDataSet, + final Function vertexRetriever) { + + this.intermediateDataSet = checkNotNull(intermediateDataSet); + this.vertexRetriever = checkNotNull(vertexRetriever); + } + + @Override + public IntermediateDataSetID getId() { + return intermediateDataSet.getId(); + } + + @Override + public ResultPartitionType getResultType() { + return intermediateDataSet.getResultType(); + } + + @Override + public DefaultLogicalVertex getProducer() { + return vertexRetriever.apply(intermediateDataSet.getProducer().getID()); + } + + @Override + public Iterable getConsumers() { + return intermediateDataSet.getConsumers().stream() + .map(JobEdge::getTarget) + .map(JobVertex::getID) + .map(vertexRetriever) + .collect(Collectors.toList()); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalTopology.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalTopology.java new file mode 100644 index 000000000000..00b8d62e7091 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalTopology.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobgraph.topology; + +import org.apache.flink.runtime.executiongraph.failover.flip1.PipelinedRegionComputeUtil; +import org.apache.flink.runtime.jobgraph.IntermediateDataSet; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.util.IterableUtils; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.function.Function; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Default implementation of {@link LogicalTopology}. + * It is an adapter of {@link JobGraph}. + */ +public class DefaultLogicalTopology implements LogicalTopology { + + private final boolean containsCoLocationConstraints; + + private final List verticesSorted; + + private final Map idToVertexMap; + + private final Map idToResultMap; + + public DefaultLogicalTopology(final JobGraph jobGraph) { + checkNotNull(jobGraph); + + this.containsCoLocationConstraints = IterableUtils.toStream(jobGraph.getVertices()) + .map(JobVertex::getCoLocationGroup) + .anyMatch(Objects::nonNull); + + this.verticesSorted = new ArrayList<>(jobGraph.getNumberOfVertices()); + this.idToVertexMap = new HashMap<>(); + this.idToResultMap = new HashMap<>(); + + buildVerticesAndResults(jobGraph); + } + + private void buildVerticesAndResults(final JobGraph jobGraph) { + final Function vertexRetriever = this::getVertex; + final Function resultRetriever = this::getResult; + + for (JobVertex jobVertex : jobGraph.getVerticesSortedTopologicallyFromSources()) { + final DefaultLogicalVertex logicalVertex = new DefaultLogicalVertex(jobVertex, resultRetriever); + this.verticesSorted.add(logicalVertex); + this.idToVertexMap.put(logicalVertex.getId(), logicalVertex); + + for (IntermediateDataSet intermediateDataSet : jobVertex.getProducedDataSets()) { + final DefaultLogicalResult logicalResult = new DefaultLogicalResult(intermediateDataSet, vertexRetriever); + idToResultMap.put(logicalResult.getId(), logicalResult); + } + } + } + + @Override + public Iterable getVertices() { + return verticesSorted; + } + + @Override + public boolean containsCoLocationConstraints() { + return containsCoLocationConstraints; + } + + private DefaultLogicalVertex getVertex(final JobVertexID vertexId) { + return Optional.ofNullable(idToVertexMap.get(vertexId)) + .orElseThrow(() -> new IllegalArgumentException("can not find vertex: " + vertexId)); + } + + private DefaultLogicalResult getResult(final IntermediateDataSetID resultId) { + return Optional.ofNullable(idToResultMap.get(resultId)) + .orElseThrow(() -> new IllegalArgumentException("can not find result: " + resultId)); + } + + public Set getLogicalPipelinedRegions() { + final Set> regionsRaw = PipelinedRegionComputeUtil.computePipelinedRegions(this); + + final Set regions = new HashSet<>(); + for (Set regionVertices : regionsRaw) { + regions.add(new LogicalPipelinedRegion(regionVertices)); + } + return regions; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalVertex.java new file mode 100644 index 000000000000..4761d76cf81f --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalVertex.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.runtime.jobgraph.topology; + +import org.apache.flink.runtime.jobgraph.IntermediateDataSet; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.jobgraph.JobEdge; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobgraph.JobVertexID; + +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Default implementation of {@link LogicalVertex}. + * It is an adapter of {@link JobVertex}. + */ +public class DefaultLogicalVertex implements LogicalVertex { + + private final JobVertex jobVertex; + + private final Function resultRetriever; + + DefaultLogicalVertex( + final JobVertex jobVertex, + final Function resultRetriever) { + + this.jobVertex = checkNotNull(jobVertex); + this.resultRetriever = checkNotNull(resultRetriever); + } + + @Override + public JobVertexID getId() { + return jobVertex.getID(); + } + + @Override + public Iterable getConsumedResults() { + return jobVertex.getInputs().stream() + .map(JobEdge::getSource) + .map(IntermediateDataSet::getId) + .map(resultRetriever) + .collect(Collectors.toList()); + } + + @Override + public Iterable getProducedResults() { + return jobVertex.getProducedDataSets().stream() + .map(IntermediateDataSet::getId) + .map(resultRetriever) + .collect(Collectors.toList()); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalResultTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalResultTest.java new file mode 100644 index 000000000000..8493b33f8df8 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalResultTest.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.jobgraph.topology; + +import org.apache.flink.runtime.jobgraph.IntermediateDataSet; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.util.IterableUtils; +import org.apache.flink.util.TestLogger; + +import org.junit.Before; +import org.junit.Test; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.createNoOpVertex; +import static org.apache.flink.runtime.io.network.partition.ResultPartitionType.PIPELINED; +import static org.apache.flink.runtime.jobgraph.DistributionPattern.ALL_TO_ALL; +import static org.apache.flink.runtime.jobgraph.topology.DefaultLogicalVertexTest.assertVertexInfoEquals; +import static org.apache.flink.runtime.jobgraph.topology.DefaultLogicalVertexTest.assertVerticesEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +/** + * Unit tests for {@link DefaultLogicalResult}. + */ +public class DefaultLogicalResultTest extends TestLogger { + + private IntermediateDataSet result; + + private DefaultLogicalResult logicalResult; + + private Map vertexMap; + + private JobVertex producerVertex; + + private Set consumerVertices; + + @Before + public void setUp() throws Exception { + buildVerticesAndResults(); + + logicalResult = new DefaultLogicalResult( + result, + vid -> new DefaultLogicalVertex(vertexMap.get(vid), rid -> null)); + } + + @Test + public void testConstructor() { + assertResultInfoEquals(result, logicalResult); + } + + @Test + public void testGetProducer() { + assertVertexInfoEquals(producerVertex, logicalResult.getProducer()); + } + + @Test + public void testGetConsumers() { + assertVerticesEquals(consumerVertices, logicalResult.getConsumers()); + } + + private void buildVerticesAndResults() { + vertexMap = new HashMap<>(); + consumerVertices = new HashSet<>(); + + final int parallelism = 3; + producerVertex = createNoOpVertex(parallelism); + vertexMap.put(producerVertex.getID(), producerVertex); + + result = producerVertex.createAndAddResultDataSet(PIPELINED); + + for (int i = 0; i < 5; i++) { + final JobVertex consumerVertex = createNoOpVertex(parallelism); + consumerVertex.connectDataSetAsInput(result, ALL_TO_ALL); + consumerVertices.add(consumerVertex); + vertexMap.put(consumerVertex.getID(), consumerVertex); + } + } + + static void assertResultsEquals( + final Iterable results, + final Iterable logicalResults) { + + final Map logicalResultMap = IterableUtils + .toStream(logicalResults) + .collect(Collectors.toMap(DefaultLogicalResult::getId, Function.identity())); + + for (IntermediateDataSet result : results) { + final DefaultLogicalResult logicalResult = logicalResultMap.remove(result.getId()); + + assertNotNull(logicalResult); + assertResultInfoEquals(result, logicalResult); + } + + // this ensures the two collections exactly matches + assertEquals(0, logicalResultMap.size()); + } + + static void assertResultInfoEquals( + final IntermediateDataSet result, + final DefaultLogicalResult logicalResult) { + + assertEquals(result.getId(), logicalResult.getId()); + assertEquals(result.getResultType(), logicalResult.getResultType()); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalTopologyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalTopologyTest.java new file mode 100644 index 000000000000..0a17db1937cb --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalTopologyTest.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.jobgraph.topology; + +import org.apache.flink.runtime.jobgraph.IntermediateDataSet; +import org.apache.flink.runtime.jobgraph.JobEdge; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; +import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; +import org.apache.flink.util.TestLogger; + +import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables; + +import org.junit.Before; +import org.junit.Test; + +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +import static junit.framework.TestCase.assertTrue; +import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.createNoOpVertex; +import static org.apache.flink.runtime.io.network.partition.ResultPartitionType.BLOCKING; +import static org.apache.flink.runtime.io.network.partition.ResultPartitionType.PIPELINED; +import static org.apache.flink.runtime.jobgraph.DistributionPattern.ALL_TO_ALL; +import static org.apache.flink.runtime.jobgraph.topology.DefaultLogicalResultTest.assertResultsEquals; +import static org.apache.flink.runtime.jobgraph.topology.DefaultLogicalVertexTest.assertVertexInfoEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; + +/** + * Unit tests for {@link DefaultLogicalTopology}. + */ +public class DefaultLogicalTopologyTest extends TestLogger { + + private JobGraph jobGraph; + + private DefaultLogicalTopology logicalTopology; + + @Before + public void setUp() throws Exception { + jobGraph = createJobGraph(false); + logicalTopology = new DefaultLogicalTopology(jobGraph); + } + + @Test + public void testGetVertices() { + // vertices from getVertices() should be topologically sorted + final Iterable jobVertices = jobGraph.getVerticesSortedTopologicallyFromSources(); + final Iterable logicalVertices = logicalTopology.getVertices(); + + assertEquals(Iterables.size(jobVertices), Iterables.size(logicalVertices)); + + final Iterator jobVertexIterator = jobVertices.iterator(); + final Iterator logicalVertexIterator = logicalVertices.iterator(); + while (jobVertexIterator.hasNext()) { + assertVertexAndConnectedResultsEquals(jobVertexIterator.next(), logicalVertexIterator.next()); + } + } + + @Test + public void testWithCoLocationConstraints() { + final DefaultLogicalTopology topology = new DefaultLogicalTopology(createJobGraph(true)); + assertTrue(topology.containsCoLocationConstraints()); + } + + @Test + public void testWithoutCoLocationConstraints() { + assertFalse(logicalTopology.containsCoLocationConstraints()); + } + + @Test + public void testGetLogicalPipelinedRegions() { + final Set regions = logicalTopology.getLogicalPipelinedRegions(); + assertEquals(2, regions.size()); + } + + private JobGraph createJobGraph(final boolean containsCoLocationConstraint) { + final JobVertex[] jobVertices = new JobVertex[3]; + final int parallelism = 3; + jobVertices[0] = createNoOpVertex("v1", parallelism); + jobVertices[1] = createNoOpVertex("v2", parallelism); + jobVertices[2] = createNoOpVertex("v3", parallelism); + jobVertices[1].connectNewDataSetAsInput(jobVertices[0], ALL_TO_ALL, PIPELINED); + jobVertices[2].connectNewDataSetAsInput(jobVertices[1], ALL_TO_ALL, BLOCKING); + + if (containsCoLocationConstraint) { + final SlotSharingGroup slotSharingGroup = new SlotSharingGroup(); + jobVertices[0].setSlotSharingGroup(slotSharingGroup); + jobVertices[1].setSlotSharingGroup(slotSharingGroup); + + final CoLocationGroup coLocationGroup = new CoLocationGroup(); + coLocationGroup.addVertex(jobVertices[0]); + coLocationGroup.addVertex(jobVertices[1]); + jobVertices[0].updateCoLocationGroup(coLocationGroup); + jobVertices[1].updateCoLocationGroup(coLocationGroup); + } + + return new JobGraph(jobVertices); + } + + private static void assertVertexAndConnectedResultsEquals( + final JobVertex jobVertex, + final DefaultLogicalVertex logicalVertex) { + + assertVertexInfoEquals(jobVertex, logicalVertex); + + final List consumedResults = jobVertex.getInputs().stream() + .map(JobEdge::getSource) + .collect(Collectors.toList()); + assertResultsEquals(consumedResults, logicalVertex.getConsumedResults()); + + final List producedResults = jobVertex.getProducedDataSets(); + assertResultsEquals(producedResults, logicalVertex.getProducedResults()); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalVertexTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalVertexTest.java new file mode 100644 index 000000000000..c569b30465a8 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/topology/DefaultLogicalVertexTest.java @@ -0,0 +1,133 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobgraph.topology; + +import org.apache.flink.runtime.jobgraph.IntermediateDataSet; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.util.IterableUtils; +import org.apache.flink.util.TestLogger; + +import org.junit.Before; +import org.junit.Test; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.createNoOpVertex; +import static org.apache.flink.runtime.io.network.partition.ResultPartitionType.BLOCKING; +import static org.apache.flink.runtime.io.network.partition.ResultPartitionType.PIPELINED; +import static org.apache.flink.runtime.jobgraph.DistributionPattern.ALL_TO_ALL; +import static org.apache.flink.runtime.jobgraph.topology.DefaultLogicalResultTest.assertResultsEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +/** + * Unit tests for {@link DefaultLogicalVertex}. + */ +public class DefaultLogicalVertexTest extends TestLogger { + + private JobVertex jobVertex; + + private DefaultLogicalVertex logicalVertex; + + private Map resultMap; + + private Set consumedResults; + + private Set producedResults; + + @Before + public void setUp() throws Exception { + buildVerticesAndResults(); + + logicalVertex = new DefaultLogicalVertex( + jobVertex, + rid -> new DefaultLogicalResult(resultMap.get(rid), vid -> null)); + } + + @Test + public void testConstructor() { + assertVertexInfoEquals(jobVertex, logicalVertex); + } + + @Test + public void testGetConsumedResults() { + assertResultsEquals(consumedResults, logicalVertex.getConsumedResults()); + } + + @Test + public void testGetProducedResults() { + assertResultsEquals(producedResults, logicalVertex.getProducedResults()); + } + + private void buildVerticesAndResults() { + resultMap = new HashMap<>(); + producedResults = new HashSet<>(); + consumedResults = new HashSet<>(); + + final int parallelism = 3; + jobVertex = createNoOpVertex(parallelism); + + for (int i = 0; i < 5; i++) { + final IntermediateDataSet producedDataSet = jobVertex.createAndAddResultDataSet(BLOCKING); + producedResults.add(producedDataSet); + resultMap.put(producedDataSet.getId(), producedDataSet); + } + + final JobVertex upStreamJobVertex = createNoOpVertex(parallelism); + for (int i = 0; i < 5; i++) { + final IntermediateDataSet consumedDataSet = upStreamJobVertex.createAndAddResultDataSet(PIPELINED); + jobVertex.connectDataSetAsInput(consumedDataSet, ALL_TO_ALL); + consumedResults.add(consumedDataSet); + resultMap.put(consumedDataSet.getId(), consumedDataSet); + } + } + + static void assertVerticesEquals( + final Iterable jobVertices, + final Iterable logicalVertices) { + + final Map logicalVertextMap = IterableUtils + .toStream(logicalVertices) + .collect(Collectors.toMap(DefaultLogicalVertex::getId, Function.identity())); + + for (JobVertex jobVertex : jobVertices) { + final DefaultLogicalVertex logicalVertex = logicalVertextMap.remove(jobVertex.getID()); + + assertNotNull(logicalVertex); + assertVertexInfoEquals(jobVertex, logicalVertex); + } + + // this ensures the two collections exactly matches + assertEquals(0, logicalVertextMap.size()); + } + + static void assertVertexInfoEquals( + final JobVertex jobVertex, + final DefaultLogicalVertex logicalVertex) { + + assertEquals(jobVertex.getID(), logicalVertex.getId()); + } +} From 8cc74a6f960ddb1023a62de6aefd725de17b43ba Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Thu, 31 Oct 2019 10:58:32 +0800 Subject: [PATCH 411/746] [FLINK-14488][python] Update python table API with temporary tables & views methods --- flink-python/pyflink/table/descriptors.py | 29 ++++++ .../pyflink/table/table_environment.py | 94 ++++++++++++++++++- .../tests/test_environment_completeness.py | 7 +- .../table/tests/test_table_environment_api.py | 50 ++++++++++ 4 files changed, 173 insertions(+), 7 deletions(-) diff --git a/flink-python/pyflink/table/descriptors.py b/flink-python/pyflink/table/descriptors.py index aa16b7c8c2c2..5e534661212d 100644 --- a/flink-python/pyflink/table/descriptors.py +++ b/flink-python/pyflink/table/descriptors.py @@ -15,6 +15,7 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ +import warnings from abc import ABCMeta from py4j.java_gateway import get_method @@ -1272,7 +1273,10 @@ def register_table_sink(self, name): :param name: Table name to be registered in the table environment. :return: This object. + + .. note:: Deprecated in 1.10. Use :func:`create_temporary_table` instead. """ + warnings.warn("Deprecated in 1.10. Use create_temporary_table instead.", DeprecationWarning) self._j_connect_table_descriptor = self._j_connect_table_descriptor.registerTableSink(name) return self @@ -1283,7 +1287,10 @@ def register_table_source(self, name): :param name: Table name to be registered in the table environment. :return: This object. + + .. note:: Deprecated in 1.10. Use :func:`create_temporary_table` instead. """ + warnings.warn("Deprecated in 1.10. Use create_temporary_table instead.", DeprecationWarning) self._j_connect_table_descriptor = \ self._j_connect_table_descriptor.registerTableSource(name) return self @@ -1295,11 +1302,33 @@ def register_table_source_and_sink(self, name): :param name: Table name to be registered in the table environment. :return: This object. + + .. note:: Deprecated in 1.10. Use :func:`create_temporary_table` instead. """ + warnings.warn("Deprecated in 1.10. Use create_temporary_table instead.", DeprecationWarning) self._j_connect_table_descriptor = \ self._j_connect_table_descriptor.registerTableSourceAndSink(name) return self + def create_temporary_table(self, path): + """ + Registers the table described by underlying properties in a given path. + + There is no distinction between source and sink at the descriptor level anymore as this + method does not perform actual class lookup. It only stores the underlying properties. The + actual source/sink lookup is performed when the table is used. + + Temporary objects can shadow permanent ones. If a permanent object in a given path exists, + it will be inaccessible in the current session. To make the permanent object available + again you can drop the corresponding temporary object. + + .. note:: The schema must be explicitly defined. + + :param path: path where to register the temporary table + """ + self._j_connect_table_descriptor.createTemporaryTable(path) + return self + class StreamTableDescriptor(ConnectTableDescriptor): """ diff --git a/flink-python/pyflink/table/table_environment.py b/flink-python/pyflink/table/table_environment.py index 6c88d9e72c20..8a0414a35981 100644 --- a/flink-python/pyflink/table/table_environment.py +++ b/flink-python/pyflink/table/table_environment.py @@ -17,6 +17,7 @@ ################################################################################ import os import tempfile +import warnings from abc import ABCMeta, abstractmethod from py4j.java_gateway import get_java_class @@ -42,7 +43,33 @@ class TableEnvironment(object): """ - The abstract base class for batch and stream TableEnvironments. + A table environment is the base class, entry point, and central context for creating Table + and SQL API programs. + + It is unified for bounded and unbounded data processing. + + A table environment is responsible for: + + - Connecting to external systems. + - Registering and retrieving :class:`Table` and other meta objects from a catalog. + - Executing SQL statements. + - Offering further configuration options. + + The path in methods such as :func:`create_temporary_view` + should be a proper SQL identifier. The syntax is following + [[catalog-name.]database-name.]object-name, where the catalog name and database are optional. + For path resolution see :func:`use_catalog` and :func:`use_database`. All keywords or other + special characters need to be escaped. + + Example: `cat.1`.`db`.`Table` resolves to an object named 'Table' (table is a reserved + keyword, thus must be escaped) in a catalog named 'cat.1' and database named 'db'. + + .. note:: + + This environment is meant for pure table programs. If you would like to convert from or to + other Flink APIs, it might be necessary to use one of the available language-specific table + environments in the corresponding bridging modules. + """ __metaclass__ = ABCMeta @@ -105,7 +132,10 @@ def register_table(self, name, table): :param name: The name under which the table will be registered. :param table: The table to register. + + .. note:: Deprecated in 1.10. Use :func:`create_temporary_view` instead. """ + warnings.warn("Deprecated in 1.10. Use create_temporary_view instead.", DeprecationWarning) self._j_tenv.registerTable(name, table._j_table) def register_table_source(self, name, table_source): @@ -256,6 +286,54 @@ def list_functions(self): j_function_name_array = self._j_tenv.listFunctions() return [item for item in j_function_name_array] + def list_temporary_tables(self): + """ + Gets the names of all temporary tables and views available in the current namespace + (the current database of the current catalog). + + :return: A list of the names of all registered temporary tables and views in the current + database of the current catalog. + + .. seealso:: :func:`list_tables` + """ + j_table_name_array = self._j_tenv.listTemporaryTables() + return [item for item in j_table_name_array] + + def list_temporary_views(self): + """ + Gets the names of all temporary views available in the current namespace (the current + database of the current catalog). + + :return: A list of the names of all registered temporary views in the current database + of the current catalog. + + .. seealso:: :func:`list_tables` + """ + j_view_name_array = self._j_tenv.listTemporaryViews() + return [item for item in j_view_name_array] + + def drop_temporary_table(self, table_path): + """ + Drops a temporary table registered in the given path. + + If a permanent table with a given path exists, it will be used + from now on for any queries that reference this path. + + :return: true if a table existed in the given path and was removed + """ + return self._j_tenv.dropTemporaryTable(table_path) + + def drop_temporary_view(self, view_path): + """ + Drops a temporary view registered in the given path. + + If a permanent table or view with a given path exists, it will be used + from now on for any queries that reference this path. + + :return: true if a view existed in the given path and was removed + """ + return self._j_tenv.dropTemporaryView(view_path) + def explain(self, table=None, extended=False): """ Returns the AST of the specified Table API and SQL queries and the execution plan to compute @@ -583,6 +661,20 @@ def register_function(self, name, function): self._j_tenv.registerFunction(name, function._judf(self._is_blink_planner, self.get_config()._j_table_config)) + def create_temporary_view(self, view_path, table): + """ + Registers a :class:`Table` API object as a temporary view similar to SQL temporary views. + + Temporary objects can shadow permanent ones. If a permanent object in a given path exists, + it will be inaccessible in the current session. To make the permanent object available + again you can drop the corresponding temporary object. + + :param view_path: The path under which the view will be registered. See also the + :class:`TableEnvironment` class description for the format of the path. + :param table: The view to register. + """ + self._j_tenv.createTemporaryView(view_path, table._j_table) + 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 4ff3d29273c1..d968953706cc 100644 --- a/flink-python/pyflink/table/tests/test_environment_completeness.py +++ b/flink-python/pyflink/table/tests/test_environment_completeness.py @@ -49,12 +49,7 @@ def excluded_methods(cls): 'create', 'loadModule', 'unloadModule', - 'listTemporaryTables', - 'createTemporaryView', - 'dropTemporaryTable', - 'listTemporaryViews', - 'from', - 'dropTemporaryView'} + 'from'} if __name__ == '__main__': diff --git a/flink-python/pyflink/table/tests/test_table_environment_api.py b/flink-python/pyflink/table/tests/test_table_environment_api.py index 19f45f2bb83a..ae50cb90fade 100644 --- a/flink-python/pyflink/table/tests/test_table_environment_api.py +++ b/flink-python/pyflink/table/tests/test_table_environment_api.py @@ -20,6 +20,7 @@ from pyflink.dataset import ExecutionEnvironment from pyflink.datastream import StreamExecutionEnvironment from pyflink.table import DataTypes, CsvTableSink, StreamTableEnvironment, EnvironmentSettings +from pyflink.table.descriptors import FileSystem, OldCsv, Schema from pyflink.table.table_config import TableConfig from pyflink.table.table_environment import BatchTableEnvironment from pyflink.table.types import RowType @@ -90,6 +91,55 @@ def test_list_tables(self): expected = ['Orders', 'Results', 'Sinks'] self.assert_equals(actual, expected) + def test_temporary_tables(self): + t_env = self.t_env + t_env.connect(FileSystem().path(os.path.join(self.tempdir + '/temp_1.csv'))) \ + .with_format(OldCsv() + .field_delimiter(',') + .field("a", DataTypes.INT()) + .field("b", DataTypes.STRING())) \ + .with_schema(Schema() + .field("a", DataTypes.INT()) + .field("b", DataTypes.STRING())) \ + .create_temporary_table("temporary_table_1") + + t_env.connect(FileSystem().path(os.path.join(self.tempdir + '/temp_2.csv'))) \ + .with_format(OldCsv() + .field_delimiter(',') + .field("a", DataTypes.INT()) + .field("b", DataTypes.STRING())) \ + .with_schema(Schema() + .field("a", DataTypes.INT()) + .field("b", DataTypes.STRING())) \ + .create_temporary_table("temporary_table_2") + + actual = t_env.list_temporary_tables() + expected = ['temporary_table_1', 'temporary_table_2'] + self.assert_equals(actual, expected) + + t_env.drop_temporary_table("temporary_table_1") + actual = t_env.list_temporary_tables() + expected = ['temporary_table_2'] + self.assert_equals(actual, expected) + + def test_temporary_views(self): + t_env = self.t_env + t_env.create_temporary_view( + "temporary_view_1", + t_env.from_elements([(1, 'Hi', 'Hello')], ['a', 'b', 'c'])) + t_env.create_temporary_view( + "temporary_view_2", + t_env.from_elements([(1, 'Hi')], ['a', 'b'])) + + actual = t_env.list_temporary_views() + expected = ['temporary_view_1', 'temporary_view_2'] + self.assert_equals(actual, expected) + + t_env.drop_temporary_view("temporary_view_1") + actual = t_env.list_temporary_views() + expected = ['temporary_view_2'] + self.assert_equals(actual, expected) + def test_explain(self): schema = RowType()\ .add('a', DataTypes.INT())\ From b6bb09aa1b71c222ff4e9e280c479e9e36121475 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Thu, 31 Oct 2019 11:36:03 +0800 Subject: [PATCH 412/746] [FLINK-14488][python] Update python table API adding from_path method --- .../pyflink/table/table_environment.py | 40 ++++++++++++++++++- .../tests/test_environment_completeness.py | 14 ++++++- .../table/tests/test_table_environment_api.py | 11 +++++ 3 files changed, 62 insertions(+), 3 deletions(-) diff --git a/flink-python/pyflink/table/table_environment.py b/flink-python/pyflink/table/table_environment.py index 8a0414a35981..99e41481631d 100644 --- a/flink-python/pyflink/table/table_environment.py +++ b/flink-python/pyflink/table/table_environment.py @@ -20,7 +20,7 @@ import warnings from abc import ABCMeta, abstractmethod -from py4j.java_gateway import get_java_class +from py4j.java_gateway import get_java_class, get_method from pyflink.serializers import BatchedSerializer, PickleSerializer from pyflink.table.catalog import Catalog @@ -201,12 +201,50 @@ def scan(self, *table_path): :param table_path: The path of the table to scan. :throws: Exception if no table is found using the given table path. :return: The resulting :class:`Table` + + .. note:: Deprecated in 1.10. Use :func:`from_path` instead. """ + warnings.warn("Deprecated in 1.10. Use from_path instead.", DeprecationWarning) gateway = get_gateway() j_table_paths = utils.to_jarray(gateway.jvm.String, table_path) j_table = self._j_tenv.scan(j_table_paths) return Table(j_table) + def from_path(self, path): + """ + Reads a registered table and returns the resulting :class:`Table`. + + A table to scan must be registered in the :class:`TableEnvironment`. + + See the documentation of :func:`use_database` or :func:`use_catalog` for the rules on the + path resolution. + + Examples: + + Reading a table from default catalog and database. + :: + + >>> tab = table_env.from_path("tableName") + + Reading a table from a registered catalog. + :: + + >>> tab = table_env.from_path("catalogName.dbName.tableName") + + Reading a table from a registered catalog with escaping. (`Table` is a reserved keyword). + Dots in e.g. a database name also must be escaped. + :: + + >>> tab = table_env.from_path("catalogName.`db.Name`.`Table`") + + :param path: The path of a table API object to scan. + :return: Either a table or virtual table (=view). + + .. seealso:: :func:`user_catalog` + .. seealso:: :func:`user_database` + """ + return Table(get_method(self._j_tenv, "from")(path)) + def insert_into(self, table, table_path, *table_path_continued): """ Writes the :class:`Table` to a :class:`TableSink` that was registered under diff --git a/flink-python/pyflink/table/tests/test_environment_completeness.py b/flink-python/pyflink/table/tests/test_environment_completeness.py index d968953706cc..50ab6e2cb413 100644 --- a/flink-python/pyflink/table/tests/test_environment_completeness.py +++ b/flink-python/pyflink/table/tests/test_environment_completeness.py @@ -48,8 +48,18 @@ def excluded_methods(cls): 'getCompletionHints', 'create', 'loadModule', - 'unloadModule', - 'from'} + 'unloadModule'} + + @classmethod + def java_method_name(cls, python_method_name): + """ + Due to 'from' is python keyword, so we use 'from_path' + in Python API corresponding 'from' in Java API. + + :param python_method_name: + :return: + """ + return {'from_path': 'from'}.get(python_method_name, python_method_name) if __name__ == '__main__': diff --git a/flink-python/pyflink/table/tests/test_table_environment_api.py b/flink-python/pyflink/table/tests/test_table_environment_api.py index ae50cb90fade..204ea07aedb6 100644 --- a/flink-python/pyflink/table/tests/test_table_environment_api.py +++ b/flink-python/pyflink/table/tests/test_table_environment_api.py @@ -140,6 +140,17 @@ def test_temporary_views(self): expected = ['temporary_view_2'] self.assert_equals(actual, expected) + def test_from_path(self): + t_env = self.t_env + t_env.create_temporary_view( + "temporary_view_1", + t_env.from_elements([(1, 'Hi', 'Hello')], ['a', 'b', 'c'])) + result = t_env.from_path("temporary_view_1") + self.assertEqual( + 'CatalogTable: (identifier: [`default_catalog`.`default_database`.`temporary_view_1`]' + ', fields: [a, b, c])', + result._j_table.getQueryOperation().asSummaryString()) + def test_explain(self): schema = RowType()\ .add('a', DataTypes.INT())\ From d83b58c4467c3451c5d952985ad2767be49f8536 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Thu, 31 Oct 2019 14:06:02 +0800 Subject: [PATCH 413/746] [FLINK-14488][python] Update python table API with insert_into --- .../pyflink/table/table_environment.py | 24 ++++++++----------- .../table/tests/test_table_environment_api.py | 15 ++++++++++++ 2 files changed, 25 insertions(+), 14 deletions(-) diff --git a/flink-python/pyflink/table/table_environment.py b/flink-python/pyflink/table/table_environment.py index 99e41481631d..488788e6cb4d 100644 --- a/flink-python/pyflink/table/table_environment.py +++ b/flink-python/pyflink/table/table_environment.py @@ -245,28 +245,24 @@ def from_path(self, path): """ return Table(get_method(self._j_tenv, "from")(path)) - def insert_into(self, table, table_path, *table_path_continued): + def insert_into(self, target_path, table): """ - Writes the :class:`Table` to a :class:`TableSink` that was registered under - the specified name. For the path resolution algorithm see - :func:`~TableEnvironment.use_database`. + Instructs to write the content of a :class:`Table` API object into a table. + + See the documentation of :func:`use_database` or :func:`use_catalog` for the rules on the + path resolution. Example: :: >>> tab = table_env.scan("tableName") - >>> table_env.insert_into(tab, "sink") + >>> table_env.insert_into("sink", tab) - :param table: :class:`Table` to write to the sink. - :param table_path: The first part of the path of the registered :class:`TableSink` to which - the :class:`Table` is written. This is to ensure at least the name of the - :class:`Table` is provided. - :param table_path_continued: The remaining part of the path of the registered - :class:`TableSink` to which the :class:`Table` is written. + :param target_path: The path of the registered :class:`TableSink` to which the + :class:`Table` is written. + :param table: table The Table to write to the sink. """ - gateway = get_gateway() - j_table_path = utils.to_jarray(gateway.jvm.String, table_path_continued) - self._j_tenv.insertInto(table._j_table, table_path, j_table_path) + self._j_tenv.insertInto(target_path, table._j_table) def list_catalogs(self): """ diff --git a/flink-python/pyflink/table/tests/test_table_environment_api.py b/flink-python/pyflink/table/tests/test_table_environment_api.py index 204ea07aedb6..a6960fa6603c 100644 --- a/flink-python/pyflink/table/tests/test_table_environment_api.py +++ b/flink-python/pyflink/table/tests/test_table_environment_api.py @@ -151,6 +151,21 @@ def test_from_path(self): ', fields: [a, b, c])', result._j_table.getQueryOperation().asSummaryString()) + def test_insert_into(self): + t_env = self.t_env + field_names = ["a", "b", "c"] + field_types = [DataTypes.BIGINT(), DataTypes.STRING(), DataTypes.STRING()] + t_env.register_table_sink( + "Sinks", + source_sink_utils.TestAppendSink(field_names, field_types)) + + t_env.insert_into("Sinks", t_env.from_elements([(1, "Hi", "Hello")], ["a", "b", "c"])) + self.t_env.execute("test") + + actual = source_sink_utils.results() + expected = ['1,Hi,Hello'] + self.assert_equals(actual, expected) + def test_explain(self): schema = RowType()\ .add('a', DataTypes.INT())\ From 6c81981516ed8a9486bdd125d75abbaae0a6bfb7 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Thu, 31 Oct 2019 11:00:05 +0800 Subject: [PATCH 414/746] [hotfix][python] Deprecate TableEnvironment.register_table_source and TableEnvironment.register_table_sink --- flink-python/pyflink/table/table_environment.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/flink-python/pyflink/table/table_environment.py b/flink-python/pyflink/table/table_environment.py index 488788e6cb4d..17e0956f61be 100644 --- a/flink-python/pyflink/table/table_environment.py +++ b/flink-python/pyflink/table/table_environment.py @@ -154,7 +154,10 @@ def register_table_source(self, name, table_source): :param name: The name under which the :class:`TableSource` is registered. :param table_source: The :class:`TableSource` to register. + + .. note:: Deprecated in 1.10. Use :func:`connect` instead. """ + warnings.warn("Deprecated in 1.10. Use connect instead.", DeprecationWarning) self._j_tenv.registerTableSource(name, table_source._j_table_source) def register_table_sink(self, name, table_sink): @@ -174,7 +177,10 @@ def register_table_sink(self, name, table_sink): :param name: The name under which the :class:`TableSink` is registered. :param table_sink: The :class:`TableSink` to register. + + .. note:: Deprecated in 1.10. Use :func:`connect` instead. """ + warnings.warn("Deprecated in 1.10. Use connect instead.", DeprecationWarning) self._j_tenv.registerTableSink(name, table_sink._j_table_sink) def scan(self, *table_path): From b9f51decb1fbb83857eb8692857960476b5f4717 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Thu, 31 Oct 2019 14:21:59 +0800 Subject: [PATCH 415/746] [hotfix][python] Clean up test_environment_completeness by removing the legacy excluded methods which are already supported --- .../pyflink/table/tests/test_environment_completeness.py | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/flink-python/pyflink/table/tests/test_environment_completeness.py b/flink-python/pyflink/table/tests/test_environment_completeness.py index 50ab6e2cb413..6261e9216b25 100644 --- a/flink-python/pyflink/table/tests/test_environment_completeness.py +++ b/flink-python/pyflink/table/tests/test_environment_completeness.py @@ -37,14 +37,9 @@ def java_class(cls): @classmethod def excluded_methods(cls): - # registerFunction and listUserDefinedFunctions should be supported when UDFs supported. - # registerCatalog, getCatalog and listTables should be supported when catalog supported in - # python. getCompletionHints has been deprecated. It will be removed in the next release. + # 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', - 'listTables', 'getCompletionHints', 'create', 'loadModule', From 22930e379d8d4557f5a960691b9b0d676f5c87d4 Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Fri, 1 Nov 2019 21:04:07 +0800 Subject: [PATCH 416/746] [FLINK-14022][table-planner][table-planner-blink] Add python udf validation check in MatchRecognize --- .../physical/stream/StreamExecMatch.scala | 6 ++ .../MatchRecognizeValidationTest.scala | 24 ++++++ .../nodes/datastream/DataStreamMatch.scala | 7 ++ .../utils/JavaUserDefinedScalarFunctions.java | 73 +++++++++++++++++++ .../MatchRecognizeValidationTest.scala | 25 ++++++- .../table/plan/PythonCalcSplitRuleTest.scala | 30 +------- 6 files changed, 135 insertions(+), 30 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecMatch.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecMatch.scala index 56b0e30ee3af..d21e1258cd2f 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecMatch.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecMatch.scala @@ -39,6 +39,7 @@ import org.apache.flink.table.planner.delegation.StreamPlanner import org.apache.flink.table.planner.plan.logical.MatchRecognize import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, StreamExecNode} import org.apache.flink.table.planner.plan.rules.physical.stream.StreamExecRetractionRules +import org.apache.flink.table.planner.plan.utils.PythonUtil.containsPythonCall import org.apache.flink.table.planner.plan.utils.RelExplainUtil._ import org.apache.flink.table.planner.plan.utils.{KeySelectorUtil, RexDefaultVisitor, SortUtil} import org.apache.flink.table.runtime.operators.`match`.{BaseRowEventComparator, RowtimeProcessFunction} @@ -75,6 +76,11 @@ class StreamExecMatch( with StreamPhysicalRel with StreamExecNode[BaseRow] { + if (logicalMatch.measures.values().exists(containsPythonCall) || + logicalMatch.patternDefinitions.values().exists(containsPythonCall)) { + throw new TableException("Python Function can not be used in MATCH_RECOGNIZE for now.") + } + override def needsUpdatesAsRetraction(input: RelNode): Boolean = true override def consumesRetractions = true diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/stream/sql/validation/MatchRecognizeValidationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/stream/sql/validation/MatchRecognizeValidationTest.scala index 35a83b505b9d..45d14063ace2 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/stream/sql/validation/MatchRecognizeValidationTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/stream/sql/validation/MatchRecognizeValidationTest.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.api.scala._ import org.apache.flink.table.api.{TableException, ValidationException} import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.WeightedAvg import org.apache.flink.table.planner.runtime.utils.UserDefinedFunctionTestUtils.ToMillis +import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.PythonScalarFunction import org.apache.flink.table.planner.utils.TableTestBase import org.apache.flink.types.Row @@ -220,6 +221,29 @@ class MatchRecognizeValidationTest extends TableTestBase { // * features. * // *************************************************************************************** + /** Python Function can not be used in MATCH_RECOGNIZE for now **/ + @Test + def testMatchPythonFunction() = { + thrown.expectMessage("Python Function can not be used in MATCH_RECOGNIZE for now.") + thrown.expect(classOf[TableException]) + + streamUtil.addFunction("pyFunc", new PythonScalarFunction("pyFunc")) + val sql = + """SELECT T.aa as ta + |FROM MyTable + |MATCH_RECOGNIZE ( + | ORDER BY proctime + | MEASURES + | A.a as aa, + | pyFunc(1,2) as bb + | PATTERN (A B) + | DEFINE + | A AS a = 1, + | B AS b = 'b' + |) AS T""".stripMargin + streamUtil.verifyExplain(sql) + } + @Test def testAllRowsPerMatch(): Unit = { thrown.expectMessage("All rows per match mode is not supported yet.") diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamMatch.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamMatch.scala index f775303306a5..67c8d6b2ce19 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamMatch.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamMatch.scala @@ -42,6 +42,7 @@ import org.apache.flink.streaming.api.datastream.DataStream import org.apache.flink.streaming.api.windowing.time.Time import scala.collection.JavaConverters._ +import scala.collection.JavaConversions._ import org.apache.flink.table.api._ import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.codegen.MatchCodeGenerator @@ -49,6 +50,7 @@ import org.apache.flink.table.plan.logical.MatchRecognize import org.apache.flink.table.plan.nodes.CommonMatchRecognize import org.apache.flink.table.plan.rules.datastream.DataStreamRetractionRules import org.apache.flink.table.plan.schema.RowSchema +import org.apache.flink.table.plan.util.PythonUtil.containsPythonCall import org.apache.flink.table.plan.util.RexDefaultVisitor import org.apache.flink.table.planner.StreamPlanner import org.apache.flink.table.runtime.`match`._ @@ -75,6 +77,11 @@ class DataStreamMatch( with CommonMatchRecognize with DataStreamRel { + if (logicalMatch.measures.values().exists(containsPythonCall) || + logicalMatch.patternDefinitions.values().exists(containsPythonCall)) { + throw new TableException("Python Function can not be used in MATCH_RECOGNIZE for now.") + } + override def needsUpdatesAsRetraction = true override def consumesRetractions = true diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/utils/JavaUserDefinedScalarFunctions.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/utils/JavaUserDefinedScalarFunctions.java index a77ad9ad172b..abc60956645f 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/utils/JavaUserDefinedScalarFunctions.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/runtime/utils/JavaUserDefinedScalarFunctions.java @@ -18,7 +18,11 @@ package org.apache.flink.table.runtime.utils; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.table.functions.ScalarFunction; +import org.apache.flink.table.functions.python.PythonEnv; +import org.apache.flink.table.functions.python.PythonFunction; import java.util.Arrays; @@ -80,4 +84,73 @@ public String eval(Integer[] a, String[] b) { } } + /** + * Test for Python Scalar Function. + */ + public static class PythonScalarFunction extends ScalarFunction implements PythonFunction { + private final String name; + + public PythonScalarFunction(String name) { + this.name = name; + } + + public int eval(int i, int j) { + return i + j; + } + + @Override + public TypeInformation getResultType(Class[] signature) { + return BasicTypeInfo.INT_TYPE_INFO; + } + + @Override + public String toString() { + return name; + } + + @Override + public byte[] getSerializedPythonFunction() { + return new byte[0]; + } + + @Override + public PythonEnv getPythonEnv() { + return null; + } + } + + /** + * Test for Python Scalar Function. + */ + public static class BooleanPythonScalarFunction extends ScalarFunction implements PythonFunction { + private final String name; + + public BooleanPythonScalarFunction(String name) { + this.name = name; + } + + public boolean eval(int i, int j) { + return i + j > 1; + } + + @Override + public TypeInformation getResultType(Class[] signature) { + return BasicTypeInfo.BOOLEAN_TYPE_INFO; + } + + @Override + public String toString() { + return name; + } + + @Override + public byte[] getSerializedPythonFunction() { + return new byte[0]; + } + + @Override + public PythonEnv getPythonEnv() { + return null; + } + } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/validation/MatchRecognizeValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/validation/MatchRecognizeValidationTest.scala index 5ab147d45e30..4048e52f3d20 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/validation/MatchRecognizeValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/validation/MatchRecognizeValidationTest.scala @@ -19,9 +19,11 @@ package org.apache.flink.table.api.stream.sql.validation import org.apache.flink.api.scala._ -import org.apache.flink.table.api.ValidationException +import org.apache.flink.table.api.{TableException, ValidationException} import org.apache.flink.table.api.scala._ +import org.apache.flink.table.runtime.utils.JavaUserDefinedScalarFunctions.PythonScalarFunction import org.apache.flink.table.utils.{StreamTableTestUtil, TableTestBase} +import org.hamcrest.Matchers import org.junit.Test class MatchRecognizeValidationTest extends TableTestBase { @@ -42,4 +44,25 @@ class MatchRecognizeValidationTest extends TableTestBase { val sql = "SELECT MATCH_PROCTIME() FROM MyTable" streamUtil.verifySql(sql, "n/a") } + + /** Python Function can not be used in MATCH_RECOGNIZE for now **/ + @Test + def testMatchPythonFunction() = { + expectedException.expectCause(Matchers.isA(classOf[TableException])) + streamUtil.addFunction("pyFunc", new PythonScalarFunction("pyFunc")) + val sql = + """SELECT T.aa as ta + |FROM MyTable + |MATCH_RECOGNIZE ( + | ORDER BY proctime + | MEASURES + | A.a as aa, + | pyFunc(1,2) as bb + | PATTERN (A B) + | DEFINE + | A AS a = 1, + | B AS b = 'b' + |) AS T""".stripMargin + streamUtil.verifySql(sql, "n/a") + } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/PythonCalcSplitRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/PythonCalcSplitRuleTest.scala index 0567a49362c6..f89c104c6076 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/PythonCalcSplitRuleTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/PythonCalcSplitRuleTest.scala @@ -18,11 +18,9 @@ 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.python.{PythonEnv, PythonFunction} -import org.apache.flink.table.functions.ScalarFunction +import org.apache.flink.table.runtime.utils.JavaUserDefinedScalarFunctions.{BooleanPythonScalarFunction, PythonScalarFunction} import org.apache.flink.table.utils.TableTestUtil._ import org.apache.flink.table.utils.TableTestBase import org.junit.Test @@ -269,29 +267,3 @@ class PythonCalcSplitRuleTest extends TableTestBase { util.verifyTable(resultTable, expected) } } - -class PythonScalarFunction(name: String) extends ScalarFunction with PythonFunction { - def eval(i: Int, j: Int): Int = i + j - - override def getResultType(signature: Array[Class[_]]): TypeInformation[_] = - BasicTypeInfo.INT_TYPE_INFO - - override def toString: String = name - - override def getSerializedPythonFunction: Array[Byte] = null - - override def getPythonEnv: PythonEnv = null -} - -class BooleanPythonScalarFunction(name: String) extends ScalarFunction with PythonFunction { - def eval(i: Int, j: Int): Boolean = i + j > 1 - - override def getResultType(signature: Array[Class[_]]): TypeInformation[_] = - BasicTypeInfo.BOOLEAN_TYPE_INFO - - override def toString: String = name - - override def getSerializedPythonFunction: Array[Byte] = null - - override def getPythonEnv: PythonEnv = null -} From 02bc04ac022870973a92e28f6bddb791dd8b270d Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Fri, 1 Nov 2019 21:04:08 +0800 Subject: [PATCH 417/746] [FLINK-14022][table-planner][table-planner-blink] Introduces SplitPythonConditionFromJoinRule which splits Python Functions from join conditions Currently, python udfs can not exist in join condition. In this commit, the SplitPythonConditionFromJoinRule is used to extract python udfs from (inner)join conditions. --- .../plan/rules/FlinkBatchRuleSets.scala | 2 + .../plan/rules/FlinkStreamRuleSets.scala | 2 + .../SplitPythonConditionFromJoinRule.scala | 86 ++++++++++++++ .../SplitPythonConditionFromJoinRuleTest.xml | 82 +++++++++++++ ...SplitPythonConditionFromJoinRuleTest.scala | 80 +++++++++++++ .../apache/flink/table/plan/Optimizer.scala | 2 +- .../table/plan/rules/FlinkRuleSets.scala | 3 + .../SplitPythonConditionFromJoinRule.scala | 86 ++++++++++++++ ...SplitPythonConditionFromJoinRuleTest.scala | 109 ++++++++++++++++++ 9 files changed, 451 insertions(+), 1 deletion(-) create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRule.scala create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRuleTest.xml create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRuleTest.scala create mode 100644 flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/SplitPythonConditionFromJoinRule.scala create mode 100644 flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/SplitPythonConditionFromJoinRuleTest.scala 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 2fd0fc418722..7582c41e8500 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 @@ -356,6 +356,8 @@ object FlinkBatchRuleSets { val LOGICAL_REWRITE: RuleSet = RuleSets.ofList( // transpose calc past snapshot CalcSnapshotTransposeRule.INSTANCE, + // Rule that splits python ScalarFunctions from join conditions + SplitPythonConditionFromJoinRule.INSTANCE, // merge calc after calc transpose FlinkCalcMergeRule.INSTANCE, // Rule that splits python ScalarFunctions from java/scala ScalarFunctions 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 33c3ee5ae5e5..ff3c6bcb4564 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 @@ -336,6 +336,8 @@ object FlinkStreamRuleSets { SplitAggregateRule.INSTANCE, // transpose calc past snapshot CalcSnapshotTransposeRule.INSTANCE, + // Rule that splits python ScalarFunctions from join conditions + SplitPythonConditionFromJoinRule.INSTANCE, // merge calc after calc transpose FlinkCalcMergeRule.INSTANCE, // Rule that splits python ScalarFunctions from java/scala ScalarFunctions. diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRule.scala new file mode 100644 index 000000000000..08efa0cec82e --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRule.scala @@ -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.table.planner.plan.rules.logical + +import org.apache.calcite.plan.RelOptRule.{none, operand} +import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelOptUtil} +import org.apache.calcite.rel.core.JoinRelType +import org.apache.calcite.rex.{RexProgram, RexProgramBuilder, RexUtil} +import org.apache.flink.table.planner.plan.nodes.logical.{FlinkLogicalCalc, FlinkLogicalJoin} +import org.apache.flink.table.planner.plan.utils.PythonUtil.containsPythonCall + +import scala.collection.JavaConversions._ + +/** + * Rule will splits the [[FlinkLogicalJoin]] which contains Python Functions in join condition + * into a [[FlinkLogicalJoin]] and a [[FlinkLogicalCalc]] with python Functions. Currently, only + * inner join is supported. + * + * After this rule is applied, there will be no Python Functions in the condition of the + * [[FlinkLogicalJoin]]. + */ +class SplitPythonConditionFromJoinRule extends RelOptRule( + operand(classOf[FlinkLogicalJoin], none), + "SplitPythonConditionFromJoinRule") { + + override def matches(call: RelOptRuleCall): Boolean = { + val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin] + val joinType: JoinRelType = join.getJoinType + // matches if it is inner join and it contains Python functions in condition + joinType == JoinRelType.INNER && Option(join.getCondition).exists(containsPythonCall) + } + + override def onMatch(call: RelOptRuleCall): Unit = { + val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin] + val rexBuilder = join.getCluster.getRexBuilder + + val joinFilters = RelOptUtil.conjunctions(join.getCondition) + val pythonFilters = joinFilters.filter(containsPythonCall) + val remainingFilters = joinFilters.filter(!containsPythonCall(_)) + + val newJoinCondition = RexUtil.composeConjunction(rexBuilder, remainingFilters) + val bottomJoin = new FlinkLogicalJoin( + join.getCluster, + join.getTraitSet, + join.getLeft, + join.getRight, + newJoinCondition, + join.getJoinType) + + val rexProgram = new RexProgramBuilder(bottomJoin.getRowType, rexBuilder).getProgram + val topCalcCondition = RexUtil.composeConjunction(rexBuilder, pythonFilters) + + val topCalc = new FlinkLogicalCalc( + join.getCluster, + join.getTraitSet, + bottomJoin, + RexProgram.create( + bottomJoin.getRowType, + rexProgram.getExprList, + topCalcCondition, + bottomJoin.getRowType, + rexBuilder)) + + call.transformTo(topCalc) + } +} + +object SplitPythonConditionFromJoinRule { + val INSTANCE = new SplitPythonConditionFromJoinRule +} diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRuleTest.xml new file mode 100644 index 000000000000..c84912719ebc --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRuleTest.xml @@ -0,0 +1,82 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRuleTest.scala new file mode 100644 index 000000000000..cf8f7a2c6598 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRuleTest.scala @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.rules.logical + +import org.apache.calcite.plan.hep.HepMatchOrder +import org.apache.flink.api.scala._ +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.planner.plan.nodes.FlinkConventions +import org.apache.flink.table.planner.plan.optimize.program._ +import org.apache.flink.table.planner.plan.rules.FlinkBatchRuleSets +import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.PythonScalarFunction +import org.apache.flink.table.planner.utils.TableTestBase +import org.junit.{Before, Test} + +/** + * Test for [[SplitPythonConditionFromJoinRule]]. + */ +class SplitPythonConditionFromJoinRuleTest extends TableTestBase { + + private val util = batchTestUtil() + + @Before + def setup(): Unit = { + val programs = new FlinkChainedProgram[BatchOptimizeContext]() + programs.addLast( + "logical", + FlinkVolcanoProgramBuilder.newBuilder + .add(FlinkBatchRuleSets.LOGICAL_OPT_RULES) + .setRequiredOutputTraits(Array(FlinkConventions.LOGICAL)) + .build()) + programs.addLast( + "logical_rewrite", + FlinkHepRuleSetProgramBuilder.newBuilder + .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_SEQUENCE) + .setHepMatchOrder(HepMatchOrder.BOTTOM_UP) + .add(FlinkBatchRuleSets.LOGICAL_REWRITE) + .build()) + util.replaceBatchProgram(programs) + + util.addTableSource[(Int, Int, Int)]("leftTable", 'a, 'b, 'c) + util.addTableSource[(Int, Int, Int)]("rightTable", 'd, 'e, 'f) + util.addFunction("pyFunc", new PythonScalarFunction("pyFunc")) + } + + @Test + def testPythonFunctionInJoinCondition(): Unit = { + val sqlQuery = "SELECT a, b, d FROM leftTable JOIN rightTable ON a=d and pyFunc(a, d)=b " + util.verifyPlan(sqlQuery) + } + + @Test + def testPythonFunctionInAboveFilter(): Unit = { + val sqlQuery = + """ + |SELECT a, d + 1 FROM( + | SELECT a, d FROM( + | SELECT a, d + | FROM leftTable JOIN rightTable ON + | a = d and pyFunc(a, a) = a + d) + | WHERE pyFunc(a, d) = a * d) + """.stripMargin + util.verifyPlan(sqlQuery) + } +} 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 4336c074c3c9..4cb8519d46d4 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 @@ -182,7 +182,7 @@ abstract class Optimizer( protected def optimizeLogicalRewritePlan(relNode: RelNode): RelNode = { val logicalRewriteRuleSet = getLogicalRewriteRuleSet if (logicalRewriteRuleSet.iterator().hasNext) { - runHepPlannerSimultaneously( + runHepPlannerSequentially( HepMatchOrder.TOP_DOWN, logicalRewriteRuleSet, relNode, 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 ea3273815740..31710214e516 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 @@ -147,6 +147,9 @@ object FlinkRuleSets { * RuleSet to do rewrite on FlinkLogicalRel */ val LOGICAL_REWRITE_RULES: RuleSet = RuleSets.ofList( + // Rule that splits python ScalarFunctions from join conditions + SplitPythonConditionFromJoinRule.INSTANCE, + CalcMergeRule.INSTANCE, PythonCalcSplitRule.SPLIT_CONDITION, PythonCalcSplitRule.SPLIT_PROJECT, PythonCalcSplitRule.PUSH_CONDITION, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/SplitPythonConditionFromJoinRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/SplitPythonConditionFromJoinRule.scala new file mode 100644 index 000000000000..43f971e65cae --- /dev/null +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/rules/logical/SplitPythonConditionFromJoinRule.scala @@ -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.table.plan.rules.logical + +import org.apache.calcite.plan.RelOptRule.{none, operand} +import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelOptUtil} +import org.apache.calcite.rel.core.JoinRelType +import org.apache.calcite.rex.{RexProgram, RexProgramBuilder, RexUtil} +import org.apache.flink.table.plan.nodes.logical.{FlinkLogicalCalc, FlinkLogicalJoin} +import org.apache.flink.table.plan.util.PythonUtil.containsPythonCall + +import scala.collection.JavaConversions._ + +/** + * Rule will splits the [[FlinkLogicalJoin]] which contains Python Functions in join condition + * into a [[FlinkLogicalJoin]] and a [[FlinkLogicalCalc]] with python Functions. Currently, only + * inner join is supported. + * + * After this rule is applied, there will be no Python Functions in the condition of the + * [[FlinkLogicalJoin]]. + */ +class SplitPythonConditionFromJoinRule extends RelOptRule( + operand(classOf[FlinkLogicalJoin], none), + "SplitPythonConditionFromJoinRule") { + + override def matches(call: RelOptRuleCall): Boolean = { + val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin] + val joinType: JoinRelType = join.getJoinType + // matches if it is inner join and it contains Python functions in condition + joinType == JoinRelType.INNER && Option(join.getCondition).exists(containsPythonCall) + } + + override def onMatch(call: RelOptRuleCall): Unit = { + val join: FlinkLogicalJoin = call.rel(0).asInstanceOf[FlinkLogicalJoin] + val rexBuilder = join.getCluster.getRexBuilder + + val joinFilters = RelOptUtil.conjunctions(join.getCondition) + val pythonFilters = joinFilters.filter(containsPythonCall) + val remainingFilters = joinFilters.filter(!containsPythonCall(_)) + + val newJoinCondition = RexUtil.composeConjunction(rexBuilder, remainingFilters) + val bottomJoin = new FlinkLogicalJoin( + join.getCluster, + join.getTraitSet, + join.getLeft, + join.getRight, + newJoinCondition, + join.getJoinType) + + val rexProgram = new RexProgramBuilder(bottomJoin.getRowType, rexBuilder).getProgram + val topCalcCondition = RexUtil.composeConjunction(rexBuilder, pythonFilters) + + val topCalc = new FlinkLogicalCalc( + join.getCluster, + join.getTraitSet, + bottomJoin, + RexProgram.create( + bottomJoin.getRowType, + rexProgram.getExprList, + topCalcCondition, + bottomJoin.getRowType, + rexBuilder)) + + call.transformTo(topCalc) + } +} + +object SplitPythonConditionFromJoinRule { + val INSTANCE = new SplitPythonConditionFromJoinRule +} diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/SplitPythonConditionFromJoinRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/SplitPythonConditionFromJoinRuleTest.scala new file mode 100644 index 000000000000..6c4e2eb55209 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/plan/SplitPythonConditionFromJoinRuleTest.scala @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.scala._ +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.runtime.utils.JavaUserDefinedScalarFunctions.PythonScalarFunction +import org.apache.flink.table.utils.TableTestBase +import org.apache.flink.table.utils.TableTestUtil._ +import org.junit.Test + +class SplitPythonConditionFromJoinRuleTest extends TableTestBase { + + @Test + def testPythonFunctionInJoinCondition(): Unit = { + val util = streamTestUtil() + val left = util.addTable[(Int, Int, Int)]("leftTable", 'a, 'b, 'c) + val right = util.addTable[(Int, Int, Int)]("rightTable", 'd, 'e, 'f) + util.tableEnv.registerFunction("pyFunc", new PythonScalarFunction("pyFunc")) + + val result = left + .join(right, "a === d && pyFunc(a, d) === b") + .select("a, b, d") + + val expected = unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamPythonCalc", + binaryNode( + "DataStreamJoin", + unaryNode( + "DataStreamCalc", + streamTableNode(left), + term("select", "a", "b") + ), + unaryNode( + "DataStreamCalc", + streamTableNode(right), + term("select", "d") + ), + term("where", "=(a, d)"), + term("join", "a, b, d"), + term("joinType", "InnerJoin") + ), + term("select", "a", "b", "d", "pyFunc(a, d) AS f0") + ), + term("select", "a", "b", "d"), + term("where", "=(f0, b)") + ) + util.verifyTable(result, expected) + } + + @Test + def testPythonFunctionInAboveFilter(): Unit = { + val util = streamTestUtil() + val left = util.addTable[(Int, Int, Int)]("leftTable", 'a, 'b, 'c) + val right = util.addTable[(Int, Int, Int)]("rightTable", 'd, 'e, 'f) + util.tableEnv.registerFunction("pyFunc", new PythonScalarFunction("pyFunc")) + + val result = left + .join(right, "a === d && pyFunc(a, d) = a + b") + .select("a, b, d") + .filter("pyFunc(a, b) = b * d") + .select("a + 1, b, d") + + val expected = unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamPythonCalc", + binaryNode( + "DataStreamJoin", + unaryNode( + "DataStreamCalc", + streamTableNode(left), + term("select", "a", "b")), + unaryNode( + "DataStreamCalc", + streamTableNode(right), + term("select", "d") + ), + term("where", "=(a, d)"), + term("join", "a, b, d"), + term("joinType", "InnerJoin") + ), + term("select", "a", "b", "d", "pyFunc(a, d) AS f0", "pyFunc(a, b) AS f1") + ), + term("select", "+(a, 1) AS _c0, b, d"), + term("where", "AND(=(f0, +(a, b)), =(f1, *(b, d)))") + ) + + util.verifyTable(result, expected) + } +} From 84f830b4f7bcb52ef6f94c96182c95aeffcd3a72 Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Fri, 1 Nov 2019 21:04:10 +0800 Subject: [PATCH 418/746] [FLINK-14022][table-planner][table-planner-blink] Add python udf validation check in join condition This closes #9969. --- .../plan/nodes/common/CommonLookupJoin.scala | 7 ++++++ .../nodes/common/CommonPhysicalJoin.scala | 8 +++++++ .../stream/StreamExecWindowJoin.scala | 7 ++++++ .../plan/batch/sql/join/LookupJoinTest.scala | 17 ++++++++++++++ .../table/validation/JoinValidationTest.scala | 13 ++++++++++- .../plan/stream/sql/join/WindowJoinTest.scala | 17 ++++++++++++++ .../flink/table/plan/nodes/CommonJoin.scala | 10 +++++++++ .../nodes/datastream/DataStreamJoin.scala | 2 ++ .../datastream/DataStreamWindowJoin.scala | 2 ++ .../sql/validation/JoinValidationTest.scala | 22 +++++++++++++++++++ .../table/validation/JoinValidationTest.scala | 17 ++++++++++++++ 11 files changed, 121 insertions(+), 1 deletion(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonLookupJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonLookupJoin.scala index f012b91fd543..6dcda8ddb164 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonLookupJoin.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonLookupJoin.scala @@ -36,6 +36,7 @@ import org.apache.flink.table.planner.functions.utils.UserDefinedFunctionUtils.{ import org.apache.flink.table.planner.plan.nodes.FlinkRelNode import org.apache.flink.table.planner.plan.utils.LookupJoinUtil._ import org.apache.flink.table.planner.plan.utils.{JoinTypeUtil, RelExplainUtil} +import org.apache.flink.table.planner.plan.utils.PythonUtil.containsPythonCall import org.apache.flink.table.planner.plan.utils.RelExplainUtil.preferExpressionFormat import org.apache.flink.table.planner.utils.TableConfigUtils.getMillisecondFromConfigDuration import org.apache.flink.table.runtime.operators.join.lookup.{AsyncLookupJoinRunner, AsyncLookupJoinWithCalcRunner, LookupJoinRunner, LookupJoinWithCalcRunner} @@ -94,6 +95,12 @@ abstract class CommonLookupJoin( tableSource.getTableSchema, calcOnTemporalTable) + if (containsPythonCall(joinInfo.getRemaining(cluster.getRexBuilder))) { + throw new TableException("Only inner join condition with equality predicates supports the " + + "Python UDF taking the inputs from the left table and the right table at the same time, " + + "e.g., ON T1.id = T2.id && pythonUdf(T1.a, T2.b)") + } + override def deriveRowType(): RelDataType = { val flinkTypeFactory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory] val rightType = if (calcOnTemporalTable.isDefined) { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPhysicalJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPhysicalJoin.scala index 7cff99004190..310406a36828 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPhysicalJoin.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPhysicalJoin.scala @@ -18,8 +18,10 @@ package org.apache.flink.table.planner.plan.nodes.common +import org.apache.flink.table.api.TableException import org.apache.flink.table.planner.plan.nodes.physical.FlinkPhysicalRel import org.apache.flink.table.planner.plan.utils.{JoinTypeUtil, JoinUtil} +import org.apache.flink.table.planner.plan.utils.PythonUtil.containsPythonCall import org.apache.flink.table.planner.plan.utils.RelExplainUtil.preferExpressionFormat import org.apache.flink.table.runtime.operators.join.FlinkJoinType @@ -49,6 +51,12 @@ abstract class CommonPhysicalJoin( extends Join(cluster, traitSet, leftRel, rightRel, condition, Set.empty[CorrelationId], joinType) with FlinkPhysicalRel { + if (containsPythonCall(condition)) { + throw new TableException("Only inner join condition with equality predicates supports the " + + "Python UDF taking the inputs from the left table and the right table at the same time, " + + "e.g., ON T1.id = T2.id && pythonUdf(T1.a, T2.b)") + } + def getJoinInfo: JoinInfo = joinInfo lazy val filterNulls: Array[Boolean] = { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecWindowJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecWindowJoin.scala index 2b096257de0c..9e85bd90e93c 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecWindowJoin.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecWindowJoin.scala @@ -30,6 +30,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.delegation.StreamPlanner import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, StreamExecNode} import org.apache.flink.table.planner.plan.utils.{JoinTypeUtil, KeySelectorUtil, UpdatingPlanChecker, WindowJoinUtil} +import org.apache.flink.table.planner.plan.utils.PythonUtil.containsPythonCall import org.apache.flink.table.planner.plan.utils.RelExplainUtil.preferExpressionFormat import org.apache.flink.table.runtime.generated.GeneratedFunction import org.apache.flink.table.runtime.operators.join.{FlinkJoinType, KeyedCoProcessOperatorWithWatermarkDelay, OuterJoinPaddingUtil, ProcTimeBoundedStreamJoin, RowTimeBoundedStreamJoin} @@ -67,6 +68,12 @@ class StreamExecWindowJoin( with StreamPhysicalRel with StreamExecNode[BaseRow] { + if (containsPythonCall(remainCondition.get)) { + throw new TableException("Only inner join condition with equality predicates supports the " + + "Python UDF taking the inputs from the left table and the right table at the same time, " + + "e.g., ON T1.id = T2.id && pythonUdf(T1.a, T2.b)") + } + // TODO remove FlinkJoinType private lazy val flinkJoinType: FlinkJoinType = JoinTypeUtil.getFlinkJoinType(joinType) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.scala index ae0414dd24d7..fb6df625766a 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.api.config.OptimizerConfigOptions import org.apache.flink.table.api.scala._ import org.apache.flink.table.planner.plan.optimize.program.FlinkBatchProgram import org.apache.flink.table.planner.plan.stream.sql.join.TestTemporalTable +import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.PythonScalarFunction import org.apache.flink.table.planner.utils.TableTestBase import org.junit.Assert.{assertTrue, fail} @@ -107,6 +108,22 @@ class LookupJoinTest extends TableTestBase { ) } + @Test + def testPythonUDFInJoinCondition(): Unit = { + thrown.expect(classOf[TableException]) + thrown.expectMessage("Only inner join condition with equality predicates supports the " + + "Python UDF taking the inputs from the left table and the right table at the same time, " + + "e.g., ON T1.id = T2.id && pythonUdf(T1.a, T2.b)") + testUtil.addFunction("pyFunc", new PythonScalarFunction("pyFunc")) + val sql = + """ + |SELECT * FROM MyTable AS T + |LEFT OUTER JOIN temporalTest FOR SYSTEM_TIME AS OF T.proctime AS D + |ON T.a = D.id AND D.age = 10 AND pyFunc(D.age, T.a) > 100 + """.stripMargin + testUtil.verifyPlan(sql) + } + @Test def testLogicalPlan(): Unit = { val sql1 = diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/JoinValidationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/JoinValidationTest.scala index 2bc17ddbab33..4347aafa7740 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/JoinValidationTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/JoinValidationTest.scala @@ -21,8 +21,9 @@ package org.apache.flink.table.planner.plan.batch.table.validation import org.apache.flink.api.scala._ import org.apache.flink.table.api.internal.TableEnvironmentImpl import org.apache.flink.table.api.scala._ -import org.apache.flink.table.api.{EnvironmentSettings, ValidationException} +import org.apache.flink.table.api.{EnvironmentSettings, TableException, ValidationException} import org.apache.flink.table.planner.runtime.utils.CollectionBatchExecTable +import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.PythonScalarFunction import org.apache.flink.table.planner.utils.TableTestBase import org.junit._ @@ -115,4 +116,14 @@ class JoinValidationTest extends TableTestBase { // Must fail. Tables are bound to different TableEnvironments. ds1.join(ds2).where("a === d").select("g.count") } + + @Test(expected = classOf[TableException]) + def testOuterJoinWithPythonFunctionInCondition(): Unit = { + val util = batchTestUtil() + val left = util.addTableSource[(Int, Long, String)]("left",'a, 'b, 'c) + val right = util.addTableSource[(Int, Long, String)]("right",'d, 'e, 'f) + val pyFunc = new PythonScalarFunction("pyFunc") + val result = left.leftOuterJoin(right, 'a === 'd && pyFunc('a, 'd) === 'a + 'd) + util.verifyPlan(result) + } } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/WindowJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/WindowJoinTest.scala index 659ce1fe450e..975024044e0c 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/WindowJoinTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/WindowJoinTest.scala @@ -22,6 +22,7 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api.TableException import org.apache.flink.table.api.scala._ import org.apache.flink.table.planner.plan.utils.WindowJoinUtil +import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.PythonScalarFunction import org.apache.flink.table.planner.utils.{StreamTableTestUtil, TableTestBase, TableTestUtil} import org.apache.calcite.rel.logical.LogicalJoin @@ -86,6 +87,22 @@ class WindowJoinTest extends TableTestBase { util.verifyPlan(sql) } + /** + * Currently only the inner join condition can support the Python UDF taking the inputs from + * the left table and the right table at the same time. + */ + @Test(expected = classOf[TableException]) + def testWindowOuterJoinWithPythonFunctionInCondition(): Unit = { + util.addFunction("pyFunc", new PythonScalarFunction("pyFunc")) + val sql = + """ + |SELECT t1.a, t2.b FROM MyTable t1 LEFT OUTER JOIN MyTable2 t2 ON + | t1.a = t2.a AND pyFunc(t1.a, t2.a) = t1.a + t2.a AND + | t1.proctime BETWEEN t2.proctime - INTERVAL '1' HOUR AND t2.proctime + INTERVAL '1' HOUR + """.stripMargin + util.verifyPlan(sql) + } + @Test def testProcessingTimeInnerJoinWithOnClause(): Unit = { val sqlQuery = diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonJoin.scala index 753ec40a516b..30953632204e 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/CommonJoin.scala @@ -21,11 +21,21 @@ import org.apache.calcite.rel.RelWriter import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.JoinRelType import org.apache.calcite.rex.RexNode +import org.apache.flink.table.api.TableException +import org.apache.flink.table.plan.util.PythonUtil.containsPythonCall import scala.collection.JavaConverters._ trait CommonJoin { + protected def validatePythonFunctionInJoinCondition(joinCondition: RexNode): Unit = { + if (containsPythonCall(joinCondition)) { + throw new TableException("Only inner join condition with equality predicates supports the " + + "Python UDF taking the inputs from the left table and the right table at the same time, " + + "e.g., ON T1.id = T2.id && pythonUdf(T1.a, T2.b)") + } + } + private[flink] def joinSelectionToString(inputType: RelDataType): String = { inputType.getFieldNames.asScala.toList.mkString(", ") } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoin.scala index 8214583b58c7..d76ade99c638 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoin.scala @@ -51,6 +51,8 @@ class DataStreamJoin( with CommonJoin with DataStreamRel { + validatePythonFunctionInJoinCondition(joinCondition) + override def deriveRowType(): RelDataType = schema.relDataType override def needsUpdatesAsRetraction: Boolean = true diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala index 0b34df934323..d336bc113c51 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala @@ -67,6 +67,8 @@ class DataStreamWindowJoin( with DataStreamRel with Logging { + validatePythonFunctionInJoinCondition(joinCondition) + override def deriveRowType(): RelDataType = schema.relDataType override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/validation/JoinValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/validation/JoinValidationTest.scala index 2a994f31a2c6..7a824d9591e4 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/validation/JoinValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/validation/JoinValidationTest.scala @@ -21,7 +21,9 @@ package org.apache.flink.table.api.stream.sql.validation import org.apache.flink.api.scala._ import org.apache.flink.table.api.TableException import org.apache.flink.table.api.scala._ +import org.apache.flink.table.runtime.utils.JavaUserDefinedScalarFunctions.PythonScalarFunction import org.apache.flink.table.utils.{StreamTableTestUtil, TableTestBase} +import org.hamcrest.Matchers import org.junit.Test class JoinValidationTest extends TableTestBase { @@ -198,4 +200,24 @@ class JoinValidationTest extends TableTestBase { streamUtil.verifySql(sql, "n/a") } + + /** + * Currently only the inner join condition can support the Python UDF taking the inputs from + * the left table and the right table at the same time. + */ + @Test + def testOuterJoinWithPythonFunctionInCondition(): Unit = { + expectedException.expectCause(Matchers.isA(classOf[TableException])) + streamUtil.addFunction("pyFunc", new PythonScalarFunction("pyFunc")) + streamUtil.addTable[(Int, Int, Long)]("leftTable", 'a, 'b, 'c) + streamUtil.addTable[(Int, Int, Long)]("rightTable", 'd, 'e, 'f) + val sql = + """ + |SELECT * + |FROM leftTable LEFT OUTER JOIN rightTable ON + | a = d AND pyFunc(a, d) = a + d + """.stripMargin + + streamUtil.verifySql(sql, "n/a") + } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/validation/JoinValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/validation/JoinValidationTest.scala index e1843a8e6d2e..5c9e6d1fa5c9 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/validation/JoinValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/table/validation/JoinValidationTest.scala @@ -23,9 +23,11 @@ import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment import org.apache.flink.table.api.scala._ import org.apache.flink.table.api.stream.table.validation.JoinValidationTest.WithoutEqualsHashCode import org.apache.flink.table.api.{TableException, ValidationException} +import org.apache.flink.table.runtime.utils.JavaUserDefinedScalarFunctions.PythonScalarFunction import org.apache.flink.table.runtime.utils.StreamTestData import org.apache.flink.table.utils.TableTestBase import org.apache.flink.types.Row +import org.hamcrest.Matchers import org.junit.Test class JoinValidationTest extends TableTestBase { @@ -228,6 +230,21 @@ class JoinValidationTest extends TableTestBase { // Must fail. Tables are bound to different TableEnvironments. in1.join(in2).where("a === d").select("g.count") } + + /** + * Currently only the inner join condition can support the Python UDF taking the inputs from + * the left table and the right table at the same time. + */ + @Test + def testOuterJoinWithPythonFunctionInCondition(): Unit = { + expectedException.expectCause(Matchers.isA(classOf[TableException])) + val util = streamTestUtil() + val left = util.addTable[(Int, Int, String)]('a, 'b, 'c) + val right = util.addTable[(Int, Int, String)]('d, 'e, 'f) + val pyFunc = new PythonScalarFunction("pyFunc") + val result = left.leftOuterJoin(right, 'a === 'd && pyFunc('a, 'd) === 'a + 'd) + util.verifyTable(result, "") + } } object JoinValidationTest { From 1fa4e3dd3e3992111a955508ac1bc1640209c51e Mon Sep 17 00:00:00 2001 From: GuoWei Ma Date: Mon, 9 Sep 2019 14:26:39 +0800 Subject: [PATCH 419/746] [FLINK-14464] Introduce the AbstractUserClassPathJobGraphRetriever This abstract class is for the JobGraphRetriever, which wants to use the user's classpath. --- .../java/org/apache/flink/util/FileUtils.java | 132 ++++++++++++++++++ .../org/apache/flink/util/FileUtilsTest.java | 129 +++++++++++++++++ ...bstractUserClassPathJobGraphRetriever.java | 57 ++++++++ ...actUserClassPathJobGraphRetrieverTest.java | 81 +++++++++++ 4 files changed, 399 insertions(+) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/AbstractUserClassPathJobGraphRetriever.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/entrypoint/component/AbstractUserClassPathJobGraphRetrieverTest.java diff --git a/flink-core/src/main/java/org/apache/flink/util/FileUtils.java b/flink-core/src/main/java/org/apache/flink/util/FileUtils.java index c4d6df76d4e1..46eba83c5165 100644 --- a/flink-core/src/main/java/org/apache/flink/util/FileUtils.java +++ b/flink-core/src/main/java/org/apache/flink/util/FileUtils.java @@ -29,19 +29,34 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; +import java.net.MalformedURLException; +import java.net.URL; import java.nio.ByteBuffer; import java.nio.channels.Channels; import java.nio.channels.SeekableByteChannel; import java.nio.channels.WritableByteChannel; import java.nio.file.AccessDeniedException; +import java.nio.file.FileVisitOption; +import java.nio.file.FileVisitResult; import java.nio.file.Files; +import java.nio.file.Paths; +import java.nio.file.SimpleFileVisitor; import java.nio.file.StandardOpenOption; +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.EnumSet; +import java.util.LinkedList; +import java.util.List; import java.util.Random; +import java.util.function.Predicate; import java.util.zip.ZipEntry; import java.util.zip.ZipInputStream; import java.util.zip.ZipOutputStream; +import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -525,6 +540,123 @@ public static Path expandDirectory(Path file, Path targetDirectory) throws IOExc return new Path(targetDirectory, rootDir); } + /** + * List the {@code directory} recursively and return the files that satisfy the {@code fileFilter}. + * + * @param directory the directory to be listed + * @param fileFilter a file filter + * @return a collection of {@code File}s + * + * @throws IOException if an I/O error occurs while listing the files in the given directory + */ + public static Collection listFilesInPath(final File directory, final Predicate fileFilter) throws IOException { + checkNotNull(directory, "directory"); + checkNotNull(fileFilter, "fileFilter"); + + if (!Files.exists(directory.toPath())) { + throw new IllegalArgumentException(String.format("The directory %s dose not exist.", directory)); + } + if (!Files.isDirectory(directory.toPath())) { + throw new IllegalArgumentException(String.format("The %s is not a directory.", directory)); + } + + final FilterFileVisitor filterFileVisitor = new FilterFileVisitor(fileFilter); + + Files.walkFileTree( + directory.toPath(), + EnumSet.of(FileVisitOption.FOLLOW_LINKS), + Integer.MAX_VALUE, + filterFileVisitor); + + return filterFileVisitor.getFiles(); + } + + /** + * Convert a collection of {@code File}s to a collection of relative path to the working dir. + * + * @param files a collection of files needed to be relatived + * @return a collection of relative {@code File}s + */ + public static Collection relativizeToWorkingDir(final Collection files) { + checkNotNull(files, "files"); + + if (files.isEmpty()) { + return Collections.emptyList(); + } + + final java.nio.file.Path workingDirPath = Paths.get(System.getProperty("user.dir")); + + final List relativeFiles = new LinkedList<>(); + + for (File file : files) { + if (file.isAbsolute()) { + relativeFiles.add(workingDirPath.relativize(file.toPath()).toFile()); + } else { + relativeFiles.add(file); + } + } + + return Collections.unmodifiableCollection(relativeFiles); + } + + /** + * Convert a collection of relative {@code File}s to a collection of relative {@code URL}s. + * + * @param relativeFiles a collection of relative {@code File}s + * @return a collection of relative URLs + * + * @throws MalformedURLException if error occurs while construct a url. + */ + public static Collection toRelativeURLs(final Collection relativeFiles) throws MalformedURLException { + checkNotNull(relativeFiles, "relativeFiles"); + + if (relativeFiles.isEmpty()) { + return Collections.emptyList(); + } + + final List urls = new LinkedList<>(); + + for (File file : relativeFiles) { + checkArgument(!file.isAbsolute(), "the relative path is required"); + urls.add( + new URL( + new URL(file.toURI().getScheme() + ":"), + file.getPath() + ) + ); + } + + return Collections.unmodifiableCollection(urls); + } + + private static final class FilterFileVisitor extends SimpleFileVisitor { + + private List files; + + private final Predicate fileFilter; + + FilterFileVisitor(Predicate fileFilter) { + this.fileFilter = checkNotNull(fileFilter); + } + + @Override + public FileVisitResult visitFile(java.nio.file.Path file, BasicFileAttributes attrs) throws IOException { + FileVisitResult fileVisitResult = super.visitFile(file, attrs); + + if (this.fileFilter.test(file.toFile())) { + if (files == null) { + files = new ArrayList<>(); + } + this.files.add(file.toFile()); + } + + return fileVisitResult; + } + + Collection getFiles() { + return files == null ? Collections.emptyList() : Collections.unmodifiableCollection(files); + } + } // ------------------------------------------------------------------------ /** diff --git a/flink-core/src/test/java/org/apache/flink/util/FileUtilsTest.java b/flink-core/src/test/java/org/apache/flink/util/FileUtilsTest.java index c3cd7ba31c90..a91de0ab6773 100644 --- a/flink-core/src/test/java/org/apache/flink/util/FileUtilsTest.java +++ b/flink-core/src/test/java/org/apache/flink/util/FileUtilsTest.java @@ -18,11 +18,14 @@ package org.apache.flink.util; +import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.core.fs.FSDataOutputStream; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; import org.apache.flink.core.testutils.CheckedThread; +import org.apache.flink.util.function.FunctionUtils; +import org.apache.commons.collections.CollectionUtils; import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; @@ -33,12 +36,17 @@ import java.io.FileOutputStream; import java.io.IOException; import java.io.OutputStream; +import java.net.MalformedURLException; +import java.net.URL; import java.nio.charset.StandardCharsets; import java.nio.file.AccessDeniedException; import java.nio.file.Files; import java.nio.file.Paths; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; import java.util.Comparator; import java.util.List; import java.util.Random; @@ -259,6 +267,73 @@ public void testCompression() throws IOException { assertDirEquals(compressDir.resolve(originalDir), extractDir.resolve(originalDir)); } + @Test + public void testListFilesInPathWithoutAnyFileReturnEmptyList() throws IOException { + final java.nio.file.Path testDir = tmp.newFolder("_test_0").toPath(); + + assertTrue(FileUtils.listFilesInPath(testDir.toFile(), f -> f.getName().endsWith(".jar")).isEmpty()); + } + + @Test + public void testListFilesInPath() throws IOException { + final java.nio.file.Path testDir = tmp.newFolder("_test_1").toPath(); + final Tuple3, Collection, Collection> result = prepareTestFiles(testDir); + + assertTrue(CollectionUtils.isEqualCollection(result.f0, + FileUtils.listFilesInPath(testDir.toFile(), f -> f.getName().endsWith(".jar")))); + } + + @Test + public void testRelativizeToWorkingDir() throws IOException { + final java.nio.file.Path testDir = tmp.newFolder("_test_2").toPath(); + final Tuple3, Collection, Collection> result = prepareTestFiles(testDir); + final Collection relativeFiles = FileUtils.relativizeToWorkingDir(result.f0); + relativeFiles.forEach(file -> assertFalse(file.isAbsolute())); + assertTrue( + CollectionUtils.isEqualCollection( + result.f1, + FileUtils.relativizeToWorkingDir(result.f0) + ) + ); + } + + @Test + public void testToRelativeURLs() throws IOException { + final java.nio.file.Path testDir = tmp.newFolder("_test_3").toPath(); + final Tuple3, Collection, Collection> result = prepareTestFiles(testDir); + + final Collection relativeURLs = FileUtils.toRelativeURLs(result.f1); + relativeURLs.forEach(url -> assertFalse(new File(url.getPath()).isAbsolute())); + + assertTrue( + CollectionUtils.isEqualCollection( + result.f2, + relativeURLs + ) + ); + } + + @Test(expected = IllegalArgumentException.class) + public void testToRelativeURLsThrowExceptionBecauseOfAbsolutePath() throws IOException { + final File tempFile = tmp.newFile(); + FileUtils.toRelativeURLs(Arrays.asList(tempFile)); + } + + @Test(expected = IllegalArgumentException.class) + public void testListDirFailsIfDirectoryDoesNotExist() throws IOException { + final String fileName = "_does_not_exists_file"; + final String doesNotExistsFilePath = tmp.getRoot() + "/" + fileName; + + FileUtils.listFilesInPath(new File(doesNotExistsFilePath), f -> f.getName().endsWith(".jar")); + } + + @Test(expected = IllegalArgumentException.class) + public void testListAFileFailsBecauseDirectoryIsExpected() throws IOException { + final String fileName = "a.jar"; + final File file = tmp.newFile(fileName); + FileUtils.listFilesInPath(file, f -> f.getName().endsWith(".jar")); + } + // ------------------------------------------------------------------------ // Utilities // ------------------------------------------------------------------------ @@ -312,6 +387,60 @@ private static void generateRandomDirs(File dir, int numFiles, int numDirs, int } } + /** + * Generate some files in the directory {@code dir}. + * @param dir the directory where the files are generated + * @return Tuple3 holding the generated files' absolute path, relative to the working directory path and relative + * url. + * @throws IOException if I/O error occurs while generating the files + */ + public static Tuple3, Collection, Collection> prepareTestFiles( + final java.nio.file.Path dir) throws IOException { + + Tuple3, Collection, Collection> result = new Tuple3<>(); + + result.f0 = generateSomeFilesInDirectoryReturnJarFiles(dir); + result.f1 = toRelativeFiles(result.f0); + result.f2 = toRelativeURLs(result.f1); + + return result; + } + + private static Collection generateSomeFilesInDirectoryReturnJarFiles( + final java.nio.file.Path dir) throws IOException { + + final java.nio.file.Path jobSubDir1 = Files.createDirectory(dir.resolve("_sub_dir1")); + final java.nio.file.Path jobSubDir2 = Files.createDirectory(dir.resolve("_sub_dir2")); + final java.nio.file.Path jarFile1 = Files.createFile(dir.resolve("file1.jar")); + final java.nio.file.Path jarFile2 = Files.createFile(dir.resolve("file2.jar")); + final java.nio.file.Path jarFile3 = Files.createFile(jobSubDir1.resolve("file3.jar")); + final java.nio.file.Path jarFile4 = Files.createFile(jobSubDir2.resolve("file4.jar")); + final Collection jarFiles = new ArrayList<>(); + + Files.createFile(dir.resolve("file1.txt")); + Files.createFile(jobSubDir2.resolve("file2.txt")); + + jarFiles.add(jarFile1.toFile()); + jarFiles.add(jarFile2.toFile()); + jarFiles.add(jarFile3.toFile()); + jarFiles.add(jarFile4.toFile()); + return jarFiles; + } + + private static Collection toRelativeFiles(Collection files) { + final java.nio.file.Path workingDir = Paths.get(System.getProperty("user.dir")); + final Collection relativeFiles = new ArrayList<>(); + files.forEach(file -> relativeFiles.add(workingDir.relativize(file.toPath()).toFile())); + return relativeFiles; + } + + private static Collection toRelativeURLs(Collection relativeFiles) throws MalformedURLException { + final Collection relativeURLs = new ArrayList<>(); + final URL context = new URL(relativeFiles.iterator().next().toURI().getScheme() + ":"); + relativeFiles.forEach(FunctionUtils.uncheckedConsumer(file -> relativeURLs.add(new URL(context, file.toString())))); + return relativeURLs; + } + /** * Generates a random content file. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/AbstractUserClassPathJobGraphRetriever.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/AbstractUserClassPathJobGraphRetriever.java new file mode 100644 index 000000000000..d446a65833d3 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/AbstractUserClassPathJobGraphRetriever.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.entrypoint.component; + +import org.apache.flink.util.FileUtils; + +import javax.annotation.Nullable; + +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +/** + * Abstract class for the JobGraphRetriever which supports getting user classpaths. + */ +public abstract class AbstractUserClassPathJobGraphRetriever implements JobGraphRetriever { + + /** User classpaths in relative form to the working directory. */ + private final List userClassPaths; + + protected AbstractUserClassPathJobGraphRetriever(@Nullable final File jobDir) throws IOException { + if (jobDir == null) { + userClassPaths = Collections.emptyList(); + } else { + final Collection jarFiles = FileUtils.listFilesInPath(jobDir, file -> file.getName().endsWith(".jar")); + final Collection relativeFiles = FileUtils.relativizeToWorkingDir(jarFiles); + this.userClassPaths = new ArrayList<>(FileUtils.toRelativeURLs(relativeFiles)); + if (this.userClassPaths.isEmpty()) { + throw new IllegalArgumentException(String.format("The job dir %s does not have any jars.", jobDir)); + } + } + } + + public List getUserClassPaths() { + return userClassPaths; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/entrypoint/component/AbstractUserClassPathJobGraphRetrieverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/entrypoint/component/AbstractUserClassPathJobGraphRetrieverTest.java new file mode 100644 index 000000000000..733ffa5eb71c --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/entrypoint/component/AbstractUserClassPathJobGraphRetrieverTest.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.runtime.entrypoint.component; + +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.util.FileUtilsTest; +import org.apache.flink.util.TestLogger; + +import org.apache.commons.collections.CollectionUtils; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import javax.annotation.Nonnull; + +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.nio.file.Path; +import java.util.Collection; + +import static org.junit.Assert.assertTrue; + +/** + * Tests for the {@link AbstractUserClassPathJobGraphRetriever}. + */ +public class AbstractUserClassPathJobGraphRetrieverTest extends TestLogger { + + @Rule + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private static class TestJobGraphRetriever extends AbstractUserClassPathJobGraphRetriever { + public TestJobGraphRetriever(@Nonnull final File jobDir) throws IOException { + super(jobDir); + } + + @Override + public JobGraph retrieveJobGraph(Configuration configuration) { + return null; + } + } + + @Test + public void testGetUserClassPath() throws IOException { + final Path testJobDir = temporaryFolder.newFolder("_test_job").toPath(); + final Tuple3, Collection, Collection> + result = FileUtilsTest.prepareTestFiles(testJobDir); + final TestJobGraphRetriever testJobGraphRetriever = new TestJobGraphRetriever(testJobDir.toFile()); + assertTrue(CollectionUtils.isEqualCollection(result.f2, testJobGraphRetriever.getUserClassPaths())); + } + + @Test(expected = IllegalArgumentException.class) + public void testTheJobGraphRetrieverThrowExceptionBecauseJobDirDoesNotHaveAnyJars() throws IOException { + final Path testJobDir = temporaryFolder.newFolder("_test_job_").toPath(); + new TestJobGraphRetriever(testJobDir.toFile()); + } + + @Test + public void testGetUserClassPathReturnEmptyListIfJobDirIsNull() throws IOException { + final TestJobGraphRetriever testJobGraphRetriever = new TestJobGraphRetriever(null); + assertTrue(testJobGraphRetriever.getUserClassPaths().isEmpty()); + } +} From 9e13af6b4acb1e3a5505bed75053a2be05299a88 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 31 Oct 2019 16:52:41 +0100 Subject: [PATCH 420/746] [FLINK-14464] Simplify file listing implementation This closes #9950. --- .../java/org/apache/flink/util/FileUtils.java | 110 +++++++--------- .../org/apache/flink/util/FileUtilsTest.java | 124 +++++++----------- ...bstractUserClassPathJobGraphRetriever.java | 24 ++-- ...actUserClassPathJobGraphRetrieverTest.java | 30 ++--- 4 files changed, 123 insertions(+), 165 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/util/FileUtils.java b/flink-core/src/main/java/org/apache/flink/util/FileUtils.java index 46eba83c5165..08ea86281d2a 100644 --- a/flink-core/src/main/java/org/apache/flink/util/FileUtils.java +++ b/flink-core/src/main/java/org/apache/flink/util/FileUtils.java @@ -48,7 +48,6 @@ import java.util.Collection; import java.util.Collections; import java.util.EnumSet; -import java.util.LinkedList; import java.util.List; import java.util.Random; import java.util.function.Predicate; @@ -56,7 +55,6 @@ import java.util.zip.ZipInputStream; import java.util.zip.ZipOutputStream; -import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -84,6 +82,8 @@ public final class FileUtils { /** The size of the buffer used for reading. */ private static final int BUFFER_SIZE = 4096; + private static final String JAR_FILE_EXTENSION = "jar"; + // ------------------------------------------------------------------------ public static void writeCompletely(WritableByteChannel channel, ByteBuffer src) throws IOException { @@ -549,21 +549,21 @@ public static Path expandDirectory(Path file, Path targetDirectory) throws IOExc * * @throws IOException if an I/O error occurs while listing the files in the given directory */ - public static Collection listFilesInPath(final File directory, final Predicate fileFilter) throws IOException { + public static Collection listFilesInDirectory(final java.nio.file.Path directory, final Predicate fileFilter) throws IOException { checkNotNull(directory, "directory"); checkNotNull(fileFilter, "fileFilter"); - if (!Files.exists(directory.toPath())) { + if (!Files.exists(directory)) { throw new IllegalArgumentException(String.format("The directory %s dose not exist.", directory)); } - if (!Files.isDirectory(directory.toPath())) { + if (!Files.isDirectory(directory)) { throw new IllegalArgumentException(String.format("The %s is not a directory.", directory)); } final FilterFileVisitor filterFileVisitor = new FilterFileVisitor(fileFilter); Files.walkFileTree( - directory.toPath(), + directory, EnumSet.of(FileVisitOption.FOLLOW_LINKS), Integer.MAX_VALUE, filterFileVisitor); @@ -572,91 +572,79 @@ public static Collection listFilesInPath(final File directory, final Predi } /** - * Convert a collection of {@code File}s to a collection of relative path to the working dir. + * Relativize the given path with respect to the given base path if it is absolute. * - * @param files a collection of files needed to be relatived - * @return a collection of relative {@code File}s + * @param basePath to relativize against + * @param pathToRelativize path which is being relativized if it is an absolute path + * @return the relativized path */ - public static Collection relativizeToWorkingDir(final Collection files) { - checkNotNull(files, "files"); - - if (files.isEmpty()) { - return Collections.emptyList(); - } - - final java.nio.file.Path workingDirPath = Paths.get(System.getProperty("user.dir")); - - final List relativeFiles = new LinkedList<>(); - - for (File file : files) { - if (file.isAbsolute()) { - relativeFiles.add(workingDirPath.relativize(file.toPath()).toFile()); - } else { - relativeFiles.add(file); - } + public static java.nio.file.Path relativizePath(java.nio.file.Path basePath, java.nio.file.Path pathToRelativize) { + if (pathToRelativize.isAbsolute()) { + return basePath.relativize(pathToRelativize); + } else { + return pathToRelativize; } - - return Collections.unmodifiableCollection(relativeFiles); } /** - * Convert a collection of relative {@code File}s to a collection of relative {@code URL}s. + * Returns the current working directory as specified by the {@code user.dir} system property. * - * @param relativeFiles a collection of relative {@code File}s - * @return a collection of relative URLs - * - * @throws MalformedURLException if error occurs while construct a url. + * @return current working directory */ - public static Collection toRelativeURLs(final Collection relativeFiles) throws MalformedURLException { - checkNotNull(relativeFiles, "relativeFiles"); - - if (relativeFiles.isEmpty()) { - return Collections.emptyList(); - } - - final List urls = new LinkedList<>(); + public static java.nio.file.Path getCurrentWorkingDirectory() { + return Paths.get(System.getProperty("user.dir")); + } - for (File file : relativeFiles) { - checkArgument(!file.isAbsolute(), "the relative path is required"); - urls.add( - new URL( - new URL(file.toURI().getScheme() + ":"), - file.getPath() - ) - ); - } + /** + * Checks whether the given file has a jar extension. + * + * @param file to check + * @return true if the file has a jar extension, otherwise false + */ + public static boolean isJarFile(java.nio.file.Path file) { + return JAR_FILE_EXTENSION.equals(org.apache.flink.shaded.guava18.com.google.common.io.Files.getFileExtension(file.toString())); + } - return Collections.unmodifiableCollection(urls); + /** + * Converts the given {@link java.nio.file.Path} into a file {@link URL}. The resulting url is + * relative iff the given path is relative. + * + * @param path to convert into a {@link URL}. + * @return URL + * @throws MalformedURLException if the path could not be converted into a file {@link URL} + */ + public static URL toURL(java.nio.file.Path path) throws MalformedURLException { + final String scheme = path.toUri().getScheme(); + return new URL(scheme, null, -1, path.toString()); } private static final class FilterFileVisitor extends SimpleFileVisitor { - private List files; + private final Predicate fileFilter; - private final Predicate fileFilter; + private final List files; - FilterFileVisitor(Predicate fileFilter) { + FilterFileVisitor(Predicate fileFilter) { this.fileFilter = checkNotNull(fileFilter); + this.files = new ArrayList<>(); } @Override public FileVisitResult visitFile(java.nio.file.Path file, BasicFileAttributes attrs) throws IOException { FileVisitResult fileVisitResult = super.visitFile(file, attrs); - if (this.fileFilter.test(file.toFile())) { - if (files == null) { - files = new ArrayList<>(); - } - this.files.add(file.toFile()); + if (fileFilter.test(file)) { + files.add(file); } return fileVisitResult; } - Collection getFiles() { - return files == null ? Collections.emptyList() : Collections.unmodifiableCollection(files); + Collection getFiles() { + return Collections.unmodifiableCollection(files); } } + // ------------------------------------------------------------------------ /** diff --git a/flink-core/src/test/java/org/apache/flink/util/FileUtilsTest.java b/flink-core/src/test/java/org/apache/flink/util/FileUtilsTest.java index a91de0ab6773..173043e85d0f 100644 --- a/flink-core/src/test/java/org/apache/flink/util/FileUtilsTest.java +++ b/flink-core/src/test/java/org/apache/flink/util/FileUtilsTest.java @@ -18,14 +18,11 @@ package org.apache.flink.util; -import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.core.fs.FSDataOutputStream; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; import org.apache.flink.core.testutils.CheckedThread; -import org.apache.flink.util.function.FunctionUtils; -import org.apache.commons.collections.CollectionUtils; import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; @@ -45,7 +42,6 @@ import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Comparator; import java.util.List; @@ -53,9 +49,13 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.junit.Assume.assumeTrue; @@ -271,67 +271,68 @@ public void testCompression() throws IOException { public void testListFilesInPathWithoutAnyFileReturnEmptyList() throws IOException { final java.nio.file.Path testDir = tmp.newFolder("_test_0").toPath(); - assertTrue(FileUtils.listFilesInPath(testDir.toFile(), f -> f.getName().endsWith(".jar")).isEmpty()); + assertThat(FileUtils.listFilesInDirectory(testDir, FileUtils::isJarFile), is(empty())); } @Test public void testListFilesInPath() throws IOException { final java.nio.file.Path testDir = tmp.newFolder("_test_1").toPath(); - final Tuple3, Collection, Collection> result = prepareTestFiles(testDir); + final Collection testFiles = prepareTestFiles(testDir); - assertTrue(CollectionUtils.isEqualCollection(result.f0, - FileUtils.listFilesInPath(testDir.toFile(), f -> f.getName().endsWith(".jar")))); + final Collection filesInDirectory = FileUtils.listFilesInDirectory(testDir, FileUtils::isJarFile); + assertThat(filesInDirectory, containsInAnyOrder(testFiles.toArray())); } @Test - public void testRelativizeToWorkingDir() throws IOException { - final java.nio.file.Path testDir = tmp.newFolder("_test_2").toPath(); - final Tuple3, Collection, Collection> result = prepareTestFiles(testDir); - final Collection relativeFiles = FileUtils.relativizeToWorkingDir(result.f0); - relativeFiles.forEach(file -> assertFalse(file.isAbsolute())); - assertTrue( - CollectionUtils.isEqualCollection( - result.f1, - FileUtils.relativizeToWorkingDir(result.f0) - ) - ); + public void testRelativizeOfAbsolutePath() throws IOException { + final java.nio.file.Path absolutePath = tmp.newFolder().toPath().toAbsolutePath(); + + final java.nio.file.Path rootPath = tmp.getRoot().toPath(); + final java.nio.file.Path relativePath = FileUtils.relativizePath(rootPath, absolutePath); + assertFalse(relativePath.isAbsolute()); + assertThat(rootPath.resolve(relativePath), is(absolutePath)); } @Test - public void testToRelativeURLs() throws IOException { - final java.nio.file.Path testDir = tmp.newFolder("_test_3").toPath(); - final Tuple3, Collection, Collection> result = prepareTestFiles(testDir); - - final Collection relativeURLs = FileUtils.toRelativeURLs(result.f1); - relativeURLs.forEach(url -> assertFalse(new File(url.getPath()).isAbsolute())); - - assertTrue( - CollectionUtils.isEqualCollection( - result.f2, - relativeURLs - ) - ); + public void testRelativizeOfRelativePath() { + final java.nio.file.Path path = Paths.get("foobar"); + assertFalse(path.isAbsolute()); + + final java.nio.file.Path relativePath = FileUtils.relativizePath(tmp.getRoot().toPath(), path); + assertThat(relativePath, is(path)); } - @Test(expected = IllegalArgumentException.class) - public void testToRelativeURLsThrowExceptionBecauseOfAbsolutePath() throws IOException { - final File tempFile = tmp.newFile(); - FileUtils.toRelativeURLs(Arrays.asList(tempFile)); + @Test + public void testAbsolutePathToURL() throws MalformedURLException { + final java.nio.file.Path absolutePath = tmp.getRoot().toPath().toAbsolutePath(); + final URL absoluteURL = FileUtils.toURL(absolutePath); + + final java.nio.file.Path transformedURL = Paths.get(absoluteURL.getPath()); + assertThat(transformedURL, is(absolutePath)); + } + + @Test + public void testRelativePathToURL() throws MalformedURLException { + final java.nio.file.Path relativePath = Paths.get("foobar"); + assertFalse(relativePath.isAbsolute()); + + final URL relativeURL = FileUtils.toURL(relativePath); + final java.nio.file.Path transformedPath = Paths.get(relativeURL.getPath()); + + assertThat(transformedPath, is(relativePath)); } @Test(expected = IllegalArgumentException.class) public void testListDirFailsIfDirectoryDoesNotExist() throws IOException { final String fileName = "_does_not_exists_file"; - final String doesNotExistsFilePath = tmp.getRoot() + "/" + fileName; - - FileUtils.listFilesInPath(new File(doesNotExistsFilePath), f -> f.getName().endsWith(".jar")); + FileUtils.listFilesInDirectory(tmp.getRoot().toPath().resolve(fileName), FileUtils::isJarFile); } @Test(expected = IllegalArgumentException.class) public void testListAFileFailsBecauseDirectoryIsExpected() throws IOException { final String fileName = "a.jar"; final File file = tmp.newFile(fileName); - FileUtils.listFilesInPath(file, f -> f.getName().endsWith(".jar")); + FileUtils.listFilesInDirectory(file.toPath(), FileUtils::isJarFile); } // ------------------------------------------------------------------------ @@ -390,57 +391,28 @@ private static void generateRandomDirs(File dir, int numFiles, int numDirs, int /** * Generate some files in the directory {@code dir}. * @param dir the directory where the files are generated - * @return Tuple3 holding the generated files' absolute path, relative to the working directory path and relative - * url. + * @return The list of generated files * @throws IOException if I/O error occurs while generating the files */ - public static Tuple3, Collection, Collection> prepareTestFiles( - final java.nio.file.Path dir) throws IOException { - - Tuple3, Collection, Collection> result = new Tuple3<>(); - - result.f0 = generateSomeFilesInDirectoryReturnJarFiles(dir); - result.f1 = toRelativeFiles(result.f0); - result.f2 = toRelativeURLs(result.f1); - - return result; - } - - private static Collection generateSomeFilesInDirectoryReturnJarFiles( - final java.nio.file.Path dir) throws IOException { - + public static Collection prepareTestFiles(final java.nio.file.Path dir) throws IOException { final java.nio.file.Path jobSubDir1 = Files.createDirectory(dir.resolve("_sub_dir1")); final java.nio.file.Path jobSubDir2 = Files.createDirectory(dir.resolve("_sub_dir2")); final java.nio.file.Path jarFile1 = Files.createFile(dir.resolve("file1.jar")); final java.nio.file.Path jarFile2 = Files.createFile(dir.resolve("file2.jar")); final java.nio.file.Path jarFile3 = Files.createFile(jobSubDir1.resolve("file3.jar")); final java.nio.file.Path jarFile4 = Files.createFile(jobSubDir2.resolve("file4.jar")); - final Collection jarFiles = new ArrayList<>(); + final Collection jarFiles = new ArrayList<>(); Files.createFile(dir.resolve("file1.txt")); Files.createFile(jobSubDir2.resolve("file2.txt")); - jarFiles.add(jarFile1.toFile()); - jarFiles.add(jarFile2.toFile()); - jarFiles.add(jarFile3.toFile()); - jarFiles.add(jarFile4.toFile()); + jarFiles.add(jarFile1); + jarFiles.add(jarFile2); + jarFiles.add(jarFile3); + jarFiles.add(jarFile4); return jarFiles; } - private static Collection toRelativeFiles(Collection files) { - final java.nio.file.Path workingDir = Paths.get(System.getProperty("user.dir")); - final Collection relativeFiles = new ArrayList<>(); - files.forEach(file -> relativeFiles.add(workingDir.relativize(file.toPath()).toFile())); - return relativeFiles; - } - - private static Collection toRelativeURLs(Collection relativeFiles) throws MalformedURLException { - final Collection relativeURLs = new ArrayList<>(); - final URL context = new URL(relativeFiles.iterator().next().toURI().getScheme() + ":"); - relativeFiles.forEach(FunctionUtils.uncheckedConsumer(file -> relativeURLs.add(new URL(context, file.toString())))); - return relativeURLs; - } - /** * Generates a random content file. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/AbstractUserClassPathJobGraphRetriever.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/AbstractUserClassPathJobGraphRetriever.java index d446a65833d3..e89cad9907b0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/AbstractUserClassPathJobGraphRetriever.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/AbstractUserClassPathJobGraphRetriever.java @@ -19,16 +19,17 @@ package org.apache.flink.runtime.entrypoint.component; import org.apache.flink.util.FileUtils; +import org.apache.flink.util.function.FunctionUtils; import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.net.URL; -import java.util.ArrayList; +import java.nio.file.Path; import java.util.Collection; import java.util.Collections; -import java.util.List; +import java.util.stream.Collectors; /** * Abstract class for the JobGraphRetriever which supports getting user classpaths. @@ -36,22 +37,23 @@ public abstract class AbstractUserClassPathJobGraphRetriever implements JobGraphRetriever { /** User classpaths in relative form to the working directory. */ - private final List userClassPaths; + private final Collection userClassPaths; - protected AbstractUserClassPathJobGraphRetriever(@Nullable final File jobDir) throws IOException { + protected AbstractUserClassPathJobGraphRetriever(@Nullable File jobDir) throws IOException { if (jobDir == null) { userClassPaths = Collections.emptyList(); } else { - final Collection jarFiles = FileUtils.listFilesInPath(jobDir, file -> file.getName().endsWith(".jar")); - final Collection relativeFiles = FileUtils.relativizeToWorkingDir(jarFiles); - this.userClassPaths = new ArrayList<>(FileUtils.toRelativeURLs(relativeFiles)); - if (this.userClassPaths.isEmpty()) { - throw new IllegalArgumentException(String.format("The job dir %s does not have any jars.", jobDir)); - } + final Path workingDirectory = FileUtils.getCurrentWorkingDirectory(); + final Collection relativeJarURLs = FileUtils.listFilesInDirectory(jobDir.toPath(), FileUtils::isJarFile) + .stream() + .map(path -> FileUtils.relativizePath(workingDirectory, path)) + .map(FunctionUtils.uncheckedFunction(FileUtils::toURL)) + .collect(Collectors.toList()); + this.userClassPaths = Collections.unmodifiableCollection(relativeJarURLs); } } - public List getUserClassPaths() { + protected Collection getUserClassPaths() { return userClassPaths; } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/entrypoint/component/AbstractUserClassPathJobGraphRetrieverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/entrypoint/component/AbstractUserClassPathJobGraphRetrieverTest.java index 733ffa5eb71c..a159195f950c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/entrypoint/component/AbstractUserClassPathJobGraphRetrieverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/entrypoint/component/AbstractUserClassPathJobGraphRetrieverTest.java @@ -18,25 +18,24 @@ package org.apache.flink.runtime.entrypoint.component; -import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.util.FileUtils; import org.apache.flink.util.FileUtilsTest; import org.apache.flink.util.TestLogger; +import org.apache.flink.util.function.FunctionUtils; -import org.apache.commons.collections.CollectionUtils; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import javax.annotation.Nonnull; - import java.io.File; import java.io.IOException; -import java.net.URL; import java.nio.file.Path; import java.util.Collection; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; /** @@ -48,29 +47,26 @@ public class AbstractUserClassPathJobGraphRetrieverTest extends TestLogger { public final TemporaryFolder temporaryFolder = new TemporaryFolder(); private static class TestJobGraphRetriever extends AbstractUserClassPathJobGraphRetriever { - public TestJobGraphRetriever(@Nonnull final File jobDir) throws IOException { + TestJobGraphRetriever(File jobDir) throws IOException { super(jobDir); } @Override public JobGraph retrieveJobGraph(Configuration configuration) { - return null; + throw new UnsupportedOperationException("This method should not be called."); } } @Test public void testGetUserClassPath() throws IOException { - final Path testJobDir = temporaryFolder.newFolder("_test_job").toPath(); - final Tuple3, Collection, Collection> - result = FileUtilsTest.prepareTestFiles(testJobDir); - final TestJobGraphRetriever testJobGraphRetriever = new TestJobGraphRetriever(testJobDir.toFile()); - assertTrue(CollectionUtils.isEqualCollection(result.f2, testJobGraphRetriever.getUserClassPaths())); - } + final File testJobDir = temporaryFolder.newFolder("_test_job"); + final Collection testFiles = FileUtilsTest.prepareTestFiles(testJobDir.toPath()); + final Path currentWorkingDirectory = FileUtils.getCurrentWorkingDirectory(); + final TestJobGraphRetriever testJobGraphRetriever = new TestJobGraphRetriever(testJobDir); - @Test(expected = IllegalArgumentException.class) - public void testTheJobGraphRetrieverThrowExceptionBecauseJobDirDoesNotHaveAnyJars() throws IOException { - final Path testJobDir = temporaryFolder.newFolder("_test_job_").toPath(); - new TestJobGraphRetriever(testJobDir.toFile()); + assertThat(testJobGraphRetriever.getUserClassPaths(), containsInAnyOrder(testFiles.stream() + .map(file -> FileUtils.relativizePath(currentWorkingDirectory, file)) + .map(FunctionUtils.uncheckedFunction(FileUtils::toURL)).toArray())); } @Test From 34a9c104f8679cf866a2de01ca6b87bd8be961a2 Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Wed, 30 Oct 2019 16:27:27 -0700 Subject: [PATCH 421/746] [FLINK-14221][table] support drop temp system functions and temp catalog functions Support dropping temp functions in FunctionCatalog. This closes #10054. --- .../flink/table/catalog/FunctionCatalog.java | 37 +++++++++++++++++++ .../table/catalog/FunctionCatalogTest.java | 37 +++++++++++++++++-- 2 files changed, 70 insertions(+), 4 deletions(-) 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 29208e6fec35..d56645321314 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 @@ -21,6 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; import org.apache.flink.table.catalog.exceptions.FunctionNotExistException; import org.apache.flink.table.delegation.PlannerTypeInferenceUtil; @@ -193,6 +194,42 @@ public void registerTempCatalogAggregateFunction( ); } + /** + * Drop a temporary system function. + * + * @param funcName name of the function + * @param ignoreIfNotExist Flag to specify behavior when the function does not exist: + * if set to false, throw an exception, + * if set to true, do nothing. + */ + public void dropTempSystemFunction(String funcName, boolean ignoreIfNotExist) { + String normalizedName = FunctionIdentifier.normalizeName(funcName); + + FunctionDefinition fd = tempSystemFunctions.remove(normalizedName); + + if (fd == null && !ignoreIfNotExist) { + throw new ValidationException(String.format("Temporary system function %s doesn't exist", funcName)); + } + } + + /** + * Drop a temporary catalog function. + * + * @param identifier identifier of the function + * @param ignoreIfNotExist Flag to specify behavior when the function does not exist: + * if set to false, throw an exception, + * if set to true, do nothing. + */ + public void dropTempCatalogFunction(ObjectIdentifier identifier, boolean ignoreIfNotExist) { + ObjectIdentifier normalizedName = FunctionIdentifier.normalizeObjectIdentifier(identifier); + + FunctionDefinition fd = tempCatalogFunctions.remove(normalizedName); + + if (fd == null && !ignoreIfNotExist) { + throw new ValidationException(String.format("Temporary catalog function %s doesn't exist", identifier)); + } + } + /** * Get names of all user defined functions, including temp system functions, temp catalog functions and catalog functions * in the current catalog and current database. diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java index ee94c00c807b..fb0205d45c2e 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java @@ -32,6 +32,7 @@ import org.junit.Before; import org.junit.Test; +import java.util.Arrays; import java.util.Collections; import java.util.HashSet; import java.util.Optional; @@ -50,6 +51,10 @@ public class FunctionCatalogTest { private ModuleManager moduleManager; private final String testCatalogName = "test"; + private final ObjectIdentifier oi = ObjectIdentifier.of( + testCatalogName, + GenericInMemoryCatalog.DEFAULT_DB, + TEST_FUNCTION_NAME); private static final String TEST_FUNCTION_NAME = "test_function"; @@ -104,10 +109,6 @@ public void testPreciseFunctionReference() throws FunctionAlreadyExistException, @Test public void testAmbiguousFunctionReference() throws FunctionAlreadyExistException, DatabaseNotExistException, ModuleAlreadyExistException { - ObjectIdentifier oi = ObjectIdentifier.of( - testCatalogName, - GenericInMemoryCatalog.DEFAULT_DB, - TEST_FUNCTION_NAME); // test no function is found assertFalse(functionCatalog.lookupFunction(FunctionIdentifier.of(TEST_FUNCTION_NAME)).isPresent()); @@ -166,6 +167,34 @@ public Optional getFunctionDefinition(String name) { } } + @Test + public void testRegisterAndDropTempSystemFunction() { + assertFalse(Arrays.asList(functionCatalog.getUserDefinedFunctions()).contains(TEST_FUNCTION_NAME)); + + functionCatalog.registerTempSystemScalarFunction(TEST_FUNCTION_NAME, new TestFunction1()); + assertTrue(Arrays.asList(functionCatalog.getUserDefinedFunctions()).contains(TEST_FUNCTION_NAME)); + + functionCatalog.dropTempSystemFunction(TEST_FUNCTION_NAME, false); + assertFalse(Arrays.asList(functionCatalog.getUserDefinedFunctions()).contains(TEST_FUNCTION_NAME)); + + functionCatalog.dropTempSystemFunction(TEST_FUNCTION_NAME, true); + assertFalse(Arrays.asList(functionCatalog.getUserDefinedFunctions()).contains(TEST_FUNCTION_NAME)); + } + + @Test + public void testRegisterAndDropTempCatalogFunction() { + assertFalse(Arrays.asList(functionCatalog.getUserDefinedFunctions()).contains(TEST_FUNCTION_NAME)); + + functionCatalog.registerTempCatalogScalarFunction(oi, new TestFunction1()); + assertTrue(Arrays.asList(functionCatalog.getUserDefinedFunctions()).contains(oi.getObjectName())); + + functionCatalog.dropTempCatalogFunction(oi, false); + assertFalse(Arrays.asList(functionCatalog.getUserDefinedFunctions()).contains(oi.getObjectName())); + + functionCatalog.dropTempCatalogFunction(oi, true); + assertFalse(Arrays.asList(functionCatalog.getUserDefinedFunctions()).contains(oi.getObjectName())); + } + /** * Testing function. */ From 15f8f3c52a1bf11ecf9f550388eee550b7fc763e Mon Sep 17 00:00:00 2001 From: Rui Li Date: Thu, 31 Oct 2019 21:15:13 +0800 Subject: [PATCH 422/746] [FLINK-14588][hive] Support Hive version 1.0.0 and 1.0.1 To support Hive 1.0.0 and 1.0.1. This closes #10062. --- flink-connectors/flink-connector-hive/pom.xml | 10 +- .../hive/HiveTableOutputFormat.java | 51 +-- .../flink/table/catalog/hive/HiveCatalog.java | 11 +- .../table/catalog/hive/client/HiveShim.java | 16 + .../catalog/hive/client/HiveShimLoader.java | 8 + .../catalog/hive/client/HiveShimV100.java | 338 ++++++++++++++++++ .../catalog/hive/client/HiveShimV101.java | 25 ++ .../catalog/hive/client/HiveShimV110.java | 287 ++------------- .../hive/util/HiveReflectionUtils.java | 25 -- .../connectors/hive/HiveRunnerShimLoader.java | 2 + .../hive/TableEnvHiveConnectorTest.java | 6 +- .../functions/hive/HiveGenericUDFTest.java | 7 +- 12 files changed, 458 insertions(+), 328 deletions(-) create mode 100644 flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV100.java create mode 100644 flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV101.java diff --git a/flink-connectors/flink-connector-hive/pom.xml b/flink-connectors/flink-connector-hive/pom.xml index e20bdebfce7d..de98156ae407 100644 --- a/flink-connectors/flink-connector-hive/pom.xml +++ b/flink-connectors/flink-connector-hive/pom.xml @@ -666,7 +666,15 @@ under the License. - + + + hive-1.0.1 + + 1.0.1 + 2.6.5 + 3.1.1 + + hive-1.1.1 diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableOutputFormat.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableOutputFormat.java index 0760ae507158..15fcbd4a5b05 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableOutputFormat.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableOutputFormat.java @@ -45,7 +45,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.common.FileUtils; +import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.Warehouse; import org.apache.hadoop.hive.metastore.api.MetaException; @@ -53,8 +53,6 @@ import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.ql.exec.FileSinkOperator; -import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils; -import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.serde2.Deserializer; import org.apache.hadoop.hive.serde2.SerDeException; import org.apache.hadoop.hive.serde2.SerDeUtils; @@ -71,8 +69,6 @@ import org.apache.hadoop.mapred.JobContextImpl; import org.apache.hadoop.mapred.JobID; import org.apache.hadoop.mapred.OutputCommitter; -import org.apache.hadoop.mapred.OutputFormat; -import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapred.SequenceFileOutputFormat; import org.apache.hadoop.mapred.TaskAttemptContext; import org.apache.hadoop.mapred.TaskAttemptContextImpl; @@ -108,6 +104,11 @@ public class HiveTableOutputFormat extends HadoopOutputFormatCommonBase imp private static final long serialVersionUID = 5167529504848109023L; + private static final PathFilter HIDDEN_FILES_PATH_FILTER = p -> { + String name = p.getName(); + return !name.startsWith("_") && !name.startsWith("."); + }; + private transient JobConf jobConf; private transient ObjectPath tablePath; private transient List partitionColumns; @@ -139,6 +140,8 @@ public class HiveTableOutputFormat extends HadoopOutputFormatCommonBase imp private transient String hiveVersion; + private transient HiveShim hiveShim; + // to convert Flink object to Hive object private transient HiveObjectConversion[] hiveConversions; @@ -167,6 +170,7 @@ public HiveTableOutputFormat(JobConf jobConf, ObjectPath tablePath, CatalogTable isDynamicPartition = isPartitioned && partitionColumns.size() > hiveTablePartition.getPartitionSpec().size(); hiveVersion = Preconditions.checkNotNull(jobConf.get(HiveCatalogValidator.CATALOG_HIVE_VERSION), "Hive version is not defined"); + hiveShim = HiveShimLoader.loadHiveShim(hiveVersion); } // Custom serialization methods @@ -209,6 +213,7 @@ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundE partitionToWriter = new HashMap<>(); tableProperties = (Properties) in.readObject(); hiveVersion = (String) in.readObject(); + hiveShim = HiveShimLoader.loadHiveShim(hiveVersion); } @Override @@ -223,7 +228,6 @@ public void finalizeGlobal(int parallelism) throws IOException { } if (isPartitioned) { if (isDynamicPartition) { - HiveShim hiveShim = HiveShimLoader.loadHiveShim(hiveVersion); FileStatus[] generatedParts = hiveShim.getFileStatusRecurse(stagingDir, partitionColumns.size() - hiveTablePartition.getPartitionSpec().size(), fs); for (FileStatus part : generatedParts) { @@ -378,16 +382,15 @@ private void moveFiles(Path srcDir, Path destDir) throws IOException { // TODO: support setting auto-purge? final boolean purge = true; // Note we assume the srcDir is a hidden dir, otherwise it will be deleted if it's a sub-dir of destDir - FileStatus[] existingFiles = fs.listStatus(destDir, FileUtils.HIDDEN_FILES_PATH_FILTER); + FileStatus[] existingFiles = fs.listStatus(destDir, HIDDEN_FILES_PATH_FILTER); if (existingFiles != null) { - HiveShim hiveShim = HiveShimLoader.loadHiveShim(hiveVersion); for (FileStatus existingFile : existingFiles) { Preconditions.checkState(hiveShim.moveToTrash(fs, existingFile.getPath(), jobConf, purge), "Failed to overwrite existing file " + existingFile); } } } - FileStatus[] srcFiles = fs.listStatus(srcDir, FileUtils.HIDDEN_FILES_PATH_FILTER); + FileStatus[] srcFiles = fs.listStatus(srcDir, HIDDEN_FILES_PATH_FILTER); for (FileStatus srcFile : srcFiles) { Path srcPath = srcFile.getPath(); Path destPath = new Path(destDir, srcPath.getName()); @@ -441,17 +444,6 @@ private List getPartitionWriters() { private HivePartitionWriter writerForLocation(String location) throws IOException { JobConf clonedConf = new JobConf(jobConf); clonedConf.set(OUTDIR, location); - OutputFormat outputFormat; - try { - StorageDescriptor sd = hiveTablePartition.getStorageDescriptor(); - Class outputFormatClz = Class.forName(sd.getOutputFormat(), true, - Thread.currentThread().getContextClassLoader()); - outputFormatClz = HiveFileFormatUtils.getOutputFormatSubstitute(outputFormatClz); - outputFormat = (OutputFormat) outputFormatClz.newInstance(); - } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) { - throw new FlinkRuntimeException("Unable to instantiate the hadoop output format", e); - } - ReflectionUtils.setConf(outputFormat, clonedConf); OutputCommitter outputCommitter = clonedConf.getOutputCommitter(); JobContext jobContext = new JobContextImpl(clonedConf, new JobID()); outputCommitter.setupJob(jobContext); @@ -474,28 +466,19 @@ private HivePartitionWriter writerForLocation(String location) throws IOExceptio SequenceFileOutputFormat.setOutputCompressionType(clonedConf, style); } } + StorageDescriptor sd = hiveTablePartition.getStorageDescriptor(); String taskPartition = String.valueOf(clonedConf.getInt("mapreduce.task.partition", -1)); Path taskPath = FileOutputFormat.getTaskOutputPath(clonedConf, taskPartition); - FileSinkOperator.RecordWriter recordWriter; - try { - recordWriter = HiveFileFormatUtils.getRecordWriter(clonedConf, outputFormat, - outputClass, isCompressed, tableProperties, taskPath, Reporter.NULL); - } catch (HiveException e) { - throw new IOException(e); - } - return new HivePartitionWriter(clonedConf, outputFormat, recordWriter, outputCommitter); + FileSinkOperator.RecordWriter recordWriter = hiveShim.getHiveRecordWriter( + clonedConf, sd.getOutputFormat(), outputClass, isCompressed, tableProperties, taskPath); + return new HivePartitionWriter(recordWriter, outputCommitter); } private static class HivePartitionWriter { - private final JobConf jobConf; - private final OutputFormat outputFormat; private final FileSinkOperator.RecordWriter recordWriter; private final OutputCommitter outputCommitter; - HivePartitionWriter(JobConf jobConf, OutputFormat outputFormat, FileSinkOperator.RecordWriter recordWriter, - OutputCommitter outputCommitter) { - this.jobConf = jobConf; - this.outputFormat = outputFormat; + HivePartitionWriter(FileSinkOperator.RecordWriter recordWriter, OutputCommitter outputCommitter) { this.recordWriter = recordWriter; this.outputCommitter = outputCommitter; } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java index 49ff7441c167..9afd55c35af2 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java @@ -85,9 +85,7 @@ import org.apache.hadoop.hive.metastore.api.UnknownDBException; import org.apache.hadoop.hive.ql.io.StorageFormatDescriptor; import org.apache.hadoop.hive.ql.io.StorageFormatFactory; -import org.apache.hadoop.hive.serde2.SerDeException; import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe; -import org.apache.hive.common.util.HiveStringUtils; import org.apache.thrift.TException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -525,12 +523,7 @@ private CatalogBaseTable instantiateCatalogTable(Table hiveTable, HiveConf hiveC fields = hiveTable.getSd().getCols(); } else { // get schema from deserializer - try { - fields = HiveReflectionUtils.getFieldsFromDeserializer(hiveShim, hiveTable.getTableName(), - HiveReflectionUtils.getDeserializer(hiveShim, hiveConf, hiveTable, true)); - } catch (SerDeException | MetaException e) { - throw new CatalogException("Failed to get Hive table schema from deserializer", e); - } + fields = hiveShim.getFieldsFromDeserializer(hiveConf, hiveTable, true); } TableSchema tableSchema = HiveTableUtil.createTableSchema(fields, hiveTable.getPartitionKeys()); @@ -1099,7 +1092,7 @@ private static Function instantiateHiveFunction(ObjectPath functionPath, Catalog return new Function( // due to https://issues.apache.org/jira/browse/HIVE-22053, we have to normalize function name ourselves - HiveStringUtils.normalizeIdentifier(functionPath.getObjectName()), + functionPath.getObjectName().trim().toLowerCase(), functionPath.getDatabaseName(), functionClassName, null, // Owner name diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShim.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShim.java index 2111099afb83..1e27fb6631e7 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShim.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShim.java @@ -27,6 +27,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData; +import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.Function; import org.apache.hadoop.hive.metastore.api.InvalidOperationException; import org.apache.hadoop.hive.metastore.api.MetaException; @@ -34,15 +35,19 @@ import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.api.UnknownDBException; +import org.apache.hadoop.hive.ql.exec.FileSinkOperator; import org.apache.hadoop.hive.ql.udf.generic.SimpleGenericUDAFParameterInfo; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.JobConf; import org.apache.thrift.TException; import java.io.IOException; import java.io.Serializable; import java.util.List; import java.util.Map; +import java.util.Properties; /** * A shim layer to support different versions of Hive. @@ -184,4 +189,15 @@ SimpleGenericUDAFParameterInfo createUDAFParameterInfo(ObjectInspector[] params, */ CatalogColumnStatisticsDataDate toFlinkDateColStats(ColumnStatisticsData hiveDateColStats); + /** + * Get Hive's FileSinkOperator.RecordWriter. + */ + FileSinkOperator.RecordWriter getHiveRecordWriter(JobConf jobConf, String outputFormatClzName, + Class outValClz, boolean isCompressed, Properties tableProps, Path outPath); + + /** + * Get Hive table schema from deserializer. + */ + List getFieldsFromDeserializer(Configuration conf, Table table, boolean skipConfError); + } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimLoader.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimLoader.java index 1645d081a113..bd4b414daaa2 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimLoader.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimLoader.java @@ -32,6 +32,8 @@ */ public class HiveShimLoader { + public static final String HIVE_VERSION_V1_0_0 = "1.0.0"; + public static final String HIVE_VERSION_V1_0_1 = "1.0.1"; public static final String HIVE_VERSION_V1_1_0 = "1.1.0"; public static final String HIVE_VERSION_V1_1_1 = "1.1.1"; public static final String HIVE_VERSION_V1_2_0 = "1.2.0"; @@ -62,6 +64,12 @@ private HiveShimLoader() { public static HiveShim loadHiveShim(String version) { return hiveShims.computeIfAbsent(version, (v) -> { + if (v.startsWith(HIVE_VERSION_V1_0_0)) { + return new HiveShimV100(); + } + if (v.startsWith(HIVE_VERSION_V1_0_1)) { + return new HiveShimV101(); + } if (v.startsWith(HIVE_VERSION_V1_1_0)) { return new HiveShimV110(); } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV100.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV100.java new file mode 100644 index 000000000000..4afef67745b5 --- /dev/null +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV100.java @@ -0,0 +1,338 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.catalog.hive.client; + +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.hive.util.HiveReflectionUtils; +import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataDate; +import org.apache.flink.table.functions.hive.FlinkHiveUDFException; +import org.apache.flink.table.functions.hive.conversion.HiveInspectors; +import org.apache.flink.util.Preconditions; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.FileUtils; +import org.apache.hadoop.hive.common.HiveStatsUtils; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.Warehouse; +import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Function; +import org.apache.hadoop.hive.metastore.api.InvalidOperationException; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.api.UnknownDBException; +import org.apache.hadoop.hive.ql.exec.FileSinkOperator; +import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils; +import org.apache.hadoop.hive.ql.io.HiveOutputFormat; +import org.apache.hadoop.hive.ql.udf.generic.SimpleGenericUDAFParameterInfo; +import org.apache.hadoop.hive.serde2.Deserializer; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantBinaryObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantBooleanObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantByteObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantDateObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantDoubleObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantFloatObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantHiveCharObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantHiveDecimalObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantHiveVarcharObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantIntObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantLongObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantShortObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantStringObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantTimestampObjectInspector; +import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.Reporter; +import org.apache.thrift.TException; + +import java.io.IOException; +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** + * Shim for Hive version 1.0.0. + */ +public class HiveShimV100 implements HiveShim { + + @Override + public IMetaStoreClient getHiveMetastoreClient(HiveConf hiveConf) { + try { + return new HiveMetaStoreClient(hiveConf); + } catch (MetaException ex) { + throw new CatalogException("Failed to create Hive Metastore client", ex); + } + } + + @Override + // 1.x client doesn't support filtering tables by type, so here we need to get all tables and filter by ourselves + public List getViews(IMetaStoreClient client, String databaseName) throws UnknownDBException, TException { + // We don't have to use reflection here because client.getAllTables(String) is supposed to be there for + // all versions. + List tableNames = client.getAllTables(databaseName); + List views = new ArrayList<>(); + for (String name : tableNames) { + Table table = client.getTable(databaseName, name); + String viewDef = table.getViewOriginalText(); + if (viewDef != null && !viewDef.isEmpty()) { + views.add(table.getTableName()); + } + } + return views; + } + + @Override + public Function getFunction(IMetaStoreClient client, String dbName, String functionName) throws NoSuchObjectException, TException { + try { + // hive-1.x doesn't throw NoSuchObjectException if function doesn't exist, instead it throws a MetaException + return client.getFunction(dbName, functionName); + } catch (MetaException e) { + // need to check the cause and message of this MetaException to decide whether it should actually be a NoSuchObjectException + if (e.getCause() instanceof NoSuchObjectException) { + throw (NoSuchObjectException) e.getCause(); + } + if (e.getMessage().startsWith(NoSuchObjectException.class.getSimpleName())) { + throw new NoSuchObjectException(e.getMessage()); + } + throw e; + } + } + + @Override + public boolean moveToTrash(FileSystem fs, Path path, Configuration conf, boolean purge) throws IOException { + try { + Method method = FileUtils.class.getDeclaredMethod("moveToTrash", FileSystem.class, Path.class, Configuration.class); + return (boolean) method.invoke(null, fs, path, conf); + } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) { + throw new IOException("Failed to move " + path + " to trash", e); + } + } + + @Override + public void alterTable(IMetaStoreClient client, String databaseName, String tableName, Table table) throws InvalidOperationException, MetaException, TException { + client.alter_table(databaseName, tableName, table); + } + + @Override + public void alterPartition(IMetaStoreClient client, String databaseName, String tableName, Partition partition) + throws InvalidOperationException, MetaException, TException { + String errorMsg = "Failed to alter partition for table %s in database %s"; + try { + Method method = client.getClass().getMethod("alter_partition", String.class, String.class, Partition.class); + method.invoke(client, databaseName, tableName, partition); + } catch (InvocationTargetException ite) { + Throwable targetEx = ite.getTargetException(); + if (targetEx instanceof TException) { + throw (TException) targetEx; + } else { + throw new CatalogException(String.format(errorMsg, tableName, databaseName), targetEx); + } + } catch (NoSuchMethodException | IllegalAccessException e) { + throw new CatalogException(String.format(errorMsg, tableName, databaseName), e); + } + } + + @Override + public SimpleGenericUDAFParameterInfo createUDAFParameterInfo(ObjectInspector[] params, boolean isWindowing, boolean distinct, boolean allColumns) { + try { + Constructor constructor = SimpleGenericUDAFParameterInfo.class.getConstructor(ObjectInspector[].class, + boolean.class, boolean.class); + return (SimpleGenericUDAFParameterInfo) constructor.newInstance(params, distinct, allColumns); + } catch (NoSuchMethodException | IllegalAccessException | InstantiationException | InvocationTargetException e) { + throw new CatalogException("Failed to create SimpleGenericUDAFParameterInfo", e); + } + } + + @Override + public Class getMetaStoreUtilsClass() { + try { + return Class.forName("org.apache.hadoop.hive.metastore.MetaStoreUtils"); + } catch (ClassNotFoundException e) { + throw new CatalogException("Failed to find class MetaStoreUtils", e); + } + } + + @Override + public Class getHiveMetaStoreUtilsClass() { + return getMetaStoreUtilsClass(); + } + + @Override + public Class getDateDataTypeClass() { + return java.sql.Date.class; + } + + @Override + public Class getTimestampDataTypeClass() { + return java.sql.Timestamp.class; + } + + @Override + public FileStatus[] getFileStatusRecurse(Path path, int level, FileSystem fs) throws IOException { + try { + Method method = HiveStatsUtils.class.getMethod("getFileStatusRecurse", Path.class, Integer.TYPE, FileSystem.class); + // getFileStatusRecurse is a static method + return (FileStatus[]) method.invoke(null, path, level, fs); + } catch (Exception ex) { + throw new CatalogException("Failed to invoke HiveStatsUtils.getFileStatusRecurse()", ex); + } + } + + @Override + public void makeSpecFromName(Map partSpec, Path currPath) { + try { + Method method = Warehouse.class.getMethod("makeSpecFromName", Map.class, Path.class); + // makeSpecFromName is a static method + method.invoke(null, partSpec, currPath); + } catch (Exception ex) { + throw new CatalogException("Failed to invoke Warehouse.makeSpecFromName()", ex); + } + } + + @Override + public ObjectInspector getObjectInspectorForConstant(PrimitiveTypeInfo primitiveTypeInfo, Object value) { + String className; + value = HiveInspectors.hivePrimitiveToWritable(value); + // Java constant object inspectors are not available until 1.2.0 -- https://issues.apache.org/jira/browse/HIVE-9766 + // So we have to use writable constant object inspectors for 1.1.x + switch (primitiveTypeInfo.getPrimitiveCategory()) { + case BOOLEAN: + className = WritableConstantBooleanObjectInspector.class.getName(); + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case BYTE: + className = WritableConstantByteObjectInspector.class.getName(); + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case SHORT: + className = WritableConstantShortObjectInspector.class.getName(); + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case INT: + className = WritableConstantIntObjectInspector.class.getName(); + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case LONG: + className = WritableConstantLongObjectInspector.class.getName(); + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case FLOAT: + className = WritableConstantFloatObjectInspector.class.getName(); + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case DOUBLE: + className = WritableConstantDoubleObjectInspector.class.getName(); + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case STRING: + className = WritableConstantStringObjectInspector.class.getName(); + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case CHAR: + className = WritableConstantHiveCharObjectInspector.class.getName(); + try { + return (ObjectInspector) Class.forName(className).getDeclaredConstructor( + CharTypeInfo.class, value.getClass()).newInstance(primitiveTypeInfo, value); + } catch (Exception e) { + throw new FlinkHiveUDFException("Failed to create writable constant object inspector", e); + } + case VARCHAR: + className = WritableConstantHiveVarcharObjectInspector.class.getName(); + try { + return (ObjectInspector) Class.forName(className).getDeclaredConstructor( + VarcharTypeInfo.class, value.getClass()).newInstance(primitiveTypeInfo, value); + } catch (Exception e) { + throw new FlinkHiveUDFException("Failed to create writable constant object inspector", e); + } + case DATE: + className = WritableConstantDateObjectInspector.class.getName(); + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case TIMESTAMP: + className = WritableConstantTimestampObjectInspector.class.getName(); + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case DECIMAL: + className = WritableConstantHiveDecimalObjectInspector.class.getName(); + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case BINARY: + className = WritableConstantBinaryObjectInspector.class.getName(); + return HiveReflectionUtils.createConstantObjectInspector(className, value); + case UNKNOWN: + case VOID: + // If type is null, we use the Constant String to replace + className = WritableConstantStringObjectInspector.class.getName(); + return HiveReflectionUtils.createConstantObjectInspector(className, value.toString()); + default: + throw new FlinkHiveUDFException( + String.format("Cannot find ConstantObjectInspector for %s", primitiveTypeInfo)); + } + } + + @Override + public ColumnStatisticsData toHiveDateColStats(CatalogColumnStatisticsDataDate flinkDateColStats) { + throw new UnsupportedOperationException("DATE column stats are not supported until Hive 1.2.0"); + } + + @Override + public boolean isDateStats(ColumnStatisticsData colStatsData) { + return false; + } + + @Override + public CatalogColumnStatisticsDataDate toFlinkDateColStats(ColumnStatisticsData hiveDateColStats) { + throw new UnsupportedOperationException("DATE column stats are not supported until Hive 1.2.0"); + } + + @Override + public FileSinkOperator.RecordWriter getHiveRecordWriter(JobConf jobConf, String outputFormatClzName, + Class outValClz, boolean isCompressed, Properties tableProps, Path outPath) { + try { + Class outputFormatClz = Class.forName(outputFormatClzName); + Class utilClass = HiveFileFormatUtils.class; + Method utilMethod = utilClass.getDeclaredMethod("getOutputFormatSubstitute", Class.class, boolean.class); + outputFormatClz = (Class) utilMethod.invoke(null, outputFormatClz, false); + Preconditions.checkState(outputFormatClz != null, "No Hive substitute output format for " + outputFormatClzName); + HiveOutputFormat outputFormat = (HiveOutputFormat) outputFormatClz.newInstance(); + utilMethod = utilClass.getDeclaredMethod("getRecordWriter", JobConf.class, HiveOutputFormat.class, + Class.class, boolean.class, Properties.class, Path.class, Reporter.class); + return (FileSinkOperator.RecordWriter) utilMethod.invoke(null, + jobConf, outputFormat, outValClz, isCompressed, tableProps, outPath, Reporter.NULL); + } catch (Exception e) { + throw new CatalogException("Failed to create Hive RecordWriter", e); + } + } + + @Override + public List getFieldsFromDeserializer(Configuration conf, Table table, boolean skipConfError) { + try { + Method utilMethod = getHiveMetaStoreUtilsClass().getMethod("getDeserializer", Configuration.class, Table.class); + Deserializer deserializer = (Deserializer) utilMethod.invoke(null, conf, table); + utilMethod = getHiveMetaStoreUtilsClass().getMethod("getFieldsFromDeserializer", String.class, Deserializer.class); + return (List) utilMethod.invoke(null, table.getTableName(), deserializer); + } catch (Exception e) { + throw new CatalogException("Failed to get table schema from deserializer", e); + } + } +} diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV101.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV101.java new file mode 100644 index 000000000000..cf8690579276 --- /dev/null +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV101.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.catalog.hive.client; + +/** + * Shim for Hive version 1.0.1. + */ +public class HiveShimV101 extends HiveShimV100 { +} diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV110.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV110.java index 91fd80af9a9c..691c14a5d083 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV110.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV110.java @@ -19,279 +19,58 @@ package org.apache.flink.table.catalog.hive.client; import org.apache.flink.table.catalog.exceptions.CatalogException; -import org.apache.flink.table.catalog.hive.util.HiveReflectionUtils; -import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataDate; -import org.apache.flink.table.functions.hive.FlinkHiveUDFException; -import org.apache.flink.table.functions.hive.conversion.HiveInspectors; +import org.apache.flink.util.Preconditions; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.common.FileUtils; -import org.apache.hadoop.hive.common.HiveStatsUtils; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.Warehouse; -import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData; -import org.apache.hadoop.hive.metastore.api.Function; -import org.apache.hadoop.hive.metastore.api.InvalidOperationException; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; -import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.api.UnknownDBException; -import org.apache.hadoop.hive.ql.udf.generic.SimpleGenericUDAFParameterInfo; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantBinaryObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantBooleanObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantByteObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantDateObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantDoubleObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantFloatObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantHiveCharObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantHiveDecimalObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantHiveVarcharObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantIntObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantLongObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantShortObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantStringObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.WritableConstantTimestampObjectInspector; -import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo; -import org.apache.thrift.TException; +import org.apache.hadoop.hive.ql.exec.FileSinkOperator; +import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils; +import org.apache.hadoop.hive.serde2.Deserializer; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.OutputFormat; +import org.apache.hadoop.mapred.Reporter; -import java.io.IOException; -import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; -import java.util.ArrayList; import java.util.List; -import java.util.Map; +import java.util.Properties; /** * Shim for Hive version 1.1.0. */ -public class HiveShimV110 implements HiveShim { +public class HiveShimV110 extends HiveShimV101 { @Override - public IMetaStoreClient getHiveMetastoreClient(HiveConf hiveConf) { + public FileSinkOperator.RecordWriter getHiveRecordWriter(JobConf jobConf, String outputFormatClzName, + Class outValClz, boolean isCompressed, Properties tableProps, Path outPath) { try { - return new HiveMetaStoreClient(hiveConf); - } catch (MetaException ex) { - throw new CatalogException("Failed to create Hive Metastore client", ex); + Class outputFormatClz = Class.forName(outputFormatClzName); + Class utilClass = HiveFileFormatUtils.class; + Method utilMethod = utilClass.getDeclaredMethod("getOutputFormatSubstitute", Class.class); + outputFormatClz = (Class) utilMethod.invoke(null, outputFormatClz); + Preconditions.checkState(outputFormatClz != null, "No Hive substitute output format for " + outputFormatClzName); + OutputFormat outputFormat = (OutputFormat) outputFormatClz.newInstance(); + utilMethod = utilClass.getDeclaredMethod("getRecordWriter", JobConf.class, OutputFormat.class, + Class.class, boolean.class, Properties.class, Path.class, Reporter.class); + return (FileSinkOperator.RecordWriter) utilMethod.invoke(null, + jobConf, outputFormat, outValClz, isCompressed, tableProps, outPath, Reporter.NULL); + } catch (Exception e) { + throw new CatalogException("Failed to create Hive RecordWriter", e); } } @Override - // 1.x client doesn't support filtering tables by type, so here we need to get all tables and filter by ourselves - public List getViews(IMetaStoreClient client, String databaseName) throws UnknownDBException, TException { - // We don't have to use reflection here because client.getAllTables(String) is supposed to be there for - // all versions. - List tableNames = client.getAllTables(databaseName); - List views = new ArrayList<>(); - for (String name : tableNames) { - Table table = client.getTable(databaseName, name); - String viewDef = table.getViewOriginalText(); - if (viewDef != null && !viewDef.isEmpty()) { - views.add(table.getTableName()); - } - } - return views; - } - - @Override - public Function getFunction(IMetaStoreClient client, String dbName, String functionName) throws NoSuchObjectException, TException { - try { - // hive-1.x doesn't throw NoSuchObjectException if function doesn't exist, instead it throws a MetaException - return client.getFunction(dbName, functionName); - } catch (MetaException e) { - // need to check the cause and message of this MetaException to decide whether it should actually be a NoSuchObjectException - if (e.getCause() instanceof NoSuchObjectException) { - throw (NoSuchObjectException) e.getCause(); - } - if (e.getMessage().startsWith(NoSuchObjectException.class.getSimpleName())) { - throw new NoSuchObjectException(e.getMessage()); - } - throw e; - } - } - - @Override - public boolean moveToTrash(FileSystem fs, Path path, Configuration conf, boolean purge) throws IOException { - try { - Method method = FileUtils.class.getDeclaredMethod("moveToTrash", FileSystem.class, Path.class, Configuration.class); - return (boolean) method.invoke(null, fs, path, conf); - } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) { - throw new IOException("Failed to move " + path + " to trash", e); - } - } - - @Override - public void alterTable(IMetaStoreClient client, String databaseName, String tableName, Table table) throws InvalidOperationException, MetaException, TException { - client.alter_table(databaseName, tableName, table); - } - - @Override - public void alterPartition(IMetaStoreClient client, String databaseName, String tableName, Partition partition) - throws InvalidOperationException, MetaException, TException { - String errorMsg = "Failed to alter partition for table %s in database %s"; - try { - Method method = client.getClass().getMethod("alter_partition", String.class, String.class, Partition.class); - method.invoke(client, databaseName, tableName, partition); - } catch (InvocationTargetException ite) { - Throwable targetEx = ite.getTargetException(); - if (targetEx instanceof TException) { - throw (TException) targetEx; - } else { - throw new CatalogException(String.format(errorMsg, tableName, databaseName), targetEx); - } - } catch (NoSuchMethodException | IllegalAccessException e) { - throw new CatalogException(String.format(errorMsg, tableName, databaseName), e); - } - } - - @Override - public SimpleGenericUDAFParameterInfo createUDAFParameterInfo(ObjectInspector[] params, boolean isWindowing, boolean distinct, boolean allColumns) { + public List getFieldsFromDeserializer(Configuration conf, Table table, boolean skipConfError) { try { - Constructor constructor = SimpleGenericUDAFParameterInfo.class.getConstructor(ObjectInspector[].class, - boolean.class, boolean.class); - return (SimpleGenericUDAFParameterInfo) constructor.newInstance(params, distinct, allColumns); - } catch (NoSuchMethodException | IllegalAccessException | InstantiationException | InvocationTargetException e) { - throw new CatalogException("Failed to create SimpleGenericUDAFParameterInfo", e); + Method utilMethod = getHiveMetaStoreUtilsClass().getMethod("getDeserializer", + Configuration.class, Table.class, boolean.class); + Deserializer deserializer = (Deserializer) utilMethod.invoke(null, conf, table, skipConfError); + utilMethod = getHiveMetaStoreUtilsClass().getMethod("getFieldsFromDeserializer", String.class, Deserializer.class); + return (List) utilMethod.invoke(null, table.getTableName(), deserializer); + } catch (Exception e) { + throw new CatalogException("Failed to get table schema from deserializer", e); } } - - @Override - public Class getMetaStoreUtilsClass() { - try { - return Class.forName("org.apache.hadoop.hive.metastore.MetaStoreUtils"); - } catch (ClassNotFoundException e) { - throw new CatalogException("Failed to find class MetaStoreUtils", e); - } - } - - @Override - public Class getHiveMetaStoreUtilsClass() { - return getMetaStoreUtilsClass(); - } - - @Override - public Class getDateDataTypeClass() { - return java.sql.Date.class; - } - - @Override - public Class getTimestampDataTypeClass() { - return java.sql.Timestamp.class; - } - - @Override - public FileStatus[] getFileStatusRecurse(Path path, int level, FileSystem fs) throws IOException { - try { - Method method = HiveStatsUtils.class.getMethod("getFileStatusRecurse", Path.class, Integer.TYPE, FileSystem.class); - // getFileStatusRecurse is a static method - return (FileStatus[]) method.invoke(null, path, level, fs); - } catch (Exception ex) { - throw new CatalogException("Failed to invoke HiveStatsUtils.getFileStatusRecurse()", ex); - } - } - - @Override - public void makeSpecFromName(Map partSpec, Path currPath) { - try { - Method method = Warehouse.class.getMethod("makeSpecFromName", Map.class, Path.class); - // makeSpecFromName is a static method - method.invoke(null, partSpec, currPath); - } catch (Exception ex) { - throw new CatalogException("Failed to invoke Warehouse.makeSpecFromName()", ex); - } - } - - @Override - public ObjectInspector getObjectInspectorForConstant(PrimitiveTypeInfo primitiveTypeInfo, Object value) { - String className; - value = HiveInspectors.hivePrimitiveToWritable(value); - // Java constant object inspectors are not available until 1.2.0 -- https://issues.apache.org/jira/browse/HIVE-9766 - // So we have to use writable constant object inspectors for 1.1.x - switch (primitiveTypeInfo.getPrimitiveCategory()) { - case BOOLEAN: - className = WritableConstantBooleanObjectInspector.class.getName(); - return HiveReflectionUtils.createConstantObjectInspector(className, value); - case BYTE: - className = WritableConstantByteObjectInspector.class.getName(); - return HiveReflectionUtils.createConstantObjectInspector(className, value); - case SHORT: - className = WritableConstantShortObjectInspector.class.getName(); - return HiveReflectionUtils.createConstantObjectInspector(className, value); - case INT: - className = WritableConstantIntObjectInspector.class.getName(); - return HiveReflectionUtils.createConstantObjectInspector(className, value); - case LONG: - className = WritableConstantLongObjectInspector.class.getName(); - return HiveReflectionUtils.createConstantObjectInspector(className, value); - case FLOAT: - className = WritableConstantFloatObjectInspector.class.getName(); - return HiveReflectionUtils.createConstantObjectInspector(className, value); - case DOUBLE: - className = WritableConstantDoubleObjectInspector.class.getName(); - return HiveReflectionUtils.createConstantObjectInspector(className, value); - case STRING: - className = WritableConstantStringObjectInspector.class.getName(); - return HiveReflectionUtils.createConstantObjectInspector(className, value); - case CHAR: - className = WritableConstantHiveCharObjectInspector.class.getName(); - try { - return (ObjectInspector) Class.forName(className).getDeclaredConstructor( - CharTypeInfo.class, value.getClass()).newInstance(primitiveTypeInfo, value); - } catch (Exception e) { - throw new FlinkHiveUDFException("Failed to create writable constant object inspector", e); - } - case VARCHAR: - className = WritableConstantHiveVarcharObjectInspector.class.getName(); - try { - return (ObjectInspector) Class.forName(className).getDeclaredConstructor( - VarcharTypeInfo.class, value.getClass()).newInstance(primitiveTypeInfo, value); - } catch (Exception e) { - throw new FlinkHiveUDFException("Failed to create writable constant object inspector", e); - } - case DATE: - className = WritableConstantDateObjectInspector.class.getName(); - return HiveReflectionUtils.createConstantObjectInspector(className, value); - case TIMESTAMP: - className = WritableConstantTimestampObjectInspector.class.getName(); - return HiveReflectionUtils.createConstantObjectInspector(className, value); - case DECIMAL: - className = WritableConstantHiveDecimalObjectInspector.class.getName(); - return HiveReflectionUtils.createConstantObjectInspector(className, value); - case BINARY: - className = WritableConstantBinaryObjectInspector.class.getName(); - return HiveReflectionUtils.createConstantObjectInspector(className, value); - case UNKNOWN: - case VOID: - // If type is null, we use the Constant String to replace - className = WritableConstantStringObjectInspector.class.getName(); - return HiveReflectionUtils.createConstantObjectInspector(className, value.toString()); - default: - throw new FlinkHiveUDFException( - String.format("Cannot find ConstantObjectInspector for %s", primitiveTypeInfo)); - } - } - - @Override - public ColumnStatisticsData toHiveDateColStats(CatalogColumnStatisticsDataDate flinkDateColStats) { - throw new UnsupportedOperationException("DATE column stats are not supported until Hive 1.2.0"); - } - - @Override - public boolean isDateStats(ColumnStatisticsData colStatsData) { - return false; - } - - @Override - public CatalogColumnStatisticsDataDate toFlinkDateColStats(ColumnStatisticsData hiveDateColStats) { - throw new UnsupportedOperationException("DATE column stats are not supported until Hive 1.2.0"); - } } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveReflectionUtils.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveReflectionUtils.java index b03eb66ff896..5224b781c446 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveReflectionUtils.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveReflectionUtils.java @@ -22,12 +22,8 @@ import org.apache.flink.table.catalog.hive.client.HiveShim; import org.apache.flink.table.functions.hive.FlinkHiveUDFException; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.serde2.Deserializer; -import org.apache.hadoop.hive.serde2.SerDeException; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import java.lang.reflect.Constructor; @@ -57,27 +53,6 @@ public static Properties getTableMetadata(HiveShim hiveShim, Table table) { } } - public static List getFieldsFromDeserializer(HiveShim hiveShim, String tableName, Deserializer deserializer) - throws SerDeException, MetaException { - try { - Method method = hiveShim.getHiveMetaStoreUtilsClass().getMethod("getFieldsFromDeserializer", String.class, Deserializer.class); - return (List) method.invoke(null, tableName, deserializer); - } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) { - throw new CatalogException("Failed to invoke MetaStoreUtils.getFieldsFromDeserializer()", e); - } - } - - public static Deserializer getDeserializer(HiveShim hiveShim, Configuration conf, Table table, boolean skipConfError) - throws MetaException { - try { - Method method = hiveShim.getHiveMetaStoreUtilsClass().getMethod("getDeserializer", Configuration.class, - Table.class, boolean.class); - return (Deserializer) method.invoke(null, conf, table, skipConfError); - } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) { - throw new CatalogException("Failed to invoke MetaStoreUtils.getDeserializer()", e); - } - } - public static List getPvals(HiveShim hiveShim, List partCols, Map partSpec) { try { Method method = hiveShim.getMetaStoreUtilsClass().getMethod("getPvals", List.class, Map.class); diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerShimLoader.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerShimLoader.java index 2a344d1b8805..590d09dbc0aa 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerShimLoader.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerShimLoader.java @@ -37,6 +37,8 @@ public static HiveRunnerShim load() { String hiveVersion = HiveShimLoader.getHiveVersion(); return hiveRunnerShims.computeIfAbsent(hiveVersion, v -> { switch (v) { + case HiveShimLoader.HIVE_VERSION_V1_0_0: + case HiveShimLoader.HIVE_VERSION_V1_0_1: case HiveShimLoader.HIVE_VERSION_V1_1_0: case HiveShimLoader.HIVE_VERSION_V1_1_1: case HiveShimLoader.HIVE_VERSION_V1_2_0: diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java index 2c50bef67884..ce00b23f066c 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java @@ -141,7 +141,8 @@ private void readWriteFormat(String format) throws Exception { hiveShell.execute("create table db1.dest (i int,s string) " + suffix); // prepare source data with Hive - hiveShell.execute("insert into db1.src values (1,'a'),(2,'b')"); + // TABLE keyword in INSERT INTO is mandatory prior to 1.1.0 + hiveShell.execute("insert into table db1.src values (1,'a'),(2,'b')"); // populate dest table with source table tableEnv.sqlUpdate("insert into db1.dest select * from db1.src"); @@ -161,7 +162,8 @@ public void testDecimal() throws Exception { hiveShell.execute("create table db1.src2 (x decimal(10,2))"); hiveShell.execute("create table db1.dest (x decimal(10,2))"); // populate src1 from Hive - hiveShell.execute("insert into db1.src1 values (1.0),(2.12),(5.123),(5.456),(123456789.12)"); + // TABLE keyword in INSERT INTO is mandatory prior to 1.1.0 + hiveShell.execute("insert into table db1.src1 values (1.0),(2.12),(5.123),(5.456),(123456789.12)"); TableEnvironment tableEnv = getTableEnvWithHiveCatalog(); // populate src2 with same data from Flink diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/functions/hive/HiveGenericUDFTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/functions/hive/HiveGenericUDFTest.java index f7c74dcd58b3..fc8c0b5c0893 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/functions/hive/HiveGenericUDFTest.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/functions/hive/HiveGenericUDFTest.java @@ -29,7 +29,6 @@ import org.apache.hadoop.hive.ql.udf.UDFUnhex; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFAbs; -import org.apache.hadoop.hive.ql.udf.generic.GenericUDFAddMonths; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFCase; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFCeil; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFCoalesce; @@ -53,6 +52,7 @@ public class HiveGenericUDFTest { private static HiveShim hiveShim = HiveShimLoader.loadHiveShim(HiveShimLoader.getHiveVersion()); private static final boolean HIVE_120_OR_LATER = HiveShimLoader.getHiveVersion().compareTo(HiveShimLoader.HIVE_VERSION_V1_2_0) >= 0; + private static final boolean HIVE_110_OR_LATER = HiveShimLoader.getHiveVersion().compareTo(HiveShimLoader.HIVE_VERSION_V1_1_0) >= 0; @Test public void testAbs() { @@ -94,9 +94,10 @@ public void testAbs() { } @Test - public void testAddMonths() { + public void testAddMonths() throws Exception { + Assume.assumeTrue(HIVE_110_OR_LATER); HiveGenericUDF udf = init( - GenericUDFAddMonths.class, + Class.forName("org.apache.hadoop.hive.ql.udf.generic.GenericUDFAddMonths"), new Object[] { null, 1 From 0828e2942af41cdfe8356ce46cd7f1540f981a5d Mon Sep 17 00:00:00 2001 From: Nicolas Deslandes Date: Tue, 29 Oct 2019 10:41:59 -0400 Subject: [PATCH 423/746] [FLINK-14562] Let RabbitMQ source close consumer and channel on close Closing method of RabbitMQ source must close consumer and channel in order to prevent leaving idle consumer This closes #10036. --- .../connectors/rabbitmq/RMQSource.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java b/flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java index d4541533f477..f079369e9248 100644 --- a/flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java +++ b/flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java @@ -177,6 +177,25 @@ public void open(Configuration config) throws Exception { @Override public void close() throws Exception { super.close(); + + try { + if (consumer != null && channel != null) { + channel.basicCancel(consumer.getConsumerTag()); + } + } catch (IOException e) { + throw new RuntimeException("Error while cancelling RMQ consumer on " + queueName + + " at " + rmqConnectionConfig.getHost(), e); + } + + try { + if (channel != null) { + channel.close(); + } + } catch (IOException e) { + throw new RuntimeException("Error while closing RMQ channel with " + queueName + + " at " + rmqConnectionConfig.getHost(), e); + } + try { if (connection != null) { connection.close(); From ba068d5657f5037ebaa1a4e59491afb89f03f74e Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Sun, 3 Nov 2019 15:12:27 +0800 Subject: [PATCH 424/746] [FLINK-14536][core] Sum the cpuCores when merging resource specs This closes #10072. --- .../org/apache/flink/api/common/operators/ResourceSpec.java | 2 +- .../org/apache/flink/api/common/operators/ResourceSpecTest.java | 2 ++ .../streaming/api/graph/StreamingJobGraphGeneratorTest.java | 2 +- 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/ResourceSpec.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/ResourceSpec.java index 2d56746ed5a5..6b450fb92bad 100755 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/ResourceSpec.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/ResourceSpec.java @@ -151,7 +151,7 @@ public ResourceSpec merge(ResourceSpec other) { } ResourceSpec target = new ResourceSpec( - Math.max(this.cpuCores, other.cpuCores), + this.cpuCores + other.cpuCores, this.heapMemoryInMB + other.heapMemoryInMB, this.directMemoryInMB + other.directMemoryInMB, this.nativeMemoryInMB + other.nativeMemoryInMB, diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/ResourceSpecTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/ResourceSpecTest.java index b00be4986329..e232f42486d3 100755 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/ResourceSpecTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/ResourceSpecTest.java @@ -141,6 +141,8 @@ public void testMerge() throws Exception { ResourceSpec rs2 = ResourceSpec.newBuilder().setCpuCores(1.0).setHeapMemoryInMB(100).build(); ResourceSpec rs3 = rs1.merge(rs2); + assertEquals(2.0, rs3.getCpuCores(), 0.000001); + assertEquals(200, rs3.getHeapMemory()); assertEquals(1.1, rs3.getGPUResource(), 0.000001); ResourceSpec rs4 = rs1.merge(rs3); 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 18b25f793694..ab25e5bbc50e 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 @@ -275,7 +275,7 @@ public void invoke(Tuple2 value) throws Exception { JobVertex sourceMapFilterVertex = jobGraph.getVerticesSortedTopologicallyFromSources().get(0); JobVertex reduceSinkVertex = jobGraph.getVerticesSortedTopologicallyFromSources().get(1); - assertTrue(sourceMapFilterVertex.getMinResources().equals(resource1.merge(resource2).merge(resource3))); + assertTrue(sourceMapFilterVertex.getMinResources().equals(resource3.merge(resource2).merge(resource1))); assertTrue(reduceSinkVertex.getPreferredResources().equals(resource4.merge(resource5))); } From a96b3dadb38a1b24d18efbcd6a6e5879121555f0 Mon Sep 17 00:00:00 2001 From: Leonard Xu Date: Mon, 4 Nov 2019 12:31:41 +0800 Subject: [PATCH 425/746] [FLINK-14549][table-planner-blink] Improve exception message when schema is not matched between query and sink This closes #10027 --- .../table/planner/sinks/TableSinkUtils.scala | 19 +++++----- .../validation/TableSinkValidationTest.scala | 35 ++++++++++++++++++- 2 files changed, 45 insertions(+), 9 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/TableSinkUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/TableSinkUtils.scala index 07e59ae446e8..8d8a578787df 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/TableSinkUtils.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/TableSinkUtils.scala @@ -50,21 +50,24 @@ object TableSinkUtils { val srcFieldTypes = query.getTableSchema.getFieldDataTypes val sinkFieldTypes = sink.getTableSchema.getFieldDataTypes - if (srcFieldTypes.length != sinkFieldTypes.length || - srcFieldTypes.zip(sinkFieldTypes).exists { case (srcF, snkF) => - !PlannerTypeUtils.isInteroperable( - fromDataTypeToLogicalType(srcF), fromDataTypeToLogicalType(snkF)) + val srcLogicalTypes = srcFieldTypes.map(t => fromDataTypeToLogicalType(t)) + val sinkLogicalTypes = sinkFieldTypes.map(t => fromDataTypeToLogicalType(t)) + + if (srcLogicalTypes.length != sinkLogicalTypes.length || + srcLogicalTypes.zip(sinkLogicalTypes).exists { + case (srcType, sinkType) => + !PlannerTypeUtils.isInteroperable(srcType, sinkType) }) { val srcFieldNames = query.getTableSchema.getFieldNames val sinkFieldNames = sink.getTableSchema.getFieldNames // format table and table sink schema strings - val srcSchema = srcFieldNames.zip(srcFieldTypes) - .map { case (n, t) => s"$n: ${t.getConversionClass.getSimpleName}" } + val srcSchema = srcFieldNames.zip(srcLogicalTypes) + .map { case (n, t) => s"$n: $t" } .mkString("[", ", ", "]") - val sinkSchema = sinkFieldNames.zip(sinkFieldTypes) - .map { case (n, t) => s"$n: ${t.getConversionClass.getSimpleName}" } + val sinkSchema = sinkFieldNames.zip(sinkLogicalTypes) + .map { case (n, t) => s"$n: $t" } .mkString("[", ", ", "]") throw new ValidationException( diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/TableSinkValidationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/TableSinkValidationTest.scala index 1b8cc44c1f40..27b026933196 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/TableSinkValidationTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/TableSinkValidationTest.scala @@ -22,8 +22,9 @@ import org.apache.flink.api.scala._ import org.apache.flink.streaming.api.TimeCharacteristic import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment import org.apache.flink.table.api.scala._ -import org.apache.flink.table.api.{TableException, ValidationException} +import org.apache.flink.table.api.{TableException, ValidationException, DataTypes, TableSchema} import org.apache.flink.table.planner.runtime.utils.{TestData, TestingAppendSink, TestingUpsertTableSink} +import org.apache.flink.table.planner.utils.MemoryTableSourceSinkUtil.DataTypeOutputFormatTableSink import org.apache.flink.table.planner.utils.{TableTestBase, TableTestUtil} import org.apache.flink.types.Row import org.junit.Test @@ -83,4 +84,36 @@ class TableSinkValidationTest extends TableTestBase { // must fail because table is not append-only env.execute() } + + @Test + def testValidateSink(): Unit = { + expectedException.expect(classOf[ValidationException]) + expectedException.expectMessage( + "Field types of query result and registered TableSink `default_catalog`." + + "`default_database`.`testSink` do not match.\n" + + "Query result schema: [a: INT, b: BIGINT, c: STRING, d: DECIMAL(10, 8)]\n" + + "TableSink schema: [a: INT, b: BIGINT, c: STRING, d: DECIMAL(10, 7)]") + + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = StreamTableEnvironment.create(env, TableTestUtil.STREAM_SETTING) + + val sourceTable = env.fromCollection(TestData.tupleData3).toTable(tEnv, 'a, 'b, 'c) + tEnv.registerTable("source", sourceTable) + val resultTable = tEnv.sqlQuery("select a, b, c, cast(b as decimal(10, 8)) as d from source") + + val sinkSchema = TableSchema.builder() + .field("a", DataTypes.INT()) + .field("b", DataTypes.BIGINT()) + .field("c", DataTypes.STRING()) + .field("d", DataTypes.DECIMAL(10, 7)) + .build() + val sink = new DataTypeOutputFormatTableSink(sinkSchema) + tEnv.registerTableSink("testSink", sink) + + tEnv.insertInto(resultTable, "testSink") + + // must fail because query result table schema is different with sink table schema + tEnv.execute("testJob") + } + } From 7ffe265f65becbca94cc0bb585297ee000ba735c Mon Sep 17 00:00:00 2001 From: caoyingjie Date: Mon, 4 Nov 2019 15:31:02 +0800 Subject: [PATCH 426/746] [FLINK-14498][runtime]Introduce NetworkBufferPool#isAvailable() for interacting with LocalBufferPool. (#9993) * [FLINK-14498][network] Introduce NetworkBufferPool#isAvailable() for non-blocking output In order to best-effort implement non-blocking output, we need to further improve the interaction between LocalBufferPool and NetworkBufferPool in non-blocking way as a supplementation of FLINK-14396. In detail, we provide the NetworkBufferPool#isAvailable to indicate the global pool state, then we could combine its state via LocalBufferPool#isAvailable method to avoid blocking in global request while task processing. Meanwhile we would refactor the process when LocalBufferPool requests global buffer. If there are no available buffers in NetworkBufferPool, the LocalBufferPool should monitor the global's available future instead of waiting 2 seconds currently in every loop retry. So we can solve the wait delay and cleanup the codes in a unified way. --- .../io/network/buffer/LocalBufferPool.java | 72 +++--- .../io/network/buffer/NetworkBufferPool.java | 74 +++++- .../BackPressureStatsTrackerImpl.java | 2 +- .../network/api/writer/RecordWriterTest.java | 3 +- .../buffer/LocalBufferPoolDestroyTest.java | 6 +- .../network/buffer/LocalBufferPoolTest.java | 41 ++++ .../network/buffer/NetworkBufferPoolTest.java | 217 +++++++++++++++++- 7 files changed, 365 insertions(+), 50 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java index 56055dbb52fe..6e4c8aee9b67 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java @@ -30,6 +30,7 @@ import java.io.IOException; import java.util.ArrayDeque; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import static org.apache.flink.util.Preconditions.checkArgument; @@ -156,8 +157,6 @@ class LocalBufferPool implements BufferPool { this.currentPoolSize = numberOfRequiredMemorySegments; this.maxNumberOfMemorySegments = maxNumberOfMemorySegments; this.bufferPoolOwner = bufferPoolOwner; - - availabilityHelper.resetAvailable(); } // ------------------------------------------------------------------------ @@ -202,17 +201,12 @@ public int bestEffortGetNumOfUsedBuffers() { @Override public Buffer requestBuffer() throws IOException { - try { - return toBuffer(requestMemorySegment(false)); - } - catch (InterruptedException e) { - throw new IOException(e); - } + return toBuffer(requestMemorySegment()); } @Override public BufferBuilder requestBufferBuilderBlocking() throws IOException, InterruptedException { - return toBufferBuilder(requestMemorySegment(true)); + return toBufferBuilder(requestMemorySegmentBlocking()); } private Buffer toBuffer(MemorySegment memorySegment) { @@ -229,27 +223,34 @@ private BufferBuilder toBufferBuilder(MemorySegment memorySegment) { return new BufferBuilder(memorySegment, this); } + private MemorySegment requestMemorySegmentBlocking() throws InterruptedException, IOException { + MemorySegment segment; + while ((segment = requestMemorySegment()) == null) { + try { + // wait until available + isAvailable().get(); + } catch (ExecutionException e) { + LOG.error("The available future is completed exceptionally.", e); + ExceptionUtils.rethrow(e); + } + } + return segment; + } + @Nullable - private MemorySegment requestMemorySegment(boolean isBlocking) throws InterruptedException, IOException { + private MemorySegment requestMemorySegment() throws IOException { MemorySegment segment = null; synchronized (availableMemorySegments) { returnExcessMemorySegments(); - // fill availableMemorySegments with at least one element, wait if required - while (availableMemorySegments.isEmpty()) { + if (availableMemorySegments.isEmpty()) { segment = requestMemorySegmentFromGlobal(); - if (segment != null || !isBlocking) { - break; - } - - availableMemorySegments.wait(2000); } - + // segment may have been released by buffer pool owner if (segment == null) { segment = availableMemorySegments.poll(); } - - if (isBlocking && isUnavailable()) { + if (segment == null) { availabilityHelper.resetUnavailable(); } } @@ -292,9 +293,8 @@ public void recycle(MemorySegment segment) { } else { listener = registeredListeners.poll(); if (listener == null) { - boolean wasUnavailable = isUnavailable(); + boolean wasUnavailable = availableMemorySegments.isEmpty(); availableMemorySegments.add(segment); - availableMemorySegments.notify(); if (wasUnavailable) { toNotify = availabilityHelper.getUnavailableToResetAvailable(); } @@ -328,21 +328,13 @@ private NotificationResult fireBufferAvailableNotification(BufferListener listen return notificationResult; } - /** - * @return true if there is no available buffers in queue and the global quota is also exhausted. - */ - private boolean isUnavailable() { - assert Thread.holdsLock(availableMemorySegments); - - return availableMemorySegments.isEmpty() && numberOfRequestedMemorySegments >= currentPoolSize; - } - /** * Destroy is called after the produce or consume phase of a task finishes. */ @Override public void lazyDestroy() { // NOTE: if you change this logic, be sure to update recycle() as well! + CompletableFuture toNotify = null; synchronized (availableMemorySegments) { if (!isDestroyed) { MemorySegment segment; @@ -355,10 +347,20 @@ public void lazyDestroy() { listener.notifyBufferDestroyed(); } + final CompletableFuture isAvailable = availabilityHelper.isAvailable(); + if (isAvailable != AVAILABLE && !isAvailable.isDone()) { + toNotify = isAvailable; + } + isDestroyed = true; } } + // notify the potential blocking request thread + if (toNotify != null) { + toNotify.complete(null); + } + try { networkBufferPool.destroyBufferPool(this); } catch (IOException e) { @@ -406,7 +408,13 @@ public void setNumBuffers(int numBuffers) throws IOException { @Override public CompletableFuture isAvailable() { - return availabilityHelper.isAvailable(); + if (numberOfRequestedMemorySegments >= currentPoolSize) { + return availabilityHelper.isAvailable(); + } else if (availabilityHelper.isAvailable() == AVAILABLE || networkBufferPool.isAvailable() == AVAILABLE) { + return AVAILABLE; + } else { + return CompletableFuture.anyOf(availabilityHelper.isAvailable(), networkBufferPool.isAvailable()); + } } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java index c08e056d4df5..4cf7fffae597 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java @@ -24,6 +24,7 @@ import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.core.memory.MemorySegmentFactory; import org.apache.flink.core.memory.MemorySegmentProvider; +import org.apache.flink.runtime.io.AvailabilityProvider; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.MathUtils; import org.apache.flink.util.Preconditions; @@ -35,13 +36,14 @@ import java.io.IOException; import java.time.Duration; +import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Set; -import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.TimeUnit; +import java.util.concurrent.CompletableFuture; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -54,7 +56,7 @@ * the buffers for the network data transfer. When new local buffer pools are created, the * NetworkBufferPool dynamically redistributes the buffers between the pools. */ -public class NetworkBufferPool implements BufferPoolFactory, MemorySegmentProvider { +public class NetworkBufferPool implements BufferPoolFactory, MemorySegmentProvider, AvailabilityProvider { private static final Logger LOG = LoggerFactory.getLogger(NetworkBufferPool.class); @@ -62,7 +64,7 @@ public class NetworkBufferPool implements BufferPoolFactory, MemorySegmentProvid private final int memorySegmentSize; - private final ArrayBlockingQueue availableMemorySegments; + private final ArrayDeque availableMemorySegments; private volatile boolean isDestroyed; @@ -78,6 +80,8 @@ public class NetworkBufferPool implements BufferPoolFactory, MemorySegmentProvid private final Duration requestSegmentsTimeout; + private final AvailabilityHelper availabilityHelper = new AvailabilityHelper(); + @VisibleForTesting public NetworkBufferPool(int numberOfSegmentsToAllocate, int segmentSize, int numberOfSegmentsToRequest) { this(numberOfSegmentsToAllocate, segmentSize, numberOfSegmentsToRequest, Duration.ofMillis(Integer.MAX_VALUE)); @@ -106,7 +110,7 @@ public NetworkBufferPool( final long sizeInLong = (long) segmentSize; try { - this.availableMemorySegments = new ArrayBlockingQueue<>(numberOfSegmentsToAllocate); + this.availableMemorySegments = new ArrayDeque<>(numberOfSegmentsToAllocate); } catch (OutOfMemoryError err) { throw new OutOfMemoryError("Could not allocate buffer queue of length " @@ -134,6 +138,8 @@ public NetworkBufferPool( ", missing (Mb): " + missingMb + "). Cause: " + err.getMessage()); } + availabilityHelper.resetAvailable(); + long allocatedMb = (sizeInLong * availableMemorySegments.size()) >> 20; LOG.info("Allocated {} MB for network buffer pool (number of memory segments: {}, bytes per segment: {}).", @@ -142,14 +148,16 @@ public NetworkBufferPool( @Nullable public MemorySegment requestMemorySegment() { - return availableMemorySegments.poll(); + synchronized (availableMemorySegments) { + return internalRequestMemorySegment(); + } } public void recycle(MemorySegment segment) { // Adds the segment back to the queue, which does not immediately free the memory // however, since this happens when references to the global pool are also released, // making the availableMemorySegments queue and its contained object reclaimable - availableMemorySegments.add(checkNotNull(segment)); + internalRecycleMemorySegments(Collections.singleton(checkNotNull(segment))); } @Override @@ -170,7 +178,12 @@ public List requestMemorySegments() throws IOException { throw new IllegalStateException("Buffer pool is destroyed."); } - final MemorySegment segment = availableMemorySegments.poll(2, TimeUnit.SECONDS); + MemorySegment segment; + synchronized (availableMemorySegments) { + if ((segment = internalRequestMemorySegment()) == null) { + availableMemorySegments.wait(2000); + } + } if (segment != null) { segments.add(segment); } @@ -199,6 +212,17 @@ public List requestMemorySegments() throws IOException { return segments; } + @Nullable + private MemorySegment internalRequestMemorySegment() { + assert Thread.holdsLock(availableMemorySegments); + + final MemorySegment segment = availableMemorySegments.poll(); + if (availableMemorySegments.isEmpty() && segment != null) { + availabilityHelper.resetUnavailable(); + } + return segment; + } + @Override public void recycleMemorySegments(Collection segments) throws IOException { recycleMemorySegments(segments, segments.size()); @@ -208,17 +232,34 @@ private void recycleMemorySegments(Collection segments, int size) synchronized (factoryLock) { numTotalRequiredBuffers -= size; - availableMemorySegments.addAll(segments); - // note: if this fails, we're fine for the buffer pool since we already recycled the segments redistributeBuffers(); } + + internalRecycleMemorySegments(segments); + } + + private void internalRecycleMemorySegments(Collection segments) { + CompletableFuture toNotify = null; + synchronized (availableMemorySegments) { + if (availableMemorySegments.isEmpty() && !segments.isEmpty()) { + toNotify = availabilityHelper.getUnavailableToResetAvailable(); + } + availableMemorySegments.addAll(segments); + availableMemorySegments.notifyAll(); + } + + if (toNotify != null) { + toNotify.complete(null); + } } public void destroy() { synchronized (factoryLock) { isDestroyed = true; + } + synchronized (availableMemorySegments) { MemorySegment segment; while ((segment = availableMemorySegments.poll()) != null) { segment.free(); @@ -235,7 +276,9 @@ public int getTotalNumberOfMemorySegments() { } public int getNumberOfAvailableMemorySegments() { - return availableMemorySegments.size(); + synchronized (availableMemorySegments) { + return availableMemorySegments.size(); + } } public int getNumberOfRegisteredBufferPools() { @@ -256,6 +299,15 @@ public int countBuffers() { return buffers; } + /** + * Returns a future that is completed when there are free segments + * in this pool. + */ + @Override + public CompletableFuture isAvailable() { + return availabilityHelper.isAvailable(); + } + // ------------------------------------------------------------------------ // BufferPoolFactory // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImpl.java index b1409b1dc099..bbcf38099a7b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImpl.java @@ -68,7 +68,7 @@ public class BackPressureStatsTrackerImpl implements BackPressureStatsTracker { private static final Logger LOG = LoggerFactory.getLogger(BackPressureStatsTrackerImpl.class); /** Maximum stack trace depth for samples. */ - static final int MAX_STACK_TRACE_DEPTH = 3; + static final int MAX_STACK_TRACE_DEPTH = 8; /** Expected class name for back pressure indicating stack trace element. */ static final String EXPECTED_CLASS_NAME = "org.apache.flink.runtime.io.network.buffer.LocalBufferPool"; 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 c7cd1bcb672e..0b3dfdfb7d6d 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 @@ -449,9 +449,8 @@ public void testIsAvailableOrNot() throws Exception { final RecordWriter recordWriter = createRecordWriter(partitionWrapper); try { - // record writer is available because of initial available local pool + // record writer is available because of initial available global pool assertTrue(recordWriter.isAvailable().isDone()); - assertEquals(recordWriter.AVAILABLE, recordWriter.isAvailable()); // request one buffer from the local pool to make it unavailable afterwards final BufferBuilder bufferBuilder = resultPartition.getBufferBuilder(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolDestroyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolDestroyTest.java index d1432b43c8d3..b42ad1f9536b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolDestroyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolDestroyTest.java @@ -108,9 +108,9 @@ public void testDestroyWhileBlockingRequest() throws Exception { * request or not */ public static boolean isInBlockingBufferRequest(StackTraceElement[] stackTrace) { - if (stackTrace.length >= 3) { - return stackTrace[0].getMethodName().equals("wait") && - stackTrace[1].getClassName().equals(LocalBufferPool.class.getName()); + if (stackTrace.length >= 8) { + return stackTrace[5].getMethodName().equals("get") && + stackTrace[7].getClassName().equals(LocalBufferPool.class.getName()); } else { return false; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java index 9c71c89d88e4..a3f4aed6ffb5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.io.network.buffer; +import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; @@ -32,6 +33,7 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -42,6 +44,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull; 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.assertTrue; @@ -403,6 +406,44 @@ public void testBoundedBuffer() throws Exception { assertEquals(1, localBufferPool.getNumberOfAvailableMemorySegments()); } + @Test + public void testIsAvailableOrNot() throws Exception { + + // the local buffer pool should be in available state initially + assertTrue(localBufferPool.isAvailable().isDone()); + + // request one buffer + final BufferBuilder bufferBuilder = checkNotNull(localBufferPool.requestBufferBuilderBlocking()); + assertFalse(localBufferPool.isAvailable().isDone()); + + // set the pool size + localBufferPool.setNumBuffers(2); + assertTrue(localBufferPool.isAvailable().isDone()); + + // drain the global buffer pool + final List segments = new ArrayList<>(numBuffers); + while (networkBufferPool.getNumberOfAvailableMemorySegments() > 0) { + segments.add(checkNotNull(networkBufferPool.requestMemorySegment())); + } + assertFalse(localBufferPool.isAvailable().isDone()); + + // recycle the requested segments to global buffer pool + for (final MemorySegment segment: segments) { + networkBufferPool.recycle(segment); + } + assertTrue(localBufferPool.isAvailable().isDone()); + + // reset the pool size + localBufferPool.setNumBuffers(1); + final CompletableFuture availableFuture = localBufferPool.isAvailable(); + assertFalse(availableFuture.isDone()); + + // recycle the requested buffer + bufferBuilder.createBufferConsumer().close(); + assertTrue(localBufferPool.isAvailable().isDone()); + assertTrue(availableFuture.isDone()); + } + // ------------------------------------------------------------------------ // Helpers // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java index 9659c7cc3e31..bd5339a1721e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java @@ -30,9 +30,18 @@ import java.io.IOException; import java.time.Duration; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.List; - +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicReference; + +import static org.apache.flink.util.Preconditions.checkNotNull; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.Matchers.hasProperty; @@ -518,4 +527,210 @@ public void go() throws Exception { globalPool.destroy(); } } + + /** + * Tests {@link NetworkBufferPool#isAvailable()}, verifying that the buffer availability is correctly + * maintained after memory segments are requested by {@link NetworkBufferPool#requestMemorySegment()} + * and recycled by {@link NetworkBufferPool#recycle(MemorySegment)}. + */ + @Test + public void testIsAvailableOrNotAfterRequestAndRecycleSingleSegment() throws Exception { + final int numBuffers = 2; + + final NetworkBufferPool globalPool = new NetworkBufferPool(numBuffers, 128, 1); + + try { + // the global pool should be in available state initially + assertTrue(globalPool.isAvailable().isDone()); + + // request the first segment + final MemorySegment segment1 = checkNotNull(globalPool.requestMemorySegment()); + assertTrue(globalPool.isAvailable().isDone()); + + // request the second segment + final MemorySegment segment2 = checkNotNull(globalPool.requestMemorySegment()); + assertFalse(globalPool.isAvailable().isDone()); + + final CompletableFuture availableFuture = globalPool.isAvailable(); + + // recycle the first segment + globalPool.recycle(segment1); + assertTrue(availableFuture.isDone()); + assertTrue(globalPool.isAvailable().isDone()); + + // recycle the second segment + globalPool.recycle(segment2); + assertTrue(globalPool.isAvailable().isDone()); + + } finally { + globalPool.destroy(); + } + } + + /** + * Tests {@link NetworkBufferPool#isAvailable()}, verifying that the buffer availability is correctly + * maintained after memory segments are requested by {@link NetworkBufferPool#requestMemorySegments()} + * and recycled by {@link NetworkBufferPool#recycleMemorySegments(Collection)}. + */ + @Test(timeout = 10000L) + public void testIsAvailableOrNotAfterRequestAndRecycleMultiSegments() throws Exception { + final int numberOfSegmentsToRequest = 5; + final int numBuffers = 2 * numberOfSegmentsToRequest; + + final NetworkBufferPool globalPool = new NetworkBufferPool(numBuffers, 128, numberOfSegmentsToRequest); + + try { + // the global pool should be in available state initially + assertTrue(globalPool.isAvailable().isDone()); + + // request 5 segments + List segments1 = globalPool.requestMemorySegments(); + assertTrue(globalPool.isAvailable().isDone()); + assertEquals(numberOfSegmentsToRequest, segments1.size()); + + // request another 5 segments + List segments2 = globalPool.requestMemorySegments(); + assertFalse(globalPool.isAvailable().isDone()); + assertEquals(numberOfSegmentsToRequest, segments2.size()); + + // request another 5 segments + final CountDownLatch latch = new CountDownLatch(1); + final List segments3 = new ArrayList<>(numberOfSegmentsToRequest); + CheckedThread asyncRequest = new CheckedThread() { + @Override + public void go() throws Exception { + // this request should be blocked until at least 5 segments are recycled + segments3.addAll(globalPool.requestMemorySegments()); + latch.countDown(); + } + }; + asyncRequest.start(); + + // recycle 5 segments + CompletableFuture availableFuture = globalPool.isAvailable(); + globalPool.recycleMemorySegments(segments1); + assertTrue(availableFuture.isDone()); + + // wait util the third request is fulfilled + latch.await(); + assertFalse(globalPool.isAvailable().isDone()); + assertEquals(numberOfSegmentsToRequest, segments3.size()); + + // recycle another 5 segments + globalPool.recycleMemorySegments(segments2); + assertTrue(globalPool.isAvailable().isDone()); + + // recycle the last 5 segments + globalPool.recycleMemorySegments(segments3); + assertTrue(globalPool.isAvailable().isDone()); + + } finally { + globalPool.destroy(); + } + } + + /** + * Tests that blocking request of multi local buffer pools can be fulfilled by recycled segments + * to the global network buffer pool. + */ + @Test(timeout = 10000L) + public void testBlockingRequestFromMultiLocalBufferPool() throws Exception { + final int localPoolRequiredSize = 5; + final int localPoolMaxSize = 10; + final int numLocalBufferPool = 2; + final int numberOfSegmentsToRequest = 10; + final int numBuffers = numLocalBufferPool * localPoolMaxSize; + + final ExecutorService executorService = Executors.newFixedThreadPool(numLocalBufferPool); + final NetworkBufferPool globalPool = new NetworkBufferPool(numBuffers, 128, numberOfSegmentsToRequest); + final List localBufferPools = new ArrayList<>(numLocalBufferPool); + + try { + // create local buffer pools + for (int i = 0; i < numLocalBufferPool; ++i) { + final BufferPool localPool = globalPool.createBufferPool(localPoolRequiredSize, localPoolMaxSize); + localBufferPools.add(localPool); + assertTrue(localPool.isAvailable().isDone()); + } + + // request some segments from the global pool in two different ways + final List segments = new ArrayList<>(numberOfSegmentsToRequest - 1); + for (int i = 0; i < numberOfSegmentsToRequest - 1; ++i) { + segments.add(globalPool.requestMemorySegment()); + } + final List exclusiveSegments = globalPool.requestMemorySegments(); + assertTrue(globalPool.isAvailable().isDone()); + for (final BufferPool localPool: localBufferPools) { + assertTrue(localPool.isAvailable().isDone()); + } + + // blocking request buffers form local buffer pools + final CountDownLatch latch = new CountDownLatch(numLocalBufferPool); + final BlockingQueue segmentsRequested = new ArrayBlockingQueue<>(numBuffers); + final AtomicReference cause = new AtomicReference<>(); + for (final BufferPool localPool: localBufferPools) { + executorService.submit(() -> { + try { + for (int num = localPoolMaxSize; num > 0; --num) { + segmentsRequested.add(localPool.requestBufferBuilderBlocking()); + } + } catch (Exception e) { + cause.set(e); + } finally { + latch.countDown(); + } + }); + } + + // wait until all available buffers are requested + while (globalPool.getNumberOfAvailableMemorySegments() > 0) { + Thread.sleep(100); + assertNull(cause.get()); + } + + final CompletableFuture globalPoolAvailableFuture = globalPool.isAvailable(); + assertFalse(globalPoolAvailableFuture.isDone()); + + final List> localPoolAvailableFutures = new ArrayList<>(numLocalBufferPool); + for (BufferPool localPool: localBufferPools) { + CompletableFuture localPoolAvailableFuture = localPool.isAvailable(); + localPoolAvailableFutures.add(localPoolAvailableFuture); + assertFalse(localPoolAvailableFuture.isDone()); + } + + // recycle the previously requested segments + for (MemorySegment segment: segments) { + globalPool.recycle(segment); + } + globalPool.recycleMemorySegments(exclusiveSegments); + + assertTrue(globalPoolAvailableFuture.isDone()); + for (CompletableFuture localPoolAvailableFuture: localPoolAvailableFutures) { + assertTrue(localPoolAvailableFuture.isDone()); + } + + // wait until all blocking buffer requests finish + latch.await(); + + assertNull(cause.get()); + assertEquals(0, globalPool.getNumberOfAvailableMemorySegments()); + assertFalse(globalPool.isAvailable().isDone()); + for (BufferPool localPool: localBufferPools) { + assertFalse(localPool.isAvailable().isDone()); + assertEquals(localPoolMaxSize, localPool.bestEffortGetNumOfUsedBuffers()); + } + + // recycle all the requested buffers + for (BufferBuilder bufferBuilder: segmentsRequested) { + bufferBuilder.createBufferConsumer().close(); + } + + } finally { + for (BufferPool bufferPool: localBufferPools) { + bufferPool.lazyDestroy(); + } + executorService.shutdown(); + globalPool.destroy(); + } + } } From 845fb95e633931eee4ff86a82c886050f318a32a Mon Sep 17 00:00:00 2001 From: Zhenghua Gao Date: Tue, 29 Oct 2019 21:50:39 +0800 Subject: [PATCH 427/746] [FLINK-14080][table-planner-blink] Introduce SqlTimestamp as internal representation of TIMESTAMP_WITHOUT_TIME_ZONE This closes #10035 --- .../flink/table/dataformat/SqlTimestamp.java | 187 +++++++++++++++++ .../typeutils/SqlTimestampSerializer.java | 189 ++++++++++++++++++ .../typeutils/SqlTimestampTypeInfo.java | 102 ++++++++++ .../table/dataformat/SqlTimestampTest.java | 97 +++++++++ .../typeutils/InternalTypeInfoTest.java | 6 +- .../typeutils/SqlTimestampSerializerTest.java | 73 +++++++ 6 files changed, 653 insertions(+), 1 deletion(-) create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/SqlTimestamp.java create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/SqlTimestampSerializer.java create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/SqlTimestampTypeInfo.java create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/SqlTimestampTest.java create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/SqlTimestampSerializerTest.java diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/SqlTimestamp.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/SqlTimestamp.java new file mode 100644 index 000000000000..53892c4d069e --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/SqlTimestamp.java @@ -0,0 +1,187 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.dataformat; + +import org.apache.flink.util.Preconditions; + +import java.sql.Timestamp; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; + +/** + * Immutable SQL TIMESTAMP type with nanosecond precision. + * + *

    This class is composite of a millisecond and nanoOfMillisecond. The millisecond part + * holds the integral second and the milli-of-second. The nanoOfMillisecond holds the + * nano-of-millisecond, which should between 0 - 999_999. + */ +public class SqlTimestamp implements Comparable { + + // the number of milliseconds in a day + private static final long MILLIS_PER_DAY = 86400000; // = 24 * 60 * 60 * 1000 + + // this field holds the integral second and the milli-of-second + private final long millisecond; + + // this field holds the nano-of-millisecond + private final int nanoOfMillisecond; + + private SqlTimestamp(long millisecond, int nanoOfMillisecond) { + Preconditions.checkArgument(nanoOfMillisecond >= 0 && nanoOfMillisecond <= 999_999); + this.millisecond = millisecond; + this.nanoOfMillisecond = nanoOfMillisecond; + } + + @Override + public int compareTo(SqlTimestamp that) { + int cmp = Long.compare(this.millisecond, that.millisecond); + if (cmp == 0) { + cmp = this.nanoOfMillisecond - that.nanoOfMillisecond; + } + return cmp; + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof SqlTimestamp)) { + return false; + } + SqlTimestamp that = (SqlTimestamp) obj; + return this.millisecond == that.millisecond && + this.nanoOfMillisecond == that.nanoOfMillisecond; + } + + @Override + public String toString() { + return toLocalDateTime().toString(); + } + + @Override + public int hashCode() { + int ret = (int) millisecond ^ (int) (millisecond >> 32); + return 31 * ret + nanoOfMillisecond; + } + + public long getMillisecond() { + return millisecond; + } + + public int getNanoOfMillisecond() { + return nanoOfMillisecond; + } + + /** + * Obtains an instance of {@code SqlTimestamp} from a millisecond. + * + *

    This returns a {@code SqlTimestamp} with the specified millisecond. + * The nanoOfMillisecond field will be set to zero. + * + * @param millisecond the number of milliseconds since January 1, 1970, 00:00:00 GMT + * A negative number is the number of milliseconds before + * January 1, 1970, 00:00:00 GMT + * @return an instance of {@code SqlTimestamp} + */ + public static SqlTimestamp fromEpochMillis(long millisecond) { + return new SqlTimestamp(millisecond, 0); + } + + /** + * Obtains an instance of {@code SqlTimestamp} from a millisecond and a nanoOfMillisecond. + * + *

    This returns a {@code SqlTimestamp} with the specified millisecond and nanoOfMillisecond. + * + * @param millisecond the number of milliseconds since January 1, 1970, 00:00:00 GMT + * A negative number is the number of milliseconds before + * January 1, 1970, 00:00:00 GMT + * @param nanoOfMillisecond the nanosecond within the millisecond, from 0 to 999,999 + * @return an instance of {@code SqlTimestamp} + */ + public static SqlTimestamp fromEpochMillis(long millisecond, int nanoOfMillisecond) { + return new SqlTimestamp(millisecond, nanoOfMillisecond); + } + + /** + * Convert this {@code SqlTimestmap} object to a {@link Timestamp}. + * + * @return An instance of {@link Timestamp} + */ + public Timestamp toTimestamp() { + return Timestamp.valueOf(toLocalDateTime()); + } + + /** + * Obtains an instance of {@code SqlTimestamp} from an instance of {@link Timestamp}. + * + *

    This returns a {@code SqlTimestamp} with the specified {@link Timestamp}. + * + * @param ts an instance of {@link Timestamp} + * @return an instance of {@code SqlTimestamp} + */ + public static SqlTimestamp fromTimestamp(Timestamp ts) { + return fromLocalDateTime(ts.toLocalDateTime()); + } + + /** + * Convert this {@code SqlTimestamp} object to a {@link LocalDateTime}. + * + * @return An instance of {@link LocalDateTime} + */ + public LocalDateTime toLocalDateTime() { + int date = (int) (millisecond / MILLIS_PER_DAY); + int time = (int) (millisecond % MILLIS_PER_DAY); + if (time < 0) { + --date; + time += MILLIS_PER_DAY; + } + long nanoOfDay = time * 1_000_000L + nanoOfMillisecond; + LocalDate localDate = LocalDate.ofEpochDay(date); + LocalTime localTime = LocalTime.ofNanoOfDay(nanoOfDay); + return LocalDateTime.of(localDate, localTime); + } + + /** + * Obtains an instance of {@code SqlTimestamp} from an instance of {@link LocalDateTime}. + * + *

    This returns a {@code SqlTimestamp} with the specified {@link LocalDateTime}. + * + * @param dateTime an instance of {@link LocalDateTime} + * @return an instance of {@code SqlTimestamp} + */ + public static SqlTimestamp fromLocalDateTime(LocalDateTime dateTime) { + long epochDay = dateTime.toLocalDate().toEpochDay(); + long nanoOfDay = dateTime.toLocalTime().toNanoOfDay(); + + long millisecond = epochDay * MILLIS_PER_DAY + nanoOfDay / 1_000_000; + int nanoOfMillisecond = (int) (nanoOfDay % 1_000_000); + + return new SqlTimestamp(millisecond, nanoOfMillisecond); + } + + /** + * Apache Calcite and Flink's planner use the number of milliseconds since epoch to represent a + * Timestamp type compactly if the number of digits of fractional seconds is between 0 - 3. + * + * @param precision the number of digits of fractional seconds + * @return true if precision is less than or equal to 3, false otherwise + */ + public static boolean isCompact(int precision) { + return precision <= 3; + } +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/SqlTimestampSerializer.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/SqlTimestampSerializer.java new file mode 100644 index 000000000000..df9769a1f213 --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/SqlTimestampSerializer.java @@ -0,0 +1,189 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.typeutils; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.table.dataformat.SqlTimestamp; + +import java.io.IOException; + +/** + * Serializer of {@link SqlTimestamp}. + * + *

    A {@link SqlTimestamp} instance can be compactly serialized as a long value(= millisecond) when + * the Timestamp type is compact. Otherwise it's serialized as a long value and a int value. + */ +public class SqlTimestampSerializer extends TypeSerializer { + + private static final long serialVersionUID = 1L; + + private final int precision; + + public SqlTimestampSerializer(int precision) { + this.precision = precision; + } + + @Override + public boolean isImmutableType() { + return true; + } + + @Override + public TypeSerializer duplicate() { + return new SqlTimestampSerializer(precision); + } + + @Override + public SqlTimestamp createInstance() { + return SqlTimestamp.fromEpochMillis(0); + } + + @Override + public SqlTimestamp copy(SqlTimestamp from) { + return from; + } + + @Override + public SqlTimestamp copy(SqlTimestamp from, SqlTimestamp reuse) { + return copy(from); + } + + @Override + public int getLength() { + return ((SqlTimestamp.isCompact(precision)) ? 8 : 12); + } + + @Override + public void serialize(SqlTimestamp record, DataOutputView target) throws IOException { + if (SqlTimestamp.isCompact(precision)) { + assert record.getNanoOfMillisecond() == 0; + target.writeLong(record.getMillisecond()); + } else { + target.writeLong(record.getMillisecond()); + target.writeInt(record.getNanoOfMillisecond()); + } + } + + @Override + public SqlTimestamp deserialize(DataInputView source) throws IOException { + if (SqlTimestamp.isCompact(precision)) { + long val = source.readLong(); + return SqlTimestamp.fromEpochMillis(val); + } else { + long longVal = source.readLong(); + int intVal = source.readInt(); + return SqlTimestamp.fromEpochMillis(longVal, intVal); + } + } + + @Override + public SqlTimestamp deserialize(SqlTimestamp reuse, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + if (SqlTimestamp.isCompact(precision)) { + target.writeLong(source.readLong()); + } else { + target.writeLong(source.readLong()); + target.writeInt(source.readInt()); + } + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + SqlTimestampSerializer that = (SqlTimestampSerializer) obj; + return precision == that.precision; + } + + @Override + public int hashCode() { + return precision; + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new SqlTimestampSerializerSnapshot(precision); + } + + /** + * {@link TypeSerializerSnapshot} for {@link SqlTimestampSerializer}. + */ + public static final class SqlTimestampSerializerSnapshot implements TypeSerializerSnapshot { + + private static final int CURRENT_VERSION = 1; + + private int previousPrecision; + + public SqlTimestampSerializerSnapshot() { + // this constructor is used when restoring from a checkpoint/savepoint. + } + + SqlTimestampSerializerSnapshot(int precision) { + this.previousPrecision = precision; + } + + @Override + public int getCurrentVersion() { + return CURRENT_VERSION; + } + + @Override + public void writeSnapshot(DataOutputView out) throws IOException { + out.writeInt(previousPrecision); + } + + @Override + public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) throws IOException { + this.previousPrecision = in.readInt(); + } + + @Override + public TypeSerializer restoreSerializer() { + return new SqlTimestampSerializer(previousPrecision); + } + + @Override + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility(TypeSerializer newSerializer) { + if (!(newSerializer instanceof SqlTimestampSerializer)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + + SqlTimestampSerializer sqlTimestampSerializer = (SqlTimestampSerializer) newSerializer; + if (previousPrecision != sqlTimestampSerializer.precision) { + return TypeSerializerSchemaCompatibility.incompatible(); + } else { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + } + } +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/SqlTimestampTypeInfo.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/SqlTimestampTypeInfo.java new file mode 100644 index 000000000000..d6f1314caa67 --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/SqlTimestampTypeInfo.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.runtime.typeutils; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.dataformat.SqlTimestamp; + +import java.util.Objects; + +/** + * TypeInformation of {@link SqlTimestamp}. + */ +public class SqlTimestampTypeInfo extends TypeInformation { + + private final int precision; + + public SqlTimestampTypeInfo(int precision) { + this.precision = precision; + } + + @Override + public boolean isBasicType() { + return true; + } + + @Override + public boolean isTupleType() { + return false; + } + + @Override + public int getArity() { + return 1; + } + + @Override + public int getTotalFields() { + return 1; + } + + @Override + public Class getTypeClass() { + return SqlTimestamp.class; + } + + @Override + public boolean isKeyType() { + return true; + } + + @Override + public TypeSerializer createSerializer(ExecutionConfig config) { + return new SqlTimestampSerializer(precision); + } + + @Override + public String toString() { + return String.format("Timestamp(%d)", precision); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof SqlTimestampTypeInfo)) { + return false; + } + + SqlTimestampTypeInfo that = (SqlTimestampTypeInfo) obj; + return this.precision == that.precision; + } + + @Override + public int hashCode() { + return Objects.hash(this.getClass().getCanonicalName(), precision); + } + + @Override + public boolean canEqual(Object obj) { + return obj instanceof SqlTimestampTypeInfo; + } + + public int getPrecision() { + return precision; + } +} diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/SqlTimestampTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/SqlTimestampTest.java new file mode 100644 index 000000000000..04e99e5a259f --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/SqlTimestampTest.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.dataformat; + +import org.junit.Assert; +import org.junit.Test; + +import java.time.LocalDateTime; + +/** + * Test for {@link SqlTimestamp}. + */ +public class SqlTimestampTest { + + @Test + public void testNormal() { + // From long to SqlTimestamp and vice versa + Assert.assertEquals(1123L, SqlTimestamp.fromEpochMillis(1123L).getMillisecond()); + Assert.assertEquals(-1123L, SqlTimestamp.fromEpochMillis(-1123L).getMillisecond()); + + Assert.assertEquals(1123L, SqlTimestamp.fromEpochMillis(1123L, 45678).getMillisecond()); + Assert.assertEquals(45678, SqlTimestamp.fromEpochMillis(1123L, 45678).getNanoOfMillisecond()); + + Assert.assertEquals(-1123L, SqlTimestamp.fromEpochMillis(-1123L, 45678).getMillisecond()); + Assert.assertEquals(45678, SqlTimestamp.fromEpochMillis(-1123L, 45678).getNanoOfMillisecond()); + + // From SqlTimestamp to SqlTimestamp and vice versa + java.sql.Timestamp t19 = java.sql.Timestamp.valueOf("1969-01-02 00:00:00.123456789"); + java.sql.Timestamp t16 = java.sql.Timestamp.valueOf("1969-01-02 00:00:00.123456"); + java.sql.Timestamp t13 = java.sql.Timestamp.valueOf("1969-01-02 00:00:00.123"); + java.sql.Timestamp t10 = java.sql.Timestamp.valueOf("1969-01-02 00:00:00"); + + Assert.assertEquals(t19, SqlTimestamp.fromTimestamp(t19).toTimestamp()); + Assert.assertEquals(t16, SqlTimestamp.fromTimestamp(t16).toTimestamp()); + Assert.assertEquals(t13, SqlTimestamp.fromTimestamp(t13).toTimestamp()); + Assert.assertEquals(t10, SqlTimestamp.fromTimestamp(t10).toTimestamp()); + + java.sql.Timestamp t2 = java.sql.Timestamp.valueOf("1979-01-02 00:00:00.123456"); + Assert.assertEquals(t2, SqlTimestamp.fromTimestamp(t2).toTimestamp()); + + java.sql.Timestamp t3 = new java.sql.Timestamp(1572333940000L); + Assert.assertEquals(t3, SqlTimestamp.fromTimestamp(t3).toTimestamp()); + + // From LocalDateTime to SqlTimestamp and vice versa + LocalDateTime ldt19 = LocalDateTime.of(1969, 1, 2, 0, 0, 0, 123456789); + LocalDateTime ldt16 = LocalDateTime.of(1969, 1, 2, 0, 0, 0, 123456000); + LocalDateTime ldt13 = LocalDateTime.of(1969, 1, 2, 0, 0, 0, 123000000); + LocalDateTime ldt10 = LocalDateTime.of(1969, 1, 2, 0, 0, 0, 0); + + Assert.assertEquals(ldt19, SqlTimestamp.fromLocalDateTime(ldt19).toLocalDateTime()); + Assert.assertEquals(ldt16, SqlTimestamp.fromLocalDateTime(ldt16).toLocalDateTime()); + Assert.assertEquals(ldt13, SqlTimestamp.fromLocalDateTime(ldt13).toLocalDateTime()); + Assert.assertEquals(ldt10, SqlTimestamp.fromLocalDateTime(ldt10).toLocalDateTime()); + + LocalDateTime ldt2 = LocalDateTime.of(1969, 1, 2, 0, 0, 0, 0); + Assert.assertEquals(ldt2, SqlTimestamp.fromLocalDateTime(ldt2).toLocalDateTime()); + + LocalDateTime ldt3 = LocalDateTime.of(1989, 1, 2, 0, 0, 0, 123456789); + Assert.assertEquals(ldt3, SqlTimestamp.fromLocalDateTime(ldt3).toLocalDateTime()); + + LocalDateTime ldt4 = LocalDateTime.of(1989, 1, 2, 0, 0, 0, 123456789); + java.sql.Timestamp t4 = java.sql.Timestamp.valueOf(ldt4); + Assert.assertEquals(SqlTimestamp.fromLocalDateTime(ldt4), SqlTimestamp.fromTimestamp(t4)); + } + + @Test + public void testToString() { + + java.sql.Timestamp t = java.sql.Timestamp.valueOf("1969-01-02 00:00:00.123456789"); + Assert.assertEquals("1969-01-02T00:00:00.123456789", SqlTimestamp.fromTimestamp(t).toString()); + + Assert.assertEquals("1970-01-01T00:00:00.123", SqlTimestamp.fromEpochMillis(123L).toString()); + Assert.assertEquals("1970-01-01T00:00:00.123456789", SqlTimestamp.fromEpochMillis(123L, 456789).toString()); + + Assert.assertEquals("1969-12-31T23:59:59.877", SqlTimestamp.fromEpochMillis(-123L).toString()); + Assert.assertEquals("1969-12-31T23:59:59.877456789", SqlTimestamp.fromEpochMillis(-123L, 456789).toString()); + + LocalDateTime ldt = LocalDateTime.of(1969, 1, 2, 0, 0, 0, 123456789); + Assert.assertEquals("1969-01-02T00:00:00.123456789", SqlTimestamp.fromLocalDateTime(ldt).toString()); + } +} diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/InternalTypeInfoTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/InternalTypeInfoTest.java index b2ae001ee179..6c90555113bb 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/InternalTypeInfoTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/InternalTypeInfoTest.java @@ -30,7 +30,11 @@ public class InternalTypeInfoTest extends TypeInformationTestBase { + + @Parameterized.Parameters + public static Collection data(){ + return Arrays.asList(new Object[][]{{0}, {3}, {6}, {9}}); + } + + private int precision; + + public SqlTimestampSerializerTest(int precision) { + super(); + this.precision = precision; + } + + @Override + protected TypeSerializer createSerializer() { + return new SqlTimestampSerializer(precision); + } + + @Override + protected int getLength() { + return (precision <= 3) ? 8 : 12; + } + + @Override + protected Class getTypeClass() { + return SqlTimestamp.class; + } + + @Override + protected SqlTimestamp[] getTestData() { + return new SqlTimestamp[] { + SqlTimestamp.fromEpochMillis(1), + SqlTimestamp.fromEpochMillis(2), + SqlTimestamp.fromEpochMillis(3), + SqlTimestamp.fromEpochMillis(4) + }; + } +} From 76225edc3667f69cc4d9c63505ce227d0755449d Mon Sep 17 00:00:00 2001 From: tison Date: Mon, 4 Nov 2019 11:08:05 +0100 Subject: [PATCH 428/746] [FLINK-14462][coordination] Remove JobGraph#allowQueuedScheduling flag because it is always true Rewrite testRestartWithSlotSharingAndNotEnoughResources --- docs/ops/deployment/mesos.md | 4 +- docs/ops/deployment/mesos.zh.md | 4 +- .../apache/flink/client/LocalExecutor.java | 2 - .../program/rest/RestClusterClient.java | 3 - .../ClassPathJobGraphRetriever.java | 1 - .../plantranslate/JobGraphGenerator.java | 2 - .../webmonitor/handlers/JarRunHandler.java | 6 +- .../runtime/executiongraph/Execution.java | 9 +- .../executiongraph/ExecutionGraph.java | 19 +- .../executiongraph/ExecutionGraphBuilder.java | 3 +- .../executiongraph/SlotProviderStrategy.java | 28 ++- .../flink/runtime/jobgraph/JobGraph.java | 11 -- .../jobmaster/slotpool/SchedulerImpl.java | 102 ++++------- .../jobmaster/slotpool/SlotProvider.java | 9 +- .../runtime/minicluster/MiniCluster.java | 8 - .../scheduler/DefaultSchedulerFactory.java | 3 +- ...ecutionGraphCheckpointCoordinatorTest.java | 1 - .../DispatcherResourceCleanupTest.java | 1 - .../runtime/dispatcher/DispatcherTest.java | 2 - .../runner/DefaultDispatcherRunnerITCase.java | 5 +- .../ZooKeeperDefaultDispatcherRunnerTest.java | 2 - .../ExecutionGraphCoLocationRestartTest.java | 1 - .../ExecutionGraphDeploymentTest.java | 49 +++-- .../ExecutionGraphNotEnoughResourceTest.java | 169 ++++++++++++++++++ .../ExecutionGraphRestartTest.java | 50 ------ .../ExecutionGraphSchedulingTest.java | 7 - .../ExecutionGraphTestUtils.java | 10 -- .../ExecutionPartitionLifecycleTest.java | 2 +- .../ExecutionVertexCancelTest.java | 4 +- .../ExecutionVertexSchedulingTest.java | 6 +- .../ProgrammedSlotProvider.java | 1 - .../executiongraph/TestingSlotProvider.java | 2 +- .../TestingSlotProviderStrategy.java | 8 +- .../utils/SimpleSlotProvider.java | 4 - .../SlotCountExceedingParallelismTest.java | 9 +- .../ScheduleWithCoLocationHintTest.java | 120 ++++++------- .../scheduler/SchedulerIsolatedTasksTest.java | 93 ++-------- .../scheduler/SchedulerTestBase.java | 3 +- .../runtime/jobmaster/JobMasterTest.java | 18 +- .../slotpool/SlotPoolCoLocationTest.java | 10 -- .../jobmaster/slotpool/SlotPoolImplTest.java | 11 -- .../slotpool/SlotPoolInteractionsTest.java | 5 - .../slotpool/SlotPoolSlotSharingTest.java | 16 -- .../DefaultExecutionSlotAllocatorTest.java | 4 +- .../LegacySchedulerBatchSchedulingTest.java | 5 +- .../flink/yarn/YarnClusterDescriptor.java | 4 - 46 files changed, 347 insertions(+), 489 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphNotEnoughResourceTest.java diff --git a/docs/ops/deployment/mesos.md b/docs/ops/deployment/mesos.md index 476e16ede6fb..3906a949b497 100644 --- a/docs/ops/deployment/mesos.md +++ b/docs/ops/deployment/mesos.md @@ -193,7 +193,6 @@ The job graph file may be generated like this way: {% highlight java %} final JobGraph jobGraph = env.getStreamGraph().getJobGraph(); -jobGraph.setAllowQueuedScheduling(true); final String jobGraphFilename = "job.graph"; File jobGraphFile = new File(jobGraphFilename); try (FileOutputStream output = new FileOutputStream(jobGraphFile); @@ -203,8 +202,7 @@ try (FileOutputStream output = new FileOutputStream(jobGraphFile); {% endhighlight %} Note: -1. Before serializing the job graph, please make sure to enable queued scheduling because slots need to be allocated lazily -2. Make sure that all Mesos processes have the user code jar on the classpath (e.g. putting them in the lib directory) +1. Make sure that all Mesos processes have the user code jar on the classpath (e.g. putting them in the lib directory) #### General configuration diff --git a/docs/ops/deployment/mesos.zh.md b/docs/ops/deployment/mesos.zh.md index 88874de94331..fec999acb177 100644 --- a/docs/ops/deployment/mesos.zh.md +++ b/docs/ops/deployment/mesos.zh.md @@ -193,7 +193,6 @@ The job graph file may be generated like this way: {% highlight java %} final JobGraph jobGraph = env.getStreamGraph().getJobGraph(); -jobGraph.setAllowQueuedScheduling(true); final String jobGraphFilename = "job.graph"; File jobGraphFile = new File(jobGraphFilename); try (FileOutputStream output = new FileOutputStream(jobGraphFile); @@ -203,8 +202,7 @@ try (FileOutputStream output = new FileOutputStream(jobGraphFile); {% endhighlight %} Note: -1. Before serializing the job graph, please make sure to enable queued scheduling because slots need to be allocated lazily -2. Make sure that all Mesos processes have the user code jar on the classpath (e.g. putting them in the lib directory) +1. Make sure that all Mesos processes have the user code jar on the classpath (e.g. putting them in the lib directory) #### General configuration 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 d866a4f10109..b1d33305a44a 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 @@ -115,8 +115,6 @@ public JobExecutionResult executePlan( baseConfiguration, 1); - jobGraph.setAllowQueuedScheduling(true); - try (final JobExecutorService executorService = createJobExecutorService(jobGraph, baseConfiguration)) { return executorService.executeJobBlocking(jobGraph); 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 7352dd4c4071..27676556b991 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 @@ -278,9 +278,6 @@ public CompletableFuture requestJobResult(@Nonnull JobID jobId) { */ @Override public CompletableFuture submitJob(@Nonnull JobGraph jobGraph) { - // we have to enable queued scheduling because slot will be allocated lazily - jobGraph.setAllowQueuedScheduling(true); - CompletableFuture jobGraphFileFuture = CompletableFuture.supplyAsync(() -> { try { final java.nio.file.Path jobGraphFile = Files.createTempFile("flink-jobgraph", ".bin"); diff --git a/flink-container/src/main/java/org/apache/flink/container/entrypoint/ClassPathJobGraphRetriever.java b/flink-container/src/main/java/org/apache/flink/container/entrypoint/ClassPathJobGraphRetriever.java index 6cb2d84a3a06..57808e332850 100644 --- a/flink-container/src/main/java/org/apache/flink/container/entrypoint/ClassPathJobGraphRetriever.java +++ b/flink-container/src/main/java/org/apache/flink/container/entrypoint/ClassPathJobGraphRetriever.java @@ -101,7 +101,6 @@ public JobGraph retrieveJobGraph(Configuration configuration) throws FlinkExcept configuration, defaultParallelism, jobId); - jobGraph.setAllowQueuedScheduling(true); jobGraph.setSavepointRestoreSettings(savepointRestoreSettings); return jobGraph; diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java index 99860b2b91f0..fae214d01c1d 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/plantranslate/JobGraphGenerator.java @@ -248,8 +248,6 @@ public JobGraph compileJobGraph(OptimizedPlan program, JobID jobId) { "This indicates that non-serializable types (like custom serializers) were registered"); } - graph.setAllowQueuedScheduling(false); - // add vertices to the graph for (JobVertex vertex : this.vertices.values()) { vertex.setInputDependencyConstraint(program.getOriginalPlan().getExecutionConfig().getDefaultInputDependencyConstraint()); diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java index baae701c89c1..dcce5a9e9416 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java @@ -98,11 +98,7 @@ protected CompletableFuture handleRequest( return jobGraph; }); - CompletableFuture jobSubmissionFuture = jarUploadFuture.thenCompose(jobGraph -> { - // we have to enable queued scheduling because slots will be allocated lazily - jobGraph.setAllowQueuedScheduling(true); - return gateway.submitJob(jobGraph, timeout); - }); + CompletableFuture jobSubmissionFuture = jarUploadFuture.thenCompose(jobGraph -> gateway.submitJob(jobGraph, timeout)); return jobSubmissionFuture .thenCombine(jarUploadFuture, (ack, jobGraph) -> new JarRunResponseBody(jobGraph.getJobID())); 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 d99d6a3c1d1b..2db95ecbf751 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 @@ -440,14 +440,7 @@ public CompletableFuture scheduleForExecution( locationPreferenceConstraint, allPreviousExecutionGraphAllocationIds); - final CompletableFuture deploymentFuture; - - if (allocationFuture.isDone() || slotProviderStrategy.isQueuedSchedulingAllowed()) { - deploymentFuture = allocationFuture.thenRun(ThrowingRunnable.unchecked(this::deploy)); - } else { - deploymentFuture = FutureUtils.completedExceptionally( - new IllegalArgumentException("The slot allocation future has not been completed yet.")); - } + final CompletableFuture deploymentFuture = allocationFuture.thenRun(ThrowingRunnable.unchecked(this::deploy)); deploymentFuture.whenComplete( (Void ignored, Throwable failure) -> { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 25a4a801c3a1..e2132423a5d1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -268,10 +268,6 @@ public class ExecutionGraph implements AccessExecutionGraph { // ------ Configuration of the Execution ------- - /** Flag to indicate whether the scheduler may queue tasks for execution, or needs to be able - * to deploy them immediately. */ - private final boolean allowQueuedScheduling; - /** The mode of scheduling. Decides how to select the initial set of tasks to be deployed. * May indicate to deploy all sources, or to deploy everything, or to deploy via backtracking * from results than need to be materialized. */ @@ -440,8 +436,7 @@ public ExecutionGraph( jobInformation.getJobId(), NettyShuffleMaster.INSTANCE, ignored -> Optional.empty()), - ScheduleMode.LAZY_FROM_SOURCES, - false); + ScheduleMode.LAZY_FROM_SOURCES); } public ExecutionGraph( @@ -459,8 +454,7 @@ public ExecutionGraph( PartitionReleaseStrategy.Factory partitionReleaseStrategyFactory, ShuffleMaster shuffleMaster, JobMasterPartitionTracker partitionTracker, - ScheduleMode scheduleMode, - boolean allowQueuedScheduling) throws IOException { + ScheduleMode scheduleMode) throws IOException { this.jobInformation = Preconditions.checkNotNull(jobInformation); @@ -468,8 +462,6 @@ public ExecutionGraph( this.scheduleMode = checkNotNull(scheduleMode); - this.allowQueuedScheduling = allowQueuedScheduling; - this.jobInformationOrBlobKey = BlobWriter.serializeAndTryOffload(jobInformation, jobInformation.getJobId(), blobWriter); this.futureExecutor = Preconditions.checkNotNull(futureExecutor); @@ -478,8 +470,7 @@ public ExecutionGraph( this.slotProviderStrategy = SlotProviderStrategy.from( scheduleMode, slotProvider, - allocationTimeout, - allowQueuedScheduling); + allocationTimeout); this.userClassLoader = Preconditions.checkNotNull(userClassLoader, "userClassLoader"); this.tasks = new ConcurrentHashMap<>(16); @@ -541,10 +532,6 @@ public int getNumberOfExecutionJobVertices() { return this.verticesInCreationOrder.size(); } - public boolean isQueuedSchedulingAllowed() { - return this.allowQueuedScheduling; - } - public SchedulingTopology getSchedulingTopology() { return executionTopology; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java index 090b5b412be3..6b0e389db579 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java @@ -180,8 +180,7 @@ public static ExecutionGraph buildGraph( partitionReleaseStrategyFactory, shuffleMaster, partitionTracker, - jobGraph.getScheduleMode(), - jobGraph.getAllowQueuedScheduling()); + jobGraph.getScheduleMode()); } catch (IOException e) { throw new JobException("Could not create the ExecutionGraph.", e); } 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 830b0522201b..cd1eac533f22 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 @@ -39,15 +39,8 @@ public abstract class SlotProviderStrategy { protected final SlotProvider slotProvider; - protected final boolean allowQueuedScheduling; - - SlotProviderStrategy(SlotProvider slotProvider, boolean allowQueuedScheduling) { + SlotProviderStrategy(SlotProvider slotProvider) { this.slotProvider = Preconditions.checkNotNull(slotProvider); - this.allowQueuedScheduling = allowQueuedScheduling; - } - - boolean isQueuedSchedulingAllowed() { - return allowQueuedScheduling; } /** @@ -80,15 +73,14 @@ public void cancelSlotRequest( public static SlotProviderStrategy from( ScheduleMode scheduleMode, SlotProvider slotProvider, - Time allocationTimeout, - boolean allowQueuedScheduling) { + Time allocationTimeout) { switch (scheduleMode) { case LAZY_FROM_SOURCES_WITH_BATCH_SLOT_REQUEST: - return new BatchSlotProviderStrategy(slotProvider, allowQueuedScheduling); + return new BatchSlotProviderStrategy(slotProvider); case LAZY_FROM_SOURCES: case EAGER: - return new NormalSlotProviderStrategy(slotProvider, allocationTimeout, allowQueuedScheduling); + return new NormalSlotProviderStrategy(slotProvider, allocationTimeout); default: throw new IllegalArgumentException(String.format("Unknown scheduling mode: %s", scheduleMode)); } @@ -100,27 +92,27 @@ SlotProvider asSlotProvider() { static class BatchSlotProviderStrategy extends SlotProviderStrategy { - BatchSlotProviderStrategy(SlotProvider slotProvider, boolean allowQueuedScheduling) { - super(slotProvider, allowQueuedScheduling); + BatchSlotProviderStrategy(SlotProvider slotProvider) { + super(slotProvider); } @Override public CompletableFuture allocateSlot(SlotRequestId slotRequestId, ScheduledUnit scheduledUnit, SlotProfile slotProfile) { - return slotProvider.allocateBatchSlot(slotRequestId, scheduledUnit, slotProfile, allowQueuedScheduling); + return slotProvider.allocateBatchSlot(slotRequestId, scheduledUnit, slotProfile); } } static class NormalSlotProviderStrategy extends SlotProviderStrategy { private final Time allocationTimeout; - NormalSlotProviderStrategy(SlotProvider slotProvider, Time allocationTimeout, boolean allowQueuedScheduling) { - super(slotProvider, allowQueuedScheduling); + NormalSlotProviderStrategy(SlotProvider slotProvider, Time allocationTimeout) { + super(slotProvider); this.allocationTimeout = Preconditions.checkNotNull(allocationTimeout); } @Override public CompletableFuture allocateSlot(SlotRequestId slotRequestId, ScheduledUnit scheduledUnit, SlotProfile slotProfile) { - return slotProvider.allocateSlot(slotRequestId, scheduledUnit, slotProfile, allowQueuedScheduling, allocationTimeout); + return slotProvider.allocateSlot(slotRequestId, scheduledUnit, slotProfile, allocationTimeout); } } } 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 31564d4ff599..f63a5184a210 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 @@ -73,9 +73,6 @@ public class JobGraph implements Serializable { /** Name of this job. */ private final String jobName; - /** flag to enable queued scheduling */ - private boolean allowQueuedScheduling; - /** The mode in which the job is scheduled */ private ScheduleMode scheduleMode = ScheduleMode.LAZY_FROM_SOURCES; @@ -220,14 +217,6 @@ public SerializedValue getSerializedExecutionConfig() { return serializedExecutionConfig; } - public void setAllowQueuedScheduling(boolean allowQueuedScheduling) { - this.allowQueuedScheduling = allowQueuedScheduling; - } - - public boolean getAllowQueuedScheduling() { - return allowQueuedScheduling; - } - public void setScheduleMode(ScheduleMode scheduleMode) { this.scheduleMode = scheduleMode; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SchedulerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SchedulerImpl.java index 4bf8eab25410..ad318858d019 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SchedulerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SchedulerImpl.java @@ -110,13 +110,11 @@ public CompletableFuture allocateSlot( SlotRequestId slotRequestId, ScheduledUnit scheduledUnit, SlotProfile slotProfile, - boolean allowQueuedScheduling, Time allocationTimeout) { return allocateSlotInternal( slotRequestId, scheduledUnit, slotProfile, - allowQueuedScheduling, allocationTimeout); } @@ -124,13 +122,11 @@ public CompletableFuture allocateSlot( public CompletableFuture allocateBatchSlot( SlotRequestId slotRequestId, ScheduledUnit scheduledUnit, - SlotProfile slotProfile, - boolean allowQueuedScheduling) { + SlotProfile slotProfile) { return allocateSlotInternal( slotRequestId, scheduledUnit, slotProfile, - allowQueuedScheduling, null); } @@ -139,7 +135,6 @@ private CompletableFuture allocateSlotInternal( SlotRequestId slotRequestId, ScheduledUnit scheduledUnit, SlotProfile slotProfile, - boolean allowQueuedScheduling, @Nullable Time allocationTimeout) { log.debug("Received slot request [{}] for task: {}", slotRequestId, scheduledUnit.getTaskToExecute()); @@ -151,7 +146,6 @@ private CompletableFuture allocateSlotInternal( slotRequestId, scheduledUnit, slotProfile, - allowQueuedScheduling, allocationTimeout); return allocationResultFuture; } @@ -161,11 +155,10 @@ private void internalAllocateSlot( SlotRequestId slotRequestId, ScheduledUnit scheduledUnit, SlotProfile slotProfile, - boolean allowQueuedScheduling, Time allocationTimeout) { CompletableFuture allocationFuture = scheduledUnit.getSlotSharingGroupId() == null ? - allocateSingleSlot(slotRequestId, slotProfile, allowQueuedScheduling, allocationTimeout) : - allocateSharedSlot(slotRequestId, scheduledUnit, slotProfile, allowQueuedScheduling, allocationTimeout); + allocateSingleSlot(slotRequestId, slotProfile, allocationTimeout) : + allocateSharedSlot(slotRequestId, scheduledUnit, slotProfile, allocationTimeout); allocationFuture.whenComplete((LogicalSlot slot, Throwable failure) -> { if (failure != null) { @@ -180,7 +173,6 @@ private void internalAllocateSlot( slotRequestId, scheduledUnit, slotProfile, - allowQueuedScheduling, allocationTimeout); } else { cancelSlotRequest( @@ -223,7 +215,6 @@ public void returnLogicalSlot(LogicalSlot logicalSlot) { private CompletableFuture allocateSingleSlot( SlotRequestId slotRequestId, SlotProfile slotProfile, - boolean allowQueuedScheduling, @Nullable Time allocationTimeout) { Optional slotAndLocality = tryAllocateFromAvailable(slotRequestId, slotProfile); @@ -236,7 +227,7 @@ private CompletableFuture allocateSingleSlot( } catch (FlinkException e) { return FutureUtils.completedExceptionally(e); } - } else if (allowQueuedScheduling) { + } else { // we allocate by requesting a new slot return requestNewAllocatedSlot(slotRequestId, slotProfile, allocationTimeout) .thenApply((PhysicalSlot allocatedSlot) -> { @@ -246,10 +237,6 @@ private CompletableFuture allocateSingleSlot( throw new CompletionException(e); } }); - } else { - // failed to allocate - return FutureUtils.completedExceptionally( - new NoResourceAvailableException("Could not allocate a simple slot for " + slotRequestId + '.')); } } @@ -318,7 +305,6 @@ private CompletableFuture allocateSharedSlot( SlotRequestId slotRequestId, ScheduledUnit scheduledUnit, SlotProfile slotProfile, - boolean allowQueuedScheduling, @Nullable Time allocationTimeout) { // allocate slot with slot sharing final SlotSharingManager multiTaskSlotManager = slotSharingManagers.computeIfAbsent( @@ -335,14 +321,12 @@ private CompletableFuture allocateSharedSlot( scheduledUnit.getCoLocationConstraint(), multiTaskSlotManager, slotProfile, - allowQueuedScheduling, allocationTimeout); } else { multiTaskSlotLocality = allocateMultiTaskSlot( scheduledUnit.getJobVertexId(), multiTaskSlotManager, slotProfile, - allowQueuedScheduling, allocationTimeout); } } catch (NoResourceAvailableException noResourceException) { @@ -363,13 +347,11 @@ private CompletableFuture allocateSharedSlot( /** * Allocates a co-located {@link SlotSharingManager.MultiTaskSlot} for the given {@link CoLocationConstraint}. * - *

    If allowQueuedScheduling is true, then the returned {@link SlotSharingManager.MultiTaskSlot} can be - * uncompleted. + *

    The returned {@link SlotSharingManager.MultiTaskSlot} can be uncompleted. * * @param coLocationConstraint for which to allocate a {@link SlotSharingManager.MultiTaskSlot} * @param multiTaskSlotManager responsible for the slot sharing group for which to allocate the slot * @param slotProfile specifying the requirements for the requested slot - * @param allowQueuedScheduling true if queued scheduling (the returned task slot must not be completed yet) is allowed, otherwise false * @param allocationTimeout timeout before the slot allocation times out * @return A {@link SlotAndLocality} which contains the allocated{@link SlotSharingManager.MultiTaskSlot} * and its locality wrt the given location preferences @@ -378,7 +360,6 @@ private SlotSharingManager.MultiTaskSlotLocality allocateCoLocatedMultiTaskSlot( CoLocationConstraint coLocationConstraint, SlotSharingManager multiTaskSlotManager, SlotProfile slotProfile, - boolean allowQueuedScheduling, @Nullable Time allocationTimeout) throws NoResourceAvailableException { final SlotRequestId coLocationSlotRequestId = coLocationConstraint.getSlotRequestId(); @@ -415,7 +396,6 @@ private SlotSharingManager.MultiTaskSlotLocality allocateCoLocatedMultiTaskSlot( coLocationConstraint.getGroupId(), multiTaskSlotManager, slotProfile, - allowQueuedScheduling, allocationTimeout); // check whether we fulfill the co-location constraint @@ -466,12 +446,11 @@ private SlotSharingManager.MultiTaskSlotLocality allocateCoLocatedMultiTaskSlot( * Allocates a {@link SlotSharingManager.MultiTaskSlot} for the given groupId which is in the * slot sharing group for which the given {@link SlotSharingManager} is responsible. * - *

    If allowQueuedScheduling is true, then the method can return an uncompleted {@link SlotSharingManager.MultiTaskSlot}. + *

    The method can return an uncompleted {@link SlotSharingManager.MultiTaskSlot}. * * @param groupId for which to allocate a new {@link SlotSharingManager.MultiTaskSlot} * @param slotSharingManager responsible for the slot sharing group for which to allocate the slot * @param slotProfile slot profile that specifies the requirements for the slot - * @param allowQueuedScheduling true if queued scheduling (the returned task slot must not be completed yet) is allowed, otherwise false * @param allocationTimeout timeout before the slot allocation times out; null if requesting a batch slot * @return A {@link SlotSharingManager.MultiTaskSlotLocality} which contains the allocated {@link SlotSharingManager.MultiTaskSlot} * and its locality wrt the given location preferences @@ -480,8 +459,7 @@ private SlotSharingManager.MultiTaskSlotLocality allocateMultiTaskSlot( AbstractID groupId, SlotSharingManager slotSharingManager, SlotProfile slotProfile, - boolean allowQueuedScheduling, - @Nullable Time allocationTimeout) throws NoResourceAvailableException { + @Nullable Time allocationTimeout) { Collection resolvedRootSlotsInfo = slotSharingManager.listResolvedRootSlotInfo(groupId); @@ -533,48 +511,44 @@ private SlotSharingManager.MultiTaskSlotLocality allocateMultiTaskSlot( return multiTaskSlotLocality; } - if (allowQueuedScheduling) { - // there is no slot immediately available --> check first for uncompleted slots at the slot sharing group - SlotSharingManager.MultiTaskSlot multiTaskSlot = slotSharingManager.getUnresolvedRootSlot(groupId); + // there is no slot immediately available --> check first for uncompleted slots at the slot sharing group + SlotSharingManager.MultiTaskSlot multiTaskSlot = slotSharingManager.getUnresolvedRootSlot(groupId); - if (multiTaskSlot == null) { - // it seems as if we have to request a new slot from the resource manager, this is always the last resort!!! - final CompletableFuture slotAllocationFuture = requestNewAllocatedSlot( - allocatedSlotRequestId, - slotProfile, - allocationTimeout); + if (multiTaskSlot == null) { + // it seems as if we have to request a new slot from the resource manager, this is always the last resort!!! + final CompletableFuture slotAllocationFuture = requestNewAllocatedSlot( + allocatedSlotRequestId, + slotProfile, + allocationTimeout); - multiTaskSlot = slotSharingManager.createRootSlot( - multiTaskSlotRequestId, - slotAllocationFuture, - allocatedSlotRequestId); + multiTaskSlot = slotSharingManager.createRootSlot( + multiTaskSlotRequestId, + slotAllocationFuture, + allocatedSlotRequestId); - slotAllocationFuture.whenComplete( - (PhysicalSlot allocatedSlot, Throwable throwable) -> { - final SlotSharingManager.TaskSlot taskSlot = slotSharingManager.getTaskSlot(multiTaskSlotRequestId); - - if (taskSlot != null) { - // still valid - if (!(taskSlot instanceof SlotSharingManager.MultiTaskSlot) || throwable != null) { - taskSlot.release(throwable); - } else { - if (!allocatedSlot.tryAssignPayload(((SlotSharingManager.MultiTaskSlot) taskSlot))) { - taskSlot.release(new FlinkException("Could not assign payload to allocated slot " + - allocatedSlot.getAllocationId() + '.')); - } - } + slotAllocationFuture.whenComplete( + (PhysicalSlot allocatedSlot, Throwable throwable) -> { + final SlotSharingManager.TaskSlot taskSlot = slotSharingManager.getTaskSlot(multiTaskSlotRequestId); + + if (taskSlot != null) { + // still valid + if (!(taskSlot instanceof SlotSharingManager.MultiTaskSlot) || throwable != null) { + taskSlot.release(throwable); } else { - slotPool.releaseSlot( - allocatedSlotRequestId, - new FlinkException("Could not find task slot with " + multiTaskSlotRequestId + '.')); + if (!allocatedSlot.tryAssignPayload(((SlotSharingManager.MultiTaskSlot) taskSlot))) { + taskSlot.release(new FlinkException("Could not assign payload to allocated slot " + + allocatedSlot.getAllocationId() + '.')); + } } - }); - } - - return SlotSharingManager.MultiTaskSlotLocality.of(multiTaskSlot, Locality.UNKNOWN); + } else { + slotPool.releaseSlot( + allocatedSlotRequestId, + new FlinkException("Could not find task slot with " + multiTaskSlotRequestId + '.')); + } + }); } - throw new NoResourceAvailableException("Could not allocate a shared slot for " + groupId + '.'); + return SlotSharingManager.MultiTaskSlotLocality.of(multiTaskSlot, Locality.UNKNOWN); } private void releaseSharedSlot( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotProvider.java index 6f3071bc0e14..36da2c7c73ce 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotProvider.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotProvider.java @@ -48,7 +48,6 @@ public interface SlotProvider { * @param slotRequestId identifying the slot request * @param scheduledUnit The task to allocate the slot for * @param slotProfile profile of the requested slot - * @param allowQueuedScheduling Whether allow the task be queued if we do not have enough resource * @param allocationTimeout after which the allocation fails with a timeout exception * @return The future of the allocation */ @@ -56,7 +55,6 @@ CompletableFuture allocateSlot( SlotRequestId slotRequestId, ScheduledUnit scheduledUnit, SlotProfile slotProfile, - boolean allowQueuedScheduling, Time allocationTimeout); /** @@ -65,14 +63,12 @@ CompletableFuture allocateSlot( * @param slotRequestId identifying the slot request * @param scheduledUnit The task to allocate the slot for * @param slotProfile profile of the requested slot - * @param allowQueuedScheduling Whether allow the task be queued if we do not have enough resource * @return The future of the allocation */ default CompletableFuture allocateBatchSlot( SlotRequestId slotRequestId, ScheduledUnit scheduledUnit, - SlotProfile slotProfile, - boolean allowQueuedScheduling) { + SlotProfile slotProfile) { throw new UnsupportedOperationException("Not properly implemented."); } @@ -80,21 +76,18 @@ default CompletableFuture allocateBatchSlot( * Allocating slot with specific requirement. * * @param scheduledUnit The task to allocate the slot for - * @param allowQueued Whether allow the task be queued if we do not have enough resource * @param slotProfile profile of the requested slot * @param allocationTimeout after which the allocation fails with a timeout exception * @return The future of the allocation */ default CompletableFuture allocateSlot( ScheduledUnit scheduledUnit, - boolean allowQueued, SlotProfile slotProfile, Time allocationTimeout) { return allocateSlot( new SlotRequestId(), scheduledUnit, slotProfile, - allowQueued, allocationTimeout); } 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 7899b540a80f..c082c6d3c2a8 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 @@ -653,21 +653,13 @@ public JobExecutionResult executeJobBlocking(JobGraph job) throws JobExecutionEx public CompletableFuture submitJob(JobGraph jobGraph) { final CompletableFuture dispatcherGatewayFuture = getDispatcherGatewayFuture(); - - // we have to allow queued scheduling in Flip-6 mode because we need to request slots - // from the ResourceManager - jobGraph.setAllowQueuedScheduling(true); - final CompletableFuture blobServerAddressFuture = createBlobServerAddress(dispatcherGatewayFuture); - final CompletableFuture jarUploadFuture = uploadAndSetJobFiles(blobServerAddressFuture, jobGraph); - final CompletableFuture acknowledgeCompletableFuture = jarUploadFuture .thenCombine( dispatcherGatewayFuture, (Void ack, DispatcherGateway dispatcherGateway) -> dispatcherGateway.submitJob(jobGraph, rpcTimeout)) .thenCompose(Function.identity()); - return acknowledgeCompletableFuture.thenApply( (Acknowledge ignored) -> new JobSubmissionResult(jobGraph.getJobID())); } 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 5de8286cb3ae..82b949502f3a 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 @@ -81,8 +81,7 @@ public SchedulerNG createInstance( final SlotProviderStrategy slotProviderStrategy = SlotProviderStrategy.from( jobGraph.getScheduleMode(), slotProvider, - slotRequestTimeout, - true); + slotRequestTimeout); return new DefaultScheduler( log, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java index ea288f08863b..a99f1c857929 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java @@ -141,7 +141,6 @@ private ExecutionGraph createExecutionGraphAndEnableCheckpointing( final ExecutionGraph executionGraph = new ExecutionGraphTestUtils.TestingExecutionGraphBuilder(jobVertex) .setRpcTimeout(timeout) .setAllocationTimeout(timeout) - .allowQueuedScheduling() .build(); executionGraph.start(ComponentMainThreadExecutorServiceAdapter.forMainThread()); 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 6b137be6dcf5..7e614e6e9f62 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 @@ -141,7 +141,6 @@ public void setup() throws Exception { testVertex.setInvokableClass(NoOpInvokable.class); jobId = new JobID(); jobGraph = new JobGraph(jobId, "testJob", testVertex); - jobGraph.setAllowQueuedScheduling(true); configuration = new Configuration(); configuration.setString(BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); 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 f136a8cdfb6e..ba952ba96cec 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 @@ -164,7 +164,6 @@ public void setUp() throws Exception { final JobVertex testVertex = new JobVertex("testVertex"); testVertex.setInvokableClass(NoOpInvokable.class); jobGraph = new JobGraph(TEST_JOB_ID, "testJob", testVertex); - jobGraph.setAllowQueuedScheduling(true); fatalErrorHandler = new TestingFatalErrorHandler(); heartbeatServices = new HeartbeatServices(1000L, 10000L); @@ -322,7 +321,6 @@ public void testJobSubmissionWithPartialResourceConfigured() throws Exception { secondVertex.setInvokableClass(NoOpInvokable.class); JobGraph jobGraphWithTwoVertices = new JobGraph(TEST_JOB_ID, "twoVerticesJob", firstVertex, secondVertex); - jobGraphWithTwoVertices.setAllowQueuedScheduling(true); CompletableFuture acknowledgeFuture = dispatcherGateway.submitJob(jobGraphWithTwoVertices, TIMEOUT); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java index 85b88241544c..410835fe2996 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java @@ -219,10 +219,7 @@ public Dispatcher createDispatcher( private static JobGraph createJobGraph() { final JobVertex testVertex = new JobVertex("testVertex"); testVertex.setInvokableClass(NoOpInvokable.class); - final JobGraph testJob = new JobGraph(TEST_JOB_ID, "testJob", testVertex); - testJob.setAllowQueuedScheduling(true); - - return testJob; + return new JobGraph(TEST_JOB_ID, "testJob", testVertex); } private DispatcherRunner createDispatcherRunner() throws Exception { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java index 5fcc8866b2a2..5081373a3d16 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java @@ -240,8 +240,6 @@ private JobGraph createJobGraphWithBlobs() throws IOException { vertex.setParallelism(1); final JobGraph jobGraph = new JobGraph("Test job graph", vertex); - jobGraph.setAllowQueuedScheduling(true); - final PermanentBlobKey permanentBlobKey = blobServer.putPermanent(jobGraph.getJobID(), new byte[256]); jobGraph.addUserJarBlobKey(permanentBlobKey); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphCoLocationRestartTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphCoLocationRestartTest.java index 132647a6df15..92548be44cd1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphCoLocationRestartTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphCoLocationRestartTest.java @@ -69,7 +69,6 @@ public void testConstraintsAfterRestart() throws Exception { new TestRestartStrategy( 1, false)) - .allowQueuedScheduling() .build(); // enable the queued scheduling for the slot pool diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java index 78fa2aba777e..439093718cc8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java @@ -26,14 +26,12 @@ import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; -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.blob.PermanentBlobService; import org.apache.flink.runtime.blob.VoidBlobWriter; import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; -import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; @@ -78,8 +76,6 @@ import org.junit.Test; import org.slf4j.LoggerFactory; -import javax.annotation.Nonnull; - import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Arrays; @@ -91,7 +87,6 @@ import java.util.Map; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.Executor; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.function.Function; @@ -171,11 +166,12 @@ public void testBuildDeploymentDescriptor() { v4.connectNewDataSetAsInput(v2, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); DirectScheduledExecutorService executor = new DirectScheduledExecutorService(); - ExecutionGraph eg = createExecutionGraphWithoutQueuedScheduling( - jobId, - new TestingSlotProvider(ignore -> new CompletableFuture<>()), - executor, - executor); + ExecutionGraph eg = new ExecutionGraphTestUtils.TestingExecutionGraphBuilder(jobId) + .setFutureExecutor(executor) + .setIoExecutor(executor) + .setSlotProvider(new TestingSlotProvider(ignore -> new CompletableFuture<>())) + .setBlobWriter(blobWriter) + .build(); eg.start(ComponentMainThreadExecutorServiceAdapter.forMainThread()); @@ -361,7 +357,7 @@ public void testAccumulatorsAndMetricsForwarding() throws Exception { } /** - * Verifies that {@link Execution#completeCancelling(Map, IOMetrics)} and {@link Execution#markFailed(Throwable, Map, IOMetrics)} + * Verifies that {@link Execution#completeCancelling(Map, IOMetrics, boolean)} and {@link Execution#markFailed(Throwable, Map, IOMetrics)} * store the given accumulators and metrics correctly. */ @Test @@ -448,7 +444,12 @@ public void testNoResourceAvailableFailure() throws Exception { DirectScheduledExecutorService directExecutor = new DirectScheduledExecutorService(); // execution graph that executes actions synchronously - ExecutionGraph eg = createExecutionGraphWithoutQueuedScheduling(jobId, slotProvider, directExecutor, TestingUtils.defaultExecutor()); + ExecutionGraph eg = new ExecutionGraphTestUtils.TestingExecutionGraphBuilder(jobId) + .setFutureExecutor(directExecutor) + .setIoExecutor(TestingUtils.defaultExecutor()) + .setSlotProvider(slotProvider) + .setBlobWriter(blobWriter) + .build(); eg.start(ComponentMainThreadExecutorServiceAdapter.forMainThread()); @@ -512,7 +513,13 @@ private Tuple2> setupExecutio DirectScheduledExecutorService executorService = new DirectScheduledExecutorService(); // execution graph that executes actions synchronously - ExecutionGraph eg = createExecutionGraphWithoutQueuedScheduling(new JobID(), slotProvider, executorService, TestingUtils.defaultExecutor()); + ExecutionGraph eg = new ExecutionGraphTestUtils.TestingExecutionGraphBuilder(new JobID()) + .setFutureExecutor(executorService) + .setIoExecutor(TestingUtils.defaultExecutor()) + .setSlotProvider(slotProvider) + .setBlobWriter(blobWriter) + .build(); + checkJobOffloaded(eg); eg.start(ComponentMainThreadExecutorServiceAdapter.forMainThread()); @@ -529,21 +536,6 @@ private Tuple2> setupExecutio return new Tuple2<>(eg, executions); } - @Nonnull - private ExecutionGraph createExecutionGraphWithoutQueuedScheduling( - JobID jobId, - SlotProvider slotProvider, - ScheduledExecutorService futureExecutor, - Executor ioExecutor) throws JobException, JobExecutionException { - return new ExecutionGraphTestUtils.TestingExecutionGraphBuilder(jobId) - .setFutureExecutor(futureExecutor) - .setIoExecutor(ioExecutor) - .setSlotProvider(slotProvider) - .setBlobWriter(blobWriter) - .setAllowQueuedScheduling(false) - .build(); - } - @Test public void testSettingIllegalMaxNumberOfCheckpointsToRetain() throws Exception { @@ -689,7 +681,6 @@ public void testExecutionGraphIsDeployedInTopologicalOrder() throws Exception { .setSlotProvider(slotProvider) .setFutureExecutor(new DirectScheduledExecutorService()) .setScheduleMode(ScheduleMode.EAGER) - .allowQueuedScheduling() .build(); executionGraph.start(ComponentMainThreadExecutorServiceAdapter.forMainThread()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphNotEnoughResourceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphNotEnoughResourceTest.java new file mode 100644 index 000000000000..e57c0b25b3c1 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphNotEnoughResourceTest.java @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.common.time.Deadline; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; +import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; +import org.apache.flink.runtime.jobgraph.DistributionPattern; +import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobgraph.ScheduleMode; +import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException; +import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; +import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; +import org.apache.flink.runtime.jobmaster.JobMasterId; +import org.apache.flink.runtime.jobmaster.slotpool.LocationPreferenceSlotSelectionStrategy; +import org.apache.flink.runtime.jobmaster.slotpool.Scheduler; +import org.apache.flink.runtime.jobmaster.slotpool.SchedulerImpl; +import org.apache.flink.runtime.jobmaster.slotpool.SlotPool; +import org.apache.flink.runtime.jobmaster.slotpool.TestingSlotPoolImpl; +import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway; +import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; +import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.TestLogger; +import org.apache.flink.util.function.ThrowingRunnable; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import static org.junit.Assert.assertEquals; + +/** + * Tests ExecutionGraph schedule behavior with not enough resource. + */ +public class ExecutionGraphNotEnoughResourceTest extends TestLogger { + + private static TestingComponentMainThreadExecutor.Resource mainThreadExecutorResource; + private static ComponentMainThreadExecutor mainThreadExecutor; + + + private static final JobID TEST_JOB_ID = new JobID(); + private static final int NUM_TASKS = 31; + + @BeforeClass + public static void setupClass() { + mainThreadExecutorResource = new TestingComponentMainThreadExecutor.Resource(); + mainThreadExecutorResource.before(); + mainThreadExecutor = mainThreadExecutorResource.getComponentMainThreadTestExecutor().getMainThreadExecutor(); + } + + @AfterClass + public static void teardownClass() { + mainThreadExecutorResource.after(); + } + + @Test + public void testRestartWithSlotSharingAndNotEnoughResources() throws Exception { + final int numRestarts = 10; + final int parallelism = 20; + + SlotPool slotPool = null; + try { + slotPool = new TestingSlotPoolImpl(TEST_JOB_ID); + final Scheduler scheduler = createSchedulerWithSlots( + parallelism - 1, slotPool, new LocalTaskManagerLocation()); + + final SlotSharingGroup sharingGroup = new SlotSharingGroup(); + + final JobVertex source = new JobVertex("source"); + source.setInvokableClass(NoOpInvokable.class); + source.setParallelism(parallelism); + source.setSlotSharingGroup(sharingGroup); + + final JobVertex sink = new JobVertex("sink"); + sink.setInvokableClass(NoOpInvokable.class); + sink.setParallelism(parallelism); + sink.setSlotSharingGroup(sharingGroup); + sink.connectNewDataSetAsInput(source, DistributionPattern.POINTWISE, ResultPartitionType.PIPELINED_BOUNDED); + + TestRestartStrategy restartStrategy = + new TestRestartStrategy(numRestarts, false); + + final ExecutionGraph eg = new ExecutionGraphTestUtils.TestingExecutionGraphBuilder(TEST_JOB_ID, source, sink) + .setSlotProvider(scheduler) + .setRestartStrategy(restartStrategy) + .setScheduleMode(ScheduleMode.EAGER) + .setAllocationTimeout(Time.milliseconds(1L)) + .build(); + + eg.start(mainThreadExecutor); + + mainThreadExecutor.execute(ThrowingRunnable.unchecked(eg::scheduleForExecution)); + + CommonTestUtils.waitUntilCondition( + () -> CompletableFuture.supplyAsync(eg::getState, mainThreadExecutor).join() == JobStatus.FAILED, + Deadline.fromNow(Duration.ofMillis(2000))); + + // the last suppressed restart is also counted + assertEquals(numRestarts + 1, CompletableFuture.supplyAsync(eg::getNumberOfRestarts, mainThreadExecutor).join().longValue()); + + final Throwable t = CompletableFuture.supplyAsync(eg::getFailureCause, mainThreadExecutor).join(); + if (!(t instanceof NoResourceAvailableException)) { + ExceptionUtils.rethrowException(t, t.getMessage()); + } + } finally { + if (slotPool != null) { + CompletableFuture.runAsync(slotPool::close, mainThreadExecutor).join(); + } + } + } + + private static Scheduler createSchedulerWithSlots( + int numSlots, + SlotPool slotPool, + TaskManagerLocation taskManagerLocation) throws Exception { + final TaskManagerGateway taskManagerGateway = new SimpleAckingTaskManagerGateway(); + final String jobManagerAddress = "foobar"; + final ResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway(); + slotPool.start(JobMasterId.generate(), jobManagerAddress, mainThreadExecutor); + slotPool.connectToResourceManager(resourceManagerGateway); + Scheduler scheduler = new SchedulerImpl(LocationPreferenceSlotSelectionStrategy.INSTANCE, slotPool); + scheduler.start(mainThreadExecutor); + + CompletableFuture.runAsync(() -> slotPool.registerTaskManager(taskManagerLocation.getResourceID()), mainThreadExecutor).join(); + + final List slotOffers = new ArrayList<>(NUM_TASKS); + for (int i = 0; i < numSlots; i++) { + final AllocationID allocationId = new AllocationID(); + final SlotOffer slotOffer = new SlotOffer(allocationId, 0, ResourceProfile.ANY); + slotOffers.add(slotOffer); + } + + CompletableFuture.runAsync(() -> slotPool.offerSlots(taskManagerLocation, taskManagerGateway, slotOffers), mainThreadExecutor).join(); + + return scheduler; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java index f102346bd599..44a7758f633a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java @@ -46,7 +46,6 @@ import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.ScheduleMode; -import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; import org.apache.flink.runtime.jobmaster.JobMasterId; @@ -64,7 +63,6 @@ import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testtasks.NoOpInvokable; -import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TestLogger; @@ -565,7 +563,6 @@ public void slotPoolExecutionGraph_ConcurrentSchedulingAndAllocationFailure_Shou .setSlotProvider(slots) .setAllocationTimeout(Time.minutes(60)) .setScheduleMode(ScheduleMode.EAGER) - .setAllowQueuedScheduling(true) .build(); eg.start(mainThreadExecutor); @@ -632,52 +629,6 @@ public void testRestartWithEagerSchedulingAndSlotSharing() throws Exception { } } - @Test - public void testRestartWithSlotSharingAndNotEnoughResources() throws Exception { - final int numRestarts = 10; - final int parallelism = 20; - - try (SlotPool slotPool = createSlotPoolImpl()) { - final Scheduler scheduler = createSchedulerWithSlots( - parallelism - 1, slotPool, new LocalTaskManagerLocation()); - - final SlotSharingGroup sharingGroup = new SlotSharingGroup(); - - final JobVertex source = new JobVertex("source"); - source.setInvokableClass(NoOpInvokable.class); - source.setParallelism(parallelism); - source.setSlotSharingGroup(sharingGroup); - - final JobVertex sink = new JobVertex("sink"); - sink.setInvokableClass(NoOpInvokable.class); - sink.setParallelism(parallelism); - sink.setSlotSharingGroup(sharingGroup); - sink.connectNewDataSetAsInput(source, DistributionPattern.POINTWISE, ResultPartitionType.PIPELINED_BOUNDED); - - TestRestartStrategy restartStrategy = - new TestRestartStrategy(numRestarts, false); - - final ExecutionGraph eg = new ExecutionGraphTestUtils.TestingExecutionGraphBuilder(TEST_JOB_ID, source, sink) - .setSlotProvider(scheduler) - .setRestartStrategy(restartStrategy) - .setScheduleMode(ScheduleMode.EAGER) - .build(); - - eg.start(mainThreadExecutor); - eg.scheduleForExecution(); - - // the last suppressed restart is also counted - assertEquals(numRestarts + 1, eg.getNumberOfRestarts()); - - assertEquals(JobStatus.FAILED, eg.getState()); - - final Throwable t = eg.getFailureCause(); - if (!(t instanceof NoResourceAvailableException)) { - ExceptionUtils.rethrowException(t, t.getMessage()); - } - } - } - /** * Tests that the {@link ExecutionGraph} can handle failures while * being in the RESTARTING state. @@ -689,7 +640,6 @@ public void testFailureWhileRestarting() throws Exception { final ExecutionGraph executionGraph = new ExecutionGraphTestUtils.TestingExecutionGraphBuilder(createJobGraph()) .setRestartStrategy(restartStrategy) .setSlotProvider(new TestingSlotProvider(ignored -> new CompletableFuture<>())) - .allowQueuedScheduling() .build(); executionGraph.start(mainThreadExecutor); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java index 0aaf85799092..5323edb503a3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSchedulingTest.java @@ -121,7 +121,6 @@ public void testScheduleSourceBeforeTarget() throws Exception { final JobID jobId = new JobID(); final JobGraph jobGraph = new JobGraph(jobId, "test", sourceVertex, targetVertex); jobGraph.setScheduleMode(ScheduleMode.EAGER); - jobGraph.setAllowQueuedScheduling(true); final CompletableFuture sourceFuture = new CompletableFuture<>(); final CompletableFuture targetFuture = new CompletableFuture<>(); @@ -195,7 +194,6 @@ public void testDeployPipelinedConnectedComponentsTogether() throws Exception { final JobID jobId = new JobID(); final JobGraph jobGraph = new JobGraph(jobId, "test", sourceVertex, targetVertex); jobGraph.setScheduleMode(ScheduleMode.EAGER); - jobGraph.setAllowQueuedScheduling(true); @SuppressWarnings({"unchecked", "rawtypes"}) final CompletableFuture[] sourceFutures = new CompletableFuture[parallelism]; @@ -292,7 +290,6 @@ public void testOneSlotFailureAbortsDeploy() throws Exception { final JobID jobId = new JobID(); final JobGraph jobGraph = new JobGraph(jobId, "test", sourceVertex, targetVertex); jobGraph.setScheduleMode(ScheduleMode.EAGER); - jobGraph.setAllowQueuedScheduling(true); // // Create the slots, futures, and the slot provider @@ -376,7 +373,6 @@ public void testEagerSchedulingWithSlotTimeout() throws Exception { final JobID jobId = new JobID(); final JobGraph jobGraph = new JobGraph(jobId, "test", vertex); jobGraph.setScheduleMode(ScheduleMode.EAGER); - jobGraph.setAllowQueuedScheduling(true); final BlockingQueue returnedSlots = new ArrayBlockingQueue<>(2); final TestingSlotOwner slotOwner = new TestingSlotOwner(); @@ -436,7 +432,6 @@ public void testSchedulingOperationCancellationWhenCancel() throws Exception { jobVertex.setParallelism(2); final JobGraph jobGraph = new JobGraph(jobVertex); jobGraph.setScheduleMode(ScheduleMode.EAGER); - jobGraph.setAllowQueuedScheduling(true); final CompletableFuture slotFuture1 = new CompletableFuture<>(); final CompletableFuture slotFuture2 = new CompletableFuture<>(); @@ -478,7 +473,6 @@ public void testSlotReleasingFailsSchedulingOperation() throws Exception { jobVertex.setInvokableClass(NoOpInvokable.class); jobVertex.setParallelism(parallelism); final JobGraph jobGraph = new JobGraph(jobVertex); - jobGraph.setAllowQueuedScheduling(true); jobGraph.setScheduleMode(ScheduleMode.EAGER); final ProgrammedSlotProvider slotProvider = new ProgrammedSlotProvider(parallelism); @@ -520,7 +514,6 @@ public void testCancellationOfIncompleteScheduling() throws Exception { jobVertex.setParallelism(parallelism); final JobGraph jobGraph = new JobGraph(jobVertex); - jobGraph.setAllowQueuedScheduling(true); jobGraph.setScheduleMode(ScheduleMode.EAGER); final TestingSlotOwner slotOwner = new TestingSlotOwner(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java index 288f5f12835f..fe3621091690 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java @@ -627,16 +627,6 @@ public TestingExecutionGraphBuilder setPartitionTracker(final JobMasterPartition return this; } - public TestingExecutionGraphBuilder allowQueuedScheduling() { - jobGraph.setAllowQueuedScheduling(true); - return this; - } - - public TestingExecutionGraphBuilder setAllowQueuedScheduling(boolean allowQueuedScheduling) { - jobGraph.setAllowQueuedScheduling(allowQueuedScheduling); - return this; - } - public TestingExecutionGraphBuilder setScheduleMode(ScheduleMode scheduleMode) { jobGraph.setScheduleMode(scheduleMode); return this; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionPartitionLifecycleTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionPartitionLifecycleTest.java index 0e1d3ddc22e1..9c0cbbd14a76 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionPartitionLifecycleTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionPartitionLifecycleTest.java @@ -233,7 +233,7 @@ private void setupExecutionGraphAndStartRunningJob(ResultPartitionType resultPar final SlotProvider slotProvider = new SlotProvider() { @Override - public CompletableFuture allocateSlot(SlotRequestId slotRequestId, ScheduledUnit scheduledUnit, SlotProfile slotProfile, boolean allowQueuedScheduling, Time allocationTimeout) { + public CompletableFuture allocateSlot(SlotRequestId slotRequestId, ScheduledUnit scheduledUnit, SlotProfile slotProfile, Time allocationTimeout) { return CompletableFuture.completedFuture( new TestingLogicalSlotBuilder() .setTaskManagerLocation(taskManagerLocation) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java index 9172348615f8..4294452944b9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java @@ -293,7 +293,7 @@ public void testScheduleOrDeployAfterCancel() { // it can occur as the result of races { vertex.scheduleForExecution( - TestingSlotProviderStrategy.from(new ProgrammedSlotProvider(1), false), + TestingSlotProviderStrategy.from(new ProgrammedSlotProvider(1)), LocationPreferenceConstraint.ALL, Collections.emptySet()); @@ -333,7 +333,7 @@ public void testActionsWhileCancelling() { AkkaUtils.getDefaultTimeout()); setVertexState(vertex, ExecutionState.CANCELING); vertex.scheduleForExecution( - TestingSlotProviderStrategy.from(new ProgrammedSlotProvider(1), false), + TestingSlotProviderStrategy.from(new ProgrammedSlotProvider(1)), LocationPreferenceConstraint.ALL, Collections.emptySet()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java index cf55f7082d1c..babb755bd26e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java @@ -57,7 +57,7 @@ public void testSlotReleasedWhenScheduledImmediately() { assertEquals(ExecutionState.CREATED, vertex.getExecutionState()); // try to deploy to the slot vertex.scheduleForExecution( - TestingSlotProviderStrategy.from(new TestingSlotProvider((i) -> future), false), + TestingSlotProviderStrategy.from(new TestingSlotProvider((i) -> future)), LocationPreferenceConstraint.ALL, Collections.emptySet()); @@ -88,7 +88,7 @@ public void testSlotReleasedWhenScheduledQueued() { assertEquals(ExecutionState.CREATED, vertex.getExecutionState()); // try to deploy to the slot vertex.scheduleForExecution( - TestingSlotProviderStrategy.from(new TestingSlotProvider(ignore -> future), true), + TestingSlotProviderStrategy.from(new TestingSlotProvider(ignore -> future)), LocationPreferenceConstraint.ALL, Collections.emptySet()); @@ -121,7 +121,7 @@ public void testScheduleToDeploying() { // try to deploy to the slot vertex.scheduleForExecution( - TestingSlotProviderStrategy.from(new TestingSlotProvider(ignore -> future), false), + TestingSlotProviderStrategy.from(new TestingSlotProvider(ignore -> future)), LocationPreferenceConstraint.ALL, Collections.emptySet()); assertEquals(ExecutionState.DEPLOYING, vertex.getExecutionState()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ProgrammedSlotProvider.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ProgrammedSlotProvider.java index 7c5a848ea87f..9bcdab21bef0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ProgrammedSlotProvider.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ProgrammedSlotProvider.java @@ -117,7 +117,6 @@ public CompletableFuture allocateSlot( SlotRequestId slotRequestId, ScheduledUnit task, SlotProfile slotProfile, - boolean allowQueued, Time allocationTimeout) { JobVertexID vertexId = task.getTaskToExecute().getVertex().getJobvertexId(); int subtask = task.getTaskToExecute().getParallelSubtaskIndex(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingSlotProvider.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingSlotProvider.java index a79527881f39..49aa555889ac 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingSlotProvider.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingSlotProvider.java @@ -56,7 +56,7 @@ public void setSlotCanceller(Consumer slotCanceller) { } @Override - public CompletableFuture allocateSlot(SlotRequestId slotRequestId, ScheduledUnit task, SlotProfile slotProfile, boolean allowQueued, Time timeout) { + public CompletableFuture allocateSlot(SlotRequestId slotRequestId, ScheduledUnit task, SlotProfile slotProfile, Time timeout) { Preconditions.checkState(!slotFutures.containsKey(slotRequestId)); final CompletableFuture slotFuture = slotFutureCreator.apply(slotRequestId); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingSlotProviderStrategy.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingSlotProviderStrategy.java index 4d4c86c38c2d..b9504e999e85 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingSlotProviderStrategy.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingSlotProviderStrategy.java @@ -28,11 +28,11 @@ */ public class TestingSlotProviderStrategy extends SlotProviderStrategy.NormalSlotProviderStrategy{ - private TestingSlotProviderStrategy(SlotProvider slotProvider, Time allocationTimeout, boolean allowQueuedScheduling) { - super(slotProvider, allocationTimeout, allowQueuedScheduling); + private TestingSlotProviderStrategy(SlotProvider slotProvider, Time allocationTimeout) { + super(slotProvider, allocationTimeout); } - public static TestingSlotProviderStrategy from(SlotProvider slotProvider, boolean allowQueuedScheduling) { - return new TestingSlotProviderStrategy(slotProvider, Time.seconds(10L), allowQueuedScheduling); + public static TestingSlotProviderStrategy from(SlotProvider slotProvider) { + return new TestingSlotProviderStrategy(slotProvider, Time.seconds(10L)); } } 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 fe55bf351177..e6257016dc9e 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,7 +46,6 @@ 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; @@ -90,7 +89,6 @@ public CompletableFuture allocateSlot( SlotRequestId slotRequestId, ScheduledUnit task, SlotProfile slotProfile, - boolean allowQueued, Time allocationTimeout) { final SlotContext slot; @@ -112,8 +110,6 @@ public CompletableFuture allocateSlot( .createTestingLogicalSlot(); allocatedSlots.put(slotRequestId, slot); return CompletableFuture.completedFuture(result); - } else if (allowQueued) { - return FutureUtils.completedExceptionally(new TimeoutException()); } else { return FutureUtils.completedExceptionally(new NoResourceAvailableException()); } 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 8e6a22f0f9b2..43bf9bab1256 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 @@ -121,14 +121,7 @@ private JobGraph createTestJobGraph( DistributionPattern.ALL_TO_ALL, ResultPartitionType.BLOCKING); - final JobGraph jobGraph = new JobGraph(jobName, sender, receiver); - - // We need to allow queued scheduling, because there are not enough slots available - // to run all tasks at once. We queue tasks and then let them finish/consume the blocking - // result one after the other. - jobGraph.setAllowQueuedScheduling(true); - - return jobGraph; + return new JobGraph(jobName, sender, receiver); } /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java index cad0af547fe7..df7b0222a7f3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java @@ -62,18 +62,18 @@ public void scheduleAllSharedAndCoLocated() throws Exception { CoLocationConstraint c6 = new CoLocationConstraint(ccg); // schedule 4 tasks from the first vertex group - LogicalSlot s1 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s2 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c2), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s3 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c3), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s4 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c4), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s5 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s6 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c2), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s7 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c3), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s8 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 4, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c5), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s9 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 5, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c6), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s10 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c4), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s11 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 4, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c5), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s12 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 5, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c6), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot s1 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot s2 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c2), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot s3 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c3), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot s4 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c4), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot s5 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot s6 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c2), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot s7 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c3), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot s8 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 4, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c5), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot s9 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 5, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c6), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot s10 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c4), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot s11 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 4, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c5), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot s12 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 5, 6, sharingGroup), sharingGroup.getSlotSharingGroupId(), c6), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); assertNotNull(s1); assertNotNull(s2); @@ -123,7 +123,7 @@ public void scheduleAllSharedAndCoLocated() throws Exception { assertTrue(testingSlotProvider.getNumberOfAvailableSlots() >= 1); LogicalSlot single = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertex(new JobVertexID(), 0, 1, null)), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertex(new JobVertexID(), 0, 1, null)), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); assertNotNull(single); s1.releaseSlot(); @@ -160,11 +160,11 @@ public void scheduleWithIntermediateRelease() throws Exception { CoLocationConstraint c1 = new CoLocationConstraint(new CoLocationGroup()); LogicalSlot s1 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertex(jid1, 0, 1, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertex(jid1, 0, 1, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); LogicalSlot s2 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertex(jid2, 0, 1, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertex(jid2, 0, 1, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot sSolo = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 0, 1, null)), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot sSolo = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 0, 1, null)), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); ResourceID taskManager = s1.getTaskManagerLocation().getResourceID(); @@ -173,7 +173,7 @@ public void scheduleWithIntermediateRelease() throws Exception { sSolo.releaseSlot(); LogicalSlot sNew = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertex(jid3, 0, 1, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertex(jid3, 0, 1, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); assertEquals(taskManager, sNew.getTaskManagerLocation().getResourceID()); assertEquals(2, testingSlotProvider.getNumberOfLocalizedAssignments()); @@ -196,14 +196,14 @@ public void scheduleWithReleaseNoResource() throws Exception { CoLocationConstraint c1 = new CoLocationConstraint(new CoLocationGroup()); LogicalSlot s1 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertex(jid1, 0, 1, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertex(jid1, 0, 1, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); s1.releaseSlot(); - testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 1, null)), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 2, null)), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 1, null)), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 2, null)), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); try { - testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 0, 1, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 0, 1, sharingGroup), sharingGroup.getSlotSharingGroupId(), c1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); fail("Scheduled even though no resource was available."); } catch (ExecutionException e) { assertTrue(e.getCause() instanceof NoResourceAvailableException); @@ -237,35 +237,35 @@ public void scheduleMixedCoLocationSlotSharing() throws Exception { SlotSharingGroup shareGroup = new SlotSharingGroup(); // first wave - testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 4, shareGroup), shareGroup.getSlotSharingGroupId()), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 4, shareGroup), shareGroup.getSlotSharingGroupId()), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 4, shareGroup), shareGroup.getSlotSharingGroupId()), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 4, shareGroup), shareGroup.getSlotSharingGroupId()), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 4, shareGroup), shareGroup.getSlotSharingGroupId()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 4, shareGroup), shareGroup.getSlotSharingGroupId()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 4, shareGroup), shareGroup.getSlotSharingGroupId()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 4, shareGroup), shareGroup.getSlotSharingGroupId()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); // second wave LogicalSlot s21 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertex(jid2, 0, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc1), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertex(jid2, 0, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); LogicalSlot s22 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertex(jid2, 2, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc2), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertex(jid2, 2, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc2), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); LogicalSlot s23 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertex(jid2, 1, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc3), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertex(jid2, 1, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc3), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); LogicalSlot s24 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertex(jid2, 3, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc4), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertex(jid2, 3, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc4), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); // third wave LogicalSlot s31 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertex(jid3, 1, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc2), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertex(jid3, 1, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc2), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); LogicalSlot s32 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertex(jid3, 2, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc3), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertex(jid3, 2, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc3), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); LogicalSlot s33 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertex(jid3, 3, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc4), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertex(jid3, 3, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc4), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); LogicalSlot s34 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertex(jid3, 0, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc1), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertex(jid3, 0, 4, shareGroup), shareGroup.getSlotSharingGroupId(), clc1), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 0, 4, shareGroup), shareGroup.getSlotSharingGroupId()), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 1, 4, shareGroup), shareGroup.getSlotSharingGroupId()), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 2, 4, shareGroup), shareGroup.getSlotSharingGroupId()), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 3, 4, shareGroup), shareGroup.getSlotSharingGroupId()), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 0, 4, shareGroup), shareGroup.getSlotSharingGroupId()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 1, 4, shareGroup), shareGroup.getSlotSharingGroupId()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 2, 4, shareGroup), shareGroup.getSlotSharingGroupId()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 3, 4, shareGroup), shareGroup.getSlotSharingGroupId()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); assertEquals(s21.getTaskManagerLocation(), s34.getTaskManagerLocation()); assertEquals(s22.getTaskManagerLocation(), s31.getTaskManagerLocation()); @@ -297,25 +297,25 @@ public void testGetsNonLocalFromSharingGroupFirst() throws Exception { // schedule something into the shared group so that both instances are in the sharing group LogicalSlot s1 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId()), false, slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId()), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); LogicalSlot s2 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId()), false, slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId()), slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); // schedule one locally to instance 1 LogicalSlot s3 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc1), false, slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); // schedule with co location constraint (yet unassigned) and a preference for // instance 1, but it can only get instance 2 LogicalSlot s4 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc2), false, slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc2), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); // schedule something into the assigned co-location constraints and check that they override the // other preferences LogicalSlot s5 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid3, 0, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId(), cc1), false, slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertexWithLocation(jid3, 0, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); LogicalSlot s6 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid3, 1, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc2), false, slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertexWithLocation(jid3, 1, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc2), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); // check that each slot got three assertEquals(s1.getTaskManagerLocation(), s3.getTaskManagerLocation()); @@ -357,9 +357,9 @@ public void testSlotReleasedInBetween() throws Exception { CoLocationConstraint cc2 = new CoLocationConstraint(ccg); LogicalSlot s1 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc1), false, slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); LogicalSlot s2 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId(), cc2), false, slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId(), cc2), slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); s1.releaseSlot(); s2.releaseSlot(); @@ -367,9 +367,9 @@ public void testSlotReleasedInBetween() throws Exception { assertEquals(2, testingSlotProvider.getNumberOfAvailableSlots()); LogicalSlot s3 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId(), cc1), false, slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); LogicalSlot s4 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc2), false, slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc2), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); // still preserves the previous instance mapping) assertEquals(loc1, s3.getTaskManagerLocation()); @@ -403,9 +403,9 @@ public void testSlotReleasedInBetweenAndNoNewLocal() throws Exception { CoLocationConstraint cc2 = new CoLocationConstraint(ccg); LogicalSlot s1 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc1), false, slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); LogicalSlot s2 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId(), cc2), false, slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId(), cc2), slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); s1.releaseSlot(); s2.releaseSlot(); @@ -413,13 +413,13 @@ public void testSlotReleasedInBetweenAndNoNewLocal() throws Exception { assertEquals(2, testingSlotProvider.getNumberOfAvailableSlots()); LogicalSlot sa = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jidx, 0, 2, null)), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertexWithLocation(jidx, 0, 2, null)), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); LogicalSlot sb = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jidx, 1, 2, null)), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertexWithLocation(jidx, 1, 2, null)), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); try { testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId(), cc1), false, slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); fail("should not be able to find a resource"); } catch (ExecutionException e) { @@ -459,14 +459,14 @@ public void testScheduleOutOfOrder() throws Exception { // and give locality preferences that hint at using the same shared slot for both // co location constraints (which we seek to prevent) LogicalSlot s1 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc1), false, slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); LogicalSlot s2 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc2), false, slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc2), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); LogicalSlot s3 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc1), false, slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); LogicalSlot s4 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc2), false, slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc2), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); // check that each slot got three assertEquals(s1.getTaskManagerLocation(), s3.getTaskManagerLocation()); @@ -504,14 +504,14 @@ public void nonColocationFollowsCoLocation() throws Exception { CoLocationConstraint cc2 = new CoLocationConstraint(ccg); LogicalSlot s1 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc1), false, slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId(), cc1), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); LogicalSlot s2 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId(), cc2), false, slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, sharingGroup, loc2), sharingGroup.getSlotSharingGroupId(), cc2), slotProfileForLocation(loc2), TestingUtils.infiniteTime()).get(); LogicalSlot s3 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId()), false, slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId()), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); LogicalSlot s4 = testingSlotProvider.allocateSlot( - new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId()), false, slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); + new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, sharingGroup, loc1), sharingGroup.getSlotSharingGroupId()), slotProfileForLocation(loc1), TestingUtils.infiniteTime()).get(); // check that each slot got two assertEquals(s1.getTaskManagerLocation(), s3.getTaskManagerLocation()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java index 75f9a70372f0..e3214b1bb774 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java @@ -26,6 +26,7 @@ import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.junit.Ignore; import org.junit.Test; import java.util.ArrayList; @@ -52,65 +53,6 @@ * Tests for scheduling individual tasks. */ public class SchedulerIsolatedTasksTest extends SchedulerTestBase { - - @Test - public void testScheduleImmediately() throws Exception { - assertEquals(0, testingSlotProvider.getNumberOfAvailableSlots()); - - testingSlotProvider.addTaskManager(2); - testingSlotProvider.addTaskManager(1); - testingSlotProvider.addTaskManager(2); - assertEquals(5, testingSlotProvider.getNumberOfAvailableSlots()); - - // schedule something into all slots - LogicalSlot s1 = testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s2 = testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s3 = testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s4 = testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s5 = testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - - // the slots should all be different - assertTrue(areAllDistinct(s1, s2, s3, s4, s5)); - - try { - testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - fail("Scheduler accepted scheduling request without available resource."); - } - catch (ExecutionException e) { - assertTrue(e.getCause() instanceof NoResourceAvailableException); - } - - // release some slots again - s3.releaseSlot(); - s4.releaseSlot(); - assertEquals(2, testingSlotProvider.getNumberOfAvailableSlots()); - - // now we can schedule some more slots - LogicalSlot s6 = testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - LogicalSlot s7 = testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); - - assertTrue(areAllDistinct(s1, s2, s3, s4, s5, s6, s7)); - - // release all - - s1.releaseSlot(); - s2.releaseSlot(); - s5.releaseSlot(); - s6.releaseSlot(); - s7.releaseSlot(); - - assertEquals(5, testingSlotProvider.getNumberOfAvailableSlots()); - - // check that slots that are released twice (accidentally) do not mess things up - - s1.releaseSlot(); - s2.releaseSlot(); - s5.releaseSlot(); - s6.releaseSlot(); - s7.releaseSlot(); - - assertEquals(5, testingSlotProvider.getNumberOfAvailableSlots()); - } @Test public void testScheduleQueueing() throws Exception { @@ -134,7 +76,7 @@ public void testScheduleQueueing() throws Exception { final AtomicBoolean errored = new AtomicBoolean(false); for (int i = 0; i < NUM_TASKS_TO_SCHEDULE; i++) { - CompletableFuture future = testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), true, SlotProfile.noRequirements(), TestingUtils.infiniteTime()); + CompletableFuture future = testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()); future.thenAcceptAsync( (LogicalSlot slot) -> { synchronized (toRelease) { @@ -179,19 +121,20 @@ public void testScheduleQueueing() throws Exception { assertEquals("All slots should be available.", totalSlots, testingSlotProvider.getNumberOfAvailableSlots()); } - + @Test + @Ignore public void testScheduleWithDyingInstances() throws Exception { final TaskManagerLocation taskManagerLocation1 = testingSlotProvider.addTaskManager(2); final TaskManagerLocation taskManagerLocation2 = testingSlotProvider.addTaskManager(2); final TaskManagerLocation taskManagerLocation3 = testingSlotProvider.addTaskManager(1); List slots = new ArrayList<>(); - slots.add(testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get()); - slots.add(testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get()); - slots.add(testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get()); - slots.add(testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get()); - slots.add(testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get()); + slots.add(testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get()); + slots.add(testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get()); + slots.add(testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get()); + slots.add(testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get()); + slots.add(testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get()); testingSlotProvider.releaseTaskManager(taskManagerLocation2.getResourceID()); @@ -212,7 +155,7 @@ public void testScheduleWithDyingInstances() throws Exception { // cannot get another slot, since all instances are dead try { - testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); fail("Scheduler served a slot from a dead instance"); } catch (ExecutionException e) { @@ -226,7 +169,7 @@ public void testScheduleWithDyingInstances() throws Exception { // that all instances have vanished assertEquals(0, testingSlotProvider.getNumberOfAvailableSlots()); } - + @Test public void testSchedulingLocation() throws Exception { final TaskManagerLocation taskManagerLocation1 = testingSlotProvider.addTaskManager(2); @@ -234,7 +177,7 @@ public void testSchedulingLocation() throws Exception { final TaskManagerLocation taskManagerLocation3 = testingSlotProvider.addTaskManager(2); // schedule something on an arbitrary instance - LogicalSlot s1 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(new LocalTaskManagerLocation())), false, SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + LogicalSlot s1 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(new LocalTaskManagerLocation())), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); // figure out how we use the location hints ResourceID firstResourceId = s1.getTaskManagerLocation().getResourceID(); @@ -256,28 +199,28 @@ public void testSchedulingLocation() throws Exception { TaskManagerLocation third = taskManagerLocations.get((index + 2) % taskManagerLocations.size()); // something that needs to go to the first instance again - LogicalSlot s2 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(s1.getTaskManagerLocation())), false, slotProfileForLocation(s1.getTaskManagerLocation()), TestingUtils.infiniteTime()).get(); + LogicalSlot s2 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(s1.getTaskManagerLocation())), slotProfileForLocation(s1.getTaskManagerLocation()), TestingUtils.infiniteTime()).get(); assertEquals(first.getResourceID(), s2.getTaskManagerLocation().getResourceID()); // first or second --> second, because first is full - LogicalSlot s3 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(first, second)), false, slotProfileForLocation(first, second), TestingUtils.infiniteTime()).get(); + LogicalSlot s3 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(first, second)), slotProfileForLocation(first, second), TestingUtils.infiniteTime()).get(); assertEquals(second.getResourceID(), s3.getTaskManagerLocation().getResourceID()); // first or third --> third (because first is full) - LogicalSlot s4 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), false, slotProfileForLocation(first, third), TestingUtils.infiniteTime()).get(); - LogicalSlot s5 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), false, slotProfileForLocation(first, third), TestingUtils.infiniteTime()).get(); + LogicalSlot s4 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), slotProfileForLocation(first, third), TestingUtils.infiniteTime()).get(); + LogicalSlot s5 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), slotProfileForLocation(first, third), TestingUtils.infiniteTime()).get(); assertEquals(third.getResourceID(), s4.getTaskManagerLocation().getResourceID()); assertEquals(third.getResourceID(), s5.getTaskManagerLocation().getResourceID()); // first or third --> second, because all others are full - LogicalSlot s6 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), false, slotProfileForLocation(first, third), TestingUtils.infiniteTime()).get(); + LogicalSlot s6 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), slotProfileForLocation(first, third), TestingUtils.infiniteTime()).get(); assertEquals(second.getResourceID(), s6.getTaskManagerLocation().getResourceID()); // release something on the first and second instance s2.releaseSlot(); s6.releaseSlot(); - LogicalSlot s7 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), false, slotProfileForLocation(first, third), TestingUtils.infiniteTime()).get(); + LogicalSlot s7 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), slotProfileForLocation(first, third), TestingUtils.infiniteTime()).get(); assertEquals(first.getResourceID(), s7.getTaskManagerLocation().getResourceID()); assertEquals(1, testingSlotProvider.getNumberOfUnconstrainedAssignments()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestBase.java index fba62593289d..bab5aead7de9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestBase.java @@ -235,9 +235,8 @@ public CompletableFuture allocateSlot( SlotRequestId slotRequestId, ScheduledUnit task, SlotProfile slotProfile, - boolean allowQueued, Time allocationTimeout) { - return scheduler.allocateSlot(task, allowQueued, slotProfile, allocationTimeout).thenApply( + return scheduler.allocateSlot(task, slotProfile, allocationTimeout).thenApply( (LogicalSlot logicalSlot) -> { switch (logicalSlot.getLocality()) { case LOCAL: diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java index e7f061ee7c69..7b7a6247a361 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java @@ -1079,16 +1079,15 @@ private void runRequestNextInputSplitTest(Function>, Colle source.setInputSplitSource(inputSplitSource); source.setInvokableClass(AbstractInvokable.class); - final JobGraph inputSplitjobGraph = new JobGraph(source); - inputSplitjobGraph.setAllowQueuedScheduling(true); + final JobGraph inputSplitJobGraph = new JobGraph(source); final ExecutionConfig executionConfig = new ExecutionConfig(); executionConfig.setRestartStrategy(RestartStrategies.fixedDelayRestart(100, 0)); - inputSplitjobGraph.setExecutionConfig(executionConfig); + inputSplitJobGraph.setExecutionConfig(executionConfig); final JobMaster jobMaster = createJobMaster( configuration, - inputSplitjobGraph, + inputSplitJobGraph, haServices, new TestingJobManagerSharedServicesBuilder().build(), heartbeatServices); @@ -1118,7 +1117,7 @@ private void runRequestNextInputSplitTest(Function>, Colle waitUntilAllExecutionsAreScheduled(jobMasterGateway); // fail the first execution to trigger a failover - jobMasterGateway.updateTaskExecutionState(new TaskExecutionState(inputSplitjobGraph.getJobID(), initialAttemptId, ExecutionState.FAILED)).get(); + jobMasterGateway.updateTaskExecutionState(new TaskExecutionState(inputSplitJobGraph.getJobID(), initialAttemptId, ExecutionState.FAILED)).get(); // wait until the job has been recovered waitUntilAllExecutionsAreScheduled(jobMasterGateway); @@ -2119,10 +2118,7 @@ private JobGraph producerConsumerJobGraph() { consumer.connectNewDataSetAsInput(producer, DistributionPattern.POINTWISE, ResultPartitionType.BLOCKING); - final JobGraph jobGraph = new JobGraph(producer, consumer); - jobGraph.setAllowQueuedScheduling(true); - - return jobGraph; + return new JobGraph(producer, consumer); } private File createSavepoint(long savepointId) throws IOException { @@ -2346,9 +2342,7 @@ private JobGraph createSingleVertexJobGraph() { final JobVertex jobVertex = new JobVertex("Test vertex"); jobVertex.setInvokableClass(NoOpInvokable.class); - final JobGraph jobGraph = new JobGraph(jobVertex); - jobGraph.setAllowQueuedScheduling(true); - return jobGraph; + return new JobGraph(jobVertex); } private static final class DummyCheckpointStorageLocation implements CompletedCheckpointStorageLocation { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolCoLocationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolCoLocationTest.java index 52cc3afa3e2f..b9601ccb1d2d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolCoLocationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolCoLocationTest.java @@ -98,7 +98,6 @@ public void testSimpleCoLocatedSlotScheduling() throws ExecutionException, Inter jobVertexId1, slotSharingGroupId, coLocationConstraint1), - true, SlotProfile.noRequirements(), TestingUtils.infiniteTime()); @@ -107,7 +106,6 @@ public void testSimpleCoLocatedSlotScheduling() throws ExecutionException, Inter jobVertexId2, slotSharingGroupId, coLocationConstraint2), - true, SlotProfile.noRequirements(), TestingUtils.infiniteTime()); @@ -116,7 +114,6 @@ public void testSimpleCoLocatedSlotScheduling() throws ExecutionException, Inter jobVertexId2, slotSharingGroupId, coLocationConstraint1), - true, SlotProfile.noRequirements(), TestingUtils.infiniteTime()); @@ -125,7 +122,6 @@ public void testSimpleCoLocatedSlotScheduling() throws ExecutionException, Inter jobVertexId1, slotSharingGroupId, coLocationConstraint2), - true, SlotProfile.noRequirements(), TestingUtils.infiniteTime()); @@ -196,7 +192,6 @@ public void testCoLocatedSlotRequestsFailBeforeResolved() throws ExecutionExcept jobVertexId1, slotSharingGroupId, coLocationConstraint1), - true, SlotProfile.noLocality(rp1), TestingUtils.infiniteTime()); @@ -205,7 +200,6 @@ public void testCoLocatedSlotRequestsFailBeforeResolved() throws ExecutionExcept jobVertexId2, slotSharingGroupId, coLocationConstraint1), - true, SlotProfile.noLocality(rp2), TestingUtils.infiniteTime()); @@ -214,7 +208,6 @@ public void testCoLocatedSlotRequestsFailBeforeResolved() throws ExecutionExcept jobVertexId3, slotSharingGroupId, coLocationConstraint1), - true, SlotProfile.noLocality(rp3), TestingUtils.infiniteTime()); @@ -274,7 +267,6 @@ public void testCoLocatedSlotRequestsFailAfterResolved() throws ExecutionExcepti jobVertexId1, slotSharingGroupId, coLocationConstraint1), - true, SlotProfile.noLocality(rp1), TestingUtils.infiniteTime()); @@ -283,7 +275,6 @@ public void testCoLocatedSlotRequestsFailAfterResolved() throws ExecutionExcepti jobVertexId2, slotSharingGroupId, coLocationConstraint1), - true, SlotProfile.noLocality(rp2), TestingUtils.infiniteTime()); @@ -304,7 +295,6 @@ public void testCoLocatedSlotRequestsFailAfterResolved() throws ExecutionExcepti jobVertexId3, slotSharingGroupId, coLocationConstraint1), - true, SlotProfile.noLocality(rp3), TestingUtils.infiniteTime()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolImplTest.java index e2658ec3e09f..d36be1ad5263 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolImplTest.java @@ -127,7 +127,6 @@ public void testAllocateSimpleSlot() throws Exception { requestId, new DummyScheduledUnit(), SlotProfile.noLocality(DEFAULT_TESTING_PROFILE), - true, timeout); assertFalse(future.isDone()); @@ -171,13 +170,11 @@ public void testAllocationFulfilledByReturnedSlot() throws Exception { new SlotRequestId(), new DummyScheduledUnit(), SlotProfile.noLocality(DEFAULT_TESTING_PROFILE), - true, timeout); CompletableFuture future2 = scheduler.allocateSlot( new SlotRequestId(), new DummyScheduledUnit(), SlotProfile.noLocality(DEFAULT_TESTING_PROFILE), - true, timeout); assertFalse(future1.isDone()); @@ -230,7 +227,6 @@ public void testAllocateWithFreeSlot() throws Exception { new SlotRequestId(), new DummyScheduledUnit(), SlotProfile.noLocality(DEFAULT_TESTING_PROFILE), - true, timeout); assertFalse(future1.isDone()); @@ -253,7 +249,6 @@ public void testAllocateWithFreeSlot() throws Exception { new SlotRequestId(), new DummyScheduledUnit(), SlotProfile.noLocality(DEFAULT_TESTING_PROFILE), - true, timeout); // second allocation fulfilled by previous slot returning @@ -283,7 +278,6 @@ public void testOfferSlot() throws Exception { new SlotRequestId(), new DummyScheduledUnit(), SlotProfile.noLocality(DEFAULT_TESTING_PROFILE), - true, timeout); assertFalse(future.isDone()); @@ -348,7 +342,6 @@ public void testReleaseResource() throws Exception { new SlotRequestId(), new DummyScheduledUnit(), SlotProfile.noLocality(DEFAULT_TESTING_PROFILE), - true, timeout); final SlotRequest slotRequest = slotRequestFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); @@ -357,7 +350,6 @@ public void testReleaseResource() throws Exception { new SlotRequestId(), new DummyScheduledUnit(), SlotProfile.noLocality(DEFAULT_TESTING_PROFILE), - true, timeout); final SlotOffer slotOffer = new SlotOffer( @@ -418,7 +410,6 @@ public void testFulfillingSlotRequestsWithUnusedOfferedSlots() throws Exception slotRequestId1, scheduledUnit, SlotProfile.noRequirements(), - true, timeout); // wait for the first slot request @@ -428,7 +419,6 @@ public void testFulfillingSlotRequestsWithUnusedOfferedSlots() throws Exception slotRequestId2, scheduledUnit, SlotProfile.noRequirements(), - true, timeout); // wait for the second slot request @@ -757,7 +747,6 @@ private CompletableFuture allocateSlot(Scheduler scheduler, SlotReq slotRequestId, new DummyScheduledUnit(), SlotProfile.noRequirements(), - true, timeout); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolInteractionsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolInteractionsTest.java index ce096acadd1b..cda9370620bb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolInteractionsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolInteractionsTest.java @@ -98,7 +98,6 @@ public void testSlotAllocationNoResourceManager() throws Exception { new SlotRequestId(), new ScheduledUnit(SchedulerTestUtils.getDummyTask()), SlotProfile.noLocality(DEFAULT_TESTING_PROFILE), - true, fastTimeout)); try { @@ -127,7 +126,6 @@ public void testCancelSlotAllocationWithoutResourceManager() throws Exception { requestId, new ScheduledUnit(SchedulerTestUtils.getDummyTask()), SlotProfile.noLocality(DEFAULT_TESTING_PROFILE), - true, fastTimeout)); try { @@ -179,7 +177,6 @@ public void testSlotAllocationTimeout() throws Exception { requestId, new DummyScheduledUnit(), SlotProfile.noLocality(DEFAULT_TESTING_PROFILE), - true, fastTimeout)); try { @@ -226,7 +223,6 @@ public void testExtraSlotsAreKept() throws Exception { requestId, new ScheduledUnit(SchedulerTestUtils.getDummyTask()), SlotProfile.noLocality(DEFAULT_TESTING_PROFILE), - true, fastTimeout)); try { @@ -281,7 +277,6 @@ public void testProviderAndOwnerSlotAllocationTimeout() throws Exception { // test the pending request is clear when timed out CompletableFuture future = testMainThreadExecutor.execute(() -> scheduler.allocateSlot( new DummyScheduledUnit(), - true, SlotProfile.noRequirements(), fastTimeout)); try { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolSlotSharingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolSlotSharingTest.java index caa3fc38c601..08aafd02411b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolSlotSharingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolSlotSharingTest.java @@ -76,7 +76,6 @@ public void testSingleQueuedSharedSlotScheduling() throws Exception { new JobVertexID(), slotSharingGroupId, null), - true, SlotProfile.noRequirements(), TestingUtils.infiniteTime()); @@ -115,7 +114,6 @@ public void testFailingQueuedSharedSlotScheduling() throws Exception { new JobVertexID(), new SlotSharingGroupId(), null), - true, SlotProfile.noRequirements(), TestingUtils.infiniteTime()); @@ -158,7 +156,6 @@ public void testQueuedSharedSlotScheduling() throws Exception { jobVertexId1, slotSharingGroupId, null), - true, SlotProfile.noRequirements(), TestingUtils.infiniteTime()); @@ -167,7 +164,6 @@ public void testQueuedSharedSlotScheduling() throws Exception { jobVertexId2, slotSharingGroupId, null), - true, SlotProfile.noRequirements(), TestingUtils.infiniteTime()); @@ -181,7 +177,6 @@ public void testQueuedSharedSlotScheduling() throws Exception { jobVertexId1, slotSharingGroupId, null), - true, SlotProfile.noRequirements(), TestingUtils.infiniteTime()); @@ -190,7 +185,6 @@ public void testQueuedSharedSlotScheduling() throws Exception { jobVertexId2, slotSharingGroupId, null), - true, SlotProfile.noRequirements(), TestingUtils.infiniteTime()); @@ -260,7 +254,6 @@ public void testQueuedMultipleSlotSharingGroups() throws Exception { jobVertexId1, slotSharingGroupId1, null), - true, SlotProfile.noRequirements(), TestingUtils.infiniteTime()); @@ -269,7 +262,6 @@ public void testQueuedMultipleSlotSharingGroups() throws Exception { jobVertexId2, slotSharingGroupId1, null), - true, SlotProfile.noRequirements(), TestingUtils.infiniteTime()); @@ -278,7 +270,6 @@ public void testQueuedMultipleSlotSharingGroups() throws Exception { jobVertexId3, slotSharingGroupId2, null), - true, SlotProfile.noRequirements(), TestingUtils.infiniteTime()); @@ -287,7 +278,6 @@ public void testQueuedMultipleSlotSharingGroups() throws Exception { jobVertexId4, slotSharingGroupId2, null), - true, SlotProfile.noRequirements(), TestingUtils.infiniteTime()); @@ -362,7 +352,6 @@ public void testSlotSharingRespectsRemainingResource() throws Exception { jobVertexId1, slotSharingGroupId, null), - true, SlotProfile.noLocality(largeRequestResource), TestingUtils.infiniteTime()); @@ -387,7 +376,6 @@ public void testSlotSharingRespectsRemainingResource() throws Exception { jobVertexId2, slotSharingGroupId, null), - true, SlotProfile.noLocality(largeRequestResource), TestingUtils.infiniteTime()); assertFalse(logicalSlotFuture2.isDone()); @@ -398,7 +386,6 @@ public void testSlotSharingRespectsRemainingResource() throws Exception { jobVertexId3, slotSharingGroupId, null), - true, SlotProfile.noLocality(smallRequestResource), TestingUtils.infiniteTime()); assertTrue(logicalSlotFuture3.isDone()); @@ -450,7 +437,6 @@ public void testRetryOnSharedSlotOverAllocated() throws InterruptedException, Ex jobVertexId1, slotSharingGroupId, null), - true, SlotProfile.noLocality(rp1), TestingUtils.infiniteTime()); @@ -459,7 +445,6 @@ public void testRetryOnSharedSlotOverAllocated() throws InterruptedException, Ex jobVertexId2, slotSharingGroupId, null), - true, SlotProfile.noLocality(rp2), TestingUtils.infiniteTime()); @@ -468,7 +453,6 @@ public void testRetryOnSharedSlotOverAllocated() throws InterruptedException, Ex jobVertexId3, slotSharingGroupId, null), - true, SlotProfile.noLocality(rp3), TestingUtils.infiniteTime()); 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 ce3f83711adc..d6eb59eddafb 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 @@ -284,8 +284,7 @@ private DefaultExecutionSlotAllocator createExecutionSlotAllocator(InputsLocatio SlotProviderStrategy.from( ScheduleMode.EAGER, slotProvider, - Time.seconds(10), - true), + Time.seconds(10)), inputsLocationsRetriever); } @@ -323,7 +322,6 @@ public CompletableFuture allocateSlot( SlotRequestId slotRequestId, ScheduledUnit task, SlotProfile slotProfile, - boolean allowQueued, Time timeout) { slotAllocationRequests.add(Tuple3.of(slotRequestId, task, slotProfile)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/LegacySchedulerBatchSchedulingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/LegacySchedulerBatchSchedulingTest.java index 03df7130daab..e2155d99e057 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/LegacySchedulerBatchSchedulingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/LegacySchedulerBatchSchedulingTest.java @@ -218,10 +218,7 @@ private JobGraph createJobGraph(int parallelism) { final JobVertex jobVertex = new JobVertex("testing task"); jobVertex.setParallelism(parallelism); jobVertex.setInvokableClass(NoOpInvokable.class); - final JobGraph jobGraph = new JobGraph(jobId, "test job", jobVertex); - jobGraph.setAllowQueuedScheduling(true); - - return jobGraph; + return new JobGraph(jobId, "test job", jobVertex); } private static class GloballyTerminalJobStatusListener implements JobStatusListener { 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 9b1618ee1c56..f958b70b8f60 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 @@ -425,10 +425,6 @@ public ClusterClient deployJobCluster( ClusterSpecification clusterSpecification, JobGraph jobGraph, boolean detached) throws ClusterDeploymentException { - - // this is required because the slots are allocated lazily - jobGraph.setAllowQueuedScheduling(true); - try { return deployInternal( clusterSpecification, From 7e3a92b7e86c334564eed77defd1d355835a4044 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 4 Nov 2019 11:10:27 +0100 Subject: [PATCH 429/746] [hotfix] Fix checkstyle violations in SchedulerIsolatedTasksTest --- .../scheduler/SchedulerIsolatedTasksTest.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java index e3214b1bb774..f1ae85719b3b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java @@ -56,12 +56,12 @@ public class SchedulerIsolatedTasksTest extends SchedulerTestBase { @Test public void testScheduleQueueing() throws Exception { - final int NUM_INSTANCES = 50; - final int NUM_SLOTS_PER_INSTANCE = 3; - final int NUM_TASKS_TO_SCHEDULE = 2000; + final int numInstances = 50; + final int numSlotsPerInstance = 3; + final int numTasksToSchedule = 2000; - for (int i = 0; i < NUM_INSTANCES; i++) { - testingSlotProvider.addTaskManager((int) (Math.random() * NUM_SLOTS_PER_INSTANCE) + 1); + for (int i = 0; i < numInstances; i++) { + testingSlotProvider.addTaskManager((int) (Math.random() * numSlotsPerInstance) + 1); } final int totalSlots = testingSlotProvider.getNumberOfAvailableSlots(); @@ -75,7 +75,7 @@ public void testScheduleQueueing() throws Exception { // flag to track errors in the concurrent thread final AtomicBoolean errored = new AtomicBoolean(false); - for (int i = 0; i < NUM_TASKS_TO_SCHEDULE; i++) { + for (int i = 0; i < numTasksToSchedule; i++) { CompletableFuture future = testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()); future.thenAcceptAsync( (LogicalSlot slot) -> { @@ -90,7 +90,7 @@ public void testScheduleQueueing() throws Exception { try { int recycled = 0; - while (recycled < NUM_TASKS_TO_SCHEDULE) { + while (recycled < numTasksToSchedule) { synchronized (toRelease) { while (toRelease.isEmpty()) { toRelease.wait(); From ee32514a3527818dc5f77364ce2308c5c1d5ee9f Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 4 Nov 2019 11:16:14 +0100 Subject: [PATCH 430/746] [hotfix] Fix checkstyle violation in SchedulerTestBase --- .../flink/runtime/jobmanager/scheduler/SchedulerTestBase.java | 1 - 1 file changed, 1 deletion(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestBase.java index bab5aead7de9..ef19c250bc79 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestBase.java @@ -264,7 +264,6 @@ public void cancelSlotRequest(SlotRequestId slotRequestId, @Nullable SlotSharing } } - /** * Test implementation of scheduler that offers a bit more introspection. */ From aa10ec88c767a312455cd59ac5f5dc6d62cd71b5 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 4 Nov 2019 11:55:02 +0100 Subject: [PATCH 431/746] [hotfix] Enable SchedulerTestBase to use different ComponentMainThreadExecutors The SchedulerTestBase has now an abstract supplier method for the ComponentMainThreadExecutor. That way subclasses can control which ComponentMainThreadExecutor is being used. --- .../ExecutionGraphCoLocationRestartTest.java | 6 ++ .../ScheduleWithCoLocationHintTest.java | 7 ++ .../scheduler/SchedulerIsolatedTasksTest.java | 17 ++++- .../scheduler/SchedulerTestBase.java | 65 ++++++++++--------- 4 files changed, 63 insertions(+), 32 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphCoLocationRestartTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphCoLocationRestartTest.java index 92548be44cd1..f69fe537ea5b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphCoLocationRestartTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphCoLocationRestartTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.executiongraph; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.jobgraph.JobStatus; @@ -46,6 +47,11 @@ public class ExecutionGraphCoLocationRestartTest extends SchedulerTestBase { private static final int NUM_TASKS = 31; + @Override + protected ComponentMainThreadExecutor getComponentMainThreadExecutor() { + return ComponentMainThreadExecutorServiceAdapter.forMainThread(); + } + @Test public void testConstraintsAfterRestart() throws Exception { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java index df7b0222a7f3..f4806b77f6b6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java @@ -21,6 +21,8 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.clusterframework.types.SlotProfile; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobmaster.LogicalSlot; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; @@ -40,6 +42,11 @@ public class ScheduleWithCoLocationHintTest extends SchedulerTestBase { + @Override + protected ComponentMainThreadExecutor getComponentMainThreadExecutor() { + return ComponentMainThreadExecutorServiceAdapter.forMainThread(); + } + @Test public void scheduleAllSharedAndCoLocated() throws Exception { JobVertexID jid1 = new JobVertexID(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java index f1ae85719b3b..7c31dbc51ca8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java @@ -21,11 +21,14 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.clusterframework.types.SlotProfile; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; +import org.apache.flink.runtime.executiongraph.TestingComponentMainThreadExecutor; import org.apache.flink.runtime.jobmaster.LogicalSlot; import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Test; @@ -54,6 +57,14 @@ */ public class SchedulerIsolatedTasksTest extends SchedulerTestBase { + @ClassRule + public static final TestingComponentMainThreadExecutor.Resource TESTING_COMPONENT_MAIN_THREAD_EXECUTOR_RESOURCE = new TestingComponentMainThreadExecutor.Resource(); + + @Override + protected ComponentMainThreadExecutor getComponentMainThreadExecutor() { + return TESTING_COMPONENT_MAIN_THREAD_EXECUTOR_RESOURCE.getComponentMainThreadTestExecutor().getMainThreadExecutor(); + } + @Test public void testScheduleQueueing() throws Exception { final int numInstances = 50; @@ -100,7 +111,7 @@ public void testScheduleQueueing() throws Exception { LogicalSlot next = iter.next(); iter.remove(); - next.releaseSlot(); + runInMainThreadExecutor(next::releaseSlot); recycled++; } } @@ -217,8 +228,8 @@ public void testSchedulingLocation() throws Exception { assertEquals(second.getResourceID(), s6.getTaskManagerLocation().getResourceID()); // release something on the first and second instance - s2.releaseSlot(); - s6.releaseSlot(); + runInMainThreadExecutor(s2::releaseSlot); + runInMainThreadExecutor(s6::releaseSlot); LogicalSlot s7 = testingSlotProvider.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), slotProfileForLocation(first, third), TestingUtils.infiniteTime()).get(); assertEquals(first.getResourceID(), s7.getTaskManagerLocation().getResourceID()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestBase.java index ef19c250bc79..d36dbd7d97ad 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestBase.java @@ -25,7 +25,6 @@ import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.clusterframework.types.SlotProfile; import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; -import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; import org.apache.flink.runtime.instance.SlotSharingGroupId; import org.apache.flink.runtime.jobgraph.JobVertexID; @@ -59,33 +58,42 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; /** * Test base for scheduler related test cases. The test are * executed with the {@link SlotPool}. */ -public class SchedulerTestBase extends TestLogger { +public abstract class SchedulerTestBase extends TestLogger { protected TestingSlotProvider testingSlotProvider; + private SlotPool slotPool; + + private TestingScheduler scheduler; + + private ComponentMainThreadExecutor componentMainThreadExecutor; + @Before public void setup() throws Exception { final JobID jobId = new JobID(); - final SlotPool slotPool = new TestingSlotPoolImpl(jobId); - final TestingScheduler testingScheduler = new TestingScheduler( + slotPool = new TestingSlotPoolImpl(jobId); + scheduler = new TestingScheduler( new HashMap<>(16), LocationPreferenceSlotSelectionStrategy.INSTANCE, slotPool); - testingSlotProvider = new TestingSlotPoolSlotProvider(slotPool, testingScheduler); + testingSlotProvider = new TestingSlotPoolSlotProvider(); final JobMasterId jobMasterId = JobMasterId.generate(); final String jobManagerAddress = "localhost"; - ComponentMainThreadExecutor executor = ComponentMainThreadExecutorServiceAdapter.forMainThread(); - slotPool.start(jobMasterId, jobManagerAddress, executor); - testingScheduler.start(executor); + componentMainThreadExecutor = getComponentMainThreadExecutor(); + slotPool.start(jobMasterId, jobManagerAddress, componentMainThreadExecutor); + scheduler.start(componentMainThreadExecutor); } + protected abstract ComponentMainThreadExecutor getComponentMainThreadExecutor(); + @After public void teardown() throws Exception { if (testingSlotProvider != null) { @@ -94,6 +102,14 @@ public void teardown() throws Exception { } } + protected final T supplyInMainThreadExecutor(Supplier supplier) { + return CompletableFuture.supplyAsync(supplier, componentMainThreadExecutor).join(); + } + + protected final void runInMainThreadExecutor(Runnable runnable) { + CompletableFuture.runAsync(runnable, componentMainThreadExecutor).join(); + } + protected interface TestingSlotProvider extends SlotProvider { TaskManagerLocation addTaskManager(int numberSlots); @@ -116,11 +132,7 @@ protected interface TestingSlotProvider extends SlotProvider { void shutdown() throws Exception; } - private static final class TestingSlotPoolSlotProvider implements TestingSlotProvider { - - private final SlotPool slotPool; - - private final TestingScheduler scheduler; + private final class TestingSlotPoolSlotProvider implements TestingSlotProvider { private final AtomicInteger numberOfLocalizedAssignments; @@ -130,11 +142,7 @@ private static final class TestingSlotPoolSlotProvider implements TestingSlotPro private final AtomicInteger numberOfHostLocalizedAssignments; - private TestingSlotPoolSlotProvider(SlotPool slotPool, TestingScheduler testingScheduler) { - this.slotPool = Preconditions.checkNotNull(slotPool); - - this.scheduler = testingScheduler; - + private TestingSlotPoolSlotProvider() { this.numberOfLocalizedAssignments = new AtomicInteger(); this.numberOfNonLocalizedAssignments = new AtomicInteger(); this.numberOfUnconstrainedAssignments = new AtomicInteger(); @@ -145,10 +153,9 @@ private TestingSlotPoolSlotProvider(SlotPool slotPool, TestingScheduler testingS public TaskManagerLocation addTaskManager(int numberSlots) { final TaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); final ResourceID resourceId = taskManagerLocation.getResourceID(); - final SlotPool slotPoolGateway = slotPool; try { - slotPoolGateway.registerTaskManager(resourceId); + supplyInMainThreadExecutor(() -> slotPool.registerTaskManager(resourceId)); } catch (Exception e) { throw new RuntimeException("Unexpected exception occurred. This indicates a programming bug.", e); } @@ -168,10 +175,10 @@ public TaskManagerLocation addTaskManager(int numberSlots) { final Collection acceptedSlotOffers; try { - acceptedSlotOffers = slotPoolGateway.offerSlots( + acceptedSlotOffers = supplyInMainThreadExecutor(() -> slotPool.offerSlots( taskManagerLocation, taskManagerGateway, - slotOffers); + slotOffers)); } catch (Exception e) { throw new RuntimeException("Unexpected exception occurred. This indicates a programming bug.", e); } @@ -184,7 +191,7 @@ public TaskManagerLocation addTaskManager(int numberSlots) { @Override public void releaseTaskManager(ResourceID resourceId) { try { - slotPool.releaseTaskManager(resourceId, null); + supplyInMainThreadExecutor(() -> slotPool.releaseTaskManager(resourceId, null)); } catch (Exception e) { throw new RuntimeException("Should not have happened.", e); } @@ -192,7 +199,7 @@ public void releaseTaskManager(ResourceID resourceId) { @Override public int getNumberOfAvailableSlots() { - return slotPool.getAvailableSlotsInformation().size(); + return supplyInMainThreadExecutor(() -> slotPool.getAvailableSlotsInformation().size()); } @Override @@ -217,17 +224,17 @@ public int getNumberOfHostLocalizedAssignments() { @Override public int getNumberOfSlots(SlotSharingGroup slotSharingGroup) { - return scheduler.getNumberOfSharedSlots(slotSharingGroup.getSlotSharingGroupId()); + return supplyInMainThreadExecutor(() -> scheduler.getNumberOfSharedSlots(slotSharingGroup.getSlotSharingGroupId())); } @Override public int getNumberOfAvailableSlotsForGroup(SlotSharingGroup slotSharingGroup, JobVertexID jobVertexId) { - return scheduler.getNumberOfAvailableSlotsForGroup(slotSharingGroup.getSlotSharingGroupId(), jobVertexId); + return supplyInMainThreadExecutor(() -> scheduler.getNumberOfAvailableSlotsForGroup(slotSharingGroup.getSlotSharingGroupId(), jobVertexId)); } @Override public void shutdown() { - slotPool.close(); + runInMainThreadExecutor(() -> slotPool.close()); } @Override @@ -236,7 +243,7 @@ public CompletableFuture allocateSlot( ScheduledUnit task, SlotProfile slotProfile, Time allocationTimeout) { - return scheduler.allocateSlot(task, slotProfile, allocationTimeout).thenApply( + return supplyInMainThreadExecutor(() -> scheduler.allocateSlot(task, slotProfile, allocationTimeout).thenApply( (LogicalSlot logicalSlot) -> { switch (logicalSlot.getLocality()) { case LOCAL: @@ -256,7 +263,7 @@ public CompletableFuture allocateSlot( } return logicalSlot; - }); + })); } @Override From 85d7374b98072819f821a49e67d8b9f836fc675e Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 4 Nov 2019 11:56:37 +0100 Subject: [PATCH 432/746] [FLINK-14462] Enable ScheduleIsolatedTasksTest#testScheduleWithDyingInstances Use the ComponentMainThreadExecutorServiceAdapter.forSingleThreadExecutor to run the test in order to support timeout values. This closes #10048. --- .../scheduler/SchedulerIsolatedTasksTest.java | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java index 7c31dbc51ca8..b41c46d6aa76 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.jobmanager.scheduler; +import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.clusterframework.types.SlotProfile; @@ -27,9 +28,9 @@ import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.util.ExceptionUtils; import org.junit.ClassRule; -import org.junit.Ignore; import org.junit.Test; import java.util.ArrayList; @@ -42,13 +43,16 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.areAllDistinct; import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getDummyTask; import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertex; +import static org.hamcrest.Matchers.instanceOf; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -134,7 +138,6 @@ public void testScheduleQueueing() throws Exception { } @Test - @Ignore public void testScheduleWithDyingInstances() throws Exception { final TaskManagerLocation taskManagerLocation1 = testingSlotProvider.addTaskManager(2); final TaskManagerLocation taskManagerLocation2 = testingSlotProvider.addTaskManager(2); @@ -156,7 +159,7 @@ public void testScheduleWithDyingInstances() throws Exception { assertTrue(slot.isAlive()); } - slot.releaseSlot(); + runInMainThreadExecutor(slot::releaseSlot); } assertEquals(3, testingSlotProvider.getNumberOfAvailableSlots()); @@ -166,11 +169,11 @@ public void testScheduleWithDyingInstances() throws Exception { // cannot get another slot, since all instances are dead try { - testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), SlotProfile.noRequirements(), TestingUtils.infiniteTime()).get(); + testingSlotProvider.allocateSlot(new ScheduledUnit(getDummyTask()), SlotProfile.noRequirements(), Time.milliseconds(10L)).get(); fail("Scheduler served a slot from a dead instance"); } catch (ExecutionException e) { - assertTrue(e.getCause() instanceof NoResourceAvailableException); + assertThat(ExceptionUtils.stripExecutionException(e), instanceOf(TimeoutException.class)); } catch (Exception e) { fail("Wrong exception type."); From f24229879336f5443e39d072a024f1d5cc85a5a7 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Fri, 1 Nov 2019 09:43:05 +0100 Subject: [PATCH 433/746] [FLINK-14372][tests] Add category AlsoRunWithSchedulerNG to KeyedStateCheckpointingITCase This closes #10065. --- .../test/checkpointing/KeyedStateCheckpointingITCase.java | 3 +++ 1 file changed, 3 insertions(+) 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 9ec23d3433e0..385832a6ced9 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 @@ -38,11 +38,13 @@ import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.TestLogger; import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.junit.rules.TemporaryFolder; import java.io.IOException; @@ -65,6 +67,7 @@ *

    It is designed to check partitioned states. */ @SuppressWarnings("serial") +@Category(AlsoRunWithSchedulerNG.class) public class KeyedStateCheckpointingITCase extends TestLogger { protected static final int MAX_MEM_STATE_SIZE = 10 * 1024 * 1024; From 9241a44e7f2e7fae0f95fc79f76ddc6ab87837db Mon Sep 17 00:00:00 2001 From: yanghua Date: Thu, 31 Oct 2019 15:46:52 +0800 Subject: [PATCH 434/746] [FLINK-12527][runtime] Remove GLOBAL_VERSION_UPDATER in ExecutionGraph This closes #10058. --- .../flink/runtime/executiongraph/ExecutionGraph.java | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index e2132423a5d1..9bd6d4dcb841 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -118,7 +118,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLongFieldUpdater; import java.util.stream.Collectors; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -170,11 +169,6 @@ */ public class ExecutionGraph implements AccessExecutionGraph { - /** In place updater for the execution graph's current global recovery version. - * Avoids having to use an AtomicLong and thus makes the frequent read access a bit faster */ - private static final AtomicLongFieldUpdater GLOBAL_VERSION_UPDATER = - AtomicLongFieldUpdater.newUpdater(ExecutionGraph.class, "globalModVersion"); - /** The log object used for debugging. */ static final Logger LOG = LoggerFactory.getLogger(ExecutionGraph.class); @@ -1373,7 +1367,7 @@ private boolean transitionState(JobStatus current, JobStatus newState, Throwable private long incrementGlobalModVersion() { incrementRestarts(); - return GLOBAL_VERSION_UPDATER.incrementAndGet(this); + return ++globalModVersion; } public void incrementRestarts() { From db8fd386f1cd1dd9f33b72b5108070fb26deaafd Mon Sep 17 00:00:00 2001 From: yanghua Date: Fri, 1 Nov 2019 09:55:14 +0800 Subject: [PATCH 435/746] [FLINK-14583][runtime] Remove progressLock from ExecutionGraph This closes #10063. --- .../executiongraph/ExecutionGraph.java | 158 ++++++++---------- 1 file changed, 74 insertions(+), 84 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 9bd6d4dcb841..5ebf33d134b4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -174,10 +174,6 @@ public class ExecutionGraph implements AccessExecutionGraph { // -------------------------------------------------------------------------------------------- - /** The lock used to secure all access to mutable fields, especially the tracking of progress - * within the job. */ - private final Object progressLock = new Object(); - /** Job specific information like the job id, job name, job configuration, etc. */ private final JobInformation jobInformation; @@ -1051,13 +1047,11 @@ else if (current == JobStatus.FAILING) { // All vertices have been cancelled and it's safe to directly go // into the canceled state. else if (current == JobStatus.RESTARTING) { - synchronized (progressLock) { - if (transitionState(current, JobStatus.CANCELED)) { - onTerminalState(JobStatus.CANCELED); + if (transitionState(current, JobStatus.CANCELED)) { + onTerminalState(JobStatus.CANCELED); - LOG.info("Canceled during restart."); - return; - } + LOG.info("Canceled during restart."); + return; } } else { @@ -1207,58 +1201,56 @@ public void restart(long expectedGlobalVersion) { assertRunningInJobMasterMainThread(); try { - synchronized (progressLock) { - // check the global version to see whether this recovery attempt is still valid - if (globalModVersion != expectedGlobalVersion) { - LOG.info("Concurrent full restart subsumed this restart."); - return; - } - - final JobStatus current = state; + // check the global version to see whether this recovery attempt is still valid + if (globalModVersion != expectedGlobalVersion) { + LOG.info("Concurrent full restart subsumed this restart."); + return; + } - if (current == JobStatus.CANCELED) { - LOG.info("Canceled job during restart. Aborting restart."); - return; - } else if (current == JobStatus.FAILED) { - LOG.info("Failed job during restart. Aborting restart."); - return; - } else if (current == JobStatus.SUSPENDED) { - LOG.info("Suspended job during restart. Aborting restart."); - return; - } else if (current != JobStatus.RESTARTING) { - throw new IllegalStateException("Can only restart job from state restarting."); - } + final JobStatus current = state; - this.currentExecutions.clear(); + if (current == JobStatus.CANCELED) { + LOG.info("Canceled job during restart. Aborting restart."); + return; + } else if (current == JobStatus.FAILED) { + LOG.info("Failed job during restart. Aborting restart."); + return; + } else if (current == JobStatus.SUSPENDED) { + LOG.info("Suspended job during restart. Aborting restart."); + return; + } else if (current != JobStatus.RESTARTING) { + throw new IllegalStateException("Can only restart job from state restarting."); + } - final Collection colGroups = new HashSet<>(); - final long resetTimestamp = System.currentTimeMillis(); + this.currentExecutions.clear(); - for (ExecutionJobVertex jv : this.verticesInCreationOrder) { + final Collection colGroups = new HashSet<>(); + final long resetTimestamp = System.currentTimeMillis(); - CoLocationGroup cgroup = jv.getCoLocationGroup(); - if (cgroup != null && !colGroups.contains(cgroup)){ - cgroup.resetConstraints(); - colGroups.add(cgroup); - } + for (ExecutionJobVertex jv : this.verticesInCreationOrder) { - jv.resetForNewExecution(resetTimestamp, expectedGlobalVersion); + CoLocationGroup cgroup = jv.getCoLocationGroup(); + if (cgroup != null && !colGroups.contains(cgroup)){ + cgroup.resetConstraints(); + colGroups.add(cgroup); } - for (int i = 0; i < stateTimestamps.length; i++) { - if (i != JobStatus.RESTARTING.ordinal()) { - // Only clear the non restarting state in order to preserve when the job was - // restarted. This is needed for the restarting time gauge - stateTimestamps[i] = 0; - } + jv.resetForNewExecution(resetTimestamp, expectedGlobalVersion); + } + + for (int i = 0; i < stateTimestamps.length; i++) { + if (i != JobStatus.RESTARTING.ordinal()) { + // Only clear the non restarting state in order to preserve when the job was + // restarted. This is needed for the restarting time gauge + stateTimestamps[i] = 0; } + } - transitionState(JobStatus.RESTARTING, JobStatus.CREATED); + transitionState(JobStatus.RESTARTING, JobStatus.CREATED); - // if we have checkpointed state, reload it into the executions - if (checkpointCoordinator != null) { - checkpointCoordinator.restoreLatestCheckpointedState(getAllVertices(), false, false); - } + // if we have checkpointed state, reload it into the executions + if (checkpointCoordinator != null) { + checkpointCoordinator.restoreLatestCheckpointedState(getAllVertices(), false, false); } scheduleForExecution(); @@ -1482,45 +1474,43 @@ private boolean tryRestartOrFail(long globalModVersionForRestart) { if (currentState == JobStatus.FAILING || currentState == JobStatus.RESTARTING) { final Throwable failureCause = this.failureCause; - synchronized (progressLock) { - if (LOG.isDebugEnabled()) { - LOG.debug("Try to restart or fail the job {} ({}) if no longer possible.", getJobName(), getJobID(), failureCause); - } else { - LOG.info("Try to restart or fail the job {} ({}) if no longer possible.", getJobName(), getJobID()); - } + if (LOG.isDebugEnabled()) { + LOG.debug("Try to restart or fail the job {} ({}) if no longer possible.", getJobName(), getJobID(), failureCause); + } else { + LOG.info("Try to restart or fail the job {} ({}) if no longer possible.", getJobName(), getJobID()); + } - final boolean isFailureCauseAllowingRestart = !(failureCause instanceof SuppressRestartsException); - final boolean isRestartStrategyAllowingRestart = restartStrategy.canRestart(); - boolean isRestartable = isFailureCauseAllowingRestart && isRestartStrategyAllowingRestart; + final boolean isFailureCauseAllowingRestart = !(failureCause instanceof SuppressRestartsException); + final boolean isRestartStrategyAllowingRestart = restartStrategy.canRestart(); + boolean isRestartable = isFailureCauseAllowingRestart && isRestartStrategyAllowingRestart; - if (isRestartable && transitionState(currentState, JobStatus.RESTARTING)) { - LOG.info("Restarting the job {} ({}).", getJobName(), getJobID()); + if (isRestartable && transitionState(currentState, JobStatus.RESTARTING)) { + LOG.info("Restarting the job {} ({}).", getJobName(), getJobID()); - RestartCallback restarter = new ExecutionGraphRestartCallback(this, globalModVersionForRestart); - FutureUtils.assertNoException( - restartStrategy - .restart(restarter, getJobMasterMainThreadExecutor()) - .exceptionally((throwable) -> { + RestartCallback restarter = new ExecutionGraphRestartCallback(this, globalModVersionForRestart); + FutureUtils.assertNoException( + restartStrategy + .restart(restarter, getJobMasterMainThreadExecutor()) + .exceptionally((throwable) -> { failGlobal(throwable); return null; })); - return true; - } - else if (!isRestartable && transitionState(currentState, JobStatus.FAILED, failureCause)) { - final String cause1 = isFailureCauseAllowingRestart ? null : - "a type of SuppressRestartsException was thrown"; - final String cause2 = isRestartStrategyAllowingRestart ? null : - "the restart strategy prevented it"; - - LOG.info("Could not restart the job {} ({}) because {}.", getJobName(), getJobID(), - StringUtils.concatenateWithAnd(cause1, cause2), failureCause); - onTerminalState(JobStatus.FAILED); - - return true; - } else { - // we must have changed the state concurrently, thus we cannot complete this operation - return false; - } + return true; + } + else if (!isRestartable && transitionState(currentState, JobStatus.FAILED, failureCause)) { + final String cause1 = isFailureCauseAllowingRestart ? null : + "a type of SuppressRestartsException was thrown"; + final String cause2 = isRestartStrategyAllowingRestart ? null : + "the restart strategy prevented it"; + + LOG.info("Could not restart the job {} ({}) because {}.", getJobName(), getJobID(), + StringUtils.concatenateWithAnd(cause1, cause2), failureCause); + onTerminalState(JobStatus.FAILED); + + return true; + } else { + // we must have changed the state concurrently, thus we cannot complete this operation + return false; } } else { // this operation is only allowed in the state FAILING or RESTARTING From a0419205799f58426188cc326f1c85f063f6e6cc Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Wed, 30 Oct 2019 16:08:54 -0700 Subject: [PATCH 436/746] [FLINK-14578][table] load/unloadModule() should throw RuntimeException rather than checked exception To adhere to FLIP to make load/unloadModule() API throw RuntimeException rather than checked exceptions. This closes #10053. --- .../flink/table/api/TableEnvironment.java | 10 +++---- .../api/internal/TableEnvironmentImpl.java | 6 ++-- .../flink/table/module/ModuleManager.java | 17 +++++------ .../table/catalog/FunctionCatalogTest.java | 7 +++-- .../ModuleAlreadyExistException.java | 28 ------------------- .../exceptions/ModuleNotFoundException.java | 28 ------------------- 6 files changed, 20 insertions(+), 76 deletions(-) delete mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/exceptions/ModuleAlreadyExistException.java delete mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/exceptions/ModuleNotFoundException.java diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java index a0a59fab5830..6fc7d0c7182d 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java @@ -28,8 +28,6 @@ import org.apache.flink.table.descriptors.ConnectorDescriptor; import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.module.Module; -import org.apache.flink.table.module.exceptions.ModuleAlreadyExistException; -import org.apache.flink.table.module.exceptions.ModuleNotFoundException; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.table.sources.TableSource; @@ -116,20 +114,20 @@ static TableEnvironment create(EnvironmentSettings settings) { /** * Loads a {@link Module} under a unique name. Modules will be kept in the loaded order. + * ValidationException is thrown when there is already a module with the same name. * * @param moduleName name of the {@link Module} * @param module the module instance - * @throws ModuleAlreadyExistException thrown when there is already a module with the same name */ - void loadModule(String moduleName, Module module) throws ModuleAlreadyExistException; + void loadModule(String moduleName, Module module); /** * Unloads a {@link Module} with given name. + * ValidationException is thrown when there is no module with the given name * * @param moduleName name of the {@link Module} - * @throws ModuleNotFoundException thrown when there is no module with the given name */ - void unloadModule(String moduleName) throws ModuleNotFoundException; + void unloadModule(String moduleName); /** * Registers a {@link ScalarFunction} under a unique name. Replaces already existing 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 10459d07346b..70cc55acbcd2 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 @@ -52,8 +52,6 @@ import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.module.Module; import org.apache.flink.table.module.ModuleManager; -import org.apache.flink.table.module.exceptions.ModuleAlreadyExistException; -import org.apache.flink.table.module.exceptions.ModuleNotFoundException; import org.apache.flink.table.operations.CatalogQueryOperation; import org.apache.flink.table.operations.CatalogSinkModifyOperation; import org.apache.flink.table.operations.ModifyOperation; @@ -207,12 +205,12 @@ public Optional getCatalog(String catalogName) { } @Override - public void loadModule(String moduleName, Module module) throws ModuleAlreadyExistException { + public void loadModule(String moduleName, Module module) { moduleManager.loadModule(moduleName, module); } @Override - public void unloadModule(String moduleName) throws ModuleNotFoundException { + public void unloadModule(String moduleName) { moduleManager.unloadModule(moduleName); } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/module/ModuleManager.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/module/ModuleManager.java index fdc26b7f653b..abb3f8eebe30 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/module/ModuleManager.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/module/ModuleManager.java @@ -18,9 +18,8 @@ package org.apache.flink.table.module; +import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.functions.FunctionDefinition; -import org.apache.flink.table.module.exceptions.ModuleAlreadyExistException; -import org.apache.flink.table.module.exceptions.ModuleNotFoundException; import org.apache.flink.util.StringUtils; import java.util.ArrayList; @@ -48,33 +47,35 @@ public ModuleManager() { /** * Load a module under a unique name. Modules will be kept in the loaded order, and new module * will be added to the end. + * ValidationException is thrown when there is already a module with the same name. * * @param name name of the module * @param module the module instance - * @throws ModuleAlreadyExistException thrown when there is already a module with the same name */ - public void loadModule(String name, Module module) throws ModuleAlreadyExistException { + public void loadModule(String name, Module module) { checkArgument(!StringUtils.isNullOrWhitespaceOnly(name), "name cannot be null or empty string"); checkNotNull(module, "module cannot be null"); if (!modules.containsKey(name)) { modules.put(name, module); } else { - throw new ModuleAlreadyExistException(name); + throw new ValidationException( + String.format("A module with name %s already exists", name)); } } /** * Unload a module with given name. + * ValidationException is thrown when there is no module with the given name. * * @param name name of the module - * @throws ModuleNotFoundException thrown when there is no module with the given name */ - public void unloadModule(String name) throws ModuleNotFoundException { + public void unloadModule(String name) { if (modules.containsKey(name)) { modules.remove(name); } else { - throw new ModuleNotFoundException(name); + throw new ValidationException( + String.format("No module with name %s exists", name)); } } diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java index fb0205d45c2e..9556d4cdbab2 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java @@ -27,7 +27,6 @@ import org.apache.flink.table.functions.ScalarFunctionDefinition; import org.apache.flink.table.module.Module; import org.apache.flink.table.module.ModuleManager; -import org.apache.flink.table.module.exceptions.ModuleAlreadyExistException; import org.junit.Before; import org.junit.Test; @@ -108,7 +107,11 @@ public void testPreciseFunctionReference() throws FunctionAlreadyExistException, } @Test - public void testAmbiguousFunctionReference() throws FunctionAlreadyExistException, DatabaseNotExistException, ModuleAlreadyExistException { + public void testAmbiguousFunctionReference() throws FunctionAlreadyExistException, DatabaseNotExistException { + ObjectIdentifier oi = ObjectIdentifier.of( + testCatalogName, + GenericInMemoryCatalog.DEFAULT_DB, + TEST_FUNCTION_NAME); // test no function is found assertFalse(functionCatalog.lookupFunction(FunctionIdentifier.of(TEST_FUNCTION_NAME)).isPresent()); diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/exceptions/ModuleAlreadyExistException.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/exceptions/ModuleAlreadyExistException.java deleted file mode 100644 index 1f27ba6a4f48..000000000000 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/exceptions/ModuleAlreadyExistException.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.table.module.exceptions; - -/** - * Exception for trying to load a module that already exists. - */ -public class ModuleAlreadyExistException extends Exception { - public ModuleAlreadyExistException(String moduleName) { - super(String.format("Module %s already exists.", moduleName)); - } -} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/exceptions/ModuleNotFoundException.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/exceptions/ModuleNotFoundException.java deleted file mode 100644 index bf575cd63071..000000000000 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/exceptions/ModuleNotFoundException.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.table.module.exceptions; - -/** - * Exception for operating on a module that does not exists. - */ -public class ModuleNotFoundException extends Exception { - public ModuleNotFoundException(String moduleName) { - super(String.format("Module %s does not exist.", moduleName)); - } -} From 047ba2452c2a2dc9b781728f45deeb3cb7b1fb14 Mon Sep 17 00:00:00 2001 From: tison Date: Tue, 5 Nov 2019 09:23:41 +0800 Subject: [PATCH 437/746] [hotfix][tests] Refactor SchedulerTestBase for removing dead code This closes #10077 . --- .../scheduler/SchedulerTestBase.java | 104 +----------------- 1 file changed, 6 insertions(+), 98 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestBase.java index d36dbd7d97ad..2ad4861a046d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestBase.java @@ -27,35 +27,29 @@ import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; import org.apache.flink.runtime.instance.SlotSharingGroupId; -import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.jobmaster.LogicalSlot; import org.apache.flink.runtime.jobmaster.SlotRequestId; import org.apache.flink.runtime.jobmaster.slotpool.LocationPreferenceSlotSelectionStrategy; +import org.apache.flink.runtime.jobmaster.slotpool.Scheduler; import org.apache.flink.runtime.jobmaster.slotpool.SchedulerImpl; import org.apache.flink.runtime.jobmaster.slotpool.SlotPool; import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; -import org.apache.flink.runtime.jobmaster.slotpool.SlotSelectionStrategy; -import org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager; import org.apache.flink.runtime.jobmaster.slotpool.TestingSlotPoolImpl; import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; -import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.Preconditions; import org.apache.flink.util.TestLogger; import org.junit.After; import org.junit.Before; -import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; -import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; @@ -66,11 +60,11 @@ */ public abstract class SchedulerTestBase extends TestLogger { - protected TestingSlotProvider testingSlotProvider; + protected TestingSlotPoolSlotProvider testingSlotProvider; private SlotPool slotPool; - private TestingScheduler scheduler; + private Scheduler scheduler; private ComponentMainThreadExecutor componentMainThreadExecutor; @@ -78,10 +72,7 @@ public abstract class SchedulerTestBase extends TestLogger { public void setup() throws Exception { final JobID jobId = new JobID(); slotPool = new TestingSlotPoolImpl(jobId); - scheduler = new TestingScheduler( - new HashMap<>(16), - LocationPreferenceSlotSelectionStrategy.INSTANCE, - slotPool); + scheduler = new SchedulerImpl(LocationPreferenceSlotSelectionStrategy.INSTANCE, slotPool); testingSlotProvider = new TestingSlotPoolSlotProvider(); @@ -110,29 +101,7 @@ protected final void runInMainThreadExecutor(Runnable runnable) { CompletableFuture.runAsync(runnable, componentMainThreadExecutor).join(); } - protected interface TestingSlotProvider extends SlotProvider { - TaskManagerLocation addTaskManager(int numberSlots); - - void releaseTaskManager(ResourceID resourceId); - - int getNumberOfAvailableSlots(); - - int getNumberOfLocalizedAssignments(); - - int getNumberOfNonLocalizedAssignments(); - - int getNumberOfUnconstrainedAssignments(); - - int getNumberOfHostLocalizedAssignments(); - - int getNumberOfSlots(SlotSharingGroup slotSharingGroup); - - int getNumberOfAvailableSlotsForGroup(SlotSharingGroup slotSharingGroup, JobVertexID jobVertexId); - - void shutdown() throws Exception; - } - - private final class TestingSlotPoolSlotProvider implements TestingSlotProvider { + protected final class TestingSlotPoolSlotProvider implements SlotProvider { private final AtomicInteger numberOfLocalizedAssignments; @@ -149,7 +118,6 @@ private TestingSlotPoolSlotProvider() { this.numberOfHostLocalizedAssignments = new AtomicInteger(); } - @Override public TaskManagerLocation addTaskManager(int numberSlots) { final TaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); final ResourceID resourceId = taskManagerLocation.getResourceID(); @@ -188,7 +156,6 @@ public TaskManagerLocation addTaskManager(int numberSlots) { return taskManagerLocation; } - @Override public void releaseTaskManager(ResourceID resourceId) { try { supplyInMainThreadExecutor(() -> slotPool.releaseTaskManager(resourceId, null)); @@ -197,42 +164,26 @@ public void releaseTaskManager(ResourceID resourceId) { } } - @Override public int getNumberOfAvailableSlots() { return supplyInMainThreadExecutor(() -> slotPool.getAvailableSlotsInformation().size()); } - @Override public int getNumberOfLocalizedAssignments() { return numberOfLocalizedAssignments.get(); } - @Override public int getNumberOfNonLocalizedAssignments() { return numberOfNonLocalizedAssignments.get(); } - @Override public int getNumberOfUnconstrainedAssignments() { return numberOfUnconstrainedAssignments.get(); } - @Override public int getNumberOfHostLocalizedAssignments() { return numberOfHostLocalizedAssignments.get(); } - @Override - public int getNumberOfSlots(SlotSharingGroup slotSharingGroup) { - return supplyInMainThreadExecutor(() -> scheduler.getNumberOfSharedSlots(slotSharingGroup.getSlotSharingGroupId())); - } - - @Override - public int getNumberOfAvailableSlotsForGroup(SlotSharingGroup slotSharingGroup, JobVertexID jobVertexId) { - return supplyInMainThreadExecutor(() -> scheduler.getNumberOfAvailableSlotsForGroup(slotSharingGroup.getSlotSharingGroupId(), jobVertexId)); - } - - @Override public void shutdown() { runInMainThreadExecutor(() -> slotPool.close()); } @@ -268,51 +219,8 @@ public CompletableFuture allocateSlot( @Override public void cancelSlotRequest(SlotRequestId slotRequestId, @Nullable SlotSharingGroupId slotSharingGroupId, Throwable cause) { - } - } - /** - * Test implementation of scheduler that offers a bit more introspection. - */ - private static final class TestingScheduler extends SchedulerImpl { - - private final Map slotSharingManagersMap; - - public TestingScheduler( - @Nonnull Map slotSharingManagersMap, - @Nonnull SlotSelectionStrategy slotSelectionStrategy, - @Nonnull SlotPool slotPoolGateway) { - - super(slotSelectionStrategy, slotPoolGateway, slotSharingManagersMap); - this.slotSharingManagersMap = slotSharingManagersMap; - } - - public int getNumberOfSharedSlots(SlotSharingGroupId slotSharingGroupId) { - final SlotSharingManager multiTaskSlotManager = slotSharingManagersMap.get(slotSharingGroupId); - - if (multiTaskSlotManager != null) { - return multiTaskSlotManager.getResolvedRootSlots().size(); - } else { - throw new FlinkRuntimeException("No MultiTaskSlotManager registered under " + slotSharingGroupId + '.'); - } - } - - public int getNumberOfAvailableSlotsForGroup(SlotSharingGroupId slotSharingGroupId, JobVertexID jobVertexId) { - final SlotSharingManager multiTaskSlotManager = slotSharingManagersMap.get(slotSharingGroupId); - - if (multiTaskSlotManager != null) { - int availableSlots = 0; - - for (SlotSharingManager.MultiTaskSlot multiTaskSlot : multiTaskSlotManager.getResolvedRootSlots()) { - if (!multiTaskSlot.contains(jobVertexId)) { - availableSlots++; - } - } - - return availableSlots; - } else { - throw new FlinkRuntimeException("No MultiTaskSlotmanager registered under " + slotSharingGroupId + '.'); - } } } + } From 503d5a28857e18c229da3030d65d4f0a1a308e3e Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Mon, 4 Nov 2019 13:02:15 +0800 Subject: [PATCH 438/746] [hotfix][runtime] Remove StreamGraphGenerator#isSlotSharingEnabled The field has been superseded by ExecutionConfig#allVerticesInSameSlotSharingGroupByDefault. This field was always true. So the removal does not affect anything in production. --- .../api/graph/StreamGraphGenerator.java | 11 ---- .../api/graph/StreamGraphGeneratorTest.java | 59 ------------------- .../graph/StreamingJobGraphGeneratorTest.java | 56 ------------------ 3 files changed, 126 deletions(-) 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 9a4eafd967e0..45435fc7e383 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 @@ -114,8 +114,6 @@ public class StreamGraphGenerator { private boolean chaining = true; - private boolean isSlotSharingEnabled = true; - private ScheduleMode scheduleMode = DEFAULT_SCHEDULE_MODE; private Collection> userArtifacts; @@ -161,11 +159,6 @@ public StreamGraphGenerator setChaining(boolean chaining) { return this; } - public StreamGraphGenerator setSlotSharingEnabled(boolean isSlotSharingEnabled) { - this.isSlotSharingEnabled = isSlotSharingEnabled; - return this; - } - public StreamGraphGenerator setScheduleMode(ScheduleMode scheduleMode) { this.scheduleMode = scheduleMode; return this; @@ -749,10 +742,6 @@ private Collection transformTwoInputTransform(TwoInputT * @param inputIds The IDs of the input operations. */ private String determineSlotSharingGroup(String specifiedGroup, Collection inputIds) { - if (!isSlotSharingEnabled) { - return null; - } - if (specifiedGroup != null) { return specifiedGroup; } else { 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 fc057e1027a4..9c288c743009 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 @@ -61,7 +61,6 @@ 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; @@ -471,39 +470,6 @@ public void testIteration() { } } - /** - * Test iteration job when disable slot sharing, check slot sharing group and co-location group. - */ - @Test - public void testIterationWithSlotSharingDisabled() { - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - DataStream source = env.fromElements(1, 2, 3).name("source"); - IterativeStream iteration = source.iterate(3000); - iteration.name("iteration").setParallelism(2); - DataStream map = iteration.map(x -> x + 1).name("map").setParallelism(2); - DataStream filter = map.filter((x) -> false).name("filter").setParallelism(2); - iteration.closeWith(filter).print(); - - List> transformations = new ArrayList<>(); - transformations.add(source.getTransformation()); - transformations.add(iteration.getTransformation()); - transformations.add(map.getTransformation()); - transformations.add(filter.getTransformation()); - - StreamGraphGenerator generator = new StreamGraphGenerator(transformations, env.getConfig(), env.getCheckpointConfig()); - generator.setSlotSharingEnabled(false); - StreamGraph streamGraph = generator.generate(); - - for (Tuple2 iterationPair : streamGraph.getIterationSourceSinkPairs()) { - assertNotNull(iterationPair.f0.getCoLocationGroup()); - assertEquals(iterationPair.f0.getCoLocationGroup(), iterationPair.f1.getCoLocationGroup()); - - assertNotNull(iterationPair.f0.getSlotSharingGroup()); - assertEquals(iterationPair.f0.getSlotSharingGroup(), iterationPair.f1.getSlotSharingGroup()); - } - } - /** * Test slot sharing is enabled. */ @@ -528,31 +494,6 @@ public void testEnableSlotSharing() { } } - /** - * Test slot sharing is disabled. - */ - @Test - public void testDisableSlotSharing() { - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - DataStream sourceDataStream = env.fromElements(1, 2, 3); - DataStream mapDataStream = sourceDataStream.map(x -> x + 1); - - final List> transformations = new ArrayList<>(); - transformations.add(sourceDataStream.getTransformation()); - transformations.add(mapDataStream.getTransformation()); - - // all stream nodes would have no group if slot sharing group is disabled - StreamGraph streamGraph = new StreamGraphGenerator( - transformations, env.getConfig(), env.getCheckpointConfig()) - .setSlotSharingEnabled(false) - .generate(); - - Collection streamNodes = streamGraph.getStreamNodes(); - for (StreamNode streamNode : streamNodes) { - assertNull(streamNode.getSlotSharingGroup()); - } - } - private static class OutputTypeConfigurableOperationWithTwoInputs extends AbstractStreamOperator implements TwoInputStreamOperator, OutputTypeConfigurable { 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 ab25e5bbc50e..0e5b684c765b 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 @@ -28,7 +28,6 @@ import org.apache.flink.api.common.operators.util.UserCodeWrapper; 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.io.DiscardingOutputFormat; import org.apache.flink.api.java.io.TypeSerializerInputFormat; import org.apache.flink.api.java.tuple.Tuple2; @@ -65,7 +64,6 @@ import org.junit.Test; import java.lang.reflect.Method; -import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -538,60 +536,6 @@ public void testIteration() { assertEquals(iterationSourceCoLocationGroup, iterationSinkCoLocationGroup); } - /** - * Test slot sharing group is enabled or disabled for iteration. - */ - @Test - public void testDisableSlotSharingForIteration() { - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - DataStream source = env.fromElements(1, 2, 3).name("source"); - IterativeStream iteration = source.iterate(3000); - iteration.name("iteration").setParallelism(2); - DataStream map = iteration.map(x -> x + 1).name("map").setParallelism(2); - DataStream filter = map.filter((x) -> false).name("filter").setParallelism(2); - iteration.closeWith(filter).print(); - - List> transformations = new ArrayList<>(); - transformations.add(source.getTransformation()); - transformations.add(iteration.getTransformation()); - transformations.add(map.getTransformation()); - transformations.add(filter.getTransformation()); - // when slot sharing group is disabled - // all job vertices except iteration vertex would have no slot sharing group - // iteration vertices would be set slot sharing group automatically - StreamGraphGenerator generator = new StreamGraphGenerator(transformations, env.getConfig(), env.getCheckpointConfig()); - generator.setSlotSharingEnabled(false); - - JobGraph jobGraph = StreamingJobGraphGenerator.createJobGraph(generator.generate()); - - SlotSharingGroup iterationSourceSlotSharingGroup = null; - SlotSharingGroup iterationSinkSlotSharingGroup = null; - - CoLocationGroup iterationSourceCoLocationGroup = null; - CoLocationGroup iterationSinkCoLocationGroup = null; - - for (JobVertex jobVertex : jobGraph.getVertices()) { - if (jobVertex.getName().startsWith(StreamGraph.ITERATION_SOURCE_NAME_PREFIX)) { - iterationSourceSlotSharingGroup = jobVertex.getSlotSharingGroup(); - iterationSourceCoLocationGroup = jobVertex.getCoLocationGroup(); - } else if (jobVertex.getName().startsWith(StreamGraph.ITERATION_SINK_NAME_PREFIX)) { - iterationSinkSlotSharingGroup = jobVertex.getSlotSharingGroup(); - iterationSinkCoLocationGroup = jobVertex.getCoLocationGroup(); - } else { - assertNull(jobVertex.getSlotSharingGroup()); - } - } - - assertNotNull(iterationSourceSlotSharingGroup); - assertNotNull(iterationSinkSlotSharingGroup); - assertEquals(iterationSourceSlotSharingGroup, iterationSinkSlotSharingGroup); - - assertNotNull(iterationSourceCoLocationGroup); - assertNotNull(iterationSinkCoLocationGroup); - assertEquals(iterationSourceCoLocationGroup, iterationSinkCoLocationGroup); - } - /** * Test default schedule mode. */ From 466b824c16e75e9836b290c9aab8fdacbcafdc78 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Mon, 4 Nov 2019 13:03:15 +0800 Subject: [PATCH 439/746] [hotifx] Remove unused constructor of StreamingJobGraphGenerator --- .../flink/streaming/api/graph/StreamingJobGraphGenerator.java | 4 ---- 1 file changed, 4 deletions(-) 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 83a81031526d..8060882a588d 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 @@ -118,10 +118,6 @@ public static JobGraph createJobGraph(StreamGraph streamGraph, @Nullable JobID j private final StreamGraphHasher defaultStreamGraphHasher; private final List legacyStreamGraphHashers; - private StreamingJobGraphGenerator(StreamGraph streamGraph) { - this(streamGraph, null); - } - private StreamingJobGraphGenerator(StreamGraph streamGraph, @Nullable JobID jobID) { this.streamGraph = streamGraph; this.defaultStreamGraphHasher = new StreamGraphHasherV2(); From fb7beb880ed49538850e3ad740f7423282003243 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Mon, 4 Nov 2019 23:50:41 +0800 Subject: [PATCH 440/746] [hotfix][table, tests] Increase slot count in blink batch tests to avoid resource deadlocks if a job has multiple logical regions This is needed before FLINK-13708 is fixed. Cases having too many queries are split as well. --- .../table/planner/runtime/utils/BatchAbstractTestBase.java | 2 +- .../planner/runtime/batch/sql/agg/AggregateITCaseBase.scala | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) 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 209e9da8f03a..3daeca58ba93 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 @@ -39,7 +39,7 @@ public class BatchAbstractTestBase { new MiniClusterResourceConfiguration.Builder() .setConfiguration(getConfiguration()) .setNumberTaskManagers(1) - .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) + .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM * 2) // TODO: Revert once FLINK-13708 has been fixed .build()); @ClassRule diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateITCaseBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateITCaseBase.scala index d8155fad0117..b784a9a814ec 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateITCaseBase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateITCaseBase.scala @@ -328,6 +328,10 @@ abstract class AggregateITCaseBase(testName: String) extends BatchTestBase { "select f0, count(*) from TableName group by f0", Seq((1, 2L), (2, 2L), (3, 2L)) // count=>long ) + } + + @Test + def testGroupBy2(): Unit = { checkQuery( Seq(("a", 1, 0), ("b", 2, 4), ("a", 2, 3)), "select f0, min(f1), min(f2) from TableName group by f0", From 299625a9763141ea8c4283559a793050a115aafd Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Mon, 4 Nov 2019 23:52:42 +0800 Subject: [PATCH 441/746] [FLINK-14060] Set vertex slot sharing based on logical pipelined regions This closes #10007. --- .../api/graph/StreamingJobGraphGenerator.java | 74 +++++++++++++--- .../graph/StreamingJobGraphGeneratorTest.java | 84 +++++++++++++++++++ 2 files changed, 146 insertions(+), 12 deletions(-) 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 8060882a588d..56d71883f27b 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 @@ -18,6 +18,7 @@ package org.apache.flink.streaming.api.graph; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.operators.ResourceSpec; @@ -39,6 +40,8 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; +import org.apache.flink.runtime.jobgraph.topology.DefaultLogicalTopology; +import org.apache.flink.runtime.jobgraph.topology.LogicalPipelinedRegion; import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; import org.apache.flink.runtime.operators.util.TaskConfig; @@ -74,6 +77,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Set; import static org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration.MINIMAL_CHECKPOINT_TIME; @@ -596,25 +600,71 @@ public static boolean isChainable(StreamEdge edge, StreamGraph streamGraph) { } private void setSlotSharingAndCoLocation() { - final HashMap slotSharingGroups = new HashMap<>(); - final HashMap> coLocationGroups = new HashMap<>(); + setSlotSharing(); + setCoLocation(); + } + + private void setSlotSharing() { + final Map specifiedSlotSharingGroups = new HashMap<>(); + final Map vertexRegionSlotSharingGroups = buildVertexRegionSlotSharingGroups(); for (Entry entry : jobVertices.entrySet()) { - final StreamNode node = streamGraph.getStreamNode(entry.getKey()); final JobVertex vertex = entry.getValue(); + final String slotSharingGroupKey = streamGraph.getStreamNode(entry.getKey()).getSlotSharingGroup(); + + final SlotSharingGroup effectiveSlotSharingGroup; + if (slotSharingGroupKey == null) { + effectiveSlotSharingGroup = null; + } else if (slotSharingGroupKey.equals(StreamGraphGenerator.DEFAULT_SLOT_SHARING_GROUP)) { + // fallback to the region slot sharing group by default + effectiveSlotSharingGroup = vertexRegionSlotSharingGroups.get(vertex.getID()); + } else { + effectiveSlotSharingGroup = specifiedSlotSharingGroups.computeIfAbsent( + slotSharingGroupKey, k -> new SlotSharingGroup()); + } - // configure slot sharing group - final String slotSharingGroupKey = node.getSlotSharingGroup(); - final SlotSharingGroup sharingGroup; + vertex.setSlotSharingGroup(effectiveSlotSharingGroup); + } + } - if (slotSharingGroupKey != null) { - sharingGroup = slotSharingGroups.computeIfAbsent( - slotSharingGroupKey, (k) -> new SlotSharingGroup()); - vertex.setSlotSharingGroup(sharingGroup); + /** + * Maps a vertex to its region slot sharing group. + * If {@link ExecutionConfig#isAllVerticesInSameSlotSharingGroupByDefault()} + * returns true, all regions will be in the same slot sharing group. + */ + private Map buildVertexRegionSlotSharingGroups() { + final Map vertexRegionSlotSharingGroups = new HashMap<>(); + final SlotSharingGroup defaultSlotSharingGroup = new SlotSharingGroup(); + + final boolean allRegionsInSameSlotSharingGroup = streamGraph.getExecutionConfig() + .isAllVerticesInSameSlotSharingGroupByDefault(); + + final Set regions = new DefaultLogicalTopology(jobGraph).getLogicalPipelinedRegions(); + for (LogicalPipelinedRegion region : regions) { + final SlotSharingGroup regionSlotSharingGroup; + if (allRegionsInSameSlotSharingGroup) { + regionSlotSharingGroup = defaultSlotSharingGroup; } else { - sharingGroup = null; + regionSlotSharingGroup = new SlotSharingGroup(); + } + + for (JobVertexID jobVertexID : region.getVertexIDs()) { + vertexRegionSlotSharingGroups.put(jobVertexID, regionSlotSharingGroup); } + } + + return vertexRegionSlotSharingGroups; + } + + private void setCoLocation() { + final Map> coLocationGroups = new HashMap<>(); + + for (Entry entry : jobVertices.entrySet()) { + + final StreamNode node = streamGraph.getStreamNode(entry.getKey()); + final JobVertex vertex = entry.getValue(); + final SlotSharingGroup sharingGroup = vertex.getSlotSharingGroup(); // configure co-location constraint final String coLocationGroupKey = node.getCoLocationGroup(); @@ -624,7 +674,7 @@ private void setSlotSharingAndCoLocation() { } Tuple2 constraint = coLocationGroups.computeIfAbsent( - coLocationGroupKey, (k) -> new Tuple2<>(sharingGroup, new CoLocationGroup())); + coLocationGroupKey, k -> new Tuple2<>(sharingGroup, new CoLocationGroup())); if (constraint.f0 != sharingGroup) { throw new IllegalStateException("Cannot co-locate operators from different slot sharing groups"); 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 0e5b684c765b..fc21a03f13fc 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 @@ -56,6 +56,7 @@ import org.apache.flink.streaming.api.transformations.PartitionTransformation; import org.apache.flink.streaming.api.transformations.ShuffleMode; import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner; +import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner; import org.apache.flink.streaming.runtime.partitioner.RescalePartitioner; import org.apache.flink.streaming.util.TestAnyModeReadingStreamOperator; import org.apache.flink.util.Collector; @@ -72,6 +73,7 @@ import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; @@ -654,4 +656,86 @@ public void testNotSupportInputSelectableOperatorIfCheckpointing() { StreamingJobGraphGenerator.createJobGraph(env.getStreamGraph()); } + + @Test + public void testSlotSharingOnAllVerticesInSameSlotSharingGroupByDefaultEnabled() { + final StreamGraph streamGraph = createStreamGraphForSlotSharingTest(); + // specify slot sharing group for map1 + streamGraph.getStreamNodes().stream() + .filter(n -> "map1".equals(n.getOperatorName())) + .findFirst() + .get() + .setSlotSharingGroup("testSlotSharingGroup"); + streamGraph.getExecutionConfig().enableAllVerticesInSameSlotSharingGroupByDefault(); + final JobGraph jobGraph = StreamingJobGraphGenerator.createJobGraph(streamGraph); + + final List verticesSorted = jobGraph.getVerticesSortedTopologicallyFromSources(); + assertEquals(4, verticesSorted.size()); + + final JobVertex source1Vertex = verticesSorted.get(0); + final JobVertex source2Vertex = verticesSorted.get(1); + final JobVertex map1Vertex = verticesSorted.get(2); + final JobVertex map2Vertex = verticesSorted.get(3); + + // all vertices should be in the same default slot sharing group + // except for map1 which has a specified slot sharing group + assertSameSlotSharingGroup(source1Vertex, source2Vertex, map2Vertex); + assertDistinctSharingGroups(source1Vertex, map1Vertex); + } + + @Test + public void testSlotSharingOnAllVerticesInSameSlotSharingGroupByDefaultDisabled() { + final StreamGraph streamGraph = createStreamGraphForSlotSharingTest(); + streamGraph.getExecutionConfig().disableAllVerticesInSameSlotSharingGroupByDefault(); + final JobGraph jobGraph = StreamingJobGraphGenerator.createJobGraph(streamGraph); + + final List verticesSorted = jobGraph.getVerticesSortedTopologicallyFromSources(); + assertEquals(4, verticesSorted.size()); + + final JobVertex source1Vertex = verticesSorted.get(0); + final JobVertex source2Vertex = verticesSorted.get(1); + final JobVertex map1Vertex = verticesSorted.get(2); + final JobVertex map2Vertex = verticesSorted.get(3); + + // vertices in the same region should be in the same slot sharing group + assertSameSlotSharingGroup(source1Vertex, map1Vertex); + + // vertices in different regions should be in different slot sharing groups + assertDistinctSharingGroups(source1Vertex, source2Vertex, map2Vertex); + } + + /** + * Create a StreamGraph as below. + * + *

    source1 --(rebalance & pipelined)--> Map1 + * + *

    source2 --(rebalance & blocking)--> Map2 + */ + private StreamGraph createStreamGraphForSlotSharingTest() { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + final DataStream source1 = env.fromElements(1, 2, 3).name("source1"); + source1.rebalance().map(v -> v).name("map1"); + + final DataStream source2 = env.fromElements(4, 5, 6).name("source2"); + final DataStream partitioned = new DataStream<>(env, new PartitionTransformation<>( + source2.getTransformation(), new RebalancePartitioner<>(), ShuffleMode.BATCH)); + partitioned.map(v -> v).name("map2"); + + return env.getStreamGraph(); + } + + private void assertSameSlotSharingGroup(JobVertex... vertices) { + for (int i = 0; i < vertices.length - 1; i++) { + assertEquals(vertices[i].getSlotSharingGroup(), vertices[i + 1].getSlotSharingGroup()); + } + } + + private void assertDistinctSharingGroups(JobVertex... vertices) { + for (int i = 0; i < vertices.length - 1; i++) { + for (int j = i + 1; j < vertices.length; j++) { + assertNotEquals(vertices[i].getSlotSharingGroup(), vertices[j].getSlotSharingGroup()); + } + } + } } From 6d1ea0f2c68c1b58f6daa692b0b3727429b11d84 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Tue, 29 Oct 2019 16:54:12 +0800 Subject: [PATCH 442/746] [hotfix][runtime] Propagate execution graph execution state update result to JobMaster Also only perform scheduler internal update if the execution graph execution state update succeeded. --- .../apache/flink/runtime/scheduler/SchedulerBase.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) 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 6a372daa6ebb..c989c1258c12 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 @@ -417,12 +417,15 @@ public CompletableFuture getTerminationFuture() { @Override public final boolean updateTaskExecutionState(final TaskExecutionState taskExecutionState) { final Optional executionVertexId = getExecutionVertexId(taskExecutionState.getID()); - if (executionVertexId.isPresent()) { - executionGraph.updateState(taskExecutionState); + + boolean updateSuccess = executionGraph.updateState(taskExecutionState); + + if (updateSuccess && executionVertexId.isPresent()) { updateTaskExecutionStateInternal(executionVertexId.get(), taskExecutionState); return true; + } else { + return false; } - return false; } protected void updateTaskExecutionStateInternal(final ExecutionVertexID executionVertexId, final TaskExecutionState taskExecutionState) { From 044eb42ed2142468f0132e678e713c901b3045cd Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Fri, 1 Nov 2019 22:14:05 +0800 Subject: [PATCH 443/746] [hotfix][runtime] Invoke sendCancelRpcCall for internally detected failures after notifying it to scheduler Also fix the issue that sendCancelRpcCall and handlePartitionCleanup may happen twice when it is non-legacy scheduling. --- .../runtime/executiongraph/Execution.java | 48 ++++++++++++------- 1 file changed, 31 insertions(+), 17 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 2db95ecbf751..4febc4052291 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 @@ -1227,11 +1227,14 @@ private void processFail(Throwable t, boolean isCallback, Map Date: Tue, 5 Nov 2019 19:07:33 +0800 Subject: [PATCH 444/746] [FLINK-14375][runtime] Avoid notifying ineffective state update to scheduler This closes #10067. --- .../runtime/scheduler/SchedulerBase.java | 34 +++++++++++++++++-- 1 file changed, 32 insertions(+), 2 deletions(-) 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 c989c1258c12..eaa688da4887 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 @@ -100,6 +100,7 @@ import java.util.concurrent.ScheduledExecutorService; import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; /** * Base class which can be used to implement {@link SchedulerNG}. @@ -420,14 +421,43 @@ public final boolean updateTaskExecutionState(final TaskExecutionState taskExecu boolean updateSuccess = executionGraph.updateState(taskExecutionState); - if (updateSuccess && executionVertexId.isPresent()) { - updateTaskExecutionStateInternal(executionVertexId.get(), taskExecutionState); + if (updateSuccess) { + checkState(executionVertexId.isPresent()); + + if (isNotifiable(executionVertexId.get(), taskExecutionState)) { + updateTaskExecutionStateInternal(executionVertexId.get(), taskExecutionState); + } return true; } else { return false; } } + private boolean isNotifiable( + final ExecutionVertexID executionVertexId, + final TaskExecutionState taskExecutionState) { + + final ExecutionVertex executionVertex = getExecutionVertex(executionVertexId); + + // only notifies FINISHED and FAILED states which are needed at the moment. + // can be refined in FLINK-14233 after the legacy scheduler is removed and + // the actions are factored out from ExecutionGraph. + switch (taskExecutionState.getExecutionState()) { + case FINISHED: + case FAILED: + // only notifies a state update if it's effective, namely it successfully + // turns the execution state to the expected value. + if (executionVertex.getExecutionState() == taskExecutionState.getExecutionState()) { + return true; + } + break; + default: + break; + } + + return false; + } + protected void updateTaskExecutionStateInternal(final ExecutionVertexID executionVertexId, final TaskExecutionState taskExecutionState) { } From 183113f8a7520c519c0ec7f0a1de1f71a34cd4fb Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Sat, 2 Nov 2019 14:33:45 +0800 Subject: [PATCH 445/746] [FLINK-14371][tests] Annotate WindowCheckpointingITCase with AlsoRunWithSchedulerNG category --- .../flink/test/checkpointing/WindowCheckpointingITCase.java | 3 +++ 1 file changed, 3 insertions(+) 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 c722311cff2d..0ea0011d373f 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 @@ -36,11 +36,13 @@ import org.apache.flink.test.checkpointing.utils.IntType; import org.apache.flink.test.checkpointing.utils.ValidatingSink; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; import org.junit.ClassRule; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -59,6 +61,7 @@ * serializability is handled correctly. */ @SuppressWarnings("serial") +@Category(AlsoRunWithSchedulerNG.class) @RunWith(Parameterized.class) public class WindowCheckpointingITCase extends TestLogger { From 99842a6ca9badb0a15ce28087a9299f372f696fc Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Sat, 2 Nov 2019 14:34:17 +0800 Subject: [PATCH 446/746] [FLINK-14371][tests] Annotate EventTimeWindowCheckpointingITCase with AlsoRunWithSchedulerNG category --- .../test/checkpointing/EventTimeWindowCheckpointingITCase.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java index de681255c483..6651db7a6ae2 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 @@ -48,6 +48,7 @@ import org.apache.flink.test.checkpointing.utils.IntType; import org.apache.flink.test.checkpointing.utils.ValidatingSink; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; @@ -57,6 +58,7 @@ import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.junit.rules.TemporaryFolder; import org.junit.rules.TestName; import org.junit.runner.RunWith; @@ -84,6 +86,7 @@ * I/O heavy variants. */ @SuppressWarnings("serial") +@Category(AlsoRunWithSchedulerNG.class) @RunWith(Parameterized.class) public class EventTimeWindowCheckpointingITCase extends TestLogger { From ef3dd92c13aeae581d96f814890599bbd54da0a0 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Tue, 5 Nov 2019 10:50:41 +0800 Subject: [PATCH 447/746] [FLINK-14371][tests] Enable ClassLoaderITCase to pass with NG scheduler Also annotate it with AlsoRunWithSchedulerNG category. This closes #10071. --- .../test/classloading/ClassLoaderITCase.java | 26 +++++++++---------- 1 file changed, 12 insertions(+), 14 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 a09142e7ee14..2d8afc0f7822 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 @@ -36,18 +36,16 @@ import org.apache.flink.test.testdata.KMeansData; import org.apache.flink.test.util.SuccessException; import org.apache.flink.test.util.TestEnvironment; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; -import org.hamcrest.Matchers; import org.junit.After; import org.junit.AfterClass; -import org.junit.Assert; import org.junit.BeforeClass; import org.junit.ClassRule; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.ExpectedException; +import org.junit.experimental.categories.Category; import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -63,13 +61,15 @@ import scala.concurrent.duration.Deadline; import scala.concurrent.duration.FiniteDuration; -import static org.hamcrest.CoreMatchers.isA; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; /** * Test job classloader. */ +@Category(AlsoRunWithSchedulerNG.class) public class ClassLoaderITCase extends TestLogger { private static final Logger LOG = LoggerFactory.getLogger(ClassLoaderITCase.class); @@ -90,13 +90,9 @@ public class ClassLoaderITCase extends TestLogger { private static final String CHECKPOINTING_CUSTOM_KV_STATE_JAR_PATH = "checkpointing_custom_kv_state-test-jar.jar"; - @ClassRule public static final TemporaryFolder FOLDER = new TemporaryFolder(); - @Rule - public ExpectedException expectedException = ExpectedException.none(); - private static MiniClusterResource miniClusterResource = null; private static final int parallelism = 4; @@ -225,7 +221,7 @@ public void testCheckpointedStreamingClassloaderJobWithCustomClassLoader() throw try { Class.forName(exception.get().getClass().getName()); - Assert.fail("Deserialization of user exception should have failed."); + fail("Deserialization of user exception should have failed."); } catch (ClassNotFoundException expected) { // expected } @@ -282,10 +278,12 @@ public void testCheckpointingCustomKvStateJobWithCustomClassLoader() throws IOEx Collections.singleton(new Path(CHECKPOINTING_CUSTOM_KV_STATE_JAR_PATH)), Collections.emptyList()); - expectedException.expectCause( - Matchers.hasProperty("cause", isA(SuccessException.class))); - - program.invokeInteractiveModeForExecution(); + try { + program.invokeInteractiveModeForExecution(); + fail("exception should happen"); + } catch (ProgramInvocationException e) { + assertTrue(ExceptionUtils.findThrowable(e, SuccessException.class).isPresent()); + } } /** From 08793a93cf910944de36ea67cdaec06975628a48 Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Thu, 24 Oct 2019 11:12:11 -0700 Subject: [PATCH 448/746] [FLINK-14418][hive] Create HiveModule to provide Hive built-in functions - Create HiveModule to provide Hive built-in functions as system functions to Flink - Unify creation of HiveShim and reuse a single HiveShim in the stack to avoid creating new instances repeatedly This closes #9988. --- .../flink/table/catalog/hive/HiveCatalog.java | 2 +- .../table/catalog/hive/client/HiveShim.java | 12 +++ .../catalog/hive/client/HiveShimV100.java | 17 ++++ .../catalog/hive/client/HiveShimV120.java | 53 +++++++++++ .../HiveFunctionDefinitionFactory.java | 32 ++++--- .../table/functions/hive/HiveGenericUDAF.java | 15 ++-- .../flink/table/module/hive/HiveModule.java | 58 ++++++++++++ .../flink/table/HiveVersionTestUtil.java | 29 ++++++ .../functions/hive/HiveGenericUDAFTest.java | 2 +- .../functions/hive/HiveGenericUDFTest.java | 4 +- .../table/module/hive/HiveModuleTest.java | 89 +++++++++++++++++++ .../src/test/resources/csv/test2.csv | 3 + 12 files changed, 286 insertions(+), 30 deletions(-) create mode 100644 flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/module/hive/HiveModule.java create mode 100644 flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/HiveVersionTestUtil.java create mode 100644 flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleTest.java create mode 100644 flink-connectors/flink-connector-hive/src/test/resources/csv/test2.csv diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java index 9afd55c35af2..82832e7e1d28 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java @@ -204,7 +204,7 @@ public Optional getTableFactory() { @Override public Optional getFunctionDefinitionFactory() { - return Optional.of(new HiveFunctionDefinitionFactory(hiveConf)); + return Optional.of(new HiveFunctionDefinitionFactory(hiveShim)); } // ------ databases ------ diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShim.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShim.java index 1e27fb6631e7..0b58724285e4 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShim.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShim.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.api.UnknownDBException; import org.apache.hadoop.hive.ql.exec.FileSinkOperator; +import org.apache.hadoop.hive.ql.exec.FunctionInfo; import org.apache.hadoop.hive.ql.udf.generic.SimpleGenericUDAFParameterInfo; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; @@ -47,7 +48,9 @@ import java.io.Serializable; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; +import java.util.Set; /** * A shim layer to support different versions of Hive. @@ -200,4 +203,13 @@ FileSinkOperator.RecordWriter getHiveRecordWriter(JobConf jobConf, String output */ List getFieldsFromDeserializer(Configuration conf, Table table, boolean skipConfError); + /** + * List names of all built-in functions. + */ + Set listBuiltInFunctions(); + + /** + * Get a Hive built-in function by name. + */ + Optional getBuiltInFunctionInfo(String name); } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV100.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV100.java index 4afef67745b5..e2f79d7c25b8 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV100.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV100.java @@ -45,6 +45,7 @@ import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.api.UnknownDBException; import org.apache.hadoop.hive.ql.exec.FileSinkOperator; +import org.apache.hadoop.hive.ql.exec.FunctionInfo; import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils; import org.apache.hadoop.hive.ql.io.HiveOutputFormat; import org.apache.hadoop.hive.ql.udf.generic.SimpleGenericUDAFParameterInfo; @@ -79,7 +80,9 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; +import java.util.Set; /** * Shim for Hive version 1.0.0. @@ -335,4 +338,18 @@ public List getFieldsFromDeserializer(Configuration conf, Table tab throw new CatalogException("Failed to get table schema from deserializer", e); } } + + @Override + public Set listBuiltInFunctions() { + // FunctionInfo doesn't have isBuiltIn() API to tell whether it's a builtin function or not + // prior to Hive 1.2.0 + throw new UnsupportedOperationException("Listing built in functions are not supported until Hive 1.2.0"); + } + + @Override + public Optional getBuiltInFunctionInfo(String name) { + // FunctionInfo doesn't have isBuiltIn() API to tell whether it's a builtin function or not + // prior to Hive 1.2.0 + throw new UnsupportedOperationException("Getting built in functions are not supported until Hive 1.2.0"); + } } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV120.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV120.java index fc929476fd9e..3a5849223019 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV120.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV120.java @@ -18,6 +18,7 @@ package org.apache.flink.table.catalog.hive.client; +import org.apache.flink.connectors.hive.FlinkHiveException; import org.apache.flink.table.catalog.exceptions.CatalogException; import org.apache.flink.table.catalog.hive.util.HiveReflectionUtils; import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataDate; @@ -31,6 +32,9 @@ import org.apache.hadoop.hive.metastore.api.InvalidOperationException; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.exec.FunctionInfo; +import org.apache.hadoop.hive.ql.exec.FunctionRegistry; +import org.apache.hadoop.hive.ql.parse.SemanticException; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; import org.apache.thrift.TException; @@ -38,6 +42,9 @@ import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; /** * Shim for Hive version 1.2.0. @@ -168,4 +175,50 @@ public CatalogColumnStatisticsDataDate toFlinkDateColStats(ColumnStatisticsData throw new CatalogException("Failed to create Flink statistics for date column", e); } } + + @Override + public Set listBuiltInFunctions() { + try { + Method method = FunctionRegistry.class.getMethod("getFunctionNames"); + // getFunctionNames is a static method + Set names = (Set) method.invoke(null); + + return names.stream() + .filter(n -> isBuiltInFunctionInfo(getFunctionInfo(n).get())) + .collect(Collectors.toSet()); + } catch (Exception ex) { + throw new CatalogException("Failed to invoke FunctionRegistry.getFunctionNames()", ex); + } + } + + @Override + public Optional getBuiltInFunctionInfo(String name) { + Optional functionInfo = getFunctionInfo(name); + + if (functionInfo.isPresent() && isBuiltInFunctionInfo(functionInfo.get())) { + return functionInfo; + } else { + return Optional.empty(); + } + } + + private Optional getFunctionInfo(String name) { + try { + return Optional.of(FunctionRegistry.getFunctionInfo(name)); + } catch (SemanticException e) { + throw new FlinkHiveException( + String.format("Failed getting function info for %s", name), e); + } catch (NullPointerException e) { + return Optional.empty(); + } + } + + private boolean isBuiltInFunctionInfo(FunctionInfo info) { + try { + Method method = FunctionInfo.class.getMethod("isBuiltIn", null); + return (boolean) method.invoke(info); + } catch (Exception ex) { + throw new CatalogException("Failed to invoke FunctionInfo.isBuiltIn()", ex); + } + } } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/factories/HiveFunctionDefinitionFactory.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/factories/HiveFunctionDefinitionFactory.java index 3e32c7fa4eac..4027474ce3c1 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/factories/HiveFunctionDefinitionFactory.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/factories/HiveFunctionDefinitionFactory.java @@ -24,8 +24,6 @@ import org.apache.flink.table.catalog.CatalogFunction; import org.apache.flink.table.catalog.config.CatalogConfig; import org.apache.flink.table.catalog.hive.client.HiveShim; -import org.apache.flink.table.catalog.hive.client.HiveShimLoader; -import org.apache.flink.table.catalog.hive.descriptors.HiveCatalogValidator; import org.apache.flink.table.factories.FunctionDefinitionFactory; import org.apache.flink.table.functions.AggregateFunctionDefinition; import org.apache.flink.table.functions.FunctionDefinition; @@ -39,7 +37,6 @@ import org.apache.flink.table.functions.hive.HiveSimpleUDF; import org.apache.flink.types.Row; -import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.exec.UDAF; import org.apache.hadoop.hive.ql.exec.UDF; import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator; @@ -57,24 +54,27 @@ public class HiveFunctionDefinitionFactory implements FunctionDefinitionFactory { private static final Logger LOG = LoggerFactory.getLogger(HiveTableFactory.class); - private final String hiveVersion; private final HiveShim hiveShim; - public HiveFunctionDefinitionFactory(HiveConf hiveConf) { - // this has to come from hiveConf, otherwise we may lose what user specifies in the yaml file - this.hiveVersion = checkNotNull(hiveConf.get(HiveCatalogValidator.CATALOG_HIVE_VERSION), - "Hive version is not defined"); - this.hiveShim = HiveShimLoader.loadHiveShim(hiveVersion); + public HiveFunctionDefinitionFactory(HiveShim hiveShim) { + checkNotNull(hiveShim, "hiveShim cannot be null"); + this.hiveShim = hiveShim; } @Override public FunctionDefinition createFunctionDefinition(String name, CatalogFunction catalogFunction) { if (Boolean.valueOf(catalogFunction.getProperties().get(CatalogConfig.IS_GENERIC))) { - FunctionDefinitionUtil.createFunctionDefinition(name, catalogFunction); + return FunctionDefinitionUtil.createFunctionDefinition(name, catalogFunction); } - String functionClassName = catalogFunction.getClassName(); + return createFunctionDefinitionFromHiveFunction(name, catalogFunction.getClassName()); + } + /** + * Create a FunctionDefinition from a Hive function's class name. + * Called directly by {@link org.apache.flink.table.module.hive.HiveModule}. + */ + public FunctionDefinition createFunctionDefinitionFromHiveFunction(String name, String functionClassName) { Class clazz; try { clazz = Thread.currentThread().getContextClassLoader().loadClass(functionClassName); @@ -114,16 +114,14 @@ public FunctionDefinition createFunctionDefinition(String name, CatalogFunction if (GenericUDAFResolver2.class.isAssignableFrom(clazz)) { LOG.info( - "Transforming Hive function '{}' into a HiveGenericUDAF with no UDAF bridging and Hive version %s", - name, hiveVersion); + "Transforming Hive function '{}' into a HiveGenericUDAF without UDAF bridging", name); - udaf = new HiveGenericUDAF(new HiveFunctionWrapper<>(functionClassName), false, hiveVersion); + udaf = new HiveGenericUDAF(new HiveFunctionWrapper<>(functionClassName), false, hiveShim); } else { LOG.info( - "Transforming Hive function '{}' into a HiveGenericUDAF with UDAF bridging and Hive version %s", - name, hiveVersion); + "Transforming Hive function '{}' into a HiveGenericUDAF with UDAF bridging", name); - udaf = new HiveGenericUDAF(new HiveFunctionWrapper<>(functionClassName), true, hiveVersion); + udaf = new HiveGenericUDAF(new HiveFunctionWrapper<>(functionClassName), true, hiveShim); } return new AggregateFunctionDefinition( diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveGenericUDAF.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveGenericUDAF.java index a6f4e3b78cd3..429f74e8bac6 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveGenericUDAF.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveGenericUDAF.java @@ -22,7 +22,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.GenericTypeInfo; import org.apache.flink.table.catalog.hive.client.HiveShim; -import org.apache.flink.table.catalog.hive.client.HiveShimLoader; import org.apache.flink.table.catalog.hive.util.HiveTypeUtil; import org.apache.flink.table.functions.AggregateFunction; import org.apache.flink.table.functions.FunctionContext; @@ -65,16 +64,16 @@ public class HiveGenericUDAF private transient boolean allIdentityConverter; private transient boolean initialized; - private final String hiveVersion; + private final HiveShim hiveShim; - public HiveGenericUDAF(HiveFunctionWrapper funcWrapper, String hiveVersion) { - this(funcWrapper, false, hiveVersion); + public HiveGenericUDAF(HiveFunctionWrapper funcWrapper, HiveShim hiveShim) { + this(funcWrapper, false, hiveShim); } - public HiveGenericUDAF(HiveFunctionWrapper funcWrapper, boolean isUDAFBridgeRequired, String hiveVersion) { + public HiveGenericUDAF(HiveFunctionWrapper funcWrapper, boolean isUDAFBridgeRequired, HiveShim hiveShim) { this.hiveFunctionWrapper = funcWrapper; this.isUDAFBridgeRequired = isUDAFBridgeRequired; - this.hiveVersion = hiveVersion; + this.hiveShim = hiveShim; } @Override @@ -84,8 +83,7 @@ public void open(FunctionContext context) throws Exception { } private void init() throws HiveException { - ObjectInspector[] inputInspectors = HiveInspectors.toInspectors(HiveShimLoader.loadHiveShim(hiveVersion), - constantArguments, argTypes); + ObjectInspector[] inputInspectors = HiveInspectors.toInspectors(hiveShim, constantArguments, argTypes); // Flink UDAF only supports Hive UDAF's PARTIAL_1 and FINAL mode @@ -119,7 +117,6 @@ private GenericUDAFEvaluator createEvaluator(ObjectInspector[] inputInspectors) resolver = (GenericUDAFResolver2) hiveFunctionWrapper.createFunction(); } - HiveShim hiveShim = HiveShimLoader.loadHiveShim(hiveVersion); return resolver.getEvaluator( hiveShim.createUDAFParameterInfo( inputInspectors, diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/module/hive/HiveModule.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/module/hive/HiveModule.java new file mode 100644 index 000000000000..048e4165d62a --- /dev/null +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/module/hive/HiveModule.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.table.module.hive; + +import org.apache.flink.table.catalog.hive.client.HiveShim; +import org.apache.flink.table.catalog.hive.client.HiveShimLoader; +import org.apache.flink.table.catalog.hive.factories.HiveFunctionDefinitionFactory; +import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.module.Module; + +import org.apache.hadoop.hive.ql.exec.FunctionInfo; + +import java.util.Optional; +import java.util.Set; + +/** + * Module to provide Hive built-in metadata. + */ +public class HiveModule implements Module { + + private final HiveFunctionDefinitionFactory factory; + private final HiveShim hiveShim; + + public HiveModule(String hiveVersion) { + this.hiveShim = HiveShimLoader.loadHiveShim(hiveVersion); + this.factory = new HiveFunctionDefinitionFactory(hiveShim); + } + + @Override + public Set listFunctions() { + return hiveShim.listBuiltInFunctions(); + } + + @Override + public Optional getFunctionDefinition(String name) { + Optional info = hiveShim.getBuiltInFunctionInfo(name); + + return info.isPresent() ? + Optional.of(factory.createFunctionDefinitionFromHiveFunction(name, info.get().getFunctionClass().getName())) + : Optional.empty(); + } +} diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/HiveVersionTestUtil.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/HiveVersionTestUtil.java new file mode 100644 index 000000000000..bf441d14390c --- /dev/null +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/HiveVersionTestUtil.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.table; + +import org.apache.flink.table.catalog.hive.client.HiveShimLoader; + +/** + * Util for testing different Hive versions. + */ +public class HiveVersionTestUtil { + public static final boolean HIVE_120_OR_LATER = HiveShimLoader.getHiveVersion().compareTo(HiveShimLoader.HIVE_VERSION_V1_2_0) >= 0; + public static final boolean HIVE_110_OR_LATER = HiveShimLoader.getHiveVersion().compareTo(HiveShimLoader.HIVE_VERSION_V1_1_0) >= 0; + +} diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/functions/hive/HiveGenericUDAFTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/functions/hive/HiveGenericUDAFTest.java index 29455230ed14..2aa1fc537b9c 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/functions/hive/HiveGenericUDAFTest.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/functions/hive/HiveGenericUDAFTest.java @@ -130,7 +130,7 @@ public void testUDAFCount() throws Exception { private static HiveGenericUDAF init(Class hiveUdfClass, Object[] constantArgs, DataType[] argTypes) throws Exception { HiveFunctionWrapper wrapper = new HiveFunctionWrapper(hiveUdfClass.getName()); - HiveGenericUDAF udf = new HiveGenericUDAF(wrapper, HiveShimLoader.getHiveVersion()); + HiveGenericUDAF udf = new HiveGenericUDAF(wrapper, HiveShimLoader.loadHiveShim(HiveShimLoader.getHiveVersion())); udf.setArgumentTypesAndConstants(constantArgs, argTypes); udf.getHiveResultType(constantArgs, argTypes); diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/functions/hive/HiveGenericUDFTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/functions/hive/HiveGenericUDFTest.java index fc8c0b5c0893..55fd8d44bdcf 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/functions/hive/HiveGenericUDFTest.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/functions/hive/HiveGenericUDFTest.java @@ -44,6 +44,8 @@ import java.math.BigDecimal; import java.util.HashMap; +import static org.apache.flink.table.HiveVersionTestUtil.HIVE_110_OR_LATER; +import static org.apache.flink.table.HiveVersionTestUtil.HIVE_120_OR_LATER; import static org.junit.Assert.assertEquals; /** @@ -51,8 +53,6 @@ */ public class HiveGenericUDFTest { private static HiveShim hiveShim = HiveShimLoader.loadHiveShim(HiveShimLoader.getHiveVersion()); - private static final boolean HIVE_120_OR_LATER = HiveShimLoader.getHiveVersion().compareTo(HiveShimLoader.HIVE_VERSION_V1_2_0) >= 0; - private static final boolean HIVE_110_OR_LATER = HiveShimLoader.getHiveVersion().compareTo(HiveShimLoader.HIVE_VERSION_V1_1_0) >= 0; @Test public void testAbs() { diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleTest.java new file mode 100644 index 000000000000..7fefe3b685c9 --- /dev/null +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleTest.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.module.hive; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.catalog.hive.client.HiveShimLoader; +import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.functions.ScalarFunction; +import org.apache.flink.table.functions.ScalarFunctionDefinition; +import org.apache.flink.table.functions.hive.HiveSimpleUDF; +import org.apache.flink.table.types.DataType; + +import org.junit.BeforeClass; +import org.junit.Test; + +import static org.apache.flink.table.HiveVersionTestUtil.HIVE_120_OR_LATER; +import static org.apache.flink.table.catalog.hive.client.HiveShimLoader.HIVE_VERSION_V1_2_0; +import static org.apache.flink.table.catalog.hive.client.HiveShimLoader.HIVE_VERSION_V2_0_0; +import static org.apache.flink.table.catalog.hive.client.HiveShimLoader.HIVE_VERSION_V2_1_1; +import static org.apache.flink.table.catalog.hive.client.HiveShimLoader.HIVE_VERSION_V2_2_0; +import static org.apache.flink.table.catalog.hive.client.HiveShimLoader.HIVE_VERSION_V2_3_4; +import static org.apache.flink.table.catalog.hive.client.HiveShimLoader.HIVE_VERSION_V3_1_1; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assume.assumeTrue; + +/** + * Test for {@link HiveModule}. + */ +public class HiveModuleTest { + @BeforeClass + public static void init() { + assumeTrue(HIVE_120_OR_LATER); + } + + @Test + public void testNumberOfBuiltinFunctions() { + String hiveVersion = HiveShimLoader.getHiveVersion(); + + if (hiveVersion.equals(HIVE_VERSION_V1_2_0)) { + assertEquals(232, new HiveModule(HiveShimLoader.getHiveVersion()).listFunctions().size()); + } else if (hiveVersion.equals(HIVE_VERSION_V2_0_0)) { + assertEquals(243, new HiveModule(HiveShimLoader.getHiveVersion()).listFunctions().size()); + } else if (hiveVersion.equals(HIVE_VERSION_V2_1_1) || hiveVersion.equals(HIVE_VERSION_V2_2_0)) { + assertEquals(253, new HiveModule(HiveShimLoader.getHiveVersion()).listFunctions().size()); + } else if (hiveVersion.equals(HIVE_VERSION_V2_3_4) || hiveVersion.equals(HIVE_VERSION_V3_1_1)) { + assertEquals(287, new HiveModule(HiveShimLoader.getHiveVersion()).listFunctions().size()); + } + } + + @Test + public void testHiveBuiltInFunction() { + FunctionDefinition fd = new HiveModule(HiveShimLoader.getHiveVersion()).getFunctionDefinition("reverse").get(); + + ScalarFunction func = ((ScalarFunctionDefinition) fd).getScalarFunction(); + HiveSimpleUDF udf = (HiveSimpleUDF) func; + + DataType[] inputType = new DataType[] { + DataTypes.STRING() + }; + + udf.setArgumentTypesAndConstants(new Object[0], inputType); + udf.getHiveResultType(new Object[0], inputType); + + udf.open(null); + + assertEquals("cba", udf.eval("abc")); + } + + @Test + public void testNonExistFunction() { + assertFalse(new HiveModule(HiveShimLoader.getHiveVersion()).getFunctionDefinition("nonexist").isPresent()); + } +} diff --git a/flink-connectors/flink-connector-hive/src/test/resources/csv/test2.csv b/flink-connectors/flink-connector-hive/src/test/resources/csv/test2.csv new file mode 100644 index 000000000000..364f81037a24 --- /dev/null +++ b/flink-connectors/flink-connector-hive/src/test/resources/csv/test2.csv @@ -0,0 +1,3 @@ +tom +jimmy +andy From a9704308802e3f71589c790a81a7e65a56ea35dc Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Fri, 1 Nov 2019 16:21:15 -0700 Subject: [PATCH 449/746] [FLINK-14419][table] Add ModuleFactory, ModuleDescriptor, ModuleDescriptorValidator for factory discovery service, and add implementations for CoreModule Add ModuleFactory, ModuleDescriptor, ModuleValidator for factory discovery service, and add implementations for CoreModule. This closes #10070. --- .../flink/table/module/ModuleManager.java | 3 +- .../descriptors/CoreModuleDescriptor.java | 40 ++++++++++++ .../CoreModuleDescriptorValidator.java} | 15 +++-- .../table/descriptors/ModuleDescriptor.java | 61 +++++++++++++++++++ .../ModuleDescriptorValidator.java | 38 ++++++++++++ .../flink/table/factories/ModuleFactory.java | 40 ++++++++++++ .../flink/table/module/CoreModuleFactory.java | 53 ++++++++++++++++ ....apache.flink.table.factories.TableFactory | 16 +++++ .../module/CoreModuleDescriptorTest.java | 57 +++++++++++++++++ .../module/CoreModuleFactoryTest.java | 52 ++++++++++++++++ 10 files changed, 369 insertions(+), 6 deletions(-) create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/CoreModuleDescriptor.java rename flink-table/flink-table-common/src/main/java/org/apache/flink/table/{module/ModuleConfig.java => descriptors/CoreModuleDescriptorValidator.java} (67%) create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/ModuleDescriptor.java create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/ModuleDescriptorValidator.java create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/ModuleFactory.java create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/CoreModuleFactory.java create mode 100644 flink-table/flink-table-common/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory create mode 100644 flink-table/flink-table-common/src/test/java/org/apache/flink/table/descriptors/module/CoreModuleDescriptorTest.java create mode 100644 flink-table/flink-table-common/src/test/java/org/apache/flink/table/factories/module/CoreModuleFactoryTest.java diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/module/ModuleManager.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/module/ModuleManager.java index abb3f8eebe30..e59b63890256 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/module/ModuleManager.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/module/ModuleManager.java @@ -29,6 +29,7 @@ import java.util.Set; import java.util.stream.Collectors; +import static org.apache.flink.table.descriptors.CoreModuleDescriptorValidator.MODULE_TYPE_CORE; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -41,7 +42,7 @@ public class ModuleManager { public ModuleManager() { this.modules = new LinkedHashMap<>(); - modules.put(ModuleConfig.CORE_MODULE_NAME, CoreModule.INSTANCE); + modules.put(MODULE_TYPE_CORE, CoreModule.INSTANCE); } /** diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/CoreModuleDescriptor.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/CoreModuleDescriptor.java new file mode 100644 index 000000000000..b3e6a19bd6a2 --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/CoreModuleDescriptor.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.table.descriptors; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.flink.table.descriptors.CoreModuleDescriptorValidator.MODULE_TYPE_CORE; + +/** + * Module descriptor for {@link org.apache.flink.table.module.CoreModule}. + */ +public class CoreModuleDescriptor extends ModuleDescriptor { + + public CoreModuleDescriptor() { + super(MODULE_TYPE_CORE); + } + + @Override + protected Map toModuleProperties() { + return Collections.unmodifiableMap(new HashMap<>()); + } +} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/ModuleConfig.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/CoreModuleDescriptorValidator.java similarity index 67% rename from flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/ModuleConfig.java rename to flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/CoreModuleDescriptorValidator.java index f4dbb6b4cf83..d852d1823058 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/ModuleConfig.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/CoreModuleDescriptorValidator.java @@ -16,12 +16,17 @@ * limitations under the License. */ -package org.apache.flink.table.module; +package org.apache.flink.table.descriptors; /** - * Configs for modules. + * Validator for {@link CoreModuleDescriptor}. */ -public class ModuleConfig { - // Name of the core module - public static final String CORE_MODULE_NAME = "core"; +public class CoreModuleDescriptorValidator extends ModuleDescriptorValidator { + public static final String MODULE_TYPE_CORE = "core"; + + @Override + public void validate(DescriptorProperties properties) { + super.validate(properties); + properties.validateValue(MODULE_TYPE, MODULE_TYPE_CORE, false); + } } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/ModuleDescriptor.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/ModuleDescriptor.java new file mode 100644 index 000000000000..27382358e6d4 --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/ModuleDescriptor.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.table.descriptors; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.util.StringUtils; + +import java.util.Map; + +import static org.apache.flink.table.descriptors.ModuleDescriptorValidator.MODULE_TYPE; +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * Describes a {@link org.apache.flink.table.module.Module}. + */ +@PublicEvolving +public abstract class ModuleDescriptor extends DescriptorBase { + + private final String type; + + /** + * Constructs a {@link ModuleDescriptor}. + * + * @param type string that identifies this catalog + */ + public ModuleDescriptor(String type) { + checkArgument(!StringUtils.isNullOrWhitespaceOnly(type), "type cannot be null or empty"); + + this.type = type; + } + + @Override + public final Map toProperties() { + final DescriptorProperties properties = new DescriptorProperties(); + properties.putString(MODULE_TYPE, type); + + properties.putProperties(toModuleProperties()); + return properties.asMap(); + } + + /** + * Converts this descriptor into a set of module properties. + */ + protected abstract Map toModuleProperties(); +} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/ModuleDescriptorValidator.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/ModuleDescriptorValidator.java new file mode 100644 index 000000000000..dd40c5844809 --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/ModuleDescriptorValidator.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.Internal; + +/** + * Validator for {@link ModuleDescriptor}. + */ +@Internal +public abstract class ModuleDescriptorValidator implements DescriptorValidator { + + /** + * Key for describing the type of the module. Used for factory discovery. + */ + public static final String MODULE_TYPE = "type"; + + @Override + public void validate(DescriptorProperties properties) { + properties.validateString(MODULE_TYPE, false, 1); + } +} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/ModuleFactory.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/ModuleFactory.java new file mode 100644 index 000000000000..947bd6e2a1c1 --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/factories/ModuleFactory.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.table.factories; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.table.module.Module; + +import java.util.Map; + +/** + * A factory to create configured module instances based on string-based properties. See + * also {@link TableFactory} for more information. + */ +@PublicEvolving +public interface ModuleFactory extends TableFactory { + + /** + * Creates and configures a {@link Module} using the given properties. + * + * @param properties normalized properties describing a module. + * @return the configured module. + */ + Module createModule(Map properties); +} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/CoreModuleFactory.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/CoreModuleFactory.java new file mode 100644 index 000000000000..aeb360059e0d --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/CoreModuleFactory.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.table.module; + +import org.apache.flink.table.factories.ModuleFactory; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.table.descriptors.CoreModuleDescriptorValidator.MODULE_TYPE_CORE; +import static org.apache.flink.table.descriptors.ModuleDescriptorValidator.MODULE_TYPE; + +/** + * Factory for {@link CoreModule}. + */ +public class CoreModuleFactory implements ModuleFactory { + + @Override + public Module createModule(Map properties) { + return CoreModule.INSTANCE; + } + + @Override + public Map requiredContext() { + Map context = new HashMap<>(); + context.put(MODULE_TYPE, MODULE_TYPE_CORE); + + return context; + } + + @Override + public List supportedProperties() { + return new ArrayList<>(); + } +} diff --git a/flink-table/flink-table-common/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink-table/flink-table-common/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory new file mode 100644 index 000000000000..456c161e7dd2 --- /dev/null +++ b/flink-table/flink-table-common/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.table.module.CoreModuleFactory diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/descriptors/module/CoreModuleDescriptorTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/descriptors/module/CoreModuleDescriptorTest.java new file mode 100644 index 000000000000..681064d163b1 --- /dev/null +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/descriptors/module/CoreModuleDescriptorTest.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.descriptors.module; + +import org.apache.flink.table.descriptors.CoreModuleDescriptor; +import org.apache.flink.table.descriptors.CoreModuleDescriptorValidator; +import org.apache.flink.table.descriptors.Descriptor; +import org.apache.flink.table.descriptors.DescriptorTestBase; +import org.apache.flink.table.descriptors.DescriptorValidator; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.table.descriptors.CoreModuleDescriptorValidator.MODULE_TYPE_CORE; +import static org.apache.flink.table.descriptors.ModuleDescriptorValidator.MODULE_TYPE; + +/** + * Tests for the {@link CoreModuleDescriptor}. + */ +public class CoreModuleDescriptorTest extends DescriptorTestBase { + + @Override + protected List descriptors() { + return Arrays.asList(new CoreModuleDescriptor()); + } + + @Override + protected List> properties() { + final Map minimumProps = new HashMap<>(); + minimumProps.put(MODULE_TYPE, MODULE_TYPE_CORE); + return Collections.singletonList(minimumProps); + } + + @Override + protected DescriptorValidator validator() { + return new CoreModuleDescriptorValidator(); + } +} diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/factories/module/CoreModuleFactoryTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/factories/module/CoreModuleFactoryTest.java new file mode 100644 index 000000000000..4599c0bcadac --- /dev/null +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/factories/module/CoreModuleFactoryTest.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.table.factories.module; + +import org.apache.flink.table.descriptors.CoreModuleDescriptor; +import org.apache.flink.table.descriptors.ModuleDescriptor; +import org.apache.flink.table.factories.ModuleFactory; +import org.apache.flink.table.factories.TableFactoryService; +import org.apache.flink.table.module.CoreModule; +import org.apache.flink.table.module.Module; + +import org.junit.Test; + +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +/** + * Test for {@link org.apache.flink.table.module.CoreModule} created by {@link org.apache.flink.table.module.CoreModuleFactory}. + */ +public class CoreModuleFactoryTest { + + @Test + public void test() { + final CoreModule expectedModule = CoreModule.INSTANCE; + + final ModuleDescriptor moduleDescriptor = new CoreModuleDescriptor(); + + final Map properties = moduleDescriptor.toProperties(); + + final Module actualModule = TableFactoryService.find(ModuleFactory.class, properties) + .createModule(properties); + + assertEquals(expectedModule, actualModule); + } +} From 2ea14169a1997434d45d6f1da6dfe9acd6bd8da3 Mon Sep 17 00:00:00 2001 From: Benchao Li Date: Wed, 6 Nov 2019 08:49:19 +0800 Subject: [PATCH 450/746] [FLINK-14546][formats] Support map type in JSON format This closes #10060 --- .../json/JsonRowDeserializationSchema.java | 25 +++++++++++++++++++ .../JsonRowDeserializationSchemaTest.java | 20 ++++++++++++--- 2 files changed, 42 insertions(+), 3 deletions(-) diff --git a/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java index 4a1ff274bd39..49fb58850de6 100644 --- a/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java +++ b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java @@ -24,6 +24,7 @@ import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.MapTypeInfo; import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.types.Row; @@ -34,6 +35,7 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.TextNode; import java.io.IOException; import java.io.Serializable; @@ -47,7 +49,10 @@ import java.time.temporal.TemporalAccessor; import java.time.temporal.TemporalQueries; import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.stream.Collectors; @@ -242,11 +247,31 @@ private Optional createContainerConverter(TypeI return Optional.of(createObjectArrayConverter(((BasicArrayTypeInfo) typeInfo).getComponentInfo())); } else if (isPrimitiveByteArray(typeInfo)) { return Optional.of(createByteArrayConverter()); + } else if (typeInfo instanceof MapTypeInfo) { + MapTypeInfo mapTypeInfo = (MapTypeInfo) typeInfo; + return Optional.of(createMapConverter(mapTypeInfo.getKeyTypeInfo(), mapTypeInfo.getValueTypeInfo())); } else { return Optional.empty(); } } + private DeserializationRuntimeConverter createMapConverter(TypeInformation keyType, TypeInformation valueType) { + DeserializationRuntimeConverter valueConverter = createConverter(valueType); + DeserializationRuntimeConverter keyConverter = createConverter(keyType); + + return (mapper, jsonNode) -> { + Iterator> fields = jsonNode.fields(); + Map result = new HashMap<>(); + while (fields.hasNext()) { + Map.Entry entry = fields.next(); + Object key = keyConverter.convert(mapper, TextNode.valueOf(entry.getKey())); + Object value = valueConverter.convert(mapper, entry.getValue()); + result.put(key, value); + } + return result; + }; + } + private DeserializationRuntimeConverter createByteArrayConverter() { return (mapper, jsonNode) -> { try { diff --git a/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java b/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java index 057301909495..8314b6381959 100644 --- a/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java +++ b/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowDeserializationSchemaTest.java @@ -32,6 +32,8 @@ import java.sql.Date; import java.sql.Time; import java.sql.Timestamp; +import java.util.HashMap; +import java.util.Map; import java.util.concurrent.ThreadLocalRandom; import static org.apache.flink.formats.utils.DeserializationSchemaMatcher.whenDeserializedWith; @@ -53,6 +55,13 @@ public void testTypeInfoDeserialization() throws Exception { String name = "asdlkjasjkdla998y1122"; byte[] bytes = new byte[1024]; ThreadLocalRandom.current().nextBytes(bytes); + Map map = new HashMap<>(); + map.put("flink", 123L); + + Map> nestedMap = new HashMap<>(); + Map innerMap = new HashMap<>(); + innerMap.put("key", 234); + nestedMap.put("inner_map", innerMap); ObjectMapper objectMapper = new ObjectMapper(); @@ -61,19 +70,24 @@ public void testTypeInfoDeserialization() throws Exception { root.put("id", id); root.put("name", name); root.put("bytes", bytes); + root.putObject("map").put("flink", 123); + root.putObject("map2map").putObject("inner_map").put("key", 234); byte[] serializedJson = objectMapper.writeValueAsBytes(root); JsonRowDeserializationSchema deserializationSchema = new JsonRowDeserializationSchema.Builder( Types.ROW_NAMED( - new String[]{"id", "name", "bytes"}, - Types.LONG, Types.STRING, Types.PRIMITIVE_ARRAY(Types.BYTE)) + new String[]{"id", "name", "bytes", "map", "map2map"}, + Types.LONG, Types.STRING, Types.PRIMITIVE_ARRAY(Types.BYTE), Types.MAP(Types.STRING, Types.LONG), + Types.MAP(Types.STRING, Types.MAP(Types.STRING, Types.INT))) ).build(); - Row row = new Row(3); + Row row = new Row(5); row.setField(0, id); row.setField(1, name); row.setField(2, bytes); + row.setField(3, map); + row.setField(4, nestedMap); assertThat(serializedJson, whenDeserializedWith(deserializationSchema).equalsTo(row)); } From 9b43f13a50848382fbd634081b82509f464e62ca Mon Sep 17 00:00:00 2001 From: "kevin.cyj" Date: Wed, 6 Nov 2019 12:54:30 +0800 Subject: [PATCH 451/746] [FLINK-14603][runtime] Notify the potential buffer consumer if the size of LocalBufferPool has been expanded. Currently, when the size of LocalBufferPool is expanded by LocalBufferPool#setNumBuffers and there are segments available in the global NetworkBufferPool, we may fail to notify the potential buffer consumer which is waiting for the LocalBufferPool to be available. This commit fixes the problem by completing the previous uncompleted available future when the size of LocalBufferPool is expanded. --- .../runtime/io/AvailabilityProvider.java | 10 ++++++ .../io/network/buffer/LocalBufferPool.java | 34 ++++++++++++++----- .../io/network/buffer/NetworkBufferPool.java | 4 +-- .../network/buffer/LocalBufferPoolTest.java | 6 ++-- 4 files changed, 42 insertions(+), 12 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 09d8ed1ac2d1..8ab59b3e5ef6 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 @@ -89,6 +89,16 @@ public CompletableFuture getUnavailableToResetAvailable() { return toNotify; } + /** + * Creates a new uncompleted future as the current state and returns the + * previous uncompleted one. + */ + public CompletableFuture getUnavailableToResetUnavailable() { + CompletableFuture toNotify = isAvailable; + isAvailable = new CompletableFuture<>(); + return toNotify; + } + /** * @return a future that is completed if the respective provider is available. */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java index 6e4c8aee9b67..72f98c0959e5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java @@ -305,9 +305,7 @@ public void recycle(MemorySegment segment) { notificationResult = fireBufferAvailableNotification(listener, segment); } - if (toNotify != null) { - toNotify.complete(null); - } + mayNotifyAvailable(toNotify); } private NotificationResult fireBufferAvailableNotification(BufferListener listener, MemorySegment segment) { @@ -348,7 +346,7 @@ public void lazyDestroy() { } final CompletableFuture isAvailable = availabilityHelper.isAvailable(); - if (isAvailable != AVAILABLE && !isAvailable.isDone()) { + if (!isAvailable(isAvailable)) { toNotify = isAvailable; } @@ -356,10 +354,7 @@ public void lazyDestroy() { } } - // notify the potential blocking request thread - if (toNotify != null) { - toNotify.complete(null); - } + mayNotifyAvailable(toNotify); try { networkBufferPool.destroyBufferPool(this); @@ -383,6 +378,7 @@ public boolean addBufferListener(BufferListener listener) { @Override public void setNumBuffers(int numBuffers) throws IOException { int numExcessBuffers; + CompletableFuture toNotify = null; synchronized (availableMemorySegments) { checkArgument(numBuffers >= numberOfRequiredMemorySegments, "Buffer pool needs at least %s buffers, but tried to set to %s", @@ -397,8 +393,16 @@ public void setNumBuffers(int numBuffers) throws IOException { returnExcessMemorySegments(); numExcessBuffers = numberOfRequestedMemorySegments - currentPoolSize; + + if (numExcessBuffers < 0 + && availableMemorySegments.isEmpty() + && isAvailable(networkBufferPool.isAvailable())) { + toNotify = availabilityHelper.getUnavailableToResetUnavailable(); + } } + mayNotifyAvailable(toNotify); + // If there is a registered owner and we have still requested more buffers than our // size, trigger a recycle via the owner. if (bufferPoolOwner != null && numExcessBuffers > 0) { @@ -429,6 +433,20 @@ public String toString() { // ------------------------------------------------------------------------ + /** + * Notifies the potential segment consumer of the new available segments by + * completing the previous uncompleted future. + */ + private void mayNotifyAvailable(@Nullable CompletableFuture toNotify) { + if (toNotify != null) { + toNotify.complete(null); + } + } + + private boolean isAvailable(CompletableFuture isAvailable) { + return isAvailable == AVAILABLE || isAvailable.isDone(); + } + private void returnMemorySegment(MemorySegment segment) { assert Thread.holdsLock(availableMemorySegments); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java index 4cf7fffae597..19a312539c10 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java @@ -229,14 +229,14 @@ public void recycleMemorySegments(Collection segments) throws IOE } private void recycleMemorySegments(Collection segments, int size) throws IOException { + internalRecycleMemorySegments(segments); + synchronized (factoryLock) { numTotalRequiredBuffers -= size; // note: if this fails, we're fine for the buffer pool since we already recycled the segments redistributeBuffers(); } - - internalRecycleMemorySegments(segments); } private void internalRecycleMemorySegments(Collection segments) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java index a3f4aed6ffb5..f22913b707d7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java @@ -414,10 +414,12 @@ public void testIsAvailableOrNot() throws Exception { // request one buffer final BufferBuilder bufferBuilder = checkNotNull(localBufferPool.requestBufferBuilderBlocking()); - assertFalse(localBufferPool.isAvailable().isDone()); + CompletableFuture availableFuture = localBufferPool.isAvailable(); + assertFalse(availableFuture.isDone()); // set the pool size localBufferPool.setNumBuffers(2); + assertTrue(availableFuture.isDone()); assertTrue(localBufferPool.isAvailable().isDone()); // drain the global buffer pool @@ -435,7 +437,7 @@ public void testIsAvailableOrNot() throws Exception { // reset the pool size localBufferPool.setNumBuffers(1); - final CompletableFuture availableFuture = localBufferPool.isAvailable(); + availableFuture = localBufferPool.isAvailable(); assertFalse(availableFuture.isDone()); // recycle the requested buffer From a9eff45741a2889aa2956fed3e43a123d7498862 Mon Sep 17 00:00:00 2001 From: yanghua Date: Tue, 5 Nov 2019 09:54:28 +0800 Subject: [PATCH 452/746] [FLINK-14600][runtime] Change type of field ExecutionGraph#verticesFinished to int This closes #10080. --- .../flink/runtime/executiongraph/ExecutionGraph.java | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 5ebf33d134b4..f787ba2a2ff8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -117,7 +117,6 @@ import java.util.concurrent.Executor; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -268,7 +267,7 @@ public class ExecutionGraph implements AccessExecutionGraph { // ------ Execution status and progress. These values are volatile, and accessed under the lock ------- - private final AtomicInteger verticesFinished; + private int verticesFinished; /** Current status of the job execution. */ private volatile JobStatus state = JobStatus.CREATED; @@ -481,8 +480,6 @@ public ExecutionGraph( this.restartStrategy = restartStrategy; this.kvStateLocationRegistry = new KvStateLocationRegistry(jobInformation.getJobId(), getAllVertices()); - this.verticesFinished = new AtomicInteger(); - this.globalModVersion = 1L; // the failover strategy must be instantiated last, so that the execution graph @@ -1381,7 +1378,7 @@ private void initFailureCause(Throwable t) { */ void vertexFinished() { assertRunningInJobMasterMainThread(); - final int numFinished = verticesFinished.incrementAndGet(); + final int numFinished = ++verticesFinished; if (numFinished == numVerticesTotal) { // done :-) @@ -1412,7 +1409,7 @@ void vertexFinished() { void vertexUnFinished() { assertRunningInJobMasterMainThread(); - verticesFinished.getAndDecrement(); + verticesFinished--; } /** From b9e75dd3f84cdf51eb66a59255e55086d2d17715 Mon Sep 17 00:00:00 2001 From: lining Date: Wed, 6 Nov 2019 22:02:32 +0800 Subject: [PATCH 453/746] [FLINK-13894][web] Add TE log link to subtask view --- .../src/app/interfaces/job-subtask.ts | 1 + ...ob-overview-drawer-subtasks.component.html | 20 ++++++++++++++++--- 2 files changed, 18 insertions(+), 3 deletions(-) diff --git a/flink-runtime-web/web-dashboard/src/app/interfaces/job-subtask.ts b/flink-runtime-web/web-dashboard/src/app/interfaces/job-subtask.ts index c7fc4efbb03f..862147a12883 100644 --- a/flink-runtime-web/web-dashboard/src/app/interfaces/job-subtask.ts +++ b/flink-runtime-web/web-dashboard/src/app/interfaces/job-subtask.ts @@ -34,4 +34,5 @@ export interface JobSubTaskInterface { 'write-records': number; 'write-records-complete': boolean; }; + 'taskmanager-id': string; } diff --git a/flink-runtime-web/web-dashboard/src/app/pages/job/overview/subtasks/job-overview-drawer-subtasks.component.html b/flink-runtime-web/web-dashboard/src/app/pages/job/overview/subtasks/job-overview-drawer-subtasks.component.html index 0d71e5db9fb9..2da866824bc7 100644 --- a/flink-runtime-web/web-dashboard/src/app/pages/job/overview/subtasks/job-overview-drawer-subtasks.component.html +++ b/flink-runtime-web/web-dashboard/src/app/pages/job/overview/subtasks/job-overview-drawer-subtasks.component.html @@ -21,7 +21,7 @@ [nzSize]="'small'" [nzLoading]="isLoading" [nzData]="listOfTask" - [nzScroll]="{x:'1430px',y:'calc( 100% - 35px )'}" + [nzScroll]="{x:'1480px',y:'calc( 100% - 35px )'}" [nzFrontPagination]="false" [nzShowPagination]="false"> @@ -36,7 +36,8 @@ Start Time Duration End Time - Status + Status + More @@ -69,9 +70,22 @@ {{ task["start_time"] | humanizeDate: 'yyyy-MM-dd HH:mm:ss' }} {{ task.duration | humanizeDuration }} {{ task["end-time"] | humanizeDate: 'yyyy-MM-dd HH:mm:ss' }} - + + + - + + + +

    + + + From 3a39d9cfaaa67c332b890577bd1b649c0f405449 Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Tue, 5 Nov 2019 12:27:48 -0800 Subject: [PATCH 454/746] [FLINK-14580][hive] add HiveModuleFactory, HiveModuleDescriptor, and HiveModuleDescriptorValidator add HiveModuleFactory, HiveModuleDescriptor, and HiveModuleDescriptorValidator for HiveModule. This closes #10092. --- .../module/hive/HiveModuleDescriptor.java | 57 +++++++++++++++ .../hive/HiveModuleDescriptorValidator.java | 37 ++++++++++ .../table/module/hive/HiveModuleFactory.java | 71 +++++++++++++++++++ ....apache.flink.table.factories.TableFactory | 1 + .../module/hive/HiveModuleDescriptorTest.java | 54 ++++++++++++++ 5 files changed, 220 insertions(+) create mode 100644 flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/module/hive/HiveModuleDescriptor.java create mode 100644 flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/module/hive/HiveModuleDescriptorValidator.java create mode 100644 flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/module/hive/HiveModuleFactory.java create mode 100644 flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleDescriptorTest.java diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/module/hive/HiveModuleDescriptor.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/module/hive/HiveModuleDescriptor.java new file mode 100644 index 000000000000..5391e7097dcd --- /dev/null +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/module/hive/HiveModuleDescriptor.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.module.hive; + +import org.apache.flink.table.descriptors.DescriptorProperties; +import org.apache.flink.table.descriptors.ModuleDescriptor; +import org.apache.flink.util.StringUtils; + +import java.util.Map; + +import static org.apache.flink.table.module.hive.HiveModuleDescriptorValidator.MODULE_HIVE_VERSION; +import static org.apache.flink.table.module.hive.HiveModuleDescriptorValidator.MODULE_TYPE_HIVE; +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * Module descriptor for {@link HiveModule}. + */ +public class HiveModuleDescriptor extends ModuleDescriptor { + private String hiveVersion; + + public HiveModuleDescriptor() { + super(MODULE_TYPE_HIVE); + } + + public HiveModuleDescriptor hiveVersion(String hiveVersion) { + checkArgument(!StringUtils.isNullOrWhitespaceOnly(hiveVersion)); + this.hiveVersion = hiveVersion; + return this; + } + + @Override + protected Map toModuleProperties() { + final DescriptorProperties properties = new DescriptorProperties(); + + if (hiveVersion != null) { + properties.putString(MODULE_HIVE_VERSION, hiveVersion); + } + + return properties.asMap(); + } +} diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/module/hive/HiveModuleDescriptorValidator.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/module/hive/HiveModuleDescriptorValidator.java new file mode 100644 index 000000000000..83773e26b88d --- /dev/null +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/module/hive/HiveModuleDescriptorValidator.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.table.module.hive; + +import org.apache.flink.table.descriptors.DescriptorProperties; +import org.apache.flink.table.descriptors.ModuleDescriptorValidator; + +/** + * Validator for {@link HiveModuleDescriptor}. + */ +public class HiveModuleDescriptorValidator extends ModuleDescriptorValidator { + public static final String MODULE_TYPE_HIVE = "hive"; + public static final String MODULE_HIVE_VERSION = "hive-version"; + + @Override + public void validate(DescriptorProperties properties) { + super.validate(properties); + properties.validateValue(MODULE_TYPE, MODULE_TYPE_HIVE, false); + properties.validateString(MODULE_HIVE_VERSION, true, 1); + } +} diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/module/hive/HiveModuleFactory.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/module/hive/HiveModuleFactory.java new file mode 100644 index 000000000000..48dcd9ff03ea --- /dev/null +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/module/hive/HiveModuleFactory.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.table.module.hive; + +import org.apache.flink.table.catalog.hive.client.HiveShimLoader; +import org.apache.flink.table.descriptors.DescriptorProperties; +import org.apache.flink.table.factories.ModuleFactory; +import org.apache.flink.table.module.Module; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.table.descriptors.ModuleDescriptorValidator.MODULE_TYPE; +import static org.apache.flink.table.module.hive.HiveModuleDescriptorValidator.MODULE_HIVE_VERSION; +import static org.apache.flink.table.module.hive.HiveModuleDescriptorValidator.MODULE_TYPE_HIVE; + +/** + * Factory for {@link HiveModule}. + */ +public class HiveModuleFactory implements ModuleFactory { + + @Override + public Module createModule(Map properties) { + final DescriptorProperties descProperties = getValidatedProperties(properties); + + final String hiveVersion = descProperties.getOptionalString(MODULE_HIVE_VERSION) + .orElse(HiveShimLoader.getHiveVersion()); + + return new HiveModule(hiveVersion); + } + + private static DescriptorProperties getValidatedProperties(Map properties) { + final DescriptorProperties descriptorProperties = new DescriptorProperties(true); + descriptorProperties.putProperties(properties); + + new HiveModuleDescriptorValidator().validate(descriptorProperties); + + return descriptorProperties; + } + + @Override + public Map requiredContext() { + Map context = new HashMap<>(); + context.put(MODULE_TYPE, MODULE_TYPE_HIVE); + + return context; + } + + @Override + public List supportedProperties() { + return new ArrayList<>(); + } +} diff --git a/flink-connectors/flink-connector-hive/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink-connectors/flink-connector-hive/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory index 27d69ee2ed0e..e1871cfae2cb 100644 --- a/flink-connectors/flink-connector-hive/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory +++ b/flink-connectors/flink-connector-hive/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory @@ -14,3 +14,4 @@ # limitations under the License. org.apache.flink.table.catalog.hive.factories.HiveCatalogFactory +org.apache.flink.table.module.hive.HiveModuleFactory diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleDescriptorTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleDescriptorTest.java new file mode 100644 index 000000000000..0f5ac3e66d57 --- /dev/null +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleDescriptorTest.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.table.module.hive; + +import org.apache.flink.table.descriptors.Descriptor; +import org.apache.flink.table.descriptors.DescriptorTestBase; +import org.apache.flink.table.descriptors.DescriptorValidator; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Tests for {@link HiveModuleDescriptor}. + */ +public class HiveModuleDescriptorTest extends DescriptorTestBase { + + @Override + protected List descriptors() { + final Descriptor descriptor = new HiveModuleDescriptor(); + + return Arrays.asList(descriptor); + } + + @Override + protected List> properties() { + final Map props1 = new HashMap<>(); + props1.put("type", "hive"); + + return Arrays.asList(props1); + } + + @Override + protected DescriptorValidator validator() { + return new HiveModuleDescriptorValidator(); + } +} From 051692a9d2ad8bfffc1087930c3ec51d692d8cc6 Mon Sep 17 00:00:00 2001 From: Yun Tang Date: Mon, 29 Jul 2019 01:24:32 +0800 Subject: [PATCH 455/746] [FLINK-13034][state backends] Add isEmpty method for MapState This closes #9255 --- docs/dev/stream/state/state.md | 1 + docs/dev/stream/state/state.zh.md | 2 +- .../flink/api/common/state/MapState.java | 9 ++++++ .../flink/cep/operator/CepOperator.java | 2 +- .../flink/cep/utils/TestSharedBuffer.java | 9 ++++++ .../client/state/ImmutableMapState.java | 5 ++++ .../client/state/ImmutableMapStateTest.java | 6 ++++ .../runtime/state/UserFacingMapState.java | 5 ++++ .../runtime/state/heap/HeapMapState.java | 6 ++++ .../flink/runtime/state/ttl/TtlMapState.java | 6 ++++ .../runtime/state/StateBackendTestBase.java | 30 +++++++++++++++++-- .../state/ttl/mock/MockInternalMapState.java | 5 ++++ .../streaming/state/RocksDBMapState.java | 12 ++++++++ .../flink/table/api/dataview/MapView.java | 11 +++++++ .../flink/table/dataview/StateMapView.scala | 2 ++ .../runtime/join/TemporalRowtimeJoin.scala | 2 +- .../table/runtime/dataview/StateMapView.java | 10 +++++++ .../temporal/TemporalRowTimeJoinOperator.java | 2 +- ...AbstractRowTimeUnboundedPrecedingOver.java | 3 +- .../window/MergingWindowSetTest.java | 10 +++++++ 20 files changed, 130 insertions(+), 8 deletions(-) diff --git a/docs/dev/stream/state/state.md b/docs/dev/stream/state/state.md index c6d4c95a7dff..14ce20023207 100644 --- a/docs/dev/stream/state/state.md +++ b/docs/dev/stream/state/state.md @@ -115,6 +115,7 @@ added using `add(T)` are folded into an aggregate using a specified `FoldFunctio retrieve an `Iterable` over all currently stored mappings. Mappings are added using `put(UK, UV)` or `putAll(Map)`. The value associated with a user key can be retrieved using `get(UK)`. The iterable views for mappings, keys and values can be retrieved using `entries()`, `keys()` and `values()` respectively. +You can also use `isEmpty()` to check whether this map contains any key-value mappings. All types of state also have a method `clear()` that clears the state for the currently active key, i.e. the key of the input element. diff --git a/docs/dev/stream/state/state.zh.md b/docs/dev/stream/state/state.zh.md index 78da6410d66a..b5d40eba282d 100644 --- a/docs/dev/stream/state/state.zh.md +++ b/docs/dev/stream/state/state.zh.md @@ -87,7 +87,7 @@ managed keyed state 接口提供不同类型状态的访问接口,这些状态 接口与 `ListState` 类似,但使用`add(T)`添加的元素会用指定的 `FoldFunction` 折叠成聚合值。 * `MapState`: 维护了一个映射列表。 你可以添加键值对到状态中,也可以获得反映当前所有映射的迭代器。使用 `put(UK,UV)` 或者 `putAll(Map)` 添加映射。 - 使用 `get(UK)` 检索特定 key。 使用 `entries()`,`keys()` 和 `values()` 分别检索映射、键和值的可迭代视图。 + 使用 `get(UK)` 检索特定 key。 使用 `entries()`,`keys()` 和 `values()` 分别检索映射、键和值的可迭代视图。你还可以通过 `isEmpty()` 来判断是否包含任何键值对。 所有类型的状态还有一个`clear()` 方法,清除当前 key 下的状态数据,也就是当前输入元素的 key。 diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/MapState.java b/flink-core/src/main/java/org/apache/flink/api/common/state/MapState.java index 7a130d49083d..94eb275eba19 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/MapState.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/MapState.java @@ -124,4 +124,13 @@ public interface MapState extends State { * @throws Exception Thrown if the system cannot access the state. */ Iterator> iterator() throws Exception; + + /** + * Returns true if this state contains no key-value mappings, otherwise false. + * + * @return True if this state contains no key-value mappings, otherwise false. + * + * @throws Exception Thrown if the system cannot access the state. + */ + boolean isEmpty() throws Exception; } diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CepOperator.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CepOperator.java index fe95c6dbbd05..2717c138fd6d 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CepOperator.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CepOperator.java @@ -530,7 +530,7 @@ boolean hasNonEmptySharedBuffer(KEY key) throws Exception { @VisibleForTesting boolean hasNonEmptyPQ(KEY key) throws Exception { setCurrentKey(key); - return elementQueueState.keys().iterator().hasNext(); + return !elementQueueState.isEmpty(); } @VisibleForTesting diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/utils/TestSharedBuffer.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/utils/TestSharedBuffer.java index 4d510cf405b0..1f5672d02752 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/utils/TestSharedBuffer.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/utils/TestSharedBuffer.java @@ -219,6 +219,15 @@ public Iterator> iterator() throws Exception { return new CountingIterator<>(values.entrySet().iterator()); } + @Override + public boolean isEmpty() throws Exception { + if (values == null) { + return true; + } + + return values.isEmpty(); + } + @Override public void clear() { stateWrites++; diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableMapState.java b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableMapState.java index 4d51b7de6be9..7a20a9648cc7 100644 --- a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableMapState.java +++ b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/ImmutableMapState.java @@ -113,6 +113,11 @@ public Iterator> iterator() { return Collections.unmodifiableSet(state.entrySet()).iterator(); } + @Override + public boolean isEmpty() { + return state.isEmpty(); + } + @Override public void clear() { throw MODIFICATION_ATTEMPT_ERROR; diff --git a/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableMapStateTest.java b/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableMapStateTest.java index 6465257b6b1a..3694c54fc2e9 100644 --- a/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableMapStateTest.java +++ b/flink-queryable-state/flink-queryable-state-client-java/src/test/java/org/apache/flink/queryablestate/client/state/ImmutableMapStateTest.java @@ -32,6 +32,7 @@ import java.util.Map; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; /** @@ -186,4 +187,9 @@ public void testClear() throws Exception { mapState.clear(); } + + @Test + public void testIsEmpty() throws Exception { + assertFalse(mapState.isEmpty()); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/UserFacingMapState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/UserFacingMapState.java index ce4d032c8825..301909bda991 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/UserFacingMapState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/UserFacingMapState.java @@ -95,4 +95,9 @@ public Iterator> iterator() throws Exception { Iterator> original = originalState.iterator(); return original != null ? original : emptyState.entrySet().iterator(); } + + @Override + public boolean isEmpty() throws Exception { + return originalState.isEmpty(); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapMapState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapMapState.java index 745e7f4f58e3..23620b854815 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapMapState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapMapState.java @@ -161,6 +161,12 @@ public Iterator> iterator() { return userMap == null ? null : userMap.entrySet().iterator(); } + @Override + public boolean isEmpty() { + Map userMap = stateTable.get(currentNamespace); + return userMap == null || userMap.isEmpty(); + } + @Override public byte[] getSerializedValue( final byte[] serializedKeyAndNamespace, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/TtlMapState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/TtlMapState.java index c3f624ab38e8..cb061742e544 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/TtlMapState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ttl/TtlMapState.java @@ -121,6 +121,12 @@ public Iterator> iterator() throws Exception { return entries().iterator(); } + @Override + public boolean isEmpty() throws Exception { + accessCallback.run(); + return original.isEmpty(); + } + @Nullable @Override public Map> getUnexpiredOrNull(@Nonnull Map> ttlValue) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java index 132fd01a6419..f90720d36713 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java @@ -2573,7 +2573,6 @@ public void testMapState() throws Exception { List expectedKeys = Arrays.asList(103, 1031, 1032); assertEquals(keys.size(), expectedKeys.size()); keys.removeAll(expectedKeys); - assertTrue(keys.isEmpty()); List values = new ArrayList<>(); for (String value : state.values()) { @@ -2582,7 +2581,6 @@ public void testMapState() throws Exception { List expectedValues = Arrays.asList("103", "1031", "1032"); assertEquals(values.size(), expectedValues.size()); values.removeAll(expectedValues); - assertTrue(values.isEmpty()); // make some more modifications backend.setCurrentKey("1"); @@ -2655,6 +2653,34 @@ public void testMapState() throws Exception { backend.dispose(); } + @Test + public void testMapStateIsEmpty() throws Exception { + MapStateDescriptor kvId = new MapStateDescriptor<>("id", Integer.class, Long.class); + + AbstractKeyedStateBackend backend = createKeyedBackend(IntSerializer.INSTANCE); + + try { + MapState state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId); + backend.setCurrentKey(1); + assertTrue(state.isEmpty()); + + int stateSize = 1024; + for (int i = 0; i < stateSize; i++) { + state.put(i, i * 2L); + assertFalse(state.isEmpty()); + } + + for (int i = 0; i < stateSize; i++) { + assertFalse(state.isEmpty()); + state.remove(i); + } + assertTrue(state.isEmpty()); + + } finally { + backend.dispose(); + } + } + /** * Verify iterator of {@link MapState} supporting arbitrary access, see [FLINK-10267] to know more details. */ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/mock/MockInternalMapState.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/mock/MockInternalMapState.java index 9b5ac10c92d0..28b2a24147aa 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/mock/MockInternalMapState.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/ttl/mock/MockInternalMapState.java @@ -86,6 +86,11 @@ public Iterator> iterator() { return entries().iterator(); } + @Override + public boolean isEmpty() { + return getInternal().isEmpty(); + } + @SuppressWarnings({"unchecked", "unused"}) static IS createState( TypeSerializer namespaceSerializer, diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBMapState.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBMapState.java index 64ce823d03fa..e7e1d2519f28 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBMapState.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBMapState.java @@ -239,6 +239,18 @@ public Map.Entry next() { }; } + @Override + public boolean isEmpty() { + final byte[] prefixBytes = serializeCurrentKeyWithGroupAndNamespace(); + + try (RocksIteratorWrapper iterator = RocksDBOperationUtils.getRocksIterator(backend.db, columnFamily)) { + + iterator.seek(prefixBytes); + + return !iterator.isValid() || !startWithKeyPrefix(prefixBytes, iterator.key()); + } + } + @Override public void clear() { try { diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/dataview/MapView.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/dataview/MapView.java index 7feb07bffca4..b7a37042adfb 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/dataview/MapView.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/dataview/MapView.java @@ -203,6 +203,17 @@ public Iterator> iterator() throws Exception { return map.entrySet().iterator(); } + /** + * Returns true if the map view contains no key-value mappings, otherwise false. + * + * @return True if the map view contains no key-value mappings, otherwise false. + * + * @throws Exception Thrown if the system cannot access the state. + */ + public boolean isEmpty() throws Exception { + return map.isEmpty(); + } + /** * Removes all entries of this map. */ diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/dataview/StateMapView.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/dataview/StateMapView.scala index 22f5f0b23459..2096cf630d60 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/dataview/StateMapView.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/dataview/StateMapView.scala @@ -50,5 +50,7 @@ class StateMapView[K, V](state: MapState[K, V]) extends MapView[K, V] { override def iterator: util.Iterator[util.Map.Entry[K, V]] = state.iterator() + override def isEmpty(): Boolean = state.isEmpty + override def clear(): Unit = state.clear() } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/runtime/join/TemporalRowtimeJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/runtime/join/TemporalRowtimeJoin.scala index f6911096f937..73c876067f47 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/runtime/join/TemporalRowtimeJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/runtime/join/TemporalRowtimeJoin.scala @@ -169,7 +169,7 @@ class TemporalRowtimeJoin( // if we have more state at any side, then update the timer, else clean it up. if (stateCleaningEnabled) { - if (lastUnprocessedTime < Long.MaxValue || rightState.iterator().hasNext) { + if (lastUnprocessedTime < Long.MaxValue || !rightState.isEmpty) { registerProcessingCleanUpTimer() } else { cleanUpLastTimer() diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/StateMapView.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/StateMapView.java index c7f2686c35f6..16d96d6297da 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/StateMapView.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/StateMapView.java @@ -107,6 +107,11 @@ public Iterator> iterator() throws Exception { return original != null ? original : emptyState.entrySet().iterator(); } + @Override + public boolean isEmpty() throws Exception { + return getMapState().isEmpty(); + } + @Override public void clear() { getMapState().clear(); @@ -191,6 +196,11 @@ public Iterator> iterator() throws Exception { return new NullAwareMapIterator<>(getMapState().iterator(), new NullMapEntryImpl()); } + @Override + public boolean isEmpty() throws Exception { + return getMapState().isEmpty() && getNullState().value() == null; + } + @Override public void clear() { getMapState().clear(); diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/temporal/TemporalRowTimeJoinOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/temporal/TemporalRowTimeJoinOperator.java index 64be99bb6ae4..f55fdd15b3a1 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/temporal/TemporalRowTimeJoinOperator.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/temporal/TemporalRowTimeJoinOperator.java @@ -189,7 +189,7 @@ public void onEventTime(InternalTimer timer) throws Excep // if we have more state at any side, then update the timer, else clean it up. if (stateCleaningEnabled) { - if (lastUnprocessedTime < Long.MAX_VALUE || rightState.iterator().hasNext()) { + if (lastUnprocessedTime < Long.MAX_VALUE || !rightState.isEmpty()) { registerProcessingCleanupTimer(); } else { cleanupLastTimer(); diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/AbstractRowTimeUnboundedPrecedingOver.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/AbstractRowTimeUnboundedPrecedingOver.java index e5a3216c7274..297f8bddc8b2 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/AbstractRowTimeUnboundedPrecedingOver.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/AbstractRowTimeUnboundedPrecedingOver.java @@ -157,8 +157,7 @@ public void onTimer( if (stateCleaningEnabled) { // we check whether there are still records which have not been processed yet - boolean noRecordsToProcess = !inputState.keys().iterator().hasNext(); - if (noRecordsToProcess) { + if (inputState.isEmpty()) { // we clean the state cleanupState(inputState, accState); function.cleanup(); diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/window/MergingWindowSetTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/window/MergingWindowSetTest.java index 7a502b6c06dc..e53751f9b3c4 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/window/MergingWindowSetTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/window/MergingWindowSetTest.java @@ -417,6 +417,11 @@ public Iterator> iterator() throws Exception { return map.entrySet().iterator(); } + @Override + public boolean isEmpty() throws Exception { + return map.isEmpty(); + } + @Override public void clear() { map.clear(); @@ -588,6 +593,11 @@ public Iterator> iterator() throws Exception { return internalMap.entrySet().iterator(); } + @Override + public boolean isEmpty() { + return internalMap.isEmpty(); + } + @Override public void clear() { internalMap.clear(); From 02c4f202bf0a5b9a40cedc480e9a9d927092080d Mon Sep 17 00:00:00 2001 From: PengFei Li Date: Tue, 15 Oct 2019 20:59:23 +0800 Subject: [PATCH 456/746] [FLINK-13469][state] Ensure resource used by StateMapSnapshot will be released if snapshot fails This closes #9301 --- .../heap/AbstractStateTableSnapshot.java | 7 -- .../state/heap/CopyOnWriteStateMap.java | 6 ++ .../heap/CopyOnWriteStateMapSnapshot.java | 15 +++- .../heap/CopyOnWriteStateTableSnapshot.java | 9 ++ .../state/heap/NestedMapsStateTable.java | 4 + .../flink/runtime/state/heap/StateTable.java | 2 +- .../state/heap/CopyOnWriteStateMapTest.java | 26 ++++++ .../state/heap/CopyOnWriteStateTableTest.java | 88 +++++++++++++++++++ .../state/heap/MockInternalKeyContext.java | 11 ++- 9 files changed, 156 insertions(+), 12 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/AbstractStateTableSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/AbstractStateTableSnapshot.java index 41e522f21e1f..f8d48a245ab3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/AbstractStateTableSnapshot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/AbstractStateTableSnapshot.java @@ -88,13 +88,6 @@ abstract class AbstractStateTableSnapshot */ protected abstract StateMapSnapshot> getStateMapSnapshotForKeyGroup(int keyGroup); - /** - * Optional hook to release resources for this snapshot at the end of its lifecycle. - */ - @Override - public void release() { - } - @Nonnull @Override public StateMetaInfoSnapshot getMetaInfoSnapshot() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateMap.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateMap.java index 7e71c0b628cf..12b3b2529ac3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateMap.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateMap.java @@ -38,6 +38,7 @@ import java.util.Iterator; import java.util.NoSuchElementException; import java.util.Objects; +import java.util.Set; import java.util.TreeSet; import java.util.stream.Stream; import java.util.stream.StreamSupport; @@ -784,6 +785,11 @@ public void releaseSnapshot(StateMapSnapshot getSnapshotVersions() { + return snapshotVersions; + } + // Meta data setter / getter and toString ----------------------------------------------------- public TypeSerializer getStateSerializer() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateMapSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateMapSnapshot.java index a39a9b05fae1..b9ea704e8c8d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateMapSnapshot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateMapSnapshot.java @@ -70,6 +70,11 @@ public class CopyOnWriteStateMapSnapshot @Nonnegative private final int numberOfEntriesInSnapshotData; + /** + * Whether this snapshot has been released. + */ + private boolean released; + /** * Creates a new {@link CopyOnWriteStateMapSnapshot}. * @@ -81,11 +86,19 @@ public class CopyOnWriteStateMapSnapshot this.snapshotData = owningStateMap.snapshotMapArrays(); this.snapshotVersion = owningStateMap.getStateMapVersion(); this.numberOfEntriesInSnapshotData = owningStateMap.size(); + this.released = false; } @Override public void release() { - owningStateMap.releaseSnapshot(this); + if (!released) { + owningStateMap.releaseSnapshot(this); + released = true; + } + } + + public boolean isReleased() { + return released; } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableSnapshot.java index 32fbdaca943d..48047ceace51 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableSnapshot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableSnapshot.java @@ -79,4 +79,13 @@ public class CopyOnWriteStateTableSnapshot extends AbstractStateTableSn return stateMapSnapshot; } + + @Override + public void release() { + for (CopyOnWriteStateMapSnapshot snapshot : stateMapSnapshots) { + if (!snapshot.isReleased()) { + snapshot.release(); + } + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/NestedMapsStateTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/NestedMapsStateTable.java index b5589ba7c4fa..42fb3054642f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/NestedMapsStateTable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/NestedMapsStateTable.java @@ -95,5 +95,9 @@ static class NestedMapsStateTableSnapshot return stateMap.stateSnapshot(); } + + @Override + public void release() { + } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTable.java index 758ff6d48c2f..921aa481e3fa 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTable.java @@ -289,7 +289,7 @@ public int getKeyGroupOffset() { } @VisibleForTesting - protected StateMap getMapForKeyGroup(int keyGroupIndex) { + StateMap getMapForKeyGroup(int keyGroupIndex) { final int pos = indexToOffset(keyGroupIndex); if (pos >= 0 && pos < keyGroupedStateMaps.length) { return keyGroupedStateMaps[pos]; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateMapTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateMapTest.java index 2709cb5d5955..6479ee960d80 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateMapTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateMapTest.java @@ -29,6 +29,8 @@ import org.junit.Assert; import org.junit.Test; +import org.mockito.Matchers; +import org.mockito.Mockito; import java.util.ArrayList; import java.util.Arrays; @@ -394,6 +396,30 @@ public void testCopyOnWriteContracts() { Assert.assertTrue(originalState5 == stateMap.get(5, 1)); } + /** + * This tests that snapshot can be released correctly. + */ + @Test + public void testSnapshotRelease() { + final CopyOnWriteStateMap stateMap = + Mockito.spy(new CopyOnWriteStateMap<>(IntSerializer.INSTANCE)); + + for (int i = 0; i < 10; i++) { + stateMap.put(i, i, i); + } + + CopyOnWriteStateMapSnapshot snapshot = stateMap.stateSnapshot(); + Assert.assertFalse(snapshot.isReleased()); + + snapshot.release(); + Assert.assertTrue(snapshot.isReleased()); + Mockito.verify(stateMap, Mockito.times(1)).releaseSnapshot(Matchers.same(snapshot)); + + // verify that snapshot will release itself only once + snapshot.release(); + Mockito.verify(stateMap, Mockito.times(1)).releaseSnapshot(Matchers.same(snapshot)); + } + @SuppressWarnings("unchecked") private static Tuple3[] convert(CopyOnWriteStateMap.StateMapEntry[] snapshot, int mapSize) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableTest.java index 0a3d27238530..d92d2f18e312 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableTest.java @@ -19,14 +19,19 @@ package org.apache.flink.runtime.state.heap; import org.apache.flink.api.common.state.StateDescriptor; +import org.apache.flink.api.common.typeutils.base.FloatSerializer; +import org.apache.flink.api.common.typeutils.base.IntSerializer; import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos; +import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.apache.flink.runtime.state.RegisteredKeyValueStateBackendMetaInfo; import org.apache.flink.runtime.state.StateSnapshot; +import org.junit.Assert; import org.junit.Test; import java.io.IOException; +import java.util.concurrent.ThreadLocalRandom; /** * Test for {@link CopyOnWriteStateTable}. @@ -71,4 +76,87 @@ public void testSerializerDuplicationInSnapshot() throws IOException { new DataOutputViewStreamWrapper( new ByteArrayOutputStreamWithPos(1024)), 0); } + + /** + * This tests that resource can be released for a successful snapshot. + */ + @Test + public void testReleaseForSuccessfulSnapshot() throws IOException { + int numberOfKeyGroups = 10; + CopyOnWriteStateTable table = createStateTableForSnapshotRelease(numberOfKeyGroups); + + ByteArrayOutputStreamWithPos byteArrayOutputStreamWithPos = new ByteArrayOutputStreamWithPos(); + DataOutputView dataOutputView = new DataOutputViewStreamWrapper(byteArrayOutputStreamWithPos); + + CopyOnWriteStateTableSnapshot snapshot = table.stateSnapshot(); + for (int group = 0; group < numberOfKeyGroups; group++) { + snapshot.writeStateInKeyGroup(dataOutputView, group); + // resource used by one key group should be released after the snapshot is successful + Assert.assertTrue(isResourceReleasedForKeyGroup(table, group)); + } + snapshot.release(); + verifyResourceIsReleasedForAllKeyGroup(table, 1); + } + + /** + * This tests that resource can be released for a failed snapshot. + */ + @Test + public void testReleaseForFailedSnapshot() throws IOException { + int numberOfKeyGroups = 10; + CopyOnWriteStateTable table = createStateTableForSnapshotRelease(numberOfKeyGroups); + + ByteArrayOutputStreamWithPos byteArrayOutputStreamWithPos = new ByteArrayOutputStreamWithPos(); + DataOutputView dataOutputView = new DataOutputViewStreamWrapper(byteArrayOutputStreamWithPos); + + CopyOnWriteStateTableSnapshot snapshot = table.stateSnapshot(); + // only snapshot part of key groups to simulate a failed snapshot + for (int group = 0; group < numberOfKeyGroups / 2; group++) { + snapshot.writeStateInKeyGroup(dataOutputView, group); + Assert.assertTrue(isResourceReleasedForKeyGroup(table, group)); + } + for (int group = numberOfKeyGroups / 2; group < numberOfKeyGroups; group++) { + Assert.assertFalse(isResourceReleasedForKeyGroup(table, group)); + } + snapshot.release(); + verifyResourceIsReleasedForAllKeyGroup(table, 2); + } + + private CopyOnWriteStateTable createStateTableForSnapshotRelease(int numberOfKeyGroups) { + RegisteredKeyValueStateBackendMetaInfo metaInfo = + new RegisteredKeyValueStateBackendMetaInfo<>( + StateDescriptor.Type.VALUE, + "test", + IntSerializer.INSTANCE, + FloatSerializer.INSTANCE); + + MockInternalKeyContext mockKeyContext = + new MockInternalKeyContext<>(0, numberOfKeyGroups - 1, numberOfKeyGroups); + CopyOnWriteStateTable table = + new CopyOnWriteStateTable<>(mockKeyContext, metaInfo, IntSerializer.INSTANCE); + + ThreadLocalRandom random = ThreadLocalRandom.current(); + for (int i = 0; i < 1000; i++) { + mockKeyContext.setCurrentKeyAndKeyGroup(i); + table.put(random.nextInt(), random.nextFloat()); + } + + return table; + } + + private void verifyResourceIsReleasedForAllKeyGroup( + CopyOnWriteStateTable table, + int snapshotVersion) { + StateMap[] stateMaps = table.getState(); + for (StateMap map : stateMaps) { + Assert.assertFalse(((CopyOnWriteStateMap) map).getSnapshotVersions().contains(snapshotVersion)); + } + } + + private boolean isResourceReleasedForKeyGroup( + CopyOnWriteStateTable table, + int keyGroup) { + CopyOnWriteStateMap stateMap = (CopyOnWriteStateMap) table.getMapForKeyGroup(keyGroup); + return !stateMap.getSnapshotVersions().contains(1); + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/MockInternalKeyContext.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/MockInternalKeyContext.java index 865ebe216628..721a6bad3081 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/MockInternalKeyContext.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/MockInternalKeyContext.java @@ -19,18 +19,23 @@ package org.apache.flink.runtime.state.heap; import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyGroupRangeAssignment; /** * Mock {@link InternalKeyContext}. */ public class MockInternalKeyContext extends InternalKeyContextImpl { + MockInternalKeyContext() { super(new KeyGroupRange(0, 0), 1); } - @Override - public void setCurrentKey(K key) { + MockInternalKeyContext(int startKeyGroup, int endKeyGroup, int numberOfKeyGroups) { + super(new KeyGroupRange(startKeyGroup, endKeyGroup), numberOfKeyGroups); + } + + public void setCurrentKeyAndKeyGroup(K key) { super.setCurrentKey(key); - super.setCurrentKeyGroupIndex(0); + super.setCurrentKeyGroupIndex(KeyGroupRangeAssignment.assignToKeyGroup(key, getNumberOfKeyGroups())); } } From 60a066abb5509b97915618d4d1a2729acced6e94 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 6 Nov 2019 13:41:30 +0100 Subject: [PATCH 457/746] [hotfix][tests] Replace mockito-based verification with property verification. --- .../runtime/state/heap/CopyOnWriteStateMapTest.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateMapTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateMapTest.java index 6479ee960d80..d761808ac029 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateMapTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateMapTest.java @@ -27,10 +27,9 @@ import org.apache.flink.runtime.state.internal.InternalKvState.StateIncrementalVisitor; import org.apache.flink.util.TestLogger; +import org.hamcrest.Matchers; import org.junit.Assert; import org.junit.Test; -import org.mockito.Matchers; -import org.mockito.Mockito; import java.util.ArrayList; import java.util.Arrays; @@ -402,7 +401,7 @@ public void testCopyOnWriteContracts() { @Test public void testSnapshotRelease() { final CopyOnWriteStateMap stateMap = - Mockito.spy(new CopyOnWriteStateMap<>(IntSerializer.INSTANCE)); + new CopyOnWriteStateMap<>(IntSerializer.INSTANCE); for (int i = 0; i < 10; i++) { stateMap.put(i, i, i); @@ -410,14 +409,15 @@ public void testSnapshotRelease() { CopyOnWriteStateMapSnapshot snapshot = stateMap.stateSnapshot(); Assert.assertFalse(snapshot.isReleased()); + Assert.assertThat(stateMap.getSnapshotVersions(), Matchers.contains(snapshot.getSnapshotVersion())); snapshot.release(); Assert.assertTrue(snapshot.isReleased()); - Mockito.verify(stateMap, Mockito.times(1)).releaseSnapshot(Matchers.same(snapshot)); + Assert.assertThat(stateMap.getSnapshotVersions(), Matchers.empty()); // verify that snapshot will release itself only once snapshot.release(); - Mockito.verify(stateMap, Mockito.times(1)).releaseSnapshot(Matchers.same(snapshot)); + Assert.assertThat(stateMap.getSnapshotVersions(), Matchers.empty()); } @SuppressWarnings("unchecked") From 87df20dcb8a3bf44ccab4eec4fa69ba25a3bb6aa Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 6 Nov 2019 13:49:44 +0100 Subject: [PATCH 458/746] [hotfix][tests] Checkstyle and common style cleanups in CopyOnWriteStateMapTest --- .../state/heap/CopyOnWriteStateMapTest.java | 95 +++++++++---------- 1 file changed, 43 insertions(+), 52 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateMapTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateMapTest.java index d761808ac029..328606a2f1e3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateMapTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateMapTest.java @@ -51,23 +51,23 @@ public void testPutGetRemoveContainsTransform() throws Exception { final CopyOnWriteStateMap> stateMap = new CopyOnWriteStateMap<>(new ArrayListSerializer<>(IntSerializer.INSTANCE)); - ArrayList state_1_1 = new ArrayList<>(); - state_1_1.add(41); - ArrayList state_2_1 = new ArrayList<>(); - state_2_1.add(42); - ArrayList state_1_2 = new ArrayList<>(); - state_1_2.add(43); - - Assert.assertNull(stateMap.putAndGetOld(1, 1, state_1_1)); - Assert.assertEquals(state_1_1, stateMap.get(1, 1)); + ArrayList state11 = new ArrayList<>(); + state11.add(41); + ArrayList state21 = new ArrayList<>(); + state21.add(42); + ArrayList state12 = new ArrayList<>(); + state12.add(43); + + Assert.assertNull(stateMap.putAndGetOld(1, 1, state11)); + Assert.assertEquals(state11, stateMap.get(1, 1)); Assert.assertEquals(1, stateMap.size()); - Assert.assertNull(stateMap.putAndGetOld(2, 1, state_2_1)); - Assert.assertEquals(state_2_1, stateMap.get(2, 1)); + Assert.assertNull(stateMap.putAndGetOld(2, 1, state21)); + Assert.assertEquals(state21, stateMap.get(2, 1)); Assert.assertEquals(2, stateMap.size()); - Assert.assertNull(stateMap.putAndGetOld(1, 2, state_1_2)); - Assert.assertEquals(state_1_2, stateMap.get(1, 2)); + Assert.assertNull(stateMap.putAndGetOld(1, 2, state12)); + Assert.assertEquals(state12, stateMap.get(1, 2)); Assert.assertEquals(3, stateMap.size()); Assert.assertTrue(stateMap.containsKey(2, 1)); @@ -77,10 +77,10 @@ public void testPutGetRemoveContainsTransform() throws Exception { Assert.assertTrue(stateMap.containsKey(2, 1)); Assert.assertEquals(3, stateMap.size()); Assert.assertNull(stateMap.get(2, 1)); - stateMap.put(2, 1, state_2_1); + stateMap.put(2, 1, state21); Assert.assertEquals(3, stateMap.size()); - Assert.assertEquals(state_2_1, stateMap.removeAndGetOld(2, 1)); + Assert.assertEquals(state21, stateMap.removeAndGetOld(2, 1)); Assert.assertFalse(stateMap.containsKey(2, 1)); Assert.assertEquals(2, stateMap.size()); @@ -92,18 +92,15 @@ public void testPutGetRemoveContainsTransform() throws Exception { Assert.assertEquals(1, stateMap.size()); StateTransformationFunction, Integer> function = - new StateTransformationFunction, Integer>() { - @Override - public ArrayList apply(ArrayList previousState, Integer value) throws Exception { - previousState.add(value); - return previousState; - } + (previousState, value) -> { + previousState.add(value); + return previousState; }; final int value = 4711; stateMap.transform(1, 1, value, function); - state_1_1 = function.apply(state_1_1, value); - Assert.assertEquals(state_1_1, stateMap.get(1, 1)); + state11 = function.apply(state11, value); + Assert.assertEquals(state11, stateMap.get(1, 1)); } /** @@ -117,14 +114,14 @@ public void testIncrementalRehash() { int insert = 0; int remove = 0; while (!stateMap.isRehashing()) { - stateMap.put(insert++, 0, new ArrayList()); + stateMap.put(insert++, 0, new ArrayList<>()); if (insert % 8 == 0) { stateMap.remove(remove++, 0); } } Assert.assertEquals(insert - remove, stateMap.size()); while (stateMap.isRehashing()) { - stateMap.put(insert++, 0, new ArrayList()); + stateMap.put(insert++, 0, new ArrayList<>()); if (insert % 8 == 0) { stateMap.remove(remove++, 0); } @@ -166,16 +163,13 @@ public void testRandomModificationsAndCopyOnWriteIsolation() throws Exception { int referencedSnapshotId = 0; final StateTransformationFunction, Integer> transformationFunction = - new StateTransformationFunction, Integer>() { - @Override - public ArrayList apply(ArrayList previousState, Integer value) throws Exception { - if (previousState == null) { - previousState = new ArrayList<>(); - } - previousState.add(value); - // we give back the original, attempting to spot errors in to copy-on-write - return previousState; + (previousState, value) -> { + if (previousState == null) { + previousState = new ArrayList<>(); } + previousState.add(value); + // we give back the original, attempting to spot errors in to copy-on-write + return previousState; }; StateIncrementalVisitor> updatingIterator = @@ -207,13 +201,13 @@ public ArrayList apply(ArrayList previousState, Integer value) break; } case 2: { - stateMap.put(key, namespace, new ArrayList()); - referenceMap.put(compositeKey, new ArrayList()); + stateMap.put(key, namespace, new ArrayList<>()); + referenceMap.put(compositeKey, new ArrayList<>()); break; } case 3: { - state = stateMap.putAndGetOld(key, namespace, new ArrayList()); - referenceState = referenceMap.put(compositeKey, new ArrayList()); + state = stateMap.putAndGetOld(key, namespace, new ArrayList<>()); + referenceState = referenceMap.put(compositeKey, new ArrayList<>()); break; } case 4: { @@ -253,6 +247,7 @@ public ArrayList apply(ArrayList previousState, Integer value) Assert.assertEquals(referenceMap.size(), stateMap.size()); if (state != null) { + Assert.assertNotNull(referenceState); // mutate the states a bit... if (random.nextBoolean() && !state.isEmpty()) { state.remove(state.size() - 1); @@ -361,11 +356,11 @@ public void testCopyOnWriteContracts() { stateMap.put(5, 1, originalState5); // no snapshot taken, we get the original back - Assert.assertTrue(stateMap.get(1, 1) == originalState1); + Assert.assertSame(stateMap.get(1, 1), originalState1); CopyOnWriteStateMapSnapshot> snapshot1 = stateMap.stateSnapshot(); // after snapshot1 is taken, we get a copy... final ArrayList copyState = stateMap.get(1, 1); - Assert.assertFalse(copyState == originalState1); + Assert.assertNotSame(copyState, originalState1); // ...and the copy is equal Assert.assertEquals(originalState1, copyState); @@ -373,26 +368,26 @@ public void testCopyOnWriteContracts() { stateMap.put(3, 1, originalState3); // on repeated lookups, we get the same copy because no further snapshot was taken - Assert.assertTrue(copyState == stateMap.get(1, 1)); + Assert.assertSame(copyState, stateMap.get(1, 1)); // we take snapshot2 CopyOnWriteStateMapSnapshot> snapshot2 = stateMap.stateSnapshot(); // after the second snapshot, copy-on-write is active again for old entries - Assert.assertFalse(copyState == stateMap.get(1, 1)); + Assert.assertNotSame(copyState, stateMap.get(1, 1)); // and equality still holds Assert.assertEquals(copyState, stateMap.get(1, 1)); // after releasing snapshot2 stateMap.releaseSnapshot(snapshot2); // we still get the original of the untouched late insert (after snapshot1) - Assert.assertTrue(originalState3 == stateMap.get(3, 1)); + Assert.assertSame(originalState3, stateMap.get(3, 1)); // but copy-on-write is still active for older inserts (before snapshot1) - Assert.assertFalse(originalState4 == stateMap.get(4, 1)); + Assert.assertNotSame(originalState4, stateMap.get(4, 1)); // after releasing snapshot1 stateMap.releaseSnapshot(snapshot1); // no copy-on-write is active - Assert.assertTrue(originalState5 == stateMap.get(5, 1)); + Assert.assertSame(originalState5, stateMap.get(5, 1)); } /** @@ -461,13 +456,9 @@ private void deepCheck( Assert.assertEquals(a.length, b.length); Comparator>> comparator = - new Comparator>>() { - - @Override - public int compare(Tuple3> o1, Tuple3> o2) { - int namespaceDiff = o1.f1 - o2.f1; - return namespaceDiff != 0 ? namespaceDiff : o1.f0 - o2.f0; - } + (o1, o2) -> { + int namespaceDiff = o1.f1 - o2.f1; + return namespaceDiff != 0 ? namespaceDiff : o1.f0 - o2.f0; }; Arrays.sort(a, comparator); From 2678506a3ed178a654ec8cb5f753bb518115def8 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 31 Oct 2019 13:31:29 +0100 Subject: [PATCH 459/746] [FLINK-14586][tests] JobMasterBuilder as top-level class --- .../runtime/jobmaster/JobMasterTest.java | 151 ++---------- .../jobmaster/utils/JobMasterBuilder.java | 220 ++++++++++++++++++ 2 files changed, 233 insertions(+), 138 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/JobMasterBuilder.java diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java index 7b7a6247a361..3c55d727ef50 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java @@ -67,7 +67,6 @@ import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.instance.SimpleSlotContext; import org.apache.flink.runtime.io.network.partition.NoOpJobMasterPartitionTracker; -import org.apache.flink.runtime.io.network.partition.PartitionTrackerFactory; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker; @@ -91,6 +90,7 @@ import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot; import org.apache.flink.runtime.jobmaster.slotpool.SlotPool; import org.apache.flink.runtime.jobmaster.slotpool.SlotPoolFactory; +import org.apache.flink.runtime.jobmaster.utils.JobMasterBuilder; import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.messages.FlinkJobNotFoundException; @@ -110,7 +110,6 @@ import org.apache.flink.runtime.scheduler.LegacySchedulerFactory; import org.apache.flink.runtime.scheduler.SchedulerNGFactory; import org.apache.flink.runtime.shuffle.NettyShuffleMaster; -import org.apache.flink.runtime.shuffle.ShuffleMaster; import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.OperatorStreamStateHandle; @@ -221,8 +220,6 @@ public class JobMasterTest extends TestLogger { private static HeartbeatServices heartbeatServices; - private static TestingHeartbeatServices testingHeartbeatService; - private Configuration configuration; private ResourceID jmResourceId; @@ -241,7 +238,6 @@ public static void setupClass() { fastHeartbeatServices = new HeartbeatServices(fastHeartbeatInterval, fastHeartbeatTimeout); heartbeatServices = new HeartbeatServices(heartbeatInterval, heartbeatTimeout); - testingHeartbeatService = new TestingHeartbeatServices(heartbeatInterval, heartbeatTimeout); } @Before @@ -311,7 +307,7 @@ public void testDeclineCheckpointInvocationWithUserException() throws Exception jobManagerSharedServices, heartbeatServices, UnregisteredJobManagerJobMetricGroupFactory.INSTANCE, - new TestingOnCompletionActions(), + new JobMasterBuilder.TestingOnCompletionActions(), testingFatalErrorHandler, JobMasterTest.class.getClassLoader(), schedulerNGFactory, @@ -440,8 +436,7 @@ public void testAllocatedSlotReportDoesNotContainStaleInformation() throws Excep final JobManagerSharedServices jobManagerSharedServices = new TestingJobManagerSharedServicesBuilder().build(); - final JobMaster jobMaster = new JobMasterBuilder() - .withJobGraph(createSingleVertexJobGraph()) + final JobMaster jobMaster = new JobMasterBuilder(createSingleVertexJobGraph(), rpcService) .withHeartbeatServices(new HeartbeatServices(5L, 1000L)) .withSlotPoolFactory(new TestingSlotPoolFactory(hasReceivedSlotOffers)) .createJobMaster(); @@ -1633,7 +1628,7 @@ public void testTriggerSavepointTimeout() throws Exception { jobManagerSharedServices, heartbeatServices, UnregisteredJobManagerJobMetricGroupFactory.INSTANCE, - new TestingOnCompletionActions(), + new JobMasterBuilder.TestingOnCompletionActions(), testingFatalErrorHandler, JobMasterTest.class.getClassLoader(), schedulerNGFactory, @@ -1732,13 +1727,11 @@ public void testTaskExecutorNotReleasedOnFailedAllocationIfPartitionIsAllocated( final TestingJobMasterPartitionTracker partitionTracker = new TestingJobMasterPartitionTracker(); partitionTracker.setIsTrackingPartitionsForFunction(ignored -> isTrackingPartitions.get()); - final JobMaster jobMaster = new JobMasterBuilder() + final JobMaster jobMaster = new JobMasterBuilder(jobGraph, rpcService) .withConfiguration(configuration) - .withJobGraph(jobGraph) .withHighAvailabilityServices(haServices) .withJobManagerSharedServices(jobManagerSharedServices) .withHeartbeatServices(heartbeatServices) - .withOnCompletionActions(new TestingOnCompletionActions()) .withPartitionTrackerFactory(ignored -> partitionTracker) .createJobMaster(); @@ -1867,13 +1860,11 @@ public void testPartitionTableCleanupOnDisconnect() throws Exception { final TestingJobMasterPartitionTracker partitionTracker = new TestingJobMasterPartitionTracker(); partitionTracker.setStopTrackingAllPartitionsConsumer(partitionCleanupTaskExecutorId::complete); - final JobMaster jobMaster = new JobMasterBuilder() + final JobMaster jobMaster = new JobMasterBuilder(jobGraph, rpcService) .withConfiguration(configuration) - .withJobGraph(jobGraph) .withHighAvailabilityServices(haServices) .withJobManagerSharedServices(jobManagerSharedServices) .withHeartbeatServices(heartbeatServices) - .withOnCompletionActions(new TestingOnCompletionActions()) .withPartitionTrackerFactory(ignored -> partitionTracker) .createJobMaster(); @@ -1909,13 +1900,11 @@ public void testPartitionReleaseOnJobTermination() throws Exception { final TestingJobMasterPartitionTracker partitionTracker = new TestingJobMasterPartitionTracker(); partitionTracker.setStopTrackingAndReleaseAllPartitionsConsumer(partitionCleanupTaskExecutorId::complete); - final JobMaster jobMaster = new JobMasterBuilder() + final JobMaster jobMaster = new JobMasterBuilder(jobGraph, rpcService) .withConfiguration(configuration) - .withJobGraph(jobGraph) .withHighAvailabilityServices(haServices) .withJobManagerSharedServices(jobManagerSharedServices) .withHeartbeatServices(heartbeatServices) - .withOnCompletionActions(new TestingOnCompletionActions()) .withPartitionTrackerFactory(ignord -> partitionTracker) .createJobMaster(); @@ -1970,6 +1959,8 @@ public void testJobFailureWhenGracefulTaskExecutorTermination() throws Exception @Test public void testJobFailureWhenTaskExecutorHeartbeatTimeout() throws Exception { + final TestingHeartbeatServices testingHeartbeatService = new TestingHeartbeatServices(heartbeatInterval, heartbeatTimeout); + runJobFailureWhenTaskExecutorTerminatesTest( testingHeartbeatService, (localTaskManagerLocation, jobMasterGateway) -> @@ -1987,7 +1978,7 @@ private void runJobFailureWhenTaskExecutorTerminatesTest( BiConsumer jobReachedRunningState, BiFunction> heartbeatConsumerFunction) throws Exception { final JobGraph jobGraph = createSingleVertexJobGraph(); - final TestingOnCompletionActions onCompletionActions = new TestingOnCompletionActions(); + final JobMasterBuilder.TestingOnCompletionActions onCompletionActions = new JobMasterBuilder.TestingOnCompletionActions(); final JobMaster jobMaster = createJobMaster( new Configuration(), jobGraph, @@ -2028,32 +2019,6 @@ private void runJobFailureWhenTaskExecutorTerminatesTest( } } - private static final class TestingOnCompletionActions implements OnCompletionActions { - - private final CompletableFuture jobReachedGloballyTerminalStateFuture = new CompletableFuture<>(); - private final CompletableFuture jobFinishedByOtherFuture = new CompletableFuture<>(); - private final CompletableFuture jobMasterFailedFuture = new CompletableFuture<>(); - - @Override - public void jobReachedGloballyTerminalState(ArchivedExecutionGraph executionGraph) { - jobReachedGloballyTerminalStateFuture.complete(executionGraph); - } - - @Override - public void jobFinishedByOther() { - jobFinishedByOtherFuture.complete(null); - } - - @Override - public void jobMasterFailed(Throwable cause) { - jobMasterFailedFuture.complete(cause); - } - - public CompletableFuture getJobReachedGloballyTerminalStateFuture() { - return jobReachedGloballyTerminalStateFuture; - } - } - private Collection registerSlotsAtJobMaster( int numberSlots, JobMasterGateway jobMasterGateway, @@ -2215,7 +2180,7 @@ private JobMaster createJobMaster( highAvailabilityServices, jobManagerSharedServices, heartbeatServices, - new TestingOnCompletionActions()); + new JobMasterBuilder.TestingOnCompletionActions()); } @Nonnull @@ -2227,106 +2192,16 @@ private JobMaster createJobMaster( HeartbeatServices heartbeatServices, OnCompletionActions onCompletionActions) throws Exception { - return new JobMasterBuilder() + return new JobMasterBuilder(jobGraph, rpcService) .withConfiguration(configuration) - .withJobGraph(jobGraph) .withHighAvailabilityServices(highAvailabilityServices) .withJobManagerSharedServices(jobManagerSharedServices) .withHeartbeatServices(heartbeatServices) .withOnCompletionActions(onCompletionActions) + .withResourceId(jmResourceId) .createJobMaster(); } - private final class JobMasterBuilder { - - private Configuration configuration = JobMasterTest.this.configuration; - - private JobGraph jobGraph = JobMasterTest.jobGraph; - - private HighAvailabilityServices highAvailabilityServices = haServices; - - private JobManagerSharedServices jobManagerSharedServices = new TestingJobManagerSharedServicesBuilder().build(); - - private HeartbeatServices heartbeatServices = JobMasterTest.heartbeatServices; - - private SlotPoolFactory slotPoolFactory = DefaultSlotPoolFactory.fromConfiguration(configuration); - - private OnCompletionActions onCompletionActions = new TestingOnCompletionActions(); - - private ShuffleMaster shuffleMaster = NettyShuffleMaster.INSTANCE; - - private PartitionTrackerFactory partitionTrackerFactory = NoOpJobMasterPartitionTracker.FACTORY; - - private JobMasterBuilder withConfiguration(Configuration configuration) { - this.configuration = configuration; - return this; - } - - private JobMasterBuilder withJobGraph(JobGraph jobGraph) { - this.jobGraph = jobGraph; - return this; - } - - private JobMasterBuilder withHighAvailabilityServices(HighAvailabilityServices highAvailabilityServices) { - this.highAvailabilityServices = highAvailabilityServices; - return this; - } - - private JobMasterBuilder withJobManagerSharedServices(JobManagerSharedServices jobManagerSharedServices) { - this.jobManagerSharedServices = jobManagerSharedServices; - return this; - } - - private JobMasterBuilder withHeartbeatServices(HeartbeatServices heartbeatServices) { - this.heartbeatServices = heartbeatServices; - return this; - } - - private JobMasterBuilder withSlotPoolFactory(SlotPoolFactory slotPoolFactory) { - this.slotPoolFactory = slotPoolFactory; - return this; - } - - private JobMasterBuilder withOnCompletionActions(OnCompletionActions onCompletionActions) { - this.onCompletionActions = onCompletionActions; - return this; - } - - private JobMasterBuilder withShuffleMaster(ShuffleMaster shuffleMaster) { - this.shuffleMaster = shuffleMaster; - return this; - } - - private JobMasterBuilder withPartitionTrackerFactory(PartitionTrackerFactory partitionTrackerFactory) { - this.partitionTrackerFactory = partitionTrackerFactory; - return this; - } - - private JobMaster createJobMaster() throws Exception { - final JobMasterConfiguration jobMasterConfiguration = JobMasterConfiguration.fromConfiguration(configuration); - final SchedulerNGFactory schedulerNGFactory = new LegacySchedulerFactory( - jobManagerSharedServices.getRestartStrategyFactory()); - - return new JobMaster( - rpcService, - jobMasterConfiguration, - jmResourceId, - jobGraph, - highAvailabilityServices, - slotPoolFactory, - DefaultSchedulerFactory.fromConfiguration(configuration), - jobManagerSharedServices, - heartbeatServices, - UnregisteredJobManagerJobMetricGroupFactory.INSTANCE, - onCompletionActions, - testingFatalErrorHandler, - JobMasterTest.class.getClassLoader(), - schedulerNGFactory, - shuffleMaster, - partitionTrackerFactory); - } - } - private JobGraph createSingleVertexJobWithRestartStrategy() throws IOException { final JobGraph jobGraph = createSingleVertexJobGraph(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/JobMasterBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/JobMasterBuilder.java new file mode 100644 index 000000000000..2b83aea3a8b0 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/JobMasterBuilder.java @@ -0,0 +1,220 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmaster.utils; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +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; +import org.apache.flink.runtime.io.network.partition.NoOpJobMasterPartitionTracker; +import org.apache.flink.runtime.io.network.partition.PartitionTrackerFactory; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmanager.OnCompletionActions; +import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; +import org.apache.flink.runtime.jobmaster.JobMaster; +import org.apache.flink.runtime.jobmaster.JobMasterConfiguration; +import org.apache.flink.runtime.jobmaster.TestingJobManagerSharedServicesBuilder; +import org.apache.flink.runtime.jobmaster.factories.UnregisteredJobManagerJobMetricGroupFactory; +import org.apache.flink.runtime.jobmaster.slotpool.DefaultSchedulerFactory; +import org.apache.flink.runtime.jobmaster.slotpool.DefaultSlotPoolFactory; +import org.apache.flink.runtime.jobmaster.slotpool.SchedulerFactory; +import org.apache.flink.runtime.jobmaster.slotpool.SlotPoolFactory; +import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.scheduler.LegacySchedulerFactory; +import org.apache.flink.runtime.scheduler.SchedulerNGFactory; +import org.apache.flink.runtime.shuffle.NettyShuffleMaster; +import org.apache.flink.runtime.shuffle.ShuffleMaster; + +import java.util.concurrent.CompletableFuture; + +/** + * A builder for the {@link JobMaster}. + */ +public class JobMasterBuilder { + + private static final long heartbeatInterval = 1000L; + private static final long heartbeatTimeout = 5_000_000L; + private static final HeartbeatServices DEFAULT_HEARTBEAT_SERVICES = new HeartbeatServices(heartbeatInterval, heartbeatTimeout); + + private Configuration configuration = new Configuration(); + + private final JobGraph jobGraph; + private final RpcService rpcService; + + private HighAvailabilityServices highAvailabilityServices; + + private JobManagerSharedServices jobManagerSharedServices = new TestingJobManagerSharedServicesBuilder().build(); + + private HeartbeatServices heartbeatServices = DEFAULT_HEARTBEAT_SERVICES; + + private SlotPoolFactory slotPoolFactory = null; + + private SchedulerFactory schedulerFactory = null; + + private SchedulerNGFactory schedulerNGFactory = null; + + private OnCompletionActions onCompletionActions = new TestingOnCompletionActions(); + + private ShuffleMaster shuffleMaster = NettyShuffleMaster.INSTANCE; + + private PartitionTrackerFactory partitionTrackerFactory = NoOpJobMasterPartitionTracker.FACTORY; + + private ResourceID jmResourceId = ResourceID.generate(); + + private FatalErrorHandler fatalErrorHandler = error -> { + }; + + public JobMasterBuilder(JobGraph jobGraph, RpcService rpcService) { + TestingHighAvailabilityServices testingHighAvailabilityServices = new TestingHighAvailabilityServices(); + testingHighAvailabilityServices.setCheckpointRecoveryFactory(new StandaloneCheckpointRecoveryFactory()); + + SettableLeaderRetrievalService rmLeaderRetrievalService = new SettableLeaderRetrievalService( + null, + null); + testingHighAvailabilityServices.setResourceManagerLeaderRetriever(rmLeaderRetrievalService); + + this.highAvailabilityServices = testingHighAvailabilityServices; + this.jobGraph = jobGraph; + this.rpcService = rpcService; + } + + public JobMasterBuilder withConfiguration(Configuration configuration) { + this.configuration = configuration; + return this; + } + + public JobMasterBuilder withHighAvailabilityServices(HighAvailabilityServices highAvailabilityServices) { + this.highAvailabilityServices = highAvailabilityServices; + return this; + } + + public JobMasterBuilder withJobManagerSharedServices(JobManagerSharedServices jobManagerSharedServices) { + this.jobManagerSharedServices = jobManagerSharedServices; + return this; + } + + public JobMasterBuilder withHeartbeatServices(HeartbeatServices heartbeatServices) { + this.heartbeatServices = heartbeatServices; + return this; + } + + public JobMasterBuilder withSlotPoolFactory(SlotPoolFactory slotPoolFactory) { + this.slotPoolFactory = slotPoolFactory; + return this; + } + + public JobMasterBuilder withFatalErrorHandler(FatalErrorHandler fatalErrorHandler) { + this.fatalErrorHandler = fatalErrorHandler; + return this; + } + + public JobMasterBuilder withSchedulerFactory(SchedulerFactory schedulerFactory) { + this.schedulerFactory = schedulerFactory; + return this; + } + + public JobMasterBuilder withSchedulerNGFactory(SchedulerNGFactory schedulerNGFactory) { + this.schedulerNGFactory = schedulerNGFactory; + return this; + } + + public JobMasterBuilder withOnCompletionActions(OnCompletionActions onCompletionActions) { + this.onCompletionActions = onCompletionActions; + return this; + } + + public JobMasterBuilder withResourceId(ResourceID resourceId) { + this.jmResourceId = resourceId; + return this; + } + + public JobMasterBuilder withShuffleMaster(ShuffleMaster shuffleMaster) { + this.shuffleMaster = shuffleMaster; + return this; + } + + public JobMasterBuilder withPartitionTrackerFactory(PartitionTrackerFactory partitionTrackerFactory) { + this.partitionTrackerFactory = partitionTrackerFactory; + return this; + } + + public JobMaster createJobMaster() throws Exception { + final JobMasterConfiguration jobMasterConfiguration = JobMasterConfiguration.fromConfiguration(configuration); + + return new JobMaster( + rpcService, + jobMasterConfiguration, + jmResourceId, + jobGraph, + highAvailabilityServices, + slotPoolFactory != null ? slotPoolFactory : DefaultSlotPoolFactory.fromConfiguration(configuration), + schedulerFactory != null ? schedulerFactory : DefaultSchedulerFactory.fromConfiguration(configuration), + jobManagerSharedServices, + heartbeatServices, + UnregisteredJobManagerJobMetricGroupFactory.INSTANCE, + onCompletionActions, + fatalErrorHandler, + JobMasterBuilder.class.getClassLoader(), + schedulerNGFactory != null ? schedulerNGFactory : new LegacySchedulerFactory(jobManagerSharedServices.getRestartStrategyFactory()), + shuffleMaster, + partitionTrackerFactory); + } + + /** + * Test {@link OnCompletionActions} implementation that exposes a {@link CompletableFuture} for every method which + * is completed with the method argument once that method is called. + */ + public static final class TestingOnCompletionActions implements OnCompletionActions { + + private final CompletableFuture jobReachedGloballyTerminalStateFuture = new CompletableFuture<>(); + private final CompletableFuture jobFinishedByOtherFuture = new CompletableFuture<>(); + private final CompletableFuture jobMasterFailedFuture = new CompletableFuture<>(); + + @Override + public void jobReachedGloballyTerminalState(ArchivedExecutionGraph executionGraph) { + jobReachedGloballyTerminalStateFuture.complete(executionGraph); + } + + @Override + public void jobFinishedByOther() { + jobFinishedByOtherFuture.complete(null); + } + + @Override + public void jobMasterFailed(Throwable cause) { + jobMasterFailedFuture.complete(cause); + } + + public CompletableFuture getJobReachedGloballyTerminalStateFuture() { + return jobReachedGloballyTerminalStateFuture; + } + + public CompletableFuture getJobFinishedByOtherFuture() { + return jobReachedGloballyTerminalStateFuture; + } + + public CompletableFuture getJobMasterFailedFuture() { + return jobReachedGloballyTerminalStateFuture; + } + } +} From 603ad1605ad46f2b967c7ba5f77d01e853577991 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 6 Nov 2019 15:34:26 +0100 Subject: [PATCH 460/746] [FLINK-14586][tests] Factor out utility createSingleVertexJobGraph() --- .../jobgraph/utils/JobGraphTestUtils.java | 36 +++++++++++++++++++ .../runtime/jobmaster/JobMasterTest.java | 21 ++++------- 2 files changed, 43 insertions(+), 14 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/utils/JobGraphTestUtils.java diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/utils/JobGraphTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/utils/JobGraphTestUtils.java new file mode 100644 index 000000000000..2b7836376b2a --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/utils/JobGraphTestUtils.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.jobgraph.utils; + +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.testtasks.NoOpInvokable; + +/** + * Test utilities for the {@link JobGraph}. + */ +public enum JobGraphTestUtils { + ; + + public static JobGraph createSingleVertexJobGraph() { + final JobVertex jobVertex = new JobVertex("Test vertex"); + jobVertex.setInvokableClass(NoOpInvokable.class); + + return new JobGraph(jobVertex); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java index 3c55d727ef50..88ebf9f2af60 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java @@ -81,6 +81,7 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; +import org.apache.flink.runtime.jobgraph.utils.JobGraphTestUtils; import org.apache.flink.runtime.jobmanager.OnCompletionActions; import org.apache.flink.runtime.jobmanager.PartitionProducerDisposedException; import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; @@ -436,7 +437,7 @@ public void testAllocatedSlotReportDoesNotContainStaleInformation() throws Excep final JobManagerSharedServices jobManagerSharedServices = new TestingJobManagerSharedServicesBuilder().build(); - final JobMaster jobMaster = new JobMasterBuilder(createSingleVertexJobGraph(), rpcService) + final JobMaster jobMaster = new JobMasterBuilder(JobGraphTestUtils.createSingleVertexJobGraph(), rpcService) .withHeartbeatServices(new HeartbeatServices(5L, 1000L)) .withSlotPoolFactory(new TestingSlotPoolFactory(hasReceivedSlotOffers)) .createJobMaster(); @@ -1719,7 +1720,7 @@ public void testReleasingTaskExecutorIfNoMoreSlotsRegistered() throws Exception public void testTaskExecutorNotReleasedOnFailedAllocationIfPartitionIsAllocated() throws Exception { final JobManagerSharedServices jobManagerSharedServices = new TestingJobManagerSharedServicesBuilder().build(); - final JobGraph jobGraph = createSingleVertexJobGraph(); + final JobGraph jobGraph = JobGraphTestUtils.createSingleVertexJobGraph(); final LocalTaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); @@ -1854,7 +1855,7 @@ private TestingResourceManagerGateway createAndRegisterTestingResourceManagerGat @Test public void testPartitionTableCleanupOnDisconnect() throws Exception { final JobManagerSharedServices jobManagerSharedServices = new TestingJobManagerSharedServicesBuilder().build(); - final JobGraph jobGraph = createSingleVertexJobGraph(); + final JobGraph jobGraph = JobGraphTestUtils.createSingleVertexJobGraph(); final CompletableFuture partitionCleanupTaskExecutorId = new CompletableFuture<>(); final TestingJobMasterPartitionTracker partitionTracker = new TestingJobMasterPartitionTracker(); @@ -1894,7 +1895,7 @@ public void testPartitionTableCleanupOnDisconnect() throws Exception { @Test public void testPartitionReleaseOnJobTermination() throws Exception { final JobManagerSharedServices jobManagerSharedServices = new TestingJobManagerSharedServicesBuilder().build(); - final JobGraph jobGraph = createSingleVertexJobGraph(); + final JobGraph jobGraph = JobGraphTestUtils.createSingleVertexJobGraph(); final CompletableFuture partitionCleanupTaskExecutorId = new CompletableFuture<>(); final TestingJobMasterPartitionTracker partitionTracker = new TestingJobMasterPartitionTracker(); @@ -1977,7 +1978,7 @@ private void runJobFailureWhenTaskExecutorTerminatesTest( HeartbeatServices heartbeatServices, BiConsumer jobReachedRunningState, BiFunction> heartbeatConsumerFunction) throws Exception { - final JobGraph jobGraph = createSingleVertexJobGraph(); + final JobGraph jobGraph = JobGraphTestUtils.createSingleVertexJobGraph(); final JobMasterBuilder.TestingOnCompletionActions onCompletionActions = new JobMasterBuilder.TestingOnCompletionActions(); final JobMaster jobMaster = createJobMaster( new Configuration(), @@ -2203,7 +2204,7 @@ private JobMaster createJobMaster( } private JobGraph createSingleVertexJobWithRestartStrategy() throws IOException { - final JobGraph jobGraph = createSingleVertexJobGraph(); + final JobGraph jobGraph = JobGraphTestUtils.createSingleVertexJobGraph(); final ExecutionConfig executionConfig = new ExecutionConfig(); executionConfig.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 0L)); @@ -2212,14 +2213,6 @@ private JobGraph createSingleVertexJobWithRestartStrategy() throws IOException { return jobGraph; } - @Nonnull - private JobGraph createSingleVertexJobGraph() { - final JobVertex jobVertex = new JobVertex("Test vertex"); - jobVertex.setInvokableClass(NoOpInvokable.class); - - return new JobGraph(jobVertex); - } - private static final class DummyCheckpointStorageLocation implements CompletedCheckpointStorageLocation { private static final long serialVersionUID = 164095949572620688L; From 28b47b734e96b7a5f363e135615d6781714ed58f Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 31 Oct 2019 13:40:39 +0100 Subject: [PATCH 461/746] [FLINK-14586][tests] Move JM partition tests into separate class --- .../JobMasterPartitionReleaseTest.java | 286 ++++++++++++++++++ .../runtime/jobmaster/JobMasterTest.java | 93 ------ 2 files changed, 286 insertions(+), 93 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterPartitionReleaseTest.java diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterPartitionReleaseTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterPartitionReleaseTest.java new file mode 100644 index 000000000000..fe7ef96fe01b --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterPartitionReleaseTest.java @@ -0,0 +1,286 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.common.time.Time; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.BlobServerOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.utils.JobGraphTestUtils; +import org.apache.flink.runtime.jobmaster.utils.JobMasterBuilder; +import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway; +import org.apache.flink.runtime.rpc.RpcUtils; +import org.apache.flink.runtime.rpc.TestingRpcService; +import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; +import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGateway; +import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; +import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; +import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; +import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.runtime.util.TestingFatalErrorHandler; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.TestLogger; + +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.IOException; +import java.util.Collection; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; + +/** + * Tests for the partition release logic of the {@link JobMaster}. + */ +public class JobMasterPartitionReleaseTest extends TestLogger { + + @ClassRule + public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private static final Time testingTimeout = Time.seconds(10L); + + private static final long heartbeatInterval = 1000L; + private static final long heartbeatTimeout = 5_000_000L; + + private static TestingRpcService rpcService; + + private static HeartbeatServices heartbeatServices; + + private Configuration configuration; + + private JobMasterId jobMasterId; + + private TestingHighAvailabilityServices haServices; + + private SettableLeaderRetrievalService rmLeaderRetrievalService; + + private TestingFatalErrorHandler testingFatalErrorHandler; + + @BeforeClass + public static void setupClass() { + rpcService = new TestingRpcService(); + + heartbeatServices = new HeartbeatServices(heartbeatInterval, heartbeatTimeout); + } + + @Before + public void setup() throws IOException { + configuration = new Configuration(); + haServices = new TestingHighAvailabilityServices(); + jobMasterId = JobMasterId.generate(); + + testingFatalErrorHandler = new TestingFatalErrorHandler(); + + haServices.setCheckpointRecoveryFactory(new StandaloneCheckpointRecoveryFactory()); + + rmLeaderRetrievalService = new SettableLeaderRetrievalService( + null, + null); + haServices.setResourceManagerLeaderRetriever(rmLeaderRetrievalService); + + configuration.setString(BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); + } + + @After + public void teardown() throws Exception { + if (testingFatalErrorHandler != null) { + testingFatalErrorHandler.rethrowError(); + } + + rpcService.clearGateways(); + } + + @AfterClass + public static void teardownClass() { + if (rpcService != null) { + rpcService.stopService(); + rpcService = null; + } + } + + @Test + public void testPartitionTableCleanupOnDisconnect() throws Exception { + final JobManagerSharedServices jobManagerSharedServices = new TestingJobManagerSharedServicesBuilder().build(); + final JobGraph jobGraph = JobGraphTestUtils.createSingleVertexJobGraph(); + + final CompletableFuture partitionCleanupTaskExecutorId = new CompletableFuture<>(); + final TestingJobMasterPartitionTracker partitionTracker = new TestingJobMasterPartitionTracker(); + partitionTracker.setStopTrackingAllPartitionsConsumer(partitionCleanupTaskExecutorId::complete); + + final JobMaster jobMaster = new JobMasterBuilder(jobGraph, rpcService) + .withConfiguration(configuration) + .withHighAvailabilityServices(haServices) + .withJobManagerSharedServices(jobManagerSharedServices) + .withHeartbeatServices(heartbeatServices) + .withPartitionTrackerFactory(ignored -> partitionTracker) + .createJobMaster(); + + final CompletableFuture disconnectTaskExecutorFuture = new CompletableFuture<>(); + final TestingTaskExecutorGateway testingTaskExecutorGateway = new TestingTaskExecutorGatewayBuilder() + .setDisconnectJobManagerConsumer((jobID, throwable) -> disconnectTaskExecutorFuture.complete(jobID)) + .createTestingTaskExecutorGateway(); + + try { + jobMaster.start(jobMasterId).get(); + + final JobMasterGateway jobMasterGateway = jobMaster.getSelfGateway(JobMasterGateway.class); + + // register a slot to establish a connection + final LocalTaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); + registerSlotsAtJobMaster(1, jobMasterGateway, testingTaskExecutorGateway, taskManagerLocation); + + jobMasterGateway.disconnectTaskManager(taskManagerLocation.getResourceID(), new Exception("test")); + disconnectTaskExecutorFuture.get(); + + assertThat(partitionCleanupTaskExecutorId.get(), equalTo(taskManagerLocation.getResourceID())); + } finally { + RpcUtils.terminateRpcEndpoint(jobMaster, testingTimeout); + } + } + + @Test + public void testPartitionReleaseOrPromotionOnJobTermination() throws Exception { + final JobManagerSharedServices jobManagerSharedServices = new TestingJobManagerSharedServicesBuilder().build(); + final JobGraph jobGraph = JobGraphTestUtils.createSingleVertexJobGraph(); + + final CompletableFuture taskDeploymentDescriptorFuture = new CompletableFuture<>(); + final CompletableFuture>> releasePartitionsFuture = new CompletableFuture<>(); + final CompletableFuture disconnectTaskExecutorFuture = new CompletableFuture<>(); + final TestingTaskExecutorGateway testingTaskExecutorGateway = new TestingTaskExecutorGatewayBuilder() + .setReleaseOrPromotePartitionsConsumer((jobId, partitionsToRelease, partitionsToPromote) -> releasePartitionsFuture.complete(Tuple2.of(jobId, partitionsToRelease))) + .setDisconnectJobManagerConsumer((jobID, throwable) -> disconnectTaskExecutorFuture.complete(jobID)) + .setSubmitTaskConsumer((tdd, ignored) -> { + taskDeploymentDescriptorFuture.complete(tdd); + return CompletableFuture.completedFuture(Acknowledge.get()); + }) + .createTestingTaskExecutorGateway(); + + final CompletableFuture taskExecutorIdForPartitionRelease = new CompletableFuture<>(); + final CompletableFuture taskExecutorIdForPartitionReleaseOrPromote = new CompletableFuture<>(); + final TestingJobMasterPartitionTracker partitionTracker = new TestingJobMasterPartitionTracker(); + partitionTracker.setStopTrackingAndReleaseAllPartitionsConsumer(taskExecutorIdForPartitionRelease::complete); + partitionTracker.setStopTrackingAndReleaseOrPromotePartitionsConsumer(taskExecutorIdForPartitionReleaseOrPromote::complete); + + final JobMaster jobMaster = new JobMasterBuilder(jobGraph, rpcService) + .withConfiguration(configuration) + .withHighAvailabilityServices(haServices) + .withJobManagerSharedServices(jobManagerSharedServices) + .withHeartbeatServices(heartbeatServices) + .withPartitionTrackerFactory(ignord -> partitionTracker) + .createJobMaster(); + + try { + jobMaster.start(jobMasterId).get(); + + final JobMasterGateway jobMasterGateway = jobMaster.getSelfGateway(JobMasterGateway.class); + + final LocalTaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); + registerSlotsAtJobMaster(1, jobMasterGateway, testingTaskExecutorGateway, taskManagerLocation); + + // update the execution state of the only execution to FINISHED + // this should trigger the job to finish + final TaskDeploymentDescriptor taskDeploymentDescriptor = taskDeploymentDescriptorFuture.get(); + jobMasterGateway.updateTaskExecutionState( + new TaskExecutionState( + taskDeploymentDescriptor.getJobId(), + taskDeploymentDescriptor.getExecutionAttemptId(), + ExecutionState.FINISHED)); + + assertThat(taskExecutorIdForPartitionReleaseOrPromote.get(), equalTo(taskManagerLocation.getResourceID())); + } finally { + RpcUtils.terminateRpcEndpoint(jobMaster, testingTimeout); + } + } + + private Collection registerSlotsAtJobMaster( + int numberSlots, + JobMasterGateway jobMasterGateway, + TaskExecutorGateway taskExecutorGateway, + TaskManagerLocation taskManagerLocation) throws ExecutionException, InterruptedException { + final AllocationIdsResourceManagerGateway allocationIdsResourceManagerGateway = new AllocationIdsResourceManagerGateway(); + rpcService.registerGateway(allocationIdsResourceManagerGateway.getAddress(), allocationIdsResourceManagerGateway); + notifyResourceManagerLeaderListeners(allocationIdsResourceManagerGateway); + + rpcService.registerGateway(taskExecutorGateway.getAddress(), taskExecutorGateway); + + jobMasterGateway.registerTaskManager(taskExecutorGateway.getAddress(), taskManagerLocation, testingTimeout).get(); + + Collection slotOffers = IntStream + .range(0, numberSlots) + .mapToObj( + index -> { + final AllocationID allocationId = allocationIdsResourceManagerGateway.takeAllocationId(); + return new SlotOffer(allocationId, index, ResourceProfile.UNKNOWN); + }) + .collect(Collectors.toList()); + + return jobMasterGateway.offerSlots(taskManagerLocation.getResourceID(), slotOffers, testingTimeout).get(); + } + + private void notifyResourceManagerLeaderListeners(TestingResourceManagerGateway testingResourceManagerGateway) { + rmLeaderRetrievalService.notifyListener(testingResourceManagerGateway.getAddress(), testingResourceManagerGateway.getFencingToken().toUUID()); + } + + private static final class AllocationIdsResourceManagerGateway extends TestingResourceManagerGateway { + private final BlockingQueue allocationIds; + + private AllocationIdsResourceManagerGateway() { + this.allocationIds = new ArrayBlockingQueue<>(10); + setRequestSlotConsumer( + slotRequest -> allocationIds.offer(slotRequest.getAllocationId()) + ); + } + + AllocationID takeAllocationId() { + try { + return allocationIds.take(); + } catch (InterruptedException e) { + ExceptionUtils.rethrow(e); + return null; + } + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java index 88ebf9f2af60..698b1928bb65 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java @@ -26,7 +26,6 @@ import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.ClosureCleaner; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; @@ -1852,98 +1851,6 @@ private TestingResourceManagerGateway createAndRegisterTestingResourceManagerGat return testingResourceManagerGateway; } - @Test - public void testPartitionTableCleanupOnDisconnect() throws Exception { - final JobManagerSharedServices jobManagerSharedServices = new TestingJobManagerSharedServicesBuilder().build(); - final JobGraph jobGraph = JobGraphTestUtils.createSingleVertexJobGraph(); - - final CompletableFuture partitionCleanupTaskExecutorId = new CompletableFuture<>(); - final TestingJobMasterPartitionTracker partitionTracker = new TestingJobMasterPartitionTracker(); - partitionTracker.setStopTrackingAllPartitionsConsumer(partitionCleanupTaskExecutorId::complete); - - final JobMaster jobMaster = new JobMasterBuilder(jobGraph, rpcService) - .withConfiguration(configuration) - .withHighAvailabilityServices(haServices) - .withJobManagerSharedServices(jobManagerSharedServices) - .withHeartbeatServices(heartbeatServices) - .withPartitionTrackerFactory(ignored -> partitionTracker) - .createJobMaster(); - - final CompletableFuture disconnectTaskExecutorFuture = new CompletableFuture<>(); - final TestingTaskExecutorGateway testingTaskExecutorGateway = new TestingTaskExecutorGatewayBuilder() - .setDisconnectJobManagerConsumer((jobID, throwable) -> disconnectTaskExecutorFuture.complete(jobID)) - .createTestingTaskExecutorGateway(); - - try { - jobMaster.start(jobMasterId).get(); - - final JobMasterGateway jobMasterGateway = jobMaster.getSelfGateway(JobMasterGateway.class); - - // register a slot to establish a connection - final LocalTaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); - registerSlotsAtJobMaster(1, jobMasterGateway, testingTaskExecutorGateway, taskManagerLocation); - - jobMasterGateway.disconnectTaskManager(taskManagerLocation.getResourceID(), new Exception("test")); - disconnectTaskExecutorFuture.get(); - - assertThat(partitionCleanupTaskExecutorId.get(), equalTo(taskManagerLocation.getResourceID())); - } finally { - RpcUtils.terminateRpcEndpoint(jobMaster, testingTimeout); - } - } - - @Test - public void testPartitionReleaseOnJobTermination() throws Exception { - final JobManagerSharedServices jobManagerSharedServices = new TestingJobManagerSharedServicesBuilder().build(); - final JobGraph jobGraph = JobGraphTestUtils.createSingleVertexJobGraph(); - - final CompletableFuture partitionCleanupTaskExecutorId = new CompletableFuture<>(); - final TestingJobMasterPartitionTracker partitionTracker = new TestingJobMasterPartitionTracker(); - partitionTracker.setStopTrackingAndReleaseAllPartitionsConsumer(partitionCleanupTaskExecutorId::complete); - - final JobMaster jobMaster = new JobMasterBuilder(jobGraph, rpcService) - .withConfiguration(configuration) - .withHighAvailabilityServices(haServices) - .withJobManagerSharedServices(jobManagerSharedServices) - .withHeartbeatServices(heartbeatServices) - .withPartitionTrackerFactory(ignord -> partitionTracker) - .createJobMaster(); - - final CompletableFuture taskDeploymentDescriptorFuture = new CompletableFuture<>(); - final CompletableFuture>> releasePartitionsFuture = new CompletableFuture<>(); - final CompletableFuture disconnectTaskExecutorFuture = new CompletableFuture<>(); - final TestingTaskExecutorGateway testingTaskExecutorGateway = new TestingTaskExecutorGatewayBuilder() - .setReleaseOrPromotePartitionsConsumer((jobId, partitionsToRelease, partitionsToPromote) -> releasePartitionsFuture.complete(Tuple2.of(jobId, partitionsToRelease))) - .setDisconnectJobManagerConsumer((jobID, throwable) -> disconnectTaskExecutorFuture.complete(jobID)) - .setSubmitTaskConsumer((tdd, ignored) -> { - taskDeploymentDescriptorFuture.complete(tdd); - return CompletableFuture.completedFuture(Acknowledge.get()); - }) - .createTestingTaskExecutorGateway(); - - try { - jobMaster.start(jobMasterId).get(); - - final JobMasterGateway jobMasterGateway = jobMaster.getSelfGateway(JobMasterGateway.class); - - final LocalTaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); - registerSlotsAtJobMaster(1, jobMasterGateway, testingTaskExecutorGateway, taskManagerLocation); - - // update the execution state of the only execution to FINISHED - // this should trigger the job to finish - final TaskDeploymentDescriptor taskDeploymentDescriptor = taskDeploymentDescriptorFuture.get(); - jobMasterGateway.updateTaskExecutionState( - new TaskExecutionState( - taskDeploymentDescriptor.getJobId(), - taskDeploymentDescriptor.getExecutionAttemptId(), - ExecutionState.FINISHED)); - - assertThat(partitionCleanupTaskExecutorId.get(), equalTo(taskManagerLocation.getResourceID())); - } finally { - RpcUtils.terminateRpcEndpoint(jobMaster, testingTimeout); - } - } - /** * Tests that the job execution is failed if the TaskExecutor disconnects from the * JobMaster. From deb2a9d369086c0b657dba171e9b6763ec2e90c9 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 31 Oct 2019 14:25:53 +0100 Subject: [PATCH 462/746] [FLINK-14586][tests] Refactor partition tests Rework test setup to be re-usable across tests. --- .../JobMasterPartitionReleaseTest.java | 229 +++++++++--------- 1 file changed, 108 insertions(+), 121 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterPartitionReleaseTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterPartitionReleaseTest.java index fe7ef96fe01b..7e5673aeb72e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterPartitionReleaseTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterPartitionReleaseTest.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; @@ -31,14 +30,13 @@ import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; -import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker; -import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.utils.JobGraphTestUtils; import org.apache.flink.runtime.jobmaster.utils.JobMasterBuilder; import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway; +import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; @@ -47,7 +45,6 @@ import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; import org.apache.flink.runtime.taskmanager.TaskExecutionState; -import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; @@ -62,12 +59,11 @@ import java.io.IOException; import java.util.Collection; +import java.util.Collections; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import java.util.stream.Collectors; -import java.util.stream.IntStream; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; @@ -82,46 +78,18 @@ public class JobMasterPartitionReleaseTest extends TestLogger { private static final Time testingTimeout = Time.seconds(10L); - private static final long heartbeatInterval = 1000L; - private static final long heartbeatTimeout = 5_000_000L; - private static TestingRpcService rpcService; - private static HeartbeatServices heartbeatServices; - - private Configuration configuration; - - private JobMasterId jobMasterId; - - private TestingHighAvailabilityServices haServices; - - private SettableLeaderRetrievalService rmLeaderRetrievalService; - private TestingFatalErrorHandler testingFatalErrorHandler; @BeforeClass public static void setupClass() { rpcService = new TestingRpcService(); - - heartbeatServices = new HeartbeatServices(heartbeatInterval, heartbeatTimeout); } @Before public void setup() throws IOException { - configuration = new Configuration(); - haServices = new TestingHighAvailabilityServices(); - jobMasterId = JobMasterId.generate(); - testingFatalErrorHandler = new TestingFatalErrorHandler(); - - haServices.setCheckpointRecoveryFactory(new StandaloneCheckpointRecoveryFactory()); - - rmLeaderRetrievalService = new SettableLeaderRetrievalService( - null, - null); - haServices.setResourceManagerLeaderRetriever(rmLeaderRetrievalService); - - configuration.setString(BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); } @After @@ -143,82 +111,33 @@ public static void teardownClass() { @Test public void testPartitionTableCleanupOnDisconnect() throws Exception { - final JobManagerSharedServices jobManagerSharedServices = new TestingJobManagerSharedServicesBuilder().build(); - final JobGraph jobGraph = JobGraphTestUtils.createSingleVertexJobGraph(); - - final CompletableFuture partitionCleanupTaskExecutorId = new CompletableFuture<>(); - final TestingJobMasterPartitionTracker partitionTracker = new TestingJobMasterPartitionTracker(); - partitionTracker.setStopTrackingAllPartitionsConsumer(partitionCleanupTaskExecutorId::complete); - - final JobMaster jobMaster = new JobMasterBuilder(jobGraph, rpcService) - .withConfiguration(configuration) - .withHighAvailabilityServices(haServices) - .withJobManagerSharedServices(jobManagerSharedServices) - .withHeartbeatServices(heartbeatServices) - .withPartitionTrackerFactory(ignored -> partitionTracker) - .createJobMaster(); - final CompletableFuture disconnectTaskExecutorFuture = new CompletableFuture<>(); final TestingTaskExecutorGateway testingTaskExecutorGateway = new TestingTaskExecutorGatewayBuilder() .setDisconnectJobManagerConsumer((jobID, throwable) -> disconnectTaskExecutorFuture.complete(jobID)) .createTestingTaskExecutorGateway(); - try { - jobMaster.start(jobMasterId).get(); - - final JobMasterGateway jobMasterGateway = jobMaster.getSelfGateway(JobMasterGateway.class); + try (final TestSetup testSetup = new TestSetup(rpcService, testingFatalErrorHandler, testingTaskExecutorGateway)) { + final JobMasterGateway jobMasterGateway = testSetup.jobMaster.getSelfGateway(JobMasterGateway.class); - // register a slot to establish a connection - final LocalTaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); - registerSlotsAtJobMaster(1, jobMasterGateway, testingTaskExecutorGateway, taskManagerLocation); - - jobMasterGateway.disconnectTaskManager(taskManagerLocation.getResourceID(), new Exception("test")); + jobMasterGateway.disconnectTaskManager(testSetup.getTaskExecutorResourceID(), new Exception("test")); disconnectTaskExecutorFuture.get(); - assertThat(partitionCleanupTaskExecutorId.get(), equalTo(taskManagerLocation.getResourceID())); - } finally { - RpcUtils.terminateRpcEndpoint(jobMaster, testingTimeout); + assertThat(testSetup.getStopTrackingPartitionsTargetResourceId().get(), equalTo(testSetup.getTaskExecutorResourceID())); } } @Test public void testPartitionReleaseOrPromotionOnJobTermination() throws Exception { - final JobManagerSharedServices jobManagerSharedServices = new TestingJobManagerSharedServicesBuilder().build(); - final JobGraph jobGraph = JobGraphTestUtils.createSingleVertexJobGraph(); - final CompletableFuture taskDeploymentDescriptorFuture = new CompletableFuture<>(); - final CompletableFuture>> releasePartitionsFuture = new CompletableFuture<>(); - final CompletableFuture disconnectTaskExecutorFuture = new CompletableFuture<>(); final TestingTaskExecutorGateway testingTaskExecutorGateway = new TestingTaskExecutorGatewayBuilder() - .setReleaseOrPromotePartitionsConsumer((jobId, partitionsToRelease, partitionsToPromote) -> releasePartitionsFuture.complete(Tuple2.of(jobId, partitionsToRelease))) - .setDisconnectJobManagerConsumer((jobID, throwable) -> disconnectTaskExecutorFuture.complete(jobID)) .setSubmitTaskConsumer((tdd, ignored) -> { taskDeploymentDescriptorFuture.complete(tdd); return CompletableFuture.completedFuture(Acknowledge.get()); }) .createTestingTaskExecutorGateway(); - final CompletableFuture taskExecutorIdForPartitionRelease = new CompletableFuture<>(); - final CompletableFuture taskExecutorIdForPartitionReleaseOrPromote = new CompletableFuture<>(); - final TestingJobMasterPartitionTracker partitionTracker = new TestingJobMasterPartitionTracker(); - partitionTracker.setStopTrackingAndReleaseAllPartitionsConsumer(taskExecutorIdForPartitionRelease::complete); - partitionTracker.setStopTrackingAndReleaseOrPromotePartitionsConsumer(taskExecutorIdForPartitionReleaseOrPromote::complete); - - final JobMaster jobMaster = new JobMasterBuilder(jobGraph, rpcService) - .withConfiguration(configuration) - .withHighAvailabilityServices(haServices) - .withJobManagerSharedServices(jobManagerSharedServices) - .withHeartbeatServices(heartbeatServices) - .withPartitionTrackerFactory(ignord -> partitionTracker) - .createJobMaster(); - - try { - jobMaster.start(jobMasterId).get(); - - final JobMasterGateway jobMasterGateway = jobMaster.getSelfGateway(JobMasterGateway.class); - - final LocalTaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); - registerSlotsAtJobMaster(1, jobMasterGateway, testingTaskExecutorGateway, taskManagerLocation); + try (final TestSetup testSetup = new TestSetup(rpcService, testingFatalErrorHandler, testingTaskExecutorGateway)) { + final JobMasterGateway jobMasterGateway = testSetup.getJobMasterGateway(); // update the execution state of the only execution to FINISHED // this should trigger the job to finish @@ -229,41 +148,10 @@ public void testPartitionReleaseOrPromotionOnJobTermination() throws Exception { taskDeploymentDescriptor.getExecutionAttemptId(), ExecutionState.FINISHED)); - assertThat(taskExecutorIdForPartitionReleaseOrPromote.get(), equalTo(taskManagerLocation.getResourceID())); - } finally { - RpcUtils.terminateRpcEndpoint(jobMaster, testingTimeout); + assertThat(testSetup.getReleasePartitionsTargetResourceId().get(), equalTo(testSetup.getTaskExecutorResourceID())); } } - private Collection registerSlotsAtJobMaster( - int numberSlots, - JobMasterGateway jobMasterGateway, - TaskExecutorGateway taskExecutorGateway, - TaskManagerLocation taskManagerLocation) throws ExecutionException, InterruptedException { - final AllocationIdsResourceManagerGateway allocationIdsResourceManagerGateway = new AllocationIdsResourceManagerGateway(); - rpcService.registerGateway(allocationIdsResourceManagerGateway.getAddress(), allocationIdsResourceManagerGateway); - notifyResourceManagerLeaderListeners(allocationIdsResourceManagerGateway); - - rpcService.registerGateway(taskExecutorGateway.getAddress(), taskExecutorGateway); - - jobMasterGateway.registerTaskManager(taskExecutorGateway.getAddress(), taskManagerLocation, testingTimeout).get(); - - Collection slotOffers = IntStream - .range(0, numberSlots) - .mapToObj( - index -> { - final AllocationID allocationId = allocationIdsResourceManagerGateway.takeAllocationId(); - return new SlotOffer(allocationId, index, ResourceProfile.UNKNOWN); - }) - .collect(Collectors.toList()); - - return jobMasterGateway.offerSlots(taskManagerLocation.getResourceID(), slotOffers, testingTimeout).get(); - } - - private void notifyResourceManagerLeaderListeners(TestingResourceManagerGateway testingResourceManagerGateway) { - rmLeaderRetrievalService.notifyListener(testingResourceManagerGateway.getAddress(), testingResourceManagerGateway.getFencingToken().toUUID()); - } - private static final class AllocationIdsResourceManagerGateway extends TestingResourceManagerGateway { private final BlockingQueue allocationIds; @@ -283,4 +171,103 @@ AllocationID takeAllocationId() { } } } + + private static class TestSetup implements AutoCloseable { + + private final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private final LocalTaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); + + private final CompletableFuture taskExecutorIdForStopTracking = new CompletableFuture<>(); + private final CompletableFuture taskExecutorIdForPartitionRelease = new CompletableFuture<>(); + + private JobMaster jobMaster; + + TestSetup(TestingRpcService rpcService, FatalErrorHandler fatalErrorHandler, TaskExecutorGateway taskExecutorGateway) throws Exception { + + temporaryFolder.create(); + + TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices(); + haServices.setCheckpointRecoveryFactory(new StandaloneCheckpointRecoveryFactory()); + + SettableLeaderRetrievalService rmLeaderRetrievalService = new SettableLeaderRetrievalService( + null, + null); + haServices.setResourceManagerLeaderRetriever(rmLeaderRetrievalService); + + final TestingJobMasterPartitionTracker partitionTracker = new TestingJobMasterPartitionTracker(); + + partitionTracker.setStopTrackingAllPartitionsConsumer(taskExecutorIdForStopTracking::complete); + partitionTracker.setStopTrackingAndReleaseAllPartitionsConsumer(taskExecutorIdForPartitionRelease::complete); + + Configuration configuration = new Configuration(); + configuration.setString(BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); + + HeartbeatServices heartbeatServices = new HeartbeatServices(1000L, 5_000_000L); + + jobMaster = new JobMasterBuilder(JobGraphTestUtils.createSingleVertexJobGraph(), rpcService) + .withConfiguration(configuration) + .withHighAvailabilityServices(haServices) + .withJobManagerSharedServices(new TestingJobManagerSharedServicesBuilder().build()) + .withFatalErrorHandler(fatalErrorHandler) + .withHeartbeatServices(heartbeatServices) + .withPartitionTrackerFactory(ignored -> partitionTracker) + .createJobMaster(); + + jobMaster.start(JobMasterId.generate()).get(); + + registerTaskExecutorAtJobMaster( + rpcService, + getJobMasterGateway(), + taskExecutorGateway, + rmLeaderRetrievalService + ); + } + + private void registerTaskExecutorAtJobMaster( + TestingRpcService rpcService, + JobMasterGateway jobMasterGateway, + TaskExecutorGateway taskExecutorGateway, + SettableLeaderRetrievalService rmLeaderRetrievalService) throws ExecutionException, InterruptedException { + + final AllocationIdsResourceManagerGateway resourceManagerGateway = new AllocationIdsResourceManagerGateway(); + rpcService.registerGateway(resourceManagerGateway.getAddress(), resourceManagerGateway); + rmLeaderRetrievalService.notifyListener(resourceManagerGateway.getAddress(), resourceManagerGateway.getFencingToken().toUUID()); + + rpcService.registerGateway(taskExecutorGateway.getAddress(), taskExecutorGateway); + + jobMasterGateway.registerTaskManager(taskExecutorGateway.getAddress(), taskManagerLocation, testingTimeout).get(); + + final AllocationID allocationId = resourceManagerGateway.takeAllocationId(); + Collection slotOffers = Collections.singleton(new SlotOffer(allocationId, 0, ResourceProfile.UNKNOWN)); + + jobMasterGateway.offerSlots(taskManagerLocation.getResourceID(), slotOffers, testingTimeout).get(); + } + + public JobMasterGateway getJobMasterGateway() { + return jobMaster.getSelfGateway(JobMasterGateway.class); + } + + public ResourceID getTaskExecutorResourceID() { + return taskManagerLocation.getResourceID(); + } + + public CompletableFuture getStopTrackingPartitionsTargetResourceId() { + return taskExecutorIdForStopTracking; + } + + public CompletableFuture getReleasePartitionsTargetResourceId() { + return taskExecutorIdForPartitionRelease; + } + + public void close() throws Exception { + try { + if (jobMaster != null) { + RpcUtils.terminateRpcEndpoint(jobMaster, testingTimeout); + } + } finally { + temporaryFolder.delete(); + } + } + } } From 0483ffcddcbb9d96e31bc4182d04ecd238c615f1 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 18 Oct 2019 15:44:44 +0200 Subject: [PATCH 463/746] [FLINK-14586][coordination] JM issues promote calls on successful job --- .../flink/runtime/jobmaster/JobMaster.java | 4 ++- .../JobMasterPartitionReleaseTest.java | 26 +++++++++++++++---- 2 files changed, 24 insertions(+), 6 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java index 8261459c4501..f50176c87cfd 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 @@ -893,7 +893,9 @@ private void jobStatusChanged( if (newJobStatus.isGloballyTerminalState()) { runAsync(() -> registeredTaskManagers.keySet() - .forEach(partitionTracker::stopTrackingAndReleasePartitionsFor)); + .forEach(newJobStatus == JobStatus.FINISHED + ? partitionTracker::stopTrackingAndReleaseOrPromotePartitionsFor + : partitionTracker::stopTrackingAndReleasePartitionsFor)); final ArchivedExecutionGraph archivedExecutionGraph = schedulerNG.requestJob(); scheduledExecutorService.execute(() -> jobCompletionActions.jobReachedGloballyTerminalState(archivedExecutionGraph)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterPartitionReleaseTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterPartitionReleaseTest.java index 7e5673aeb72e..be114f7c784d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterPartitionReleaseTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterPartitionReleaseTest.java @@ -64,6 +64,7 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.function.Function; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; @@ -127,7 +128,16 @@ public void testPartitionTableCleanupOnDisconnect() throws Exception { } @Test - public void testPartitionReleaseOrPromotionOnJobTermination() throws Exception { + public void testPartitionReleaseOrPromotionOnJobSuccess() throws Exception { + testPartitionReleaseOrPromotionOnJobTermination(TestSetup::getReleaseOrPromotePartitionsTargetResourceId, ExecutionState.FINISHED); + } + + @Test + public void testPartitionReleaseOrPromotionOnJobFailure() throws Exception { + testPartitionReleaseOrPromotionOnJobTermination(TestSetup::getReleasePartitionsTargetResourceId, ExecutionState.FAILED); + } + + private void testPartitionReleaseOrPromotionOnJobTermination(Function> taskExecutorCallSelector, ExecutionState finalExecutionState) throws Exception { final CompletableFuture taskDeploymentDescriptorFuture = new CompletableFuture<>(); final TestingTaskExecutorGateway testingTaskExecutorGateway = new TestingTaskExecutorGatewayBuilder() .setSubmitTaskConsumer((tdd, ignored) -> { @@ -139,16 +149,16 @@ public void testPartitionReleaseOrPromotionOnJobTermination() throws Exception { try (final TestSetup testSetup = new TestSetup(rpcService, testingFatalErrorHandler, testingTaskExecutorGateway)) { final JobMasterGateway jobMasterGateway = testSetup.getJobMasterGateway(); - // update the execution state of the only execution to FINISHED + // update the execution state of the only execution to target state // this should trigger the job to finish final TaskDeploymentDescriptor taskDeploymentDescriptor = taskDeploymentDescriptorFuture.get(); jobMasterGateway.updateTaskExecutionState( new TaskExecutionState( taskDeploymentDescriptor.getJobId(), taskDeploymentDescriptor.getExecutionAttemptId(), - ExecutionState.FINISHED)); + finalExecutionState)); - assertThat(testSetup.getReleasePartitionsTargetResourceId().get(), equalTo(testSetup.getTaskExecutorResourceID())); + assertThat(taskExecutorCallSelector.apply(testSetup).get(), equalTo(testSetup.getTaskExecutorResourceID())); } } @@ -180,10 +190,11 @@ private static class TestSetup implements AutoCloseable { private final CompletableFuture taskExecutorIdForStopTracking = new CompletableFuture<>(); private final CompletableFuture taskExecutorIdForPartitionRelease = new CompletableFuture<>(); + private final CompletableFuture taskExecutorIdForPartitionReleaseOrPromote = new CompletableFuture<>(); private JobMaster jobMaster; - TestSetup(TestingRpcService rpcService, FatalErrorHandler fatalErrorHandler, TaskExecutorGateway taskExecutorGateway) throws Exception { + public TestSetup(TestingRpcService rpcService, FatalErrorHandler fatalErrorHandler, TaskExecutorGateway taskExecutorGateway) throws Exception { temporaryFolder.create(); @@ -199,6 +210,7 @@ private static class TestSetup implements AutoCloseable { partitionTracker.setStopTrackingAllPartitionsConsumer(taskExecutorIdForStopTracking::complete); partitionTracker.setStopTrackingAndReleaseAllPartitionsConsumer(taskExecutorIdForPartitionRelease::complete); + partitionTracker.setStopTrackingAndReleaseOrPromotePartitionsConsumer(taskExecutorIdForPartitionReleaseOrPromote::complete); Configuration configuration = new Configuration(); configuration.setString(BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); @@ -260,6 +272,10 @@ public CompletableFuture getReleasePartitionsTargetResourceId() { return taskExecutorIdForPartitionRelease; } + public CompletableFuture getReleaseOrPromotePartitionsTargetResourceId() { + return taskExecutorIdForPartitionReleaseOrPromote; + } + public void close() throws Exception { try { if (jobMaster != null) { From b8500691c6b1e23c61c9408b5562d67bb2fc2336 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 5 Nov 2019 15:30:13 +0100 Subject: [PATCH 464/746] [FLINK-12342][yarn] Remove container requests in order to reduce excess containers This commit changes the order in which the container requests are removed when onContainersAllocated is being called. The idea is to remove the container requests as fast as possible in order to avoid allocating excess containers as described in YARN-1902. This closes #10089. --- .../flink/yarn/YarnResourceManager.java | 75 ++++++++++--------- 1 file changed, 40 insertions(+), 35 deletions(-) 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 3a70086b67a5..1cc093d7bad0 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 @@ -370,47 +370,22 @@ public void onContainersCompleted(final List statuses) { @Override public void onContainersAllocated(List containers) { runAsync(() -> { + log.info("Received {} containers with {} pending container requests.", containers.size(), numPendingContainerRequests); final Collection pendingRequests = getPendingRequests(); final Iterator pendingRequestsIterator = pendingRequests.iterator(); - for (Container container : containers) { - log.info( - "Received new container: {} - Remaining pending container requests: {}", - container.getId(), - numPendingContainerRequests); + // number of allocated containers can be larger than the number of pending container requests + final int numAcceptedContainers = Math.min(containers.size(), numPendingContainerRequests); + final List requiredContainers = containers.subList(0, numAcceptedContainers); + final List excessContainers = containers.subList(numAcceptedContainers, containers.size()); - if (numPendingContainerRequests > 0) { - removeContainerRequest(pendingRequestsIterator.next()); - - final String containerIdStr = container.getId().toString(); - final ResourceID resourceId = new ResourceID(containerIdStr); - - workerNodeMap.put(resourceId, new YarnWorkerNode(container)); - - try { - // Context information used to start a TaskExecutor Java process - ContainerLaunchContext taskExecutorLaunchContext = createTaskExecutorLaunchContext( - container.getResource(), - containerIdStr, - container.getNodeId().getHost()); - - nodeManagerClient.startContainer(container, taskExecutorLaunchContext); - } catch (Throwable t) { - log.error("Could not start TaskManager in container {}.", container.getId(), t); - - // release the failed container - workerNodeMap.remove(resourceId); - resourceManagerClient.releaseAssignedContainer(container.getId()); - // and ask for a new one - requestYarnContainerIfRequired(); - } - } else { - // return the excessive containers - log.info("Returning excess container {}.", container.getId()); - resourceManagerClient.releaseAssignedContainer(container.getId()); - } + for (int i = 0; i < requiredContainers.size(); i++) { + removeContainerRequest(pendingRequestsIterator.next()); } + excessContainers.forEach(this::returnExcessContainer); + requiredContainers.forEach(this::startTaskExecutorInContainer); + // if we are waiting for no further containers, we can go to the // regular heartbeat interval if (numPendingContainerRequests <= 0) { @@ -419,6 +394,36 @@ public void onContainersAllocated(List containers) { }); } + private void startTaskExecutorInContainer(Container container) { + final String containerIdStr = container.getId().toString(); + final ResourceID resourceId = new ResourceID(containerIdStr); + + workerNodeMap.put(resourceId, new YarnWorkerNode(container)); + + try { + // Context information used to start a TaskExecutor Java process + ContainerLaunchContext taskExecutorLaunchContext = createTaskExecutorLaunchContext( + container.getResource(), + containerIdStr, + container.getNodeId().getHost()); + + nodeManagerClient.startContainer(container, taskExecutorLaunchContext); + } catch (Throwable t) { + log.error("Could not start TaskManager in container {}.", container.getId(), t); + + // release the failed container + workerNodeMap.remove(resourceId); + resourceManagerClient.releaseAssignedContainer(container.getId()); + // and ask for a new one + requestYarnContainerIfRequired(); + } + } + + private void returnExcessContainer(Container excessContainer) { + log.info("Returning excess container {}.", excessContainer.getId()); + resourceManagerClient.releaseAssignedContainer(excessContainer.getId()); + } + private void removeContainerRequest(AMRMClient.ContainerRequest pendingContainerRequest) { numPendingContainerRequests--; From 155885cc6d182bf0f6519a9992b21b957444b4fe Mon Sep 17 00:00:00 2001 From: Hwanju Kim Date: Mon, 4 Nov 2019 23:13:03 -0800 Subject: [PATCH 465/746] [FLINK-14589] Redundant slot requests with the same AllocationID leads to inconsistent slot table When a slot request is redundantly made with the same AllocationID to a slot index other than the already allocated one, slot table becomes inconsistent having two slot indices allocated but one AllocationID assigned to only the latest slot index. This can lead to slot leakage. This patch prevents such redundent slot request from rendering inconsistent slot allocation state by rejecting the request. This closes #10099. --- .../taskexecutor/slot/TaskSlotTable.java | 7 ++++- .../taskexecutor/slot/TaskSlotTableTest.java | 28 +++++++++++++++++++ 2 files changed, 34 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java index 41036b3ec86a..2c361a1677b2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java @@ -193,7 +193,12 @@ public SlotReport createSlotReport(ResourceID resourceId) { public boolean allocateSlot(int index, JobID jobId, AllocationID allocationId, Time slotTimeout) { checkInit(); - TaskSlot taskSlot = taskSlots.get(index); + TaskSlot taskSlot = allocationIDTaskSlotMap.get(allocationId); + if (taskSlot != null) { + LOG.info("Allocation ID {} is already allocated in {}.", allocationId, taskSlot); + return false; + } + taskSlot = taskSlots.get(index); boolean result = taskSlot.allocate(jobId, allocationId); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableTest.java index ef3fc06d54a5..46ba55c30c2a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableTest.java @@ -36,6 +36,7 @@ import java.util.Collection; import java.util.Collections; import java.util.HashSet; +import java.util.Iterator; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; @@ -85,6 +86,33 @@ public void testTryMarkSlotActive() throws SlotNotFoundException { } } + /** + * Tests that redundant slot allocation with the same AllocationID to a different slot is rejected. + */ + @Test + public void testRedundantSlotAllocation() { + final TaskSlotTable taskSlotTable = createTaskSlotTable(Collections.nCopies(2, ResourceProfile.UNKNOWN)); + + try { + taskSlotTable.start(new TestingSlotActionsBuilder().build()); + + final JobID jobId = new JobID(); + final AllocationID allocationId = new AllocationID(); + + assertThat(taskSlotTable.allocateSlot(0, jobId, allocationId, SLOT_TIMEOUT), is(true)); + assertThat(taskSlotTable.allocateSlot(1, jobId, allocationId, SLOT_TIMEOUT), is(false)); + + assertThat(taskSlotTable.isAllocated(0, jobId, allocationId), is(true)); + assertThat(taskSlotTable.isSlotFree(1), is(true)); + + Iterator allocatedSlots = taskSlotTable.getAllocatedSlots(jobId); + assertThat(allocatedSlots.next().getIndex(), is(0)); + assertThat(allocatedSlots.hasNext(), is(false)); + } finally { + taskSlotTable.stop(); + } + } + @Nonnull private TaskSlotTable createTaskSlotTable(final Collection resourceProfiles) { return new TaskSlotTable( From 28cf30dbadabf829627781dc8aa05fbacb7ffe5a Mon Sep 17 00:00:00 2001 From: Jiayi Liao Date: Thu, 7 Nov 2019 13:28:12 +0800 Subject: [PATCH 466/746] [FLINK-14608][formats] Remove java stream from JsonRowDeserializationSchema This closes #10107 --- .../json/JsonRowDeserializationSchema.java | 23 ++++++++++++------- 1 file changed, 15 insertions(+), 8 deletions(-) diff --git a/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java index 49fb58850de6..b38934fcaf23 100644 --- a/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java +++ b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java @@ -39,6 +39,7 @@ import java.io.IOException; import java.io.Serializable; +import java.lang.reflect.Array; import java.sql.Date; import java.sql.Time; import java.sql.Timestamp; @@ -59,8 +60,6 @@ import static java.lang.String.format; import static java.time.format.DateTimeFormatter.ISO_LOCAL_DATE; -import static java.util.Spliterators.spliterator; -import static java.util.stream.StreamSupport.stream; import static org.apache.flink.formats.json.TimeFormats.RFC3339_TIMESTAMP_FORMAT; import static org.apache.flink.formats.json.TimeFormats.RFC3339_TIME_FORMAT; import static org.apache.flink.util.Preconditions.checkArgument; @@ -289,7 +288,7 @@ private boolean isPrimitiveByteArray(TypeInformation typeInfo) { private DeserializationRuntimeConverter createObjectArrayConverter(TypeInformation elementTypeInfo) { DeserializationRuntimeConverter elementConverter = createConverter(elementTypeInfo); - return assembleArrayConverter(elementConverter); + return assembleArrayConverter(elementTypeInfo, elementConverter); } private DeserializationRuntimeConverter createRowConverter(RowTypeInfo typeInfo) { @@ -429,13 +428,21 @@ private Object convertField( } } - private DeserializationRuntimeConverter assembleArrayConverter(DeserializationRuntimeConverter elementConverter) { + private DeserializationRuntimeConverter assembleArrayConverter( + TypeInformation elementType, + DeserializationRuntimeConverter elementConverter) { + + final Class elementClass = elementType.getTypeClass(); + return (mapper, jsonNode) -> { - ArrayNode node = (ArrayNode) jsonNode; + final ArrayNode node = (ArrayNode) jsonNode; + final Object[] array = (Object[]) Array.newInstance(elementClass, node.size()); + for (int i = 0; i < node.size(); i++) { + final JsonNode innerNode = node.get(i); + array[i] = elementConverter.convert(mapper, innerNode); + } - return stream(spliterator(node.elements(), node.size(), 0), false) - .map(innerNode -> elementConverter.convert(mapper, innerNode)) - .toArray(); + return array; }; } } From 59339ac185cb80c926e0cf00b7b31dc29df2a18f Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Fri, 1 Nov 2019 12:42:42 +0100 Subject: [PATCH 467/746] [FLINK-13702][flink-table-planner] Fix NullSerializer & NullAwareMapSerializer snapshotting --- flink-table/flink-table-common/pom.xml | 14 +++ .../table/dataview/ListViewTypeInfo.java | 2 +- .../flink/table/dataview/MapViewTypeInfo.java | 2 +- .../dataview/NullAwareMapSerializer.java | 2 +- .../NullAwareMapSerializerSnapshot.java | 69 ++++++++++++++ .../flink/table/dataview/NullSerializer.java | 17 +++- .../dataview/NullAwareMapSerializerTest.java | 95 +++++++++++++++++++ 7 files changed, 197 insertions(+), 4 deletions(-) create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/dataview/NullAwareMapSerializerSnapshot.java create mode 100644 flink-table/flink-table-common/src/test/java/org/apache/flink/table/dataview/NullAwareMapSerializerTest.java diff --git a/flink-table/flink-table-common/pom.xml b/flink-table/flink-table-common/pom.xml index 926d086194bb..c0f112b948aa 100644 --- a/flink-table/flink-table-common/pom.xml +++ b/flink-table/flink-table-common/pom.xml @@ -43,6 +43,20 @@ under the License. flink-core ${project.version} + + + org.apache.flink + flink-core + ${project.version} + test-jar + test + + + org.apache.flink + flink-test-utils-junit + ${project.version} + test + diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/dataview/ListViewTypeInfo.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/dataview/ListViewTypeInfo.java index 951b1402c82a..93513301f20e 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/dataview/ListViewTypeInfo.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/dataview/ListViewTypeInfo.java @@ -94,7 +94,7 @@ public boolean isKeyType() { @Override public TypeSerializer> createSerializer(ExecutionConfig config) { if (nullSerializer) { - return (TypeSerializer>) (TypeSerializer) new NullSerializer(); + return (TypeSerializer>) (TypeSerializer) NullSerializer.INSTANCE; } else { TypeSerializer elementSerializer = elementType.createSerializer(config); return new ListViewSerializer<>(new ListSerializer<>(elementSerializer)); diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/dataview/MapViewTypeInfo.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/dataview/MapViewTypeInfo.java index af40f28f2ec6..cf4fb097b8be 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/dataview/MapViewTypeInfo.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/dataview/MapViewTypeInfo.java @@ -114,7 +114,7 @@ public boolean isKeyType() { @Override public TypeSerializer> createSerializer(ExecutionConfig config) { if (nullSerializer) { - return (TypeSerializer>) (TypeSerializer) new NullSerializer(); + return (TypeSerializer>) (TypeSerializer) NullSerializer.INSTANCE; } else { TypeSerializer keySer = keyType.createSerializer(config); TypeSerializer valueSer = valueType.createSerializer(config); diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/dataview/NullAwareMapSerializer.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/dataview/NullAwareMapSerializer.java index dc0b691124ea..896d4d7a1f38 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/dataview/NullAwareMapSerializer.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/dataview/NullAwareMapSerializer.java @@ -175,6 +175,6 @@ public int hashCode() { @Override public TypeSerializerSnapshot> snapshotConfiguration() { - throw new UnsupportedOperationException(); + return new NullAwareMapSerializerSnapshot<>(this); } } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/dataview/NullAwareMapSerializerSnapshot.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/dataview/NullAwareMapSerializerSnapshot.java new file mode 100644 index 000000000000..fa3df1e6a2f8 --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/dataview/NullAwareMapSerializerSnapshot.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.dataview; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; + +import java.util.Map; + +/** + * A {@link TypeSerializerSnapshot} for the {@link NullAwareMapSerializer}. + * + * @param the key type of the map entries. + * @param the value type of the map entries. + */ +@Internal +public class NullAwareMapSerializerSnapshot extends CompositeTypeSerializerSnapshot, NullAwareMapSerializer> { + private static final int CURRENT_VERSION = 1; + + /** + * Constructor for read instantiation. + */ + public NullAwareMapSerializerSnapshot() { + super(NullAwareMapSerializer.class); + } + + /** + * Constructor to create the snapshot for writing. + */ + public NullAwareMapSerializerSnapshot(NullAwareMapSerializer mapViewSerializer) { + super(mapViewSerializer); + } + + @Override + public int getCurrentOuterSnapshotVersion() { + return CURRENT_VERSION; + } + + @Override + @SuppressWarnings("unchecked") + protected NullAwareMapSerializer createOuterSerializerWithNestedSerializers(TypeSerializer[] nestedSerializers) { + TypeSerializer keySerializer = (TypeSerializer) nestedSerializers[0]; + TypeSerializer valueSerializer = (TypeSerializer) nestedSerializers[1]; + return new NullAwareMapSerializer<>(keySerializer, valueSerializer); + } + + @Override + protected TypeSerializer[] getNestedSerializers(NullAwareMapSerializer outerSerializer) { + return new TypeSerializer[]{outerSerializer.getKeySerializer(), outerSerializer.getValueSerializer()}; + } +} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/dataview/NullSerializer.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/dataview/NullSerializer.java index 8964d957f93b..f9809c194e43 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/dataview/NullSerializer.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/dataview/NullSerializer.java @@ -19,6 +19,7 @@ package org.apache.flink.table.dataview; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; import org.apache.flink.core.memory.DataInputView; @@ -33,6 +34,10 @@ public class NullSerializer extends TypeSerializerSingleton { private static final long serialVersionUID = -5381596724707742625L; + public static final NullSerializer INSTANCE = new NullSerializer(); + + private NullSerializer() {} + @Override public boolean isImmutableType() { return true; @@ -81,6 +86,16 @@ public void copy(DataInputView source, DataOutputView target) throws IOException @Override public TypeSerializerSnapshot snapshotConfiguration() { - throw new UnsupportedOperationException(); + return new NullSerializerSnapshot(); + } + + /** + * Serializer configuration snapshot for compatibility and format evolution. + */ + @SuppressWarnings("WeakerAccess") + public static final class NullSerializerSnapshot extends SimpleTypeSerializerSnapshot { + public NullSerializerSnapshot() { + super(() -> NullSerializer.INSTANCE); + } } } diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/dataview/NullAwareMapSerializerTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/dataview/NullAwareMapSerializerTest.java new file mode 100644 index 000000000000..869a470113d2 --- /dev/null +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/dataview/NullAwareMapSerializerTest.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.dataview; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Random; +import java.util.TreeMap; + +/** + * Tests for {@link NullAwareMapSerializer}. + */ +public class NullAwareMapSerializerTest extends SerializerTestBase> { + @Override + protected TypeSerializer> createSerializer() { + return new NullAwareMapSerializer<>(LongSerializer.INSTANCE, StringSerializer.INSTANCE); + } + + @Override + protected int getLength() { + return -1; + } + + @Override + @SuppressWarnings("unchecked") + protected Class> getTypeClass() { + return (Class>) (Class) Map.class; + } + + @Override + @SuppressWarnings("unchecked") + protected Map[] getTestData() { + final Random rnd = new Random(123654789); + + // empty maps + final Map map1 = Collections.emptyMap(); + final Map map2 = new HashMap<>(); + final Map map3 = new TreeMap<>(); + + // single element maps + final Map map4 = Collections.singletonMap(0L, "hello"); + final Map map5 = new HashMap<>(); + map5.put(12345L, "12345L"); + final Map map6 = new TreeMap<>(); + map6.put(777888L, "777888L"); + + // longer maps + final Map map7 = new HashMap<>(); + for (int i = 0; i < rnd.nextInt(200); i++) { + map7.put(rnd.nextLong(), Long.toString(rnd.nextLong())); + } + + final Map map8 = new TreeMap<>(); + for (int i = 0; i < rnd.nextInt(200); i++) { + map8.put(rnd.nextLong(), Long.toString(rnd.nextLong())); + } + + // null-value maps + final Map map9 = Collections.singletonMap(0L, null); + final Map map10 = new HashMap<>(); + map10.put(999L, null); + final Map map11 = new TreeMap<>(); + map11.put(666L, null); + + // null-key maps + final Map map12 = Collections.singletonMap(null, ""); + + return (Map[]) new Map[] { + map1, map2, map3, map4, map5, map6, map7, map8, map9, map10, map11, map12 + }; + + } +} From dcb29bd072bdca3a2a576a0a9fe771798d6b364f Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Fri, 1 Nov 2019 12:43:12 +0100 Subject: [PATCH 468/746] [FLINK-13702][flink-table-planner] Fix accumulator type of CollectAggFunction --- .../aggfunctions/CollectAggFunction.java | 22 +++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/CollectAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/CollectAggFunction.java index aaad3ed063a1..95049b0c4d9b 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/CollectAggFunction.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/CollectAggFunction.java @@ -18,13 +18,20 @@ package org.apache.flink.table.planner.functions.aggfunctions; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.typeutils.MapTypeInfo; +import org.apache.flink.api.java.typeutils.PojoField; +import org.apache.flink.api.java.typeutils.PojoTypeInfo; import org.apache.flink.table.api.dataview.MapView; +import org.apache.flink.table.dataview.MapViewTypeInfo; import org.apache.flink.table.functions.AggregateFunction; +import org.apache.flink.util.WrappingRuntimeException; +import java.util.ArrayList; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Objects; @@ -128,4 +135,19 @@ public Map getValue(CollectAccumulator accumulator) { public TypeInformation> getResultType() { return new MapTypeInfo<>(elementType, Types.INT); } + + @Override + @SuppressWarnings("unchecked") + public TypeInformation> getAccumulatorType() { + try { + Class> clazz = (Class>) (Class) CollectAccumulator.class; + List pojoFields = new ArrayList<>(); + pojoFields.add(new PojoField( + clazz.getDeclaredField("map"), + new MapViewTypeInfo<>(elementType, BasicTypeInfo.INT_TYPE_INFO))); + return new PojoTypeInfo<>(clazz, pojoFields); + } catch (NoSuchFieldException e) { + throw new WrappingRuntimeException(e); + } + } } From 960ac40d4bfde146ec83c835f51d877c72725dbd Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Fri, 1 Nov 2019 12:44:06 +0100 Subject: [PATCH 469/746] [FLINK-13702][flink-table-planner] Fix digest of GenericRelDataType --- .../flink/table/planner/plan/schema/GenericRelDataType.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/GenericRelDataType.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/GenericRelDataType.scala index ecea88a5541f..2143fa7e08f1 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/GenericRelDataType.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/GenericRelDataType.scala @@ -64,6 +64,6 @@ class GenericRelDataType( * [[ArraySqlType]], [[MapSqlType]]... use generateTypeString to equals and hashcode. */ override def generateTypeString(sb: lang.StringBuilder, withDetail: Boolean): Unit = { - sb.append(toString) + sb.append(s"ANY('${genericType.getTypeInformation}')") } } From 0b28e830d7366126a91ca9faa38cb19a8f66a9b6 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Fri, 1 Nov 2019 15:08:38 +0100 Subject: [PATCH 470/746] [FLINK-13702][table-planner-blink] Fixed BinaryGeneric & BinaryString materialization --- .../FirstValueWithRetractAggFunction.java | 6 +- .../LastValueWithRetractAggFunction.java | 4 +- .../table/planner/codegen/CodeGenUtils.scala | 5 +- .../codegen/SortCodeGeneratorTest.java | 16 +- .../dataformat/AbstractBinaryWriter.java | 31 +-- .../flink/table/dataformat/BinaryArray.java | 6 +- .../flink/table/dataformat/BinaryFormat.java | 87 ++----- .../flink/table/dataformat/BinaryGeneric.java | 49 ++-- .../flink/table/dataformat/BinaryMap.java | 12 +- .../flink/table/dataformat/BinaryRow.java | 8 +- .../flink/table/dataformat/BinarySection.java | 74 ++++++ .../flink/table/dataformat/BinaryString.java | 239 +++++++++++------- .../table/dataformat/BinaryStringUtil.java | 119 ++++----- .../flink/table/dataformat/BinaryWriter.java | 5 +- .../dataformat/DataFormatConverters.java | 2 +- .../table/dataformat/LazyBinaryFormat.java | 73 +++--- .../flink/table/dataformat/NestedRow.java | 5 +- .../typeutils/BinaryGenericSerializer.java | 76 +++--- .../flink/table/dataformat/BaseRowTest.java | 11 +- .../table/dataformat/BinaryArrayTest.java | 10 +- .../flink/table/dataformat/BinaryRowTest.java | 63 +++-- .../table/dataformat/BinaryStringTest.java | 17 +- .../flink/table/dataformat/NestedRowTest.java | 10 +- .../typeutils/BaseArraySerializerTest.java | 2 +- .../typeutils/BaseMapSerializerTest.java | 2 +- .../BinaryGenericSerializerTest.java | 21 +- .../table/utils/BinaryGenericAsserter.java | 78 ++++++ 27 files changed, 607 insertions(+), 424 deletions(-) create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinarySection.java create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/utils/BinaryGenericAsserter.java diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/FirstValueWithRetractAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/FirstValueWithRetractAggFunction.java index 7bcbbc246b17..a81d1d86b464 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/FirstValueWithRetractAggFunction.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/FirstValueWithRetractAggFunction.java @@ -70,11 +70,9 @@ public GenericRow createAccumulator() { acc.setField(0, null); acc.setField(1, null); acc.setField(2, new BinaryGeneric<>( - new MapView<>(getResultType(), new ListTypeInfo<>(Types.LONG)), - getValueToOrderMapViewSerializer())); + new MapView<>(getResultType(), new ListTypeInfo<>(Types.LONG)))); acc.setField(3, new BinaryGeneric<>( - new MapView<>(Types.LONG, new ListTypeInfo<>(getResultType())), - getOrderToValueMapViewSerializer())); + new MapView<>(Types.LONG, new ListTypeInfo<>(getResultType())))); return acc; } diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/LastValueWithRetractAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/LastValueWithRetractAggFunction.java index 97f3c998b8d9..cd250a4eb709 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/LastValueWithRetractAggFunction.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/LastValueWithRetractAggFunction.java @@ -70,9 +70,9 @@ public GenericRow createAccumulator() { acc.setField(0, null); acc.setField(1, null); acc.setField(2, new BinaryGeneric<>( - new MapView<>(getResultType(), new ListTypeInfo<>(Types.LONG)), getValueToOrderMapViewSerializer())); + new MapView<>(getResultType(), new ListTypeInfo<>(Types.LONG)))); acc.setField(3, new BinaryGeneric<>( - new MapView<>(Types.LONG, new ListTypeInfo<>(getResultType())), getOrderToValueMapViewSerializer())); + new MapView<>(Types.LONG, new ListTypeInfo<>(getResultType())))); return acc; } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala index 473b78861eca..2d2b43c55254 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala @@ -655,8 +655,9 @@ object CodeGenUtils { case ROW => val ser = ctx.addReusableTypeSerializer(t) s"$writerTerm.writeRow($indexTerm, $fieldValTerm, $ser)" - - case ANY => s"$writerTerm.writeGeneric($indexTerm, $fieldValTerm)" + case ANY => + val ser = ctx.addReusableTypeSerializer(t) + s"$writerTerm.writeGeneric($indexTerm, $fieldValTerm, $ser)" } private def isConverterIdentity(t: DataType): Boolean = { diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/codegen/SortCodeGeneratorTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/codegen/SortCodeGeneratorTest.java index c055078e2700..5e1aeaf7b4ca 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/codegen/SortCodeGeneratorTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/codegen/SortCodeGeneratorTest.java @@ -48,6 +48,7 @@ import org.apache.flink.table.runtime.operators.sort.BinaryInMemorySortBuffer; import org.apache.flink.table.runtime.types.InternalSerializers; import org.apache.flink.table.runtime.typeutils.AbstractRowSerializer; +import org.apache.flink.table.runtime.typeutils.BinaryGenericSerializer; import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.ArrayType; @@ -82,6 +83,8 @@ import java.util.concurrent.ThreadLocalRandom; import static org.apache.flink.table.types.logical.LogicalTypeRoot.INTEGER; +import static org.apache.flink.table.utils.BinaryGenericAsserter.equivalent; +import static org.junit.Assert.assertThat; /** * Random test for sort code generator. @@ -262,7 +265,7 @@ private Object[] generateValues(LogicalType type) { } break; case ANY: - seeds[i] = new BinaryGeneric<>(rnd.nextInt(), IntSerializer.INSTANCE); + seeds[i] = new BinaryGeneric<>(rnd.nextInt()); break; default: throw new RuntimeException("Not support!"); @@ -314,7 +317,7 @@ private Object value1(LogicalType type, Random rnd) { case ROW: return GenericRow.of(new Object[]{null}); case ANY: - return new BinaryGeneric<>(rnd.nextInt(), IntSerializer.INSTANCE); + return new BinaryGeneric<>(rnd.nextInt()); default: throw new RuntimeException("Not support!"); } @@ -355,7 +358,7 @@ private Object value2(LogicalType type, Random rnd) { return GenericRow.of(GenericRow.of(new Object[]{null})); } case ANY: - return new BinaryGeneric<>(rnd.nextInt(), IntSerializer.INSTANCE); + return new BinaryGeneric<>(rnd.nextInt()); default: throw new RuntimeException("Not support!"); } @@ -396,7 +399,7 @@ private Object value3(LogicalType type, Random rnd) { return GenericRow.of(GenericRow.of(rnd.nextInt())); } case ANY: - return new BinaryGeneric<>(rnd.nextInt(), IntSerializer.INSTANCE); + return new BinaryGeneric<>(rnd.nextInt()); default: throw new RuntimeException("Not support!"); } @@ -562,6 +565,11 @@ private void testInner() throws Exception { Object o2 = TypeGetterSetters.get(result.get(i), keys[j], keyTypes[j]); if (keyTypes[j] instanceof VarBinaryType) { Assert.assertArrayEquals(msg, (byte[]) o1, (byte[]) o2); + } else if (keyTypes[j] instanceof TypeInformationAnyType) { + assertThat( + msg, + (BinaryGeneric) o1, + equivalent((BinaryGeneric) o2, new BinaryGenericSerializer<>(IntSerializer.INSTANCE))); } else { Assert.assertEquals(msg, o1, o2); } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/AbstractBinaryWriter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/AbstractBinaryWriter.java index 5091c3cddef0..6499528de45c 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/AbstractBinaryWriter.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/AbstractBinaryWriter.java @@ -17,12 +17,14 @@ package org.apache.flink.table.dataformat; +import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.core.memory.MemorySegmentFactory; import org.apache.flink.table.runtime.typeutils.BaseArraySerializer; import org.apache.flink.table.runtime.typeutils.BaseMapSerializer; import org.apache.flink.table.runtime.typeutils.BaseRowSerializer; +import org.apache.flink.table.runtime.typeutils.BinaryGenericSerializer; import org.apache.flink.table.runtime.util.SegmentsUtil; import java.io.IOException; @@ -30,8 +32,6 @@ import java.nio.charset.StandardCharsets; import java.util.Arrays; -import static org.apache.flink.table.dataformat.BinaryFormat.MAX_FIX_PART_DATA_SIZE; - /** * Use the special format to write data to a {@link MemorySegment} (its capacity grows * automatically). @@ -91,7 +91,7 @@ public void writeString(int pos, BinaryString input) { private void writeBytes(int pos, byte[] bytes) { int len = bytes.length; - if (len <= MAX_FIX_PART_DATA_SIZE) { + if (len <= BinaryFormat.MAX_FIX_PART_DATA_SIZE) { writeBytesToFixLenPart(segment, getFieldOffset(pos), bytes, len); } else { writeBytesToVarLenPart(pos, bytes, len); @@ -118,24 +118,11 @@ private DataOutputViewStreamWrapper getOutputView() { } @Override - public void writeGeneric(int pos, BinaryGeneric input) { - if (input.getSegments() == null) { - int beforeCursor = cursor; - try { - input.getJavaObjectSerializer().serialize(input.getJavaObject(), getOutputView()); - } catch (IOException e) { - throw new RuntimeException(e); - } - int size = cursor - beforeCursor; - final int roundedSize = roundNumberOfBytesToNearestWord(size); - int paddingBytes = roundedSize - size; - ensureCapacity(paddingBytes); - setOffsetAndSize(pos, beforeCursor, size); - zeroBytes(cursor, paddingBytes); - cursor += paddingBytes; - } else { - writeSegmentsToVarLenPart(pos, input.getSegments(), input.getOffset(), input.getSizeInBytes()); - } + @SuppressWarnings("unchecked") + public void writeGeneric(int pos, BinaryGeneric input, BinaryGenericSerializer serializer) { + TypeSerializer innerSerializer = serializer.getInnerSerializer(); + input.ensureMaterialized(innerSerializer); + writeSegmentsToVarLenPart(pos, input.getSegments(), input.getOffset(), input.getSizeInBytes()); } @Override @@ -152,7 +139,7 @@ public void writeRow(int pos, BaseRow input, BaseRowSerializer serializer) { @Override public void writeBinary(int pos, byte[] bytes) { int len = bytes.length; - if (len <= MAX_FIX_PART_DATA_SIZE) { + if (len <= BinaryFormat.MAX_FIX_PART_DATA_SIZE) { writeBytesToFixLenPart(segment, getFieldOffset(pos), bytes, len); } else { writeBytesToVarLenPart(pos, bytes, len); diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArray.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArray.java index acf5adc32c92..523d96aced0d 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArray.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArray.java @@ -35,7 +35,7 @@ * *

    {@code BinaryArray} are influenced by Apache Spark UnsafeArrayData. */ -public final class BinaryArray extends BinaryFormat implements BaseArray { +public final class BinaryArray extends BinarySection implements BaseArray { /** * Offset for Arrays. @@ -174,7 +174,7 @@ public BinaryString getString(int pos) { assertIndexIsValid(pos); int fieldOffset = getElementOffset(pos, 8); final long offsetAndSize = SegmentsUtil.getLong(segments, fieldOffset); - return BinaryString.readBinaryStringFieldFromSegments( + return BinaryFormat.readBinaryStringFieldFromSegments( segments, offset, fieldOffset, offsetAndSize); } @@ -205,7 +205,7 @@ public byte[] getBinary(int pos) { assertIndexIsValid(pos); int fieldOffset = getElementOffset(pos, 8); final long offsetAndSize = SegmentsUtil.getLong(segments, fieldOffset); - return readBinaryFieldFromSegments( + return BinaryFormat.readBinaryFieldFromSegments( segments, offset, fieldOffset, offsetAndSize); } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryFormat.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryFormat.java index 2a6ccc4e9da4..a95ac03469c2 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryFormat.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryFormat.java @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this 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, @@ -21,10 +22,9 @@ import org.apache.flink.table.runtime.util.SegmentsUtil; /** - * Binary format that in {@link MemorySegment}s. + * Binary format spanning {@link MemorySegment}s. */ -public abstract class BinaryFormat { - +public interface BinaryFormat { /** * It decides whether to put data in FixLenPart or VarLenPart. See more in {@link BinaryRow}. * @@ -36,8 +36,7 @@ public abstract class BinaryFormat { * 1-bit mark(1) = 0, 31-bits offset to the data, and 4-bytes length of data. * Data is stored in variable-length part. */ - static final int MAX_FIX_PART_DATA_SIZE = 7; - + int MAX_FIX_PART_DATA_SIZE = 7; /** * To get the mark in highest bit of long. * Form: 10000000 00000000 ... (8 bytes) @@ -45,8 +44,7 @@ public abstract class BinaryFormat { *

    This is used to decide whether the data is stored in fixed-length part or variable-length * part. see {@link #MAX_FIX_PART_DATA_SIZE} for more information. */ - private static final long HIGHEST_FIRST_BIT = 0x80L << 56; - + long HIGHEST_FIRST_BIT = 0x80L << 56; /** * To get the 7 bits length in second bit to eighth bit out of a long. * Form: 01111111 00000000 ... (8 bytes) @@ -54,58 +52,13 @@ public abstract class BinaryFormat { *

    This is used to get the length of the data which is stored in this long. * see {@link #MAX_FIX_PART_DATA_SIZE} for more information. */ - private static final long HIGHEST_SECOND_TO_EIGHTH_BIT = 0x7FL << 56; - - protected MemorySegment[] segments; - protected int offset; - protected int sizeInBytes; - - public BinaryFormat() {} - - public BinaryFormat(MemorySegment[] segments, int offset, int sizeInBytes) { - this.segments = segments; - this.offset = offset; - this.sizeInBytes = sizeInBytes; - } - - public final void pointTo(MemorySegment segment, int offset, int sizeInBytes) { - pointTo(new MemorySegment[] {segment}, offset, sizeInBytes); - } - - public void pointTo(MemorySegment[] segments, int offset, int sizeInBytes) { - this.segments = segments; - this.offset = offset; - this.sizeInBytes = sizeInBytes; - } - - public MemorySegment[] getSegments() { - return segments; - } - - public int getOffset() { - return offset; - } + long HIGHEST_SECOND_TO_EIGHTH_BIT = 0x7FL << 56; - public int getSizeInBytes() { - return sizeInBytes; - } + MemorySegment[] getSegments(); - @Override - public boolean equals(Object o) { - return this == o || o != null && - getClass() == o.getClass() && - binaryEquals((BinaryFormat) o); - } + int getOffset(); - protected boolean binaryEquals(BinaryFormat that) { - return sizeInBytes == that.sizeInBytes && - SegmentsUtil.equals(segments, offset, that.segments, that.offset, sizeInBytes); - } - - @Override - public int hashCode() { - return SegmentsUtil.hash(segments, offset, sizeInBytes); - } + int getSizeInBytes(); /** * Get binary, if len less than 8, will be include in variablePartOffsetAndLen. @@ -117,7 +70,9 @@ public int hashCode() { * @param variablePartOffsetAndLen a long value, real data or offset and len. */ static byte[] readBinaryFieldFromSegments( - MemorySegment[] segments, int baseOffset, int fieldOffset, + MemorySegment[] segments, + int baseOffset, + int fieldOffset, long variablePartOffsetAndLen) { long mark = variablePartOffsetAndLen & HIGHEST_FIRST_BIT; if (mark == 0) { @@ -145,7 +100,9 @@ static byte[] readBinaryFieldFromSegments( * @param variablePartOffsetAndLen a long value, real data or offset and len. */ static BinaryString readBinaryStringFieldFromSegments( - MemorySegment[] segments, int baseOffset, int fieldOffset, + MemorySegment[] segments, + int baseOffset, + int fieldOffset, long variablePartOffsetAndLen) { long mark = variablePartOffsetAndLen & HIGHEST_FIRST_BIT; if (mark == 0) { diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryGeneric.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryGeneric.java index 390e43b68bdc..5200f36e08d1 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryGeneric.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryGeneric.java @@ -31,48 +31,28 @@ */ public final class BinaryGeneric extends LazyBinaryFormat { - private final TypeSerializer javaObjectSer; - - public BinaryGeneric(T javaObject, TypeSerializer javaObjectSer) { + public BinaryGeneric(T javaObject) { super(javaObject); - this.javaObjectSer = javaObjectSer; } - public BinaryGeneric(MemorySegment[] segments, int offset, int sizeInBytes, - TypeSerializer javaObjectSer) { + public BinaryGeneric(MemorySegment[] segments, int offset, int sizeInBytes) { super(segments, offset, sizeInBytes); - this.javaObjectSer = javaObjectSer; } - public BinaryGeneric(MemorySegment[] segments, int offset, int sizeInBytes, T javaObject, - TypeSerializer javaObjectSer) { + public BinaryGeneric(MemorySegment[] segments, int offset, int sizeInBytes, T javaObject) { super(segments, offset, sizeInBytes, javaObject); - this.javaObjectSer = javaObjectSer; - } - - public TypeSerializer getJavaObjectSerializer() { - return javaObjectSer; } @Override - public void materialize() { + protected BinarySection materialize(TypeSerializer serializer) { try { - byte[] bytes = InstantiationUtil.serializeToByteArray(javaObjectSer, javaObject); - pointTo(new MemorySegment[] {MemorySegmentFactory.wrap(bytes)}, 0, bytes.length); + byte[] bytes = InstantiationUtil.serializeToByteArray(serializer, javaObject); + return new BinarySection(new MemorySegment[] {MemorySegmentFactory.wrap(bytes)}, 0, bytes.length); } catch (IOException e) { throw new RuntimeException(e); } } - public BinaryGeneric copy() { - ensureMaterialized(); - byte[] bytes = SegmentsUtil.copyToBytes(segments, offset, sizeInBytes); - T newJavaObject = javaObject == null ? null : javaObjectSer.copy(javaObject); - return new BinaryGeneric<>(new MemorySegment[] {MemorySegmentFactory.wrap(bytes)}, 0, sizeInBytes, - newJavaObject, - javaObjectSer); - } - static BinaryGeneric readBinaryGenericFieldFromSegments( MemorySegment[] segments, int baseOffset, long offsetAndSize) { final int size = ((int) offsetAndSize); @@ -91,4 +71,21 @@ public static T getJavaObjectFromBinaryGeneric(BinaryGeneric value, TypeS } return value.getJavaObject(); } + + @Override + public boolean equals(Object o) { + throw new UnsupportedOperationException("BinaryGeneric cannot be compared"); + } + + @Override + public int hashCode() { + throw new UnsupportedOperationException("BinaryGeneric does not have a hashCode"); + } + + @Override + public String toString() { + return "BinaryGeneric{" + + "javaObject=" + javaObject + + '}'; + } } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryMap.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryMap.java index 8660feea9c5e..679cff9c73a5 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryMap.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryMap.java @@ -34,7 +34,7 @@ * *

    {@code BinaryMap} are influenced by Apache Spark UnsafeMapData. */ -public final class BinaryMap extends BinaryFormat implements BaseMap { +public final class BinaryMap extends BinarySection implements BaseMap { private final BinaryArray keys; private final BinaryArray values; @@ -102,13 +102,13 @@ public int hashCode() { } public static BinaryMap valueOf(BinaryArray key, BinaryArray value) { - checkArgument(key.getSegments().length == 1 && value.getSegments().length == 1); - byte[] bytes = new byte[4 + key.getSizeInBytes() + value.getSizeInBytes()]; + checkArgument(key.segments.length == 1 && value.getSegments().length == 1); + byte[] bytes = new byte[4 + key.sizeInBytes + value.sizeInBytes]; MemorySegment segment = MemorySegmentFactory.wrap(bytes); - segment.putInt(0, key.getSizeInBytes()); - key.getSegments()[0].copyTo(key.getOffset(), segment, 4, key.getSizeInBytes()); + segment.putInt(0, key.sizeInBytes); + key.getSegments()[0].copyTo(key.getOffset(), segment, 4, key.sizeInBytes); value.getSegments()[0].copyTo( - value.getOffset(), segment, 4 + key.getSizeInBytes(), value.getSizeInBytes()); + value.getOffset(), segment, 4 + key.sizeInBytes, value.sizeInBytes); BinaryMap map = new BinaryMap(); map.pointTo(segment, 0, bytes.length); return map; diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryRow.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryRow.java index bc98c43f1ede..75f7e24cb088 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryRow.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryRow.java @@ -52,7 +52,7 @@ * The difference is that BinaryRow is placed on a discontinuous memory, and the variable length * type can also be placed on a fixed length area (If it's short enough). */ -public final class BinaryRow extends BinaryFormat implements BaseRow { +public final class BinaryRow extends BinarySection implements BaseRow { public static final boolean LITTLE_ENDIAN = (ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN); private static final long FIRST_BYTE_ZERO = LITTLE_ENDIAN ? ~0xFFL : ~(0xFFL << 56L); @@ -287,7 +287,7 @@ public BinaryString getString(int pos) { assertIndexIsValid(pos); int fieldOffset = getFieldOffset(pos); final long offsetAndLen = segments[0].getLong(fieldOffset); - return BinaryString.readBinaryStringFieldFromSegments(segments, offset, fieldOffset, offsetAndLen); + return BinaryFormat.readBinaryStringFieldFromSegments(segments, offset, fieldOffset, offsetAndLen); } @Override @@ -315,7 +315,7 @@ public byte[] getBinary(int pos) { assertIndexIsValid(pos); int fieldOffset = getFieldOffset(pos); final long offsetAndLen = segments[0].getLong(fieldOffset); - return readBinaryFieldFromSegments(segments, offset, fieldOffset, offsetAndLen); + return BinaryFormat.readBinaryFieldFromSegments(segments, offset, fieldOffset, offsetAndLen); } @Override @@ -411,7 +411,7 @@ public boolean equalsWithoutHeader(BaseRow o) { } private boolean equalsFrom(Object o, int startIndex) { - if (o != null && o instanceof BinaryRow) { + if (o instanceof BinaryRow) { BinaryRow other = (BinaryRow) o; return sizeInBytes == other.sizeInBytes && SegmentsUtil.equals( diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinarySection.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinarySection.java new file mode 100644 index 000000000000..afde68b721bd --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinarySection.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.table.dataformat; + +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.table.runtime.util.SegmentsUtil; + +/** + * Binary section of memory. + */ +public class BinarySection implements BinaryFormat { + + protected MemorySegment[] segments; + protected int offset; + protected int sizeInBytes; + + public BinarySection() {} + + public BinarySection(MemorySegment[] segments, int offset, int sizeInBytes) { + this.segments = segments; + this.offset = offset; + this.sizeInBytes = sizeInBytes; + } + + public final void pointTo(MemorySegment segment, int offset, int sizeInBytes) { + pointTo(new MemorySegment[] {segment}, offset, sizeInBytes); + } + + public void pointTo(MemorySegment[] segments, int offset, int sizeInBytes) { + this.segments = segments; + this.offset = offset; + this.sizeInBytes = sizeInBytes; + } + + public MemorySegment[] getSegments() { + return segments; + } + + public int getOffset() { + return offset; + } + + public int getSizeInBytes() { + return sizeInBytes; + } + + @Override + public boolean equals(Object o) { + return this == o || o != null && + getClass() == o.getClass() && + sizeInBytes == ((BinarySection) o).sizeInBytes && + SegmentsUtil.equals(segments, offset, ((BinarySection) o).segments, ((BinarySection) o).offset, sizeInBytes); + } + + @Override + public int hashCode() { + return SegmentsUtil.hash(segments, offset, sizeInBytes); + } +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryString.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryString.java index f3db1ebf66c8..78240a68ba32 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryString.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryString.java @@ -18,6 +18,7 @@ package org.apache.flink.table.dataformat; import org.apache.flink.api.common.typeinfo.TypeInfo; +import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.core.memory.MemorySegmentFactory; import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfoFactory; @@ -116,7 +117,7 @@ public int numChars() { ensureMaterialized(); if (inFirstSegment()) { int len = 0; - for (int i = 0; i < sizeInBytes; i += numBytesForFirstByte(getByteOneSegment(i))) { + for (int i = 0; i < binarySection.sizeInBytes; i += numBytesForFirstByte(getByteOneSegment(i))) { len++; } return len; @@ -127,10 +128,10 @@ public int numChars() { private int numCharsMultiSegs() { int len = 0; - int segSize = segments[0].size(); + int segSize = binarySection.segments[0].size(); SegmentAndOffset index = firstSegmentAndOffset(segSize); int i = 0; - while (i < sizeInBytes) { + while (i < binarySection.sizeInBytes) { int charBytes = numBytesForFirstByte(index.value()); i += charBytes; len++; @@ -141,7 +142,7 @@ private int numCharsMultiSegs() { /** * Returns the {@code byte} value at the specified index. An index ranges from {@code 0} to - * {@code getSizeInBytes() - 1}. + * {@code binarySection.sizeInBytes - 1}. * * @param index the index of the {@code byte} value. * @return the {@code byte} value at the specified index of this UTF-8 bytes. @@ -151,12 +152,12 @@ private int numCharsMultiSegs() { */ public byte byteAt(int index) { ensureMaterialized(); - int globalOffset = offset + index; - int size = segments[0].size(); + int globalOffset = binarySection.offset + index; + int size = binarySection.segments[0].size(); if (globalOffset < size) { - return segments[0].get(globalOffset); + return binarySection.segments[0].get(globalOffset); } else { - return segments[globalOffset / size].get(globalOffset % size); + return binarySection.segments[globalOffset / size].get(globalOffset % size); } } @@ -165,12 +166,12 @@ public byte byteAt(int index) { */ public byte[] getBytes() { ensureMaterialized(); - return SegmentsUtil.getBytes(segments, offset, sizeInBytes); + return SegmentsUtil.getBytes(binarySection.segments, binarySection.offset, binarySection.sizeInBytes); } @Override public boolean equals(Object o) { - if (o != null && o instanceof BinaryString) { + if (o instanceof BinaryString) { BinaryString other = (BinaryString) o; if (javaObject != null && other.javaObject != null) { return javaObject.equals(other.javaObject); @@ -178,28 +179,62 @@ public boolean equals(Object o) { ensureMaterialized(); other.ensureMaterialized(); - return binaryEquals(other); + return binarySection.equals(other.binarySection); } else { return false; } } + @Override + public int hashCode() { + ensureMaterialized(); + return binarySection.hashCode(); + } + @Override public String toString() { if (javaObject == null) { - byte[] bytes = SegmentsUtil.allocateReuseBytes(sizeInBytes); - SegmentsUtil.copyToBytes(segments, offset, bytes, 0, sizeInBytes); - javaObject = StringUtf8Utils.decodeUTF8(bytes, 0, sizeInBytes); + byte[] bytes = SegmentsUtil.allocateReuseBytes(binarySection.sizeInBytes); + SegmentsUtil.copyToBytes(binarySection.segments, binarySection.offset, bytes, 0, binarySection.sizeInBytes); + javaObject = StringUtf8Utils.decodeUTF8(bytes, 0, binarySection.sizeInBytes); } return javaObject; } @Override - public void materialize() { + public MemorySegment[] getSegments() { + ensureMaterialized(); + return super.getSegments(); + } + + @Override + public int getOffset() { + ensureMaterialized(); + return super.getOffset(); + } + + @Override + public int getSizeInBytes() { + ensureMaterialized(); + return super.getSizeInBytes(); + } + + public void ensureMaterialized() { + ensureMaterialized(null); + } + + @Override + protected BinarySection materialize(TypeSerializer serializer) { + if (serializer != null) { + throw new IllegalArgumentException("BinaryString does not support custom serializers"); + } + byte[] bytes = StringUtf8Utils.encodeUTF8(javaObject); - segments = new MemorySegment[] {MemorySegmentFactory.wrap(bytes)}; - offset = 0; - sizeInBytes = bytes.length; + return new BinarySection( + new MemorySegment[]{MemorySegmentFactory.wrap(bytes)}, + 0, + bytes.length + ); } /** @@ -207,9 +242,9 @@ public void materialize() { */ public BinaryString copy() { ensureMaterialized(); - byte[] copy = SegmentsUtil.copyToBytes(segments, offset, sizeInBytes); + byte[] copy = SegmentsUtil.copyToBytes(binarySection.segments, binarySection.offset, binarySection.sizeInBytes); return new BinaryString(new MemorySegment[] {MemorySegmentFactory.wrap(copy)}, - 0, sizeInBytes, javaObject); + 0, binarySection.sizeInBytes, javaObject); } /** @@ -227,19 +262,20 @@ public int compareTo(@Nonnull BinaryString other) { ensureMaterialized(); other.ensureMaterialized(); - if (segments.length == 1 && other.segments.length == 1) { + if (binarySection.segments.length == 1 && other.binarySection.segments.length == 1) { - int len = Math.min(sizeInBytes, other.sizeInBytes); - MemorySegment seg1 = segments[0]; - MemorySegment seg2 = other.segments[0]; + int len = Math.min(binarySection.sizeInBytes, other.binarySection.sizeInBytes); + MemorySegment seg1 = binarySection.segments[0]; + MemorySegment seg2 = other.binarySection.segments[0]; for (int i = 0; i < len; i++) { - int res = (seg1.get(offset + i) & 0xFF) - (seg2.get(other.offset + i) & 0xFF); + int res = + (seg1.get(binarySection.offset + i) & 0xFF) - (seg2.get(other.binarySection.offset + i) & 0xFF); if (res != 0) { return res; } } - return sizeInBytes - other.sizeInBytes; + return binarySection.sizeInBytes - other.binarySection.sizeInBytes; } // if there are multi segments. @@ -251,20 +287,20 @@ public int compareTo(@Nonnull BinaryString other) { */ private int compareMultiSegments(BinaryString other) { - if (sizeInBytes == 0 || other.sizeInBytes == 0) { - return sizeInBytes - other.sizeInBytes; + if (binarySection.sizeInBytes == 0 || other.binarySection.sizeInBytes == 0) { + return binarySection.sizeInBytes - other.binarySection.sizeInBytes; } - int len = Math.min(sizeInBytes, other.sizeInBytes); + int len = Math.min(binarySection.sizeInBytes, other.binarySection.sizeInBytes); - MemorySegment seg1 = segments[0]; - MemorySegment seg2 = other.segments[0]; + MemorySegment seg1 = binarySection.segments[0]; + MemorySegment seg2 = other.binarySection.segments[0]; - int segmentSize = segments[0].size(); - int otherSegmentSize = other.segments[0].size(); + int segmentSize = binarySection.segments[0].size(); + int otherSegmentSize = other.binarySection.segments[0].size(); - int sizeOfFirst1 = segmentSize - offset; - int sizeOfFirst2 = otherSegmentSize - other.offset; + int sizeOfFirst1 = segmentSize - binarySection.offset; + int sizeOfFirst2 = otherSegmentSize - other.binarySection.offset; int varSegIndex1 = 1; int varSegIndex2 = 1; @@ -272,12 +308,12 @@ private int compareMultiSegments(BinaryString other) { // find the first segment of this string. while (sizeOfFirst1 <= 0) { sizeOfFirst1 += segmentSize; - seg1 = segments[varSegIndex1++]; + seg1 = binarySection.segments[varSegIndex1++]; } while (sizeOfFirst2 <= 0) { sizeOfFirst2 += otherSegmentSize; - seg2 = other.segments[varSegIndex2++]; + seg2 = other.binarySection.segments[varSegIndex2++]; } int offset1 = segmentSize - sizeOfFirst1; @@ -299,20 +335,20 @@ private int compareMultiSegments(BinaryString other) { len -= needCompare; // next segment if (sizeOfFirst1 < sizeOfFirst2) { //I am smaller - seg1 = segments[varSegIndex1++]; + seg1 = binarySection.segments[varSegIndex1++]; offset1 = 0; offset2 += needCompare; sizeOfFirst1 = segmentSize; sizeOfFirst2 -= needCompare; } else if (sizeOfFirst1 > sizeOfFirst2) { //other is smaller - seg2 = other.segments[varSegIndex2++]; + seg2 = other.binarySection.segments[varSegIndex2++]; offset2 = 0; offset1 += needCompare; sizeOfFirst2 = otherSegmentSize; sizeOfFirst1 -= needCompare; } else { // same, should go ahead both. - seg1 = segments[varSegIndex1++]; - seg2 = other.segments[varSegIndex2++]; + seg1 = binarySection.segments[varSegIndex1++]; + seg2 = other.binarySection.segments[varSegIndex2++]; offset1 = 0; offset2 = 0; sizeOfFirst1 = segmentSize; @@ -323,7 +359,7 @@ private int compareMultiSegments(BinaryString other) { checkArgument(needCompare == len); - return sizeInBytes - other.sizeInBytes; + return binarySection.sizeInBytes - other.binarySection.sizeInBytes; } /** @@ -343,27 +379,27 @@ private int compareMultiSegments(BinaryString other) { */ public BinaryString substring(int beginIndex, int endIndex) { ensureMaterialized(); - if (endIndex <= beginIndex || beginIndex >= sizeInBytes) { + if (endIndex <= beginIndex || beginIndex >= binarySection.sizeInBytes) { return EMPTY_UTF8; } if (inFirstSegment()) { - MemorySegment segment = segments[0]; + MemorySegment segment = binarySection.segments[0]; int i = 0; int c = 0; - while (i < sizeInBytes && c < beginIndex) { - i += numBytesForFirstByte(segment.get(i + offset)); + while (i < binarySection.sizeInBytes && c < beginIndex) { + i += numBytesForFirstByte(segment.get(i + binarySection.offset)); c += 1; } int j = i; - while (i < sizeInBytes && c < endIndex) { - i += numBytesForFirstByte(segment.get(i + offset)); + while (i < binarySection.sizeInBytes && c < endIndex) { + i += numBytesForFirstByte(segment.get(i + binarySection.offset)); c += 1; } if (i > j) { byte[] bytes = new byte[i - j]; - segment.get(offset + j, bytes, 0, i - j); + segment.get(binarySection.offset + j, bytes, 0, i - j); return fromBytes(bytes); } else { return EMPTY_UTF8; @@ -374,11 +410,11 @@ public BinaryString substring(int beginIndex, int endIndex) { } private BinaryString substringMultiSegs(final int start, final int until) { - int segSize = segments[0].size(); + int segSize = binarySection.segments[0].size(); SegmentAndOffset index = firstSegmentAndOffset(segSize); int i = 0; int c = 0; - while (i < sizeInBytes && c < start) { + while (i < binarySection.sizeInBytes && c < start) { int charSize = numBytesForFirstByte(index.value()); i += charSize; index.skipBytes(charSize, segSize); @@ -386,7 +422,7 @@ private BinaryString substringMultiSegs(final int start, final int until) { } int j = i; - while (i < sizeInBytes && c < until) { + while (i < binarySection.sizeInBytes && c < until) { int charSize = numBytesForFirstByte(index.value()); i += charSize; index.skipBytes(charSize, segSize); @@ -394,7 +430,7 @@ private BinaryString substringMultiSegs(final int start, final int until) { } if (i > j) { - return fromBytes(SegmentsUtil.copyToBytes(segments, offset + j, i - j)); + return fromBytes(SegmentsUtil.copyToBytes(binarySection.segments, binarySection.offset + j, i - j)); } else { return EMPTY_UTF8; } @@ -410,12 +446,12 @@ private BinaryString substringMultiSegs(final int start, final int until) { public boolean contains(final BinaryString s) { ensureMaterialized(); s.ensureMaterialized(); - if (s.sizeInBytes == 0) { + if (s.binarySection.sizeInBytes == 0) { return true; } int find = SegmentsUtil.find( - segments, offset, sizeInBytes, - s.segments, s.offset, s.sizeInBytes); + binarySection.segments, binarySection.offset, binarySection.sizeInBytes, + s.binarySection.segments, s.binarySection.offset, s.binarySection.sizeInBytes); return find != -1; } @@ -446,7 +482,7 @@ public boolean startsWith(final BinaryString prefix) { public boolean endsWith(final BinaryString suffix) { ensureMaterialized(); suffix.ensureMaterialized(); - return matchAt(suffix, sizeInBytes - suffix.sizeInBytes); + return matchAt(suffix, binarySection.sizeInBytes - suffix.binarySection.sizeInBytes); } /** @@ -461,9 +497,9 @@ public BinaryString trim() { ensureMaterialized(); if (inFirstSegment()) { int s = 0; - int e = this.sizeInBytes - 1; + int e = this.binarySection.sizeInBytes - 1; // skip all of the space (0x20) in the left side - while (s < this.sizeInBytes && getByteOneSegment(s) == 0x20) { + while (s < this.binarySection.sizeInBytes && getByteOneSegment(s) == 0x20) { s++; } // skip all of the space (0x20) in the right side @@ -483,11 +519,11 @@ public BinaryString trim() { private BinaryString trimMultiSegs() { int s = 0; - int e = this.sizeInBytes - 1; - int segSize = segments[0].size(); + int e = this.binarySection.sizeInBytes - 1; + int segSize = binarySection.segments[0].size(); SegmentAndOffset front = firstSegmentAndOffset(segSize); // skip all of the space (0x20) in the left side - while (s < this.sizeInBytes && front.value() == 0x20) { + while (s < this.binarySection.sizeInBytes && front.value() == 0x20) { s++; front.nextByte(segSize); } @@ -518,7 +554,7 @@ private BinaryString trimMultiSegs() { public int indexOf(BinaryString str, int fromIndex) { ensureMaterialized(); str.ensureMaterialized(); - if (str.sizeInBytes == 0) { + if (str.binarySection.sizeInBytes == 0) { return 0; } if (inFirstSegment()) { @@ -526,21 +562,21 @@ public int indexOf(BinaryString str, int fromIndex) { int byteIdx = 0; // position is char int charIdx = 0; - while (byteIdx < sizeInBytes && charIdx < fromIndex) { + while (byteIdx < binarySection.sizeInBytes && charIdx < fromIndex) { byteIdx += numBytesForFirstByte(getByteOneSegment(byteIdx)); charIdx++; } do { - if (byteIdx + str.sizeInBytes > sizeInBytes) { + if (byteIdx + str.binarySection.sizeInBytes > binarySection.sizeInBytes) { return -1; } - if (SegmentsUtil.equals(segments, offset + byteIdx, - str.segments, str.offset, str.sizeInBytes)) { + if (SegmentsUtil.equals(binarySection.segments, binarySection.offset + byteIdx, + str.binarySection.segments, str.binarySection.offset, str.binarySection.sizeInBytes)) { return charIdx; } byteIdx += numBytesForFirstByte(getByteOneSegment(byteIdx)); charIdx++; - } while (byteIdx < sizeInBytes); + } while (byteIdx < binarySection.sizeInBytes); return -1; } else { @@ -553,27 +589,27 @@ private int indexOfMultiSegs(BinaryString str, int fromIndex) { int byteIdx = 0; // position is char int charIdx = 0; - int segSize = segments[0].size(); + int segSize = binarySection.segments[0].size(); SegmentAndOffset index = firstSegmentAndOffset(segSize); - while (byteIdx < sizeInBytes && charIdx < fromIndex) { + while (byteIdx < binarySection.sizeInBytes && charIdx < fromIndex) { int charBytes = numBytesForFirstByte(index.value()); byteIdx += charBytes; charIdx++; index.skipBytes(charBytes, segSize); } do { - if (byteIdx + str.sizeInBytes > sizeInBytes) { + if (byteIdx + str.binarySection.sizeInBytes > binarySection.sizeInBytes) { return -1; } - if (SegmentsUtil.equals(segments, offset + byteIdx, - str.segments, str.offset, str.sizeInBytes)) { + if (SegmentsUtil.equals(binarySection.segments, binarySection.offset + byteIdx, + str.binarySection.segments, str.binarySection.offset, str.binarySection.sizeInBytes)) { return charIdx; } int charBytes = numBytesForFirstByte(index.segment.get(index.offset)); byteIdx += charBytes; charIdx++; index.skipBytes(charBytes, segSize); - } while (byteIdx < sizeInBytes); + } while (byteIdx < binarySection.sizeInBytes); return -1; } @@ -587,14 +623,14 @@ public BinaryString toUpperCase() { if (javaObject != null) { return javaToUpperCase(); } - if (sizeInBytes == 0) { + if (binarySection.sizeInBytes == 0) { return EMPTY_UTF8; } - int size = segments[0].size(); + int size = binarySection.segments[0].size(); SegmentAndOffset segmentAndOffset = startSegmentAndOffset(size); - byte[] bytes = new byte[sizeInBytes]; + byte[] bytes = new byte[binarySection.sizeInBytes]; bytes[0] = (byte) Character.toTitleCase(segmentAndOffset.value()); - for (int i = 0; i < sizeInBytes; i++) { + for (int i = 0; i < binarySection.sizeInBytes; i++) { byte b = segmentAndOffset.value(); if (numBytesForFirstByte(b) != 1) { // fallback @@ -624,14 +660,14 @@ public BinaryString toLowerCase() { if (javaObject != null) { return javaToLowerCase(); } - if (sizeInBytes == 0) { + if (binarySection.sizeInBytes == 0) { return EMPTY_UTF8; } - int size = segments[0].size(); + int size = binarySection.segments[0].size(); SegmentAndOffset segmentAndOffset = startSegmentAndOffset(size); - byte[] bytes = new byte[sizeInBytes]; + byte[] bytes = new byte[binarySection.sizeInBytes]; bytes[0] = (byte) Character.toTitleCase(segmentAndOffset.value()); - for (int i = 0; i < sizeInBytes; i++) { + for (int i = 0; i < binarySection.sizeInBytes; i++) { byte b = segmentAndOffset.value(); if (numBytesForFirstByte(b) != 1) { // fallback @@ -657,11 +693,11 @@ private BinaryString javaToLowerCase() { // ------------------------------------------------------------------------------------------ byte getByteOneSegment(int i) { - return segments[0].get(offset + i); + return binarySection.segments[0].get(binarySection.offset + i); } boolean inFirstSegment() { - return sizeInBytes + offset <= segments[0].size(); + return binarySection.sizeInBytes + binarySection.offset <= binarySection.segments[0].size(); } private boolean matchAt(final BinaryString s, int pos) { @@ -669,41 +705,50 @@ private boolean matchAt(final BinaryString s, int pos) { } private boolean matchAtOneSeg(final BinaryString s, int pos) { - return s.sizeInBytes + pos <= sizeInBytes && pos >= 0 && - segments[0].equalTo(s.segments[0], offset + pos, s.offset, s.sizeInBytes); + return s.binarySection.sizeInBytes + pos <= binarySection.sizeInBytes && pos >= 0 && + binarySection.segments[0].equalTo( + s.binarySection.segments[0], + binarySection.offset + pos, + s.binarySection.offset, + s.binarySection.sizeInBytes); } private boolean matchAtVarSeg(final BinaryString s, int pos) { - return s.sizeInBytes + pos <= sizeInBytes && pos >= 0 && - SegmentsUtil.equals(segments, offset + pos, s.segments, s.offset, s.sizeInBytes); + return s.binarySection.sizeInBytes + pos <= binarySection.sizeInBytes && pos >= 0 && + SegmentsUtil.equals( + binarySection.segments, + binarySection.offset + pos, + s.binarySection.segments, + s.binarySection.offset, + s.binarySection.sizeInBytes); } BinaryString copyBinaryStringInOneSeg(int start, int len) { byte[] newBytes = new byte[len]; - segments[0].get(offset + start, newBytes, 0, len); + binarySection.segments[0].get(binarySection.offset + start, newBytes, 0, len); return fromBytes(newBytes); } BinaryString copyBinaryString(int start, int end) { int len = end - start + 1; byte[] newBytes = new byte[len]; - SegmentsUtil.copyToBytes(segments, offset + start, newBytes, 0, len); + SegmentsUtil.copyToBytes(binarySection.segments, binarySection.offset + start, newBytes, 0, len); return fromBytes(newBytes); } SegmentAndOffset firstSegmentAndOffset(int segSize) { - int segIndex = offset / segSize; - return new SegmentAndOffset(segIndex, offset % segSize); + int segIndex = binarySection.offset / segSize; + return new SegmentAndOffset(segIndex, binarySection.offset % segSize); } SegmentAndOffset lastSegmentAndOffset(int segSize) { - int lastOffset = offset + sizeInBytes - 1; + int lastOffset = binarySection.offset + binarySection.sizeInBytes - 1; int segIndex = lastOffset / segSize; return new SegmentAndOffset(segIndex, lastOffset % segSize); } private SegmentAndOffset startSegmentAndOffset(int segSize) { - return inFirstSegment() ? new SegmentAndOffset(0, offset) : firstSegmentAndOffset(segSize); + return inFirstSegment() ? new SegmentAndOffset(0, binarySection.offset) : firstSegmentAndOffset(segSize); } /** @@ -716,13 +761,13 @@ class SegmentAndOffset { private SegmentAndOffset(int segIndex, int offset) { this.segIndex = segIndex; - this.segment = segments[segIndex]; + this.segment = binarySection.segments[segIndex]; this.offset = offset; } private void assignSegment() { - segment = segIndex >= 0 && segIndex < segments.length ? - segments[segIndex] : null; + segment = segIndex >= 0 && segIndex < binarySection.segments.length ? + binarySection.segments[segIndex] : null; } void previousByte(int segSize) { diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryStringUtil.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryStringUtil.java index 6faa51a4f24a..021a8afbbc1f 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryStringUtil.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryStringUtil.java @@ -582,9 +582,9 @@ public static BinaryString keyValue(BinaryString str, byte split1, byte split2, int byteIdx = 0; // position of last split1 int lastSplit1Idx = -1; - while (byteIdx < str.sizeInBytes) { + while (byteIdx < str.getSizeInBytes()) { // If find next split1 in str, process current kv - if (str.segments[0].get(str.offset + byteIdx) == split1) { + if (str.getSegments()[0].get(str.getOffset() + byteIdx) == split1) { int currentKeyIdx = lastSplit1Idx + 1; // If key of current kv is keyName, return the value directly BinaryString value = findValueOfKey(str, split2, keyName, currentKeyIdx, byteIdx); @@ -597,7 +597,7 @@ public static BinaryString keyValue(BinaryString str, byte split1, byte split2, } // process the string which is not ends with split1 int currentKeyIdx = lastSplit1Idx + 1; - return findValueOfKey(str, split2, keyName, currentKeyIdx, str.sizeInBytes); + return findValueOfKey(str, split2, keyName, currentKeyIdx, str.getSizeInBytes()); } else { return keyValueSlow(str, split1, split2, keyName); } @@ -609,16 +609,16 @@ private static BinaryString findValueOfKey( BinaryString keyName, int start, int end) { - int keyNameLen = keyName.sizeInBytes; + int keyNameLen = keyName.getSizeInBytes(); for (int idx = start; idx < end; idx++) { - if (str.segments[0].get(str.offset + idx) == split) { + if (str.getSegments()[0].get(str.getOffset() + idx) == split) { if (idx == start + keyNameLen && - str.segments[0].equalTo(keyName.segments[0], str.offset + start, - keyName.offset, keyNameLen)) { + str.getSegments()[0].equalTo(keyName.getSegments()[0], str.getOffset() + start, + keyName.getOffset(), keyNameLen)) { int valueIdx = idx + 1; int valueLen = end - valueIdx; byte[] bytes = new byte[valueLen]; - str.segments[0].get(str.offset + valueIdx, bytes, 0, valueLen); + str.getSegments()[0].get(str.getOffset() + valueIdx, bytes, 0, valueLen); return fromBytes(bytes, 0, valueLen); } else { return null; @@ -637,7 +637,7 @@ private static BinaryString keyValueSlow( int byteIdx = 0; // position of last split1 int lastSplit1Idx = -1; - while (byteIdx < str.sizeInBytes) { + while (byteIdx < str.getSizeInBytes()) { // If find next split1 in str, process current kv if (str.byteAt(byteIdx) == split1) { int currentKeyIdx = lastSplit1Idx + 1; @@ -650,7 +650,7 @@ private static BinaryString keyValueSlow( byteIdx++; } int currentKeyIdx = lastSplit1Idx + 1; - return findValueOfKeySlow(str, split2, keyName, currentKeyIdx, str.sizeInBytes); + return findValueOfKeySlow(str, split2, keyName, currentKeyIdx, str.getSizeInBytes()); } private static BinaryString findValueOfKeySlow( @@ -659,14 +659,14 @@ private static BinaryString findValueOfKeySlow( BinaryString keyName, int start, int end) { - int keyNameLen = keyName.sizeInBytes; + int keyNameLen = keyName.getSizeInBytes(); for (int idx = start; idx < end; idx++) { if (str.byteAt(idx) == split) { if (idx == start + keyNameLen && - SegmentsUtil.equals(str.segments, str.offset + start, keyName.segments, - keyName.offset, keyNameLen)) { + SegmentsUtil.equals(str.getSegments(), str.getOffset() + start, keyName.getSegments(), + keyName.getOffset(), keyNameLen)) { int valueIdx = idx + 1; - byte[] bytes = SegmentsUtil.copyToBytes(str.segments, str.offset + valueIdx, end - valueIdx); + byte[] bytes = SegmentsUtil.copyToBytes(str.getSegments(), str.getOffset() + valueIdx, end - valueIdx); return fromBytes(bytes); } else { return null; @@ -740,8 +740,8 @@ public static BinaryString concat(Iterable inputs) { int offset = 0; for (BinaryString input : inputs) { if (input != null) { - int len = input.sizeInBytes; - SegmentsUtil.copyToBytes(input.segments, input.offset, result, offset, len); + int len = input.getSizeInBytes(); + SegmentsUtil.copyToBytes(input.getSegments(), input.getOffset(), result, offset, len); offset += len; } } @@ -771,7 +771,7 @@ public static BinaryString concatWs(BinaryString separator, Iterable= str.sizeInBytes) { + if (searchIdx >= str.getSizeInBytes()) { return EMPTY_UTF8; } else { - return str.copyBinaryStringInOneSeg(searchIdx, str.sizeInBytes - searchIdx); + return str.copyBinaryStringInOneSeg(searchIdx, str.getSizeInBytes() - searchIdx); } } else { return trimLeftSlow(str, trimStr); @@ -952,9 +957,9 @@ public static BinaryString trimLeft(BinaryString str, BinaryString trimStr) { private static BinaryString trimLeftSlow(BinaryString str, BinaryString trimStr) { int searchIdx = 0; - int segSize = str.segments[0].size(); + int segSize = str.getSegments()[0].size(); BinaryString.SegmentAndOffset front = str.firstSegmentAndOffset(segSize); - while (searchIdx < str.sizeInBytes) { + while (searchIdx < str.getSizeInBytes()) { int charBytes = numBytesForFirstByte(front.value()); BinaryString currentChar = str.copyBinaryString(searchIdx, searchIdx + charBytes - 1); if (trimStr.contains(currentChar)) { @@ -964,18 +969,18 @@ private static BinaryString trimLeftSlow(BinaryString str, BinaryString trimStr) break; } } - if (searchIdx == str.sizeInBytes) { + if (searchIdx == str.getSizeInBytes()) { // empty string return EMPTY_UTF8; } else { - return str.copyBinaryString(searchIdx, str.sizeInBytes - 1); + return str.copyBinaryString(searchIdx, str.getSizeInBytes() - 1); } } public static BinaryString trimRight(BinaryString str) { str.ensureMaterialized(); if (str.inFirstSegment()) { - int e = str.sizeInBytes - 1; + int e = str.getSizeInBytes() - 1; // skip all of the space (0x20) in the right side while (e >= 0 && str.getByteOneSegment(e) == 0x20) { e--; @@ -993,8 +998,8 @@ public static BinaryString trimRight(BinaryString str) { } private static BinaryString trimRightSlow(BinaryString str) { - int e = str.sizeInBytes - 1; - int segSize = str.segments[0].size(); + int e = str.getSizeInBytes() - 1; + int segSize = str.getSegments()[0].size(); BinaryString.SegmentAndOffset behind = str.lastSegmentAndOffset(segSize); // skip all of the space (0x20) in the right side while (e >= 0 && behind.value() == 0x20) { @@ -1032,10 +1037,10 @@ public static BinaryString trimRight(BinaryString str, BinaryString trimStr) { int charIdx = 0; int byteIdx = 0; // each element in charLens is length of character in the source string - int[] charLens = new int[str.sizeInBytes]; + int[] charLens = new int[str.getSizeInBytes()]; // each element in charStartPos is start position of first byte in the source string - int[] charStartPos = new int[str.sizeInBytes]; - while (byteIdx < str.sizeInBytes) { + int[] charStartPos = new int[str.getSizeInBytes()]; + while (byteIdx < str.getSizeInBytes()) { charStartPos[charIdx] = byteIdx; charLens[charIdx] = numBytesForFirstByte(str.getByteOneSegment(byteIdx)); byteIdx += charLens[charIdx]; @@ -1043,7 +1048,7 @@ public static BinaryString trimRight(BinaryString str, BinaryString trimStr) { } // searchIdx points to the first character which is not in trim string from the right // end. - int searchIdx = str.sizeInBytes - 1; + int searchIdx = str.getSizeInBytes() - 1; charIdx -= 1; while (charIdx >= 0) { BinaryString currentChar = str.copyBinaryStringInOneSeg( @@ -1069,13 +1074,13 @@ public static BinaryString trimRight(BinaryString str, BinaryString trimStr) { private static BinaryString trimRightSlow(BinaryString str, BinaryString trimStr) { int charIdx = 0; int byteIdx = 0; - int segSize = str.segments[0].size(); + int segSize = str.getSegments()[0].size(); BinaryString.SegmentAndOffset index = str.firstSegmentAndOffset(segSize); // each element in charLens is length of character in the source string - int[] charLens = new int[str.sizeInBytes]; + int[] charLens = new int[str.getSizeInBytes()]; // each element in charStartPos is start position of first byte in the source string - int[] charStartPos = new int[str.sizeInBytes]; - while (byteIdx < str.sizeInBytes) { + int[] charStartPos = new int[str.getSizeInBytes()]; + while (byteIdx < str.getSizeInBytes()) { charStartPos[charIdx] = byteIdx; int charBytes = numBytesForFirstByte(index.value()); charLens[charIdx] = charBytes; @@ -1085,7 +1090,7 @@ private static BinaryString trimRightSlow(BinaryString str, BinaryString trimStr } // searchIdx points to the first character which is not in trim string from the right // end. - int searchIdx = str.sizeInBytes - 1; + int searchIdx = str.getSizeInBytes() - 1; charIdx -= 1; while (charIdx >= 0) { BinaryString currentChar = str.copyBinaryString( diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryWriter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryWriter.java index 8163f57b158c..3a559b7768fe 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryWriter.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryWriter.java @@ -21,6 +21,7 @@ import org.apache.flink.table.runtime.typeutils.BaseArraySerializer; import org.apache.flink.table.runtime.typeutils.BaseMapSerializer; import org.apache.flink.table.runtime.typeutils.BaseRowSerializer; +import org.apache.flink.table.runtime.typeutils.BinaryGenericSerializer; import org.apache.flink.table.types.logical.DecimalType; import org.apache.flink.table.types.logical.LogicalType; @@ -68,7 +69,7 @@ public interface BinaryWriter { void writeRow(int pos, BaseRow value, BaseRowSerializer type); - void writeGeneric(int pos, BinaryGeneric value); + void writeGeneric(int pos, BinaryGeneric value, BinaryGenericSerializer serializer); /** * Finally, complete write to set real size to binary. @@ -124,7 +125,7 @@ static void write(BinaryWriter writer, int pos, writer.writeRow(pos, (BaseRow) o, (BaseRowSerializer) serializer); break; case ANY: - writer.writeGeneric(pos, (BinaryGeneric) o); + writer.writeGeneric(pos, (BinaryGeneric) o, (BinaryGenericSerializer) serializer); break; case BINARY: case VARBINARY: diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/DataFormatConverters.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/DataFormatConverters.java index b6e6bb2012c5..1185457b5c25 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/DataFormatConverters.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/DataFormatConverters.java @@ -620,7 +620,7 @@ public GenericConverter(TypeSerializer serializer) { @Override BinaryGeneric toInternalImpl(T value) { - return new BinaryGeneric<>(value, serializer); + return new BinaryGeneric<>(value); } @Override diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/LazyBinaryFormat.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/LazyBinaryFormat.java index be29ee186e34..5f91820ae0d8 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/LazyBinaryFormat.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/LazyBinaryFormat.java @@ -17,7 +17,11 @@ package org.apache.flink.table.dataformat; +import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.util.WrappingRuntimeException; + +import java.io.IOException; /** * Lazy binary format. @@ -37,59 +41,77 @@ * It can lazy the conversions as much as possible. Only when it is needed can it be converted * into the required form. */ -public abstract class LazyBinaryFormat extends BinaryFormat { +public abstract class LazyBinaryFormat implements BinaryFormat { - protected T javaObject; + T javaObject; + BinarySection binarySection; public LazyBinaryFormat() { this(null, -1, -1, null); } public LazyBinaryFormat(MemorySegment[] segments, int offset, int sizeInBytes, T javaObject) { - super(segments, offset, sizeInBytes); - this.javaObject = javaObject; + this(javaObject, new BinarySection(segments, offset, sizeInBytes)); } public LazyBinaryFormat(MemorySegment[] segments, int offset, int sizeInBytes) { - this(segments, offset, sizeInBytes, null); + this(null, new BinarySection(segments, offset, sizeInBytes)); } public LazyBinaryFormat(T javaObject) { - this(null, -1, -1, javaObject); + this(javaObject, null); + } + + public LazyBinaryFormat(T javaObject, BinarySection binarySection) { + this.javaObject = javaObject; + this.binarySection = binarySection; } public T getJavaObject() { return javaObject; } + /** + * Must be public as it is used during code generation. + */ public void setJavaObject(T javaObject) { this.javaObject = javaObject; } @Override public MemorySegment[] getSegments() { - ensureMaterialized(); - return segments; + if (binarySection == null) { + throw new IllegalStateException("Lazy Binary Format was not materialized"); + } + return binarySection.segments; } @Override public int getOffset() { - ensureMaterialized(); - return offset; + if (binarySection == null) { + throw new IllegalStateException("Lazy Binary Format was not materialized"); + } + return binarySection.offset; } @Override public int getSizeInBytes() { - ensureMaterialized(); - return sizeInBytes; + if (binarySection == null) { + throw new IllegalStateException("Lazy Binary Format was not materialized"); + } + return binarySection.sizeInBytes; } /** * Ensure we have materialized binary format. */ - public void ensureMaterialized() { - if (segments == null) { - materialize(); + public final void ensureMaterialized(TypeSerializer serializer) { + if (binarySection == null) { + try { + this.binarySection = materialize(serializer); + } catch (IOException e) { + throw new WrappingRuntimeException(e); + } } } @@ -98,24 +120,5 @@ public void ensureMaterialized() { * Inherited classes need to hold the information they need. * (For example, BinaryGeneric needs javaObjectSerializer). */ - public abstract void materialize(); - - @Override - public boolean equals(Object o) { - if (o != null && o instanceof LazyBinaryFormat) { - LazyBinaryFormat other = (LazyBinaryFormat) o; - - ensureMaterialized(); - other.ensureMaterialized(); - return binaryEquals(other); - } else { - return false; - } - } - - @Override - public int hashCode() { - ensureMaterialized(); - return super.hashCode(); - } + protected abstract BinarySection materialize(TypeSerializer serializer) throws IOException; } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/NestedRow.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/NestedRow.java index 252eec9a6de6..f31a138daf7a 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/NestedRow.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/NestedRow.java @@ -21,6 +21,7 @@ import org.apache.flink.core.memory.MemorySegmentFactory; import org.apache.flink.table.runtime.util.SegmentsUtil; +import static org.apache.flink.table.dataformat.BinaryFormat.readBinaryFieldFromSegments; import static org.apache.flink.table.dataformat.BinaryRow.calculateBitSetWidthInBytes; import static org.apache.flink.util.Preconditions.checkArgument; @@ -32,7 +33,7 @@ * has a possibility to cross the boundary of a segment, while the fixed-length part of {@link BinaryRow} * must fit into its first memory segment. */ -public final class NestedRow extends BinaryFormat implements BaseRow { +public final class NestedRow extends BinarySection implements BaseRow { private final int arity; private final int nullBitsSizeInBytes; @@ -224,7 +225,7 @@ public BinaryString getString(int pos) { assertIndexIsValid(pos); int fieldOffset = getFieldOffset(pos); final long offsetAndLen = SegmentsUtil.getLong(segments, fieldOffset); - return BinaryString.readBinaryStringFieldFromSegments(segments, offset, fieldOffset, offsetAndLen); + return BinaryFormat.readBinaryStringFieldFromSegments(segments, offset, fieldOffset, offsetAndLen); } @Override diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BinaryGenericSerializer.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BinaryGenericSerializer.java index b0ca78ff754b..29ba7eba655f 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BinaryGenericSerializer.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BinaryGenericSerializer.java @@ -19,18 +19,15 @@ package org.apache.flink.table.runtime.typeutils; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; -import org.apache.flink.api.java.typeutils.runtime.DataInputViewStream; -import org.apache.flink.api.java.typeutils.runtime.DataOutputViewStream; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.core.memory.MemorySegmentFactory; import org.apache.flink.table.dataformat.BinaryGeneric; import org.apache.flink.table.runtime.util.SegmentsUtil; -import org.apache.flink.util.InstantiationUtil; import java.io.IOException; @@ -55,12 +52,19 @@ public boolean isImmutableType() { @Override public BinaryGeneric createInstance() { - return new BinaryGeneric<>(serializer.createInstance(), serializer); + return new BinaryGeneric<>(serializer.createInstance()); } @Override public BinaryGeneric copy(BinaryGeneric from) { - return from.copy(); + from.ensureMaterialized(serializer); + byte[] bytes = SegmentsUtil.copyToBytes(from.getSegments(), from.getOffset(), from.getSizeInBytes()); + T newJavaObject = from.getJavaObject() == null ? null : serializer.copy(from.getJavaObject()); + return new BinaryGeneric<>( + new MemorySegment[]{MemorySegmentFactory.wrap(bytes)}, + 0, + bytes.length, + newJavaObject); } @Override @@ -75,6 +79,7 @@ public int getLength() { @Override public void serialize(BinaryGeneric record, DataOutputView target) throws IOException { + record.ensureMaterialized(serializer); target.writeInt(record.getSizeInBytes()); SegmentsUtil.copyToView(record.getSegments(), record.getOffset(), record.getSizeInBytes(), target); } @@ -86,7 +91,8 @@ public BinaryGeneric deserialize(DataInputView source) throws IOException { source.readFully(bytes); return new BinaryGeneric<>( new MemorySegment[] {MemorySegmentFactory.wrap(bytes)}, - 0, bytes.length, serializer); + 0, + bytes.length); } @Override @@ -103,7 +109,7 @@ public void copy(DataInputView source, DataOutputView target) throws IOException @Override public BinaryGenericSerializer duplicate() { - return this; + return new BinaryGenericSerializer<>(serializer.duplicate()); } @Override @@ -127,63 +133,41 @@ public int hashCode() { @Override public TypeSerializerSnapshot> snapshotConfiguration() { - return new BinaryGenericSerializerSnapshot<>(serializer); + return new BinaryGenericSerializerSnapshot<>(this); + } + + public TypeSerializer getInnerSerializer() { + return serializer; } /** * {@link TypeSerializerSnapshot} for {@link BinaryGenericSerializer}. */ - public static final class BinaryGenericSerializerSnapshot implements TypeSerializerSnapshot> { - private static final int CURRENT_VERSION = 3; - - private TypeSerializer previousSerializer; + public static final class BinaryGenericSerializerSnapshot extends CompositeTypeSerializerSnapshot, BinaryGenericSerializer> { @SuppressWarnings("unused") public BinaryGenericSerializerSnapshot() { - // this constructor is used when restoring from a checkpoint/savepoint. - } - - BinaryGenericSerializerSnapshot(TypeSerializer serializer) { - this.previousSerializer = serializer; + super(BinaryGenericSerializer.class); } - @Override - public int getCurrentVersion() { - return CURRENT_VERSION; - } - - @Override - public void writeSnapshot(DataOutputView out) throws IOException { - InstantiationUtil.serializeObject(new DataOutputViewStream(out), previousSerializer); + public BinaryGenericSerializerSnapshot(BinaryGenericSerializer serializerInstance) { + super(serializerInstance); } @Override - public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) throws IOException { - try { - this.previousSerializer = InstantiationUtil.deserializeObject( - new DataInputViewStream(in), userCodeClassLoader); - } catch (ClassNotFoundException e) { - throw new IOException(e); - } + protected int getCurrentOuterSnapshotVersion() { + return 0; } @Override - public TypeSerializer> restoreSerializer() { - return new BinaryGenericSerializer<>(previousSerializer); + protected TypeSerializer[] getNestedSerializers(BinaryGenericSerializer outerSerializer) { + return new TypeSerializer[]{outerSerializer.serializer}; } @Override - public TypeSerializerSchemaCompatibility> resolveSchemaCompatibility(TypeSerializer> newSerializer) { - if (!(newSerializer instanceof BinaryGenericSerializer)) { - return TypeSerializerSchemaCompatibility.incompatible(); - } - - BinaryGenericSerializer newBinaryGenericSerializer = (BinaryGenericSerializer) newSerializer; - if (!previousSerializer.equals(newBinaryGenericSerializer.serializer)) { - return TypeSerializerSchemaCompatibility.incompatible(); - } else { - return TypeSerializerSchemaCompatibility.compatibleAsIs(); - } + @SuppressWarnings("unchecked") + protected BinaryGenericSerializer createOuterSerializerWithNestedSerializers(TypeSerializer[] nestedSerializers) { + return new BinaryGenericSerializer<>((TypeSerializer) nestedSerializers[0]); } } } diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BaseRowTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BaseRowTest.java index a86e0011c1f1..82019d04b2d9 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BaseRowTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BaseRowTest.java @@ -22,6 +22,7 @@ import org.apache.flink.table.runtime.typeutils.BaseArraySerializer; import org.apache.flink.table.runtime.typeutils.BaseMapSerializer; import org.apache.flink.table.runtime.typeutils.BaseRowSerializer; +import org.apache.flink.table.runtime.typeutils.BinaryGenericSerializer; import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.RowType; @@ -30,9 +31,11 @@ import java.math.BigDecimal; +import static org.apache.flink.table.utils.BinaryGenericAsserter.equivalent; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; /** @@ -48,11 +51,13 @@ public class BaseRowTest { private BinaryMap map; private BinaryRow underRow; private byte[] bytes; + private BinaryGenericSerializer genericSerializer; @Before public void before() { str = BinaryString.fromString("haha"); - generic = new BinaryGeneric<>("haha", StringSerializer.INSTANCE); + generic = new BinaryGeneric<>("haha"); + genericSerializer = new BinaryGenericSerializer<>(StringSerializer.INSTANCE); decimal1 = Decimal.fromLong(10, 5, 0); decimal2 = Decimal.fromBigDecimal(new BigDecimal(11), 20, 0); array = new BinaryArray(); @@ -98,7 +103,7 @@ private BinaryRow getBinaryRow() { writer.writeFloat(5, 5); writer.writeDouble(6, 6); writer.writeString(8, str); - writer.writeGeneric(9, generic); + writer.writeGeneric(9, generic, genericSerializer); writer.writeDecimal(10, decimal1, 5); writer.writeDecimal(11, decimal2, 20); writer.writeArray(12, array, new BaseArraySerializer(DataTypes.INT().getLogicalType(), null)); @@ -193,7 +198,7 @@ private void testAll(BaseRow row) { assertEquals(5, (int) row.getFloat(5)); assertEquals(6, (int) row.getDouble(6)); assertEquals(str, row.getString(8)); - assertEquals(generic, row.getGeneric(9)); + assertThat(row.getGeneric(9), equivalent(generic, genericSerializer)); assertEquals(decimal1, row.getDecimal(10, 5, 0)); assertEquals(decimal2, row.getDecimal(11, 20, 0)); assertEquals(array, row.getArray(12)); diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryArrayTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryArrayTest.java index f64907e95c6e..9de5cdfa259c 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryArrayTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryArrayTest.java @@ -25,6 +25,7 @@ import org.apache.flink.table.runtime.typeutils.BaseArraySerializer; import org.apache.flink.table.runtime.typeutils.BaseMapSerializer; import org.apache.flink.table.runtime.typeutils.BaseRowSerializer; +import org.apache.flink.table.runtime.typeutils.BinaryGenericSerializer; import org.apache.flink.table.runtime.util.SegmentsUtil; import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.RowType; @@ -36,7 +37,9 @@ import java.math.BigDecimal; import static org.apache.flink.table.dataformat.BinaryString.fromString; +import static org.apache.flink.table.utils.BinaryGenericAsserter.equivalent; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; /** @@ -460,13 +463,14 @@ public void testDecimal() { public void testGeneric() { BinaryArray array = new BinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); - BinaryGeneric generic = new BinaryGeneric<>("hahah", StringSerializer.INSTANCE); - writer.writeGeneric(0, generic); + BinaryGeneric generic = new BinaryGeneric<>("hahah"); + BinaryGenericSerializer serializer = new BinaryGenericSerializer<>(StringSerializer.INSTANCE); + writer.writeGeneric(0, generic, serializer); writer.setNullAt(1); writer.complete(); BinaryGeneric newGeneric = array.getGeneric(0); - assertEquals(generic, newGeneric); + assertThat(newGeneric, equivalent(generic, serializer)); assertTrue(array.isNullAt(1)); } diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryRowTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryRowTest.java index fa870b401c65..3a90aaa17845 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryRowTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryRowTest.java @@ -37,6 +37,7 @@ import org.apache.flink.table.runtime.typeutils.BaseArraySerializer; import org.apache.flink.table.runtime.typeutils.BaseMapSerializer; import org.apache.flink.table.runtime.typeutils.BaseRowSerializer; +import org.apache.flink.table.runtime.typeutils.BinaryGenericSerializer; import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer; import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.RowType; @@ -68,11 +69,13 @@ import static org.apache.flink.table.dataformat.BinaryString.fromBytes; import static org.apache.flink.table.dataformat.BinaryString.fromString; import static org.apache.flink.table.dataformat.DataFormatTestUtil.MyObj; +import static org.apache.flink.table.utils.BinaryGenericAsserter.equivalent; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; /** @@ -444,18 +447,18 @@ public void testDecimal() { public void testGeneric() { BinaryRow row = new BinaryRow(3); BinaryRowWriter writer = new BinaryRowWriter(row); - BinaryGeneric hahah = new BinaryGeneric<>("hahah", StringSerializer.INSTANCE); - writer.writeGeneric(0, hahah); + BinaryGenericSerializer binarySerializer = new BinaryGenericSerializer<>(StringSerializer.INSTANCE); + BinaryGeneric hahah = new BinaryGeneric<>("hahah"); + writer.writeGeneric(0, hahah, binarySerializer); writer.setNullAt(1); - hahah.ensureMaterialized(); - writer.writeGeneric(2, hahah); + writer.writeGeneric(2, hahah, binarySerializer); writer.complete(); BinaryGeneric generic0 = row.getGeneric(0); - assertEquals(hahah, generic0); + assertThat(generic0, equivalent(hahah, binarySerializer)); assertTrue(row.isNullAt(1)); BinaryGeneric generic2 = row.getGeneric(2); - assertEquals(hahah, generic2); + assertThat(generic2, equivalent(hahah, binarySerializer)); } @Test @@ -619,18 +622,18 @@ public void testGenericObject() throws Exception { GenericTypeInfo info = new GenericTypeInfo<>(MyObj.class); TypeSerializer genericSerializer = info.createSerializer(new ExecutionConfig()); + BinaryGenericSerializer binarySerializer = new BinaryGenericSerializer<>(genericSerializer); BinaryRow row = new BinaryRow(4); BinaryRowWriter writer = new BinaryRowWriter(row); writer.writeInt(0, 0); - BinaryGeneric myObj1 = new BinaryGeneric<>(new MyObj(0, 1), genericSerializer); - writer.writeGeneric(1, myObj1); - BinaryGeneric myObj2 = new BinaryGeneric<>(new MyObj(123, 5.0), genericSerializer); - myObj2.ensureMaterialized(); - writer.writeGeneric(2, myObj2); - BinaryGeneric myObj3 = new BinaryGeneric<>(new MyObj(1, 1), genericSerializer); - writer.writeGeneric(3, myObj3); + BinaryGeneric myObj1 = new BinaryGeneric<>(new MyObj(0, 1)); + writer.writeGeneric(1, myObj1, binarySerializer); + BinaryGeneric myObj2 = new BinaryGeneric<>(new MyObj(123, 5.0)); + writer.writeGeneric(2, myObj2, binarySerializer); + BinaryGeneric myObj3 = new BinaryGeneric<>(new MyObj(1, 1)); + writer.writeGeneric(3, myObj3, binarySerializer); writer.complete(); assertTestGenericObjectRow(row, genericSerializer); @@ -670,12 +673,30 @@ public void testDateAndTimeAsGenericObject() { LocalDateTime localDateTime = LocalDateTime.of(localDate, localTime); writer.writeInt(0, 0); - writer.writeGeneric(1, new BinaryGeneric<>(new Date(123), SqlDateSerializer.INSTANCE)); - writer.writeGeneric(2, new BinaryGeneric<>(new Time(456), SqlTimeSerializer.INSTANCE)); - writer.writeGeneric(3, new BinaryGeneric<>(new Timestamp(789), SqlTimestampSerializer.INSTANCE)); - writer.writeGeneric(4, new BinaryGeneric<>(localDate, LocalDateSerializer.INSTANCE)); - writer.writeGeneric(5, new BinaryGeneric<>(localTime, LocalTimeSerializer.INSTANCE)); - writer.writeGeneric(6, new BinaryGeneric<>(localDateTime, LocalDateTimeSerializer.INSTANCE)); + writer.writeGeneric( + 1, + new BinaryGeneric<>(new Date(123)), + new BinaryGenericSerializer<>(SqlDateSerializer.INSTANCE)); + writer.writeGeneric( + 2, + new BinaryGeneric<>(new Time(456)), + new BinaryGenericSerializer<>(SqlTimeSerializer.INSTANCE)); + writer.writeGeneric( + 3, + new BinaryGeneric<>(new Timestamp(789)), + new BinaryGenericSerializer<>(SqlTimestampSerializer.INSTANCE)); + writer.writeGeneric( + 4, + new BinaryGeneric<>(localDate), + new BinaryGenericSerializer<>(LocalDateSerializer.INSTANCE)); + writer.writeGeneric( + 5, + new BinaryGeneric<>(localTime), + new BinaryGenericSerializer<>(LocalTimeSerializer.INSTANCE)); + writer.writeGeneric( + 6, + new BinaryGeneric<>(localDateTime), + new BinaryGenericSerializer<>(LocalDateTimeSerializer.INSTANCE)); writer.complete(); assertEquals(new Date(123), BinaryGeneric.getJavaObjectFromBinaryGeneric( @@ -781,7 +802,7 @@ public void testZeroOutPaddingGeneric() { random.nextBytes(bytes); writer.writeBinary(0, bytes); writer.reset(); - writer.writeGeneric(0, new BinaryGeneric<>(new MyObj(0, 1), genericSerializer)); + writer.writeGeneric(0, new BinaryGeneric<>(new MyObj(0, 1)), new BinaryGenericSerializer<>(genericSerializer)); writer.complete(); int hash1 = row.hashCode(); @@ -789,7 +810,7 @@ public void testZeroOutPaddingGeneric() { random.nextBytes(bytes); writer.writeBinary(0, bytes); writer.reset(); - writer.writeGeneric(0, new BinaryGeneric<>(new MyObj(0, 1), genericSerializer)); + writer.writeGeneric(0, new BinaryGeneric<>(new MyObj(0, 1)), new BinaryGenericSerializer<>(genericSerializer)); writer.complete(); int hash2 = row.hashCode(); diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryStringTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryStringTest.java index 015c7ff518a6..c6a7649a6479 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryStringTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryStringTest.java @@ -54,6 +54,7 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; @@ -596,25 +597,25 @@ private DecimalData(String str, int precision, int scale) { @Test public void testEmptyString() { BinaryString str2 = fromString("hahahahah"); - BinaryString str3 = new BinaryString(); + BinaryString str3; { MemorySegment[] segments = new MemorySegment[2]; segments[0] = MemorySegmentFactory.wrap(new byte[10]); segments[1] = MemorySegmentFactory.wrap(new byte[10]); - str3.pointTo(segments, 15, 0); + str3 = BinaryString.fromAddress(segments, 15, 0); } assertTrue(BinaryString.EMPTY_UTF8.compareTo(str2) < 0); assertTrue(str2.compareTo(BinaryString.EMPTY_UTF8) > 0); - assertTrue(BinaryString.EMPTY_UTF8.compareTo(str3) == 0); - assertTrue(str3.compareTo(BinaryString.EMPTY_UTF8) == 0); + assertEquals(0, BinaryString.EMPTY_UTF8.compareTo(str3)); + assertEquals(0, str3.compareTo(BinaryString.EMPTY_UTF8)); - assertFalse(BinaryString.EMPTY_UTF8.equals(str2)); - assertFalse(str2.equals(BinaryString.EMPTY_UTF8)); + assertNotEquals(BinaryString.EMPTY_UTF8, str2); + assertNotEquals(str2, BinaryString.EMPTY_UTF8); - assertTrue(BinaryString.EMPTY_UTF8.equals(str3)); - assertTrue(str3.equals(BinaryString.EMPTY_UTF8)); + assertEquals(BinaryString.EMPTY_UTF8, str3); + assertEquals(str3, BinaryString.EMPTY_UTF8); } @Test diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/NestedRowTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/NestedRowTest.java index 2ba02371633c..23e3713f2bc2 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/NestedRowTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/NestedRowTest.java @@ -26,6 +26,8 @@ import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.runtime.typeutils.BaseRowSerializer; +import org.apache.flink.table.runtime.typeutils.BinaryGenericSerializer; +import org.apache.flink.table.runtime.typeutils.BinaryStringSerializer; import org.apache.flink.table.types.logical.LogicalType; import org.junit.Test; @@ -137,7 +139,7 @@ private BinaryRow getBinaryRow() { gRow.setField(1, 5L); gRow.setField(2, BinaryString.fromString("12345678")); gRow.setField(3, null); - gRow.setField(4, new BinaryGeneric<>(new MyObj(15, 5), genericSerializer)); + gRow.setField(4, new BinaryGeneric<>(new MyObj(15, 5))); BaseRowSerializer serializer = new BaseRowSerializer( new LogicalType[]{ @@ -150,9 +152,9 @@ private BinaryRow getBinaryRow() { new TypeSerializer[]{ IntSerializer.INSTANCE, LongSerializer.INSTANCE, - StringSerializer.INSTANCE, - StringSerializer.INSTANCE, - genericSerializer + BinaryStringSerializer.INSTANCE, + BinaryStringSerializer.INSTANCE, + new BinaryGenericSerializer<>(genericSerializer) }); writer.writeRow(0, gRow, serializer); writer.complete(); diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/BaseArraySerializerTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/BaseArraySerializerTest.java index 100aef066a2d..35bbc69d2aca 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/BaseArraySerializerTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/BaseArraySerializerTest.java @@ -84,7 +84,7 @@ public void testExecutionConfigWithKryo() throws Exception { MyObj inputObj = new MyObj(114514, 1919810); BaseArray inputArray = new GenericArray(new BinaryGeneric[] { - new BinaryGeneric<>(inputObj, new KryoSerializer<>(MyObj.class, config)) + new BinaryGeneric<>(inputObj) }, 1); byte[] serialized; diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/BaseMapSerializerTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/BaseMapSerializerTest.java index afd1cf4cd554..02c0b67a6cc7 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/BaseMapSerializerTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/BaseMapSerializerTest.java @@ -82,7 +82,7 @@ public void testExecutionConfigWithKryo() throws Exception { int inputKey = 998244353; MyObj inputObj = new MyObj(114514, 1919810); Map javaMap = new HashMap<>(); - javaMap.put(inputKey, new BinaryGeneric<>(inputObj, new KryoSerializer<>(MyObj.class, config))); + javaMap.put(inputKey, new BinaryGeneric<>(inputObj)); BaseMap inputMap = new GenericMap(javaMap); byte[] serialized; diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/BinaryGenericSerializerTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/BinaryGenericSerializerTest.java index d472392d8b8d..f35227c36958 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/BinaryGenericSerializerTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/BinaryGenericSerializerTest.java @@ -21,11 +21,22 @@ import org.apache.flink.api.common.typeutils.SerializerTestBase; import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.table.dataformat.BinaryGeneric; +import org.apache.flink.table.utils.BinaryGenericAsserter; +import org.apache.flink.testutils.DeeplyEqualsChecker; /** * A test for the {@link BinaryGenericSerializer}. */ public class BinaryGenericSerializerTest extends SerializerTestBase> { + public BinaryGenericSerializerTest() { + super(new DeeplyEqualsChecker() + .withCustomCheck( + (o, o2) -> o instanceof BinaryGeneric && o2 instanceof BinaryGeneric, + (o, o2, checker) -> BinaryGenericAsserter.equivalent( + (BinaryGeneric) o2, + new BinaryGenericSerializer<>(StringSerializer.INSTANCE)).matches(o) + )); + } @Override protected BinaryGenericSerializer createSerializer() { @@ -45,11 +56,11 @@ protected Class> getTypeClass() { @Override protected BinaryGeneric[] getTestData() { return new BinaryGeneric[] { - new BinaryGeneric<>("1", StringSerializer.INSTANCE), - new BinaryGeneric<>("2", StringSerializer.INSTANCE), - new BinaryGeneric<>("3", StringSerializer.INSTANCE), - new BinaryGeneric<>("4", StringSerializer.INSTANCE), - new BinaryGeneric<>("5", StringSerializer.INSTANCE) + new BinaryGeneric<>("1"), + new BinaryGeneric<>("2"), + new BinaryGeneric<>("3"), + new BinaryGeneric<>("4"), + new BinaryGeneric<>("5") }; } } diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/utils/BinaryGenericAsserter.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/utils/BinaryGenericAsserter.java new file mode 100644 index 000000000000..98af4a96c451 --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/utils/BinaryGenericAsserter.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.table.utils; + +import org.apache.flink.table.dataformat.BinaryGeneric; +import org.apache.flink.table.runtime.typeutils.BinaryGenericSerializer; +import org.apache.flink.table.runtime.util.SegmentsUtil; + +import org.hamcrest.Description; +import org.hamcrest.TypeSafeMatcher; + +import java.util.Arrays; + +/** + * A {@link org.hamcrest.Matcher} that allows equality check on {@link BinaryGeneric}s. + */ +public class BinaryGenericAsserter extends TypeSafeMatcher { + private final BinaryGeneric expected; + private final BinaryGenericSerializer serializer; + + private BinaryGenericAsserter( + BinaryGeneric expected, + BinaryGenericSerializer serializer) { + this.expected = expected; + this.serializer = serializer; + } + + /** + * Checks that the {@link BinaryGeneric} is equivalent to the expected one. The serializer will be used + * to ensure both objects are materialized into the binary form. + * + * @param expected the expected object + * @param serializer serializer used to materialize the underlying java object + * @return binary equality matcher + */ + @SuppressWarnings("unchecked") + public static BinaryGenericAsserter equivalent(BinaryGeneric expected, BinaryGenericSerializer serializer) { + expected.ensureMaterialized(serializer.getInnerSerializer()); + return new BinaryGenericAsserter(expected, serializer); + } + + @Override + @SuppressWarnings("unchecked") + protected boolean matchesSafely(BinaryGeneric item) { + item.ensureMaterialized(serializer.getInnerSerializer()); + expected.ensureMaterialized(serializer.getInnerSerializer()); + + return item.getSizeInBytes() == expected.getSizeInBytes() && + SegmentsUtil.equals( + item.getSegments(), + item.getOffset(), + expected.getSegments(), + expected.getOffset(), + item.getSizeInBytes()); + } + + @Override + public void describeTo(Description description) { + byte[] bytes = SegmentsUtil.getBytes(expected.getSegments(), expected.getOffset(), expected.getSizeInBytes()); + description.appendText(Arrays.toString(bytes)); + } +} From 6d865e26da224e6cdfae0845ee7d00cb405cb9c3 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 7 Nov 2019 10:48:16 +0100 Subject: [PATCH 471/746] [FLINK-14639][metrics][docs] Fix methods signature --- docs/monitoring/metrics.md | 2 +- docs/monitoring/metrics.zh.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/monitoring/metrics.md b/docs/monitoring/metrics.md index 7f6598ba187f..948d193f9403 100644 --- a/docs/monitoring/metrics.md +++ b/docs/monitoring/metrics.md @@ -433,7 +433,7 @@ You can configure which delimiter to use for the identifier (default: `.`) by se ### User Scope -You can define a user scope by calling `MetricGroup#addGroup(String name)`, `MetricGroup#addGroup(int name)` or `Metric#addGroup(String key, String value)`. +You can define a user scope by calling `MetricGroup#addGroup(String name)`, `MetricGroup#addGroup(int name)` or `MetricGroup#addGroup(String key, String value)`. These methods affect what `MetricGroup#getMetricIdentifier` and `MetricGroup#getScopeComponents` return.

    diff --git a/docs/monitoring/metrics.zh.md b/docs/monitoring/metrics.zh.md index a2b552d88042..798cbab516fb 100644 --- a/docs/monitoring/metrics.zh.md +++ b/docs/monitoring/metrics.zh.md @@ -433,7 +433,7 @@ You can configure which delimiter to use for the identifier (default: `.`) by se ### User Scope -You can define a user scope by calling `MetricGroup#addGroup(String name)`, `MetricGroup#addGroup(int name)` or `Metric#addGroup(String key, String value)`. +You can define a user scope by calling `MetricGroup#addGroup(String name)`, `MetricGroup#addGroup(int name)` or `MetricGroup#addGroup(String key, String value)`. These methods affect what `MetricGroup#getMetricIdentifier` and `MetricGroup#getScopeComponents` return.
    From edeec8d7420185d1c49b2739827bd921d2c2d485 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Sat, 26 Oct 2019 00:26:18 +0200 Subject: [PATCH 472/746] [hotfix][runtime] Replace all occurrences of letter to mail to unify wording of variables and documentation. --- .../runtime/tasks/mailbox/Mailbox.java | 10 ++-- .../tasks/mailbox/MailboxStateException.java | 2 +- .../runtime/tasks/mailbox/TaskMailbox.java | 10 ++-- .../tasks/mailbox/TaskMailboxImpl.java | 18 ++++---- .../execution/MailboxExecutorFactory.java | 2 +- .../execution/MailboxExecutorImpl.java | 2 +- .../mailbox/execution/MailboxProcessor.java | 46 +++++++++---------- .../tasks/mailbox/TaskMailboxImplTest.java | 14 +++--- .../execution/TaskMailboxProcessorTest.java | 2 +- 9 files changed, 53 insertions(+), 53 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/Mailbox.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/Mailbox.java index 6bc7501121b7..bc77fb393acc 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/Mailbox.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/Mailbox.java @@ -32,19 +32,19 @@ */ public interface Mailbox { /** - * Returns an optional with either the oldest letter from the mailbox (head of queue) if the mailbox is not empty or + * Returns an optional with either the oldest mail from the mailbox (head of queue) if the mailbox is not empty or * an empty optional otherwise. * - * @return an optional with either the oldest letter from the mailbox (head of queue) if the mailbox is not empty or + * @return an optional with either the oldest mail from the mailbox (head of queue) if the mailbox is not empty or * an empty optional otherwise. * @throws MailboxStateException if mailbox is already closed. */ Optional tryTake(int priority) throws MailboxStateException; /** - * This method returns the oldest letter from the mailbox (head of queue) or blocks until a letter is available. + * This method returns the oldest mail from the mailbox (head of queue) or blocks until a mail is available. * - * @return the oldest letter from the mailbox (head of queue). + * @return the oldest mail from the mailbox (head of queue). * @throws InterruptedException on interruption. * @throws MailboxStateException if mailbox is already closed. */ @@ -52,7 +52,7 @@ public interface Mailbox { Mail take(int priority) throws InterruptedException, MailboxStateException; /** - * Enqueues the given letter to the mailbox and blocks until there is capacity for a successful put. + * Enqueues the given mail to the mailbox and blocks until there is capacity for a successful put. * * @param mail the mail to enqueue. * @throws MailboxStateException if the mailbox is quiesced or closed. diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxStateException.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxStateException.java index a3168c72dbfb..75bc93c7b6c5 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxStateException.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxStateException.java @@ -20,7 +20,7 @@ /** * This exception signals that a method of the mailbox was invoked in a state that does not support the invocation, - * e.g. on the attempt to put a letter into a closed mailbox. + * e.g. on the attempt to put a mail into a closed mailbox. */ public class MailboxStateException extends Exception { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java index 78311cb61109..a9155ac3c39f 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java @@ -27,15 +27,15 @@ * In the open state, the mailbox supports put and take operations. * In the quiesced state, the mailbox supports only take operations. * - *

    Additionally, letters have a priority that can be used to retrieve only relevant letters. + *

    Additionally, mails have a priority that can be used to retrieve only relevant mails. */ public interface TaskMailbox extends Mailbox { /** - * The minimal priority for letters. The priority is used when no operator is associated with the letter. + * The minimal priority for mails. The priority is used when no operator is associated with the mail. */ int MIN_PRIORITY = -1; /** - * The maximal priority for letters. This priority indicates that the message should be performed before any letter + * The maximal priority for mails. This priority indicates that the message should be performed before any mail * associated with an operator. */ int MAX_PRIORITY = Integer.MAX_VALUE; @@ -55,9 +55,9 @@ enum State { /** * Close the mailbox. In this state, all pending and future put operations and all pending and future take - * operations will throw {@link MailboxStateException}. Returns all letters that were still enqueued. + * operations will throw {@link MailboxStateException}. Returns all mails that were still enqueued. * - * @return list with all letters that where enqueued in the mailbox at the time of closing. + * @return list with all mails that where enqueued in the mailbox at the time of closing. */ @Nonnull List close(); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java index 61889507a3b7..276ded4d7fdf 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java @@ -46,7 +46,7 @@ public class TaskMailboxImpl implements TaskMailbox { private final ReentrantLock lock; /** - * Internal queue of letters. + * Internal queue of mails. */ @GuardedBy("lock") private final LinkedList queue; @@ -58,7 +58,7 @@ public class TaskMailboxImpl implements TaskMailbox { private final Condition notEmpty; /** - * Number of letters in the mailbox. We track it separately from the queue#size to avoid locking on {@link #hasMail()}. + * Number of mails in the mailbox. We track it separately from the queue#size to avoid locking on {@link #hasMail()}. */ @GuardedBy("lock") private volatile int count; @@ -98,11 +98,11 @@ public Optional tryTake(int priority) throws MailboxStateException { final ReentrantLock lock = this.lock; lock.lockInterruptibly(); try { - Mail headLetter; - while ((headLetter = takeHeadInternal(priorty)) == null) { + Mail headMail; + while ((headMail = takeHeadInternal(priorty)) == null) { notEmpty.await(); } - return headLetter; + return headMail; } finally { lock.unlock(); } @@ -172,7 +172,7 @@ private Mail takeHeadInternal(int priority) throws MailboxStateException { return null; } - private void drainAllLetters(List drainInto) { + private void drainAllMails(List drainInto) { assert lock.isHeldByCurrentThread(); for (Mail mail : queue) { drainInto.add(mail.getRunnable()); @@ -229,12 +229,12 @@ public List close() { if (state == State.CLOSED) { return Collections.emptyList(); } - List droppedLetters = new ArrayList<>(count); - drainAllLetters(droppedLetters); + List droppedMails = new ArrayList<>(count); + drainAllMails(droppedMails); state = State.CLOSED; // to unblock all notEmpty.signalAll(); - return droppedLetters; + return droppedMails; } finally { lock.unlock(); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorFactory.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorFactory.java index 6449491b549c..1b9420c80879 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorFactory.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorFactory.java @@ -24,7 +24,7 @@ public interface MailboxExecutorFactory { /** - * Creates a new executor for the given priority. The priority is used when enqueuing new letters as well as + * Creates a new executor for the given priority. The priority is used when enqueuing new mails as well as * yielding. * * @param priority the priority of the mailbox executor. diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java index f44bb6bac5f3..d85d178b52d7 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java @@ -32,7 +32,7 @@ */ public final class MailboxExecutorImpl implements MailboxExecutor { - /** Reference to the thread that executes the mailbox letters. */ + /** Reference to the thread that executes the mailbox mails. */ @Nonnull private final Thread taskMailboxThread; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java index d627f75c00cc..20ca2c9b21bf 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java @@ -66,10 +66,10 @@ public class MailboxProcessor { /** Action that is repeatedly executed if no action request is in the mailbox. Typically record processing. */ private final MailboxDefaultAction mailboxDefaultAction; - /** The thread that executes the mailbox letters. */ + /** The thread that executes the mailbox mails. */ private final Thread mailboxThread; - /** A pre-created instance of mailbox executor that executes all letters. */ + /** A pre-created instance of mailbox executor that executes all mails. */ private final MailboxExecutor mainMailboxExecutor; /** Control flag to terminate the mailbox loop. Must only be accessed from mailbox thread. */ @@ -83,14 +83,14 @@ public class MailboxProcessor { private SuspendedMailboxDefaultAction suspendedDefaultAction; /** Special action that is used to terminate the mailbox loop. */ - private final Runnable mailboxPoisonLetter; + private final Runnable mailboxPoisonMail; public MailboxProcessor(MailboxDefaultAction mailboxDefaultAction) { this.mailboxDefaultAction = Preconditions.checkNotNull(mailboxDefaultAction); this.mailbox = new TaskMailboxImpl(); this.mailboxThread = Thread.currentThread(); this.mainMailboxExecutor = new MailboxExecutorImpl(mailbox, mailboxThread, TaskMailbox.MIN_PRIORITY); - this.mailboxPoisonLetter = () -> mailboxLoopRunning = false; + this.mailboxPoisonMail = () -> mailboxLoopRunning = false; this.mailboxLoopRunning = true; this.suspendedDefaultAction = null; } @@ -122,10 +122,10 @@ public void prepareClose() { * {@link java.util.concurrent.RunnableFuture} that are still contained in the mailbox. */ public void close() { - List droppedLetters = mailbox.close(); - if (!droppedLetters.isEmpty()) { - LOG.debug("Closing the mailbox dropped letters {}.", droppedLetters); - FutureUtils.cancelRunnableFutures(droppedLetters); + List droppedMails = mailbox.close(); + if (!droppedMails.isEmpty()) { + LOG.debug("Closing the mailbox dropped mails {}.", droppedMails); + FutureUtils.cancelRunnableFutures(droppedMails); } } @@ -154,11 +154,11 @@ public void runMailboxLoop() throws Exception { } /** - * Reports a throwable for rethrowing from the mailbox thread. This will clear and cancel all other pending letters. + * Reports a throwable for rethrowing from the mailbox thread. This will clear and cancel all other pending mails. * @param throwable to report by rethrowing from the mailbox loop. */ public void reportThrowable(Throwable throwable) { - sendPriorityLetter( + sendPriorityMail( () -> { throw new WrappingRuntimeException(throwable); }, @@ -170,16 +170,16 @@ public void reportThrowable(Throwable throwable) { */ public void allActionsCompleted() { mailbox.runExclusively(() -> { - // keep state check and poison letter enqueuing atomic, such that no intermediate #close may cause a - // MailboxStateException in #sendPriorityLetter. + // keep state check and poison mail enqueuing atomic, such that no intermediate #close may cause a + // MailboxStateException in #sendPriorityMail. if (mailbox.getState() == TaskMailbox.State.OPEN) { - sendPriorityLetter(mailboxPoisonLetter, "poison letter"); + sendPriorityMail(mailboxPoisonMail, "poison mail"); } }); } - private void sendPriorityLetter(Runnable priorityLetter, String descriptionFormat, Object... descriptionArgs) { - mainMailboxExecutor.executeFirst(priorityLetter, descriptionFormat, descriptionArgs); + private void sendPriorityMail(Runnable priorityMail, String descriptionFormat, Object... descriptionArgs) { + mainMailboxExecutor.executeFirst(priorityMail, descriptionFormat, descriptionArgs); } /** @@ -198,11 +198,11 @@ private boolean processMail(TaskMailbox mailbox) throws MailboxStateException, I return true; } - // TODO consider batched draining into list and/or limit number of executed letters - // Take letters in a non-blockingly and execute them. - Optional maybeLetter; - while (isMailboxLoopRunning() && (maybeLetter = mailbox.tryTake(MIN_PRIORITY)).isPresent()) { - maybeLetter.get().run(); + // TODO consider batched draining into list and/or limit number of executed mails + // Take mails in a non-blockingly and execute them. + Optional maybeMail; + while (isMailboxLoopRunning() && (maybeMail = mailbox.tryTake(MIN_PRIORITY)).isPresent()) { + maybeMail.get().run(); } // If the default action is currently not available, we can run a blocking mailbox execution until the default @@ -242,9 +242,9 @@ private boolean isMailboxLoopRunning() { * Helper method to make sure that the mailbox loop will check the control flow flags in the next iteration. */ private void ensureControlFlowSignalCheck() { - // Make sure that mailbox#hasMail is true via a dummy letter so that the flag change is noticed. + // Make sure that mailbox#hasMail is true via a dummy mail so that the flag change is noticed. if (!mailbox.hasMail()) { - sendPriorityLetter(() -> {}, "signal check"); + sendPriorityMail(() -> {}, "signal check"); } } @@ -281,7 +281,7 @@ public void resume() { if (isMailboxThread()) { resumeInternal(); } else { - sendPriorityLetter(this::resumeInternal, "resume default action"); + sendPriorityMail(this::resumeInternal, "resume default action"); } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImplTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImplTest.java index 0b6653ec060b..a4cd996eba77 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImplTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImplTest.java @@ -42,7 +42,7 @@ public class TaskMailboxImplTest { private static final Runnable NO_OP = () -> {}; - private static final Runnable POISON_LETTER = NO_OP; + private static final Runnable POISON_MAIL = NO_OP; private static final int DEFAULT_PRIORITY = 0; /** * Object under test. @@ -86,7 +86,7 @@ public void testContracts() throws Exception { Assert.assertFalse(taskMailbox.hasMail()); for (int i = 0; i < 10; ++i) { - final Mail mail = new Mail(NO_OP, DEFAULT_PRIORITY, "letter, DEFAULT_PRIORITY"); + final Mail mail = new Mail(NO_OP, DEFAULT_PRIORITY, "mail, DEFAULT_PRIORITY"); testObjects.add(mail); taskMailbox.put(mail); Assert.assertTrue(taskMailbox.hasMail()); @@ -230,12 +230,12 @@ private void testUnblocksInternal( */ private void testPutTake(FunctionWithException takeMethod) throws Exception { final int numThreads = 10; - final int numLettersPerThread = 1000; + final int numMailsPerThread = 1000; final int[] results = new int[numThreads]; Thread[] writerThreads = new Thread[numThreads]; Thread readerThread = new Thread(ThrowingRunnable.unchecked(() -> { Mail mail; - while ((mail = takeMethod.apply(taskMailbox)).getRunnable() != POISON_LETTER) { + while ((mail = takeMethod.apply(taskMailbox)).getRunnable() != POISON_MAIL) { mail.run(); } })); @@ -244,7 +244,7 @@ private void testPutTake(FunctionWithException takeMet for (int i = 0; i < writerThreads.length; ++i) { final int threadId = i; writerThreads[i] = new Thread(ThrowingRunnable.unchecked(() -> { - for (int k = 0; k < numLettersPerThread; ++k) { + for (int k = 0; k < numMailsPerThread; ++k) { taskMailbox.put(new Mail(() -> ++results[threadId], DEFAULT_PRIORITY, "result " + k)); } })); @@ -258,11 +258,11 @@ private void testPutTake(FunctionWithException takeMet writerThread.join(); } - taskMailbox.put(new Mail(POISON_LETTER, DEFAULT_PRIORITY, "POISON_LETTER, DEFAULT_PRIORITY")); + taskMailbox.put(new Mail(POISON_MAIL, DEFAULT_PRIORITY, "POISON_MAIL, DEFAULT_PRIORITY")); readerThread.join(); for (int perThreadResult : results) { - Assert.assertEquals(numLettersPerThread, perThreadResult); + Assert.assertEquals(numMailsPerThread, perThreadResult); } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/TaskMailboxProcessorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/TaskMailboxProcessorTest.java index cf58da39388f..9cac93457ec2 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/TaskMailboxProcessorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/TaskMailboxProcessorTest.java @@ -67,7 +67,7 @@ public void testShutdown() { } @Test - public void testRunDefaultActionAndLetters() throws Exception { + public void testRunDefaultActionAndMails() throws Exception { AtomicBoolean stop = new AtomicBoolean(false); MailboxThread mailboxThread = new MailboxThread() { @Override From bf7a402caf0197ba9640070444737974a775a8f6 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Sat, 26 Oct 2019 00:31:30 +0200 Subject: [PATCH 473/746] [hotfix][runtime] Replace MailboxStateException with IllegalStateException. MailboxStateException adds no information and makes using code throwing it more clunky then necessary. (Handling exception is limited to rethrowing since it indicates a programmatic error). --- .../runtime/tasks/mailbox/Mailbox.java | 16 ++++---- .../tasks/mailbox/MailboxStateException.java | 41 ------------------- .../runtime/tasks/mailbox/TaskMailbox.java | 4 +- .../tasks/mailbox/TaskMailboxImpl.java | 22 +++++----- .../mailbox/execution/MailboxExecutor.java | 4 +- .../execution/MailboxExecutorImpl.java | 31 +++++--------- .../mailbox/execution/MailboxProcessor.java | 3 +- .../tasks/mailbox/TaskMailboxImplTest.java | 34 +++++++-------- 8 files changed, 52 insertions(+), 103 deletions(-) delete mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxStateException.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/Mailbox.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/Mailbox.java index bc77fb393acc..ae89ca16c201 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/Mailbox.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/Mailbox.java @@ -37,33 +37,33 @@ public interface Mailbox { * * @return an optional with either the oldest mail from the mailbox (head of queue) if the mailbox is not empty or * an empty optional otherwise. - * @throws MailboxStateException if mailbox is already closed. + * @throws IllegalStateException if mailbox is already closed. */ - Optional tryTake(int priority) throws MailboxStateException; + Optional tryTake(int priority); /** * This method returns the oldest mail from the mailbox (head of queue) or blocks until a mail is available. * * @return the oldest mail from the mailbox (head of queue). * @throws InterruptedException on interruption. - * @throws MailboxStateException if mailbox is already closed. + * @throws IllegalStateException if mailbox is already closed. */ @Nonnull - Mail take(int priority) throws InterruptedException, MailboxStateException; + Mail take(int priority) throws InterruptedException; /** * Enqueues the given mail to the mailbox and blocks until there is capacity for a successful put. * * @param mail the mail to enqueue. - * @throws MailboxStateException if the mailbox is quiesced or closed. + * @throws IllegalStateException if the mailbox is quiesced or closed. */ - void put(Mail mail) throws MailboxStateException; + void put(Mail mail); /** * Adds the given action to the head of the mailbox. * * @param mail the mail to enqueue. - * @throws MailboxStateException if the mailbox is quiesced or closed. + * @throws IllegalStateException if the mailbox is quiesced or closed. */ - void putFirst(Mail mail) throws MailboxStateException; + void putFirst(Mail mail); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxStateException.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxStateException.java deleted file mode 100644 index 75bc93c7b6c5..000000000000 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxStateException.java +++ /dev/null @@ -1,41 +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.mailbox; - -/** - * This exception signals that a method of the mailbox was invoked in a state that does not support the invocation, - * e.g. on the attempt to put a mail into a closed mailbox. - */ -public class MailboxStateException extends Exception { - - MailboxStateException() { - } - - MailboxStateException(String message) { - super(message); - } - - MailboxStateException(String message, Throwable cause) { - super(message, cause); - } - - MailboxStateException(Throwable cause) { - super(cause); - } -} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java index a9155ac3c39f..358628e4678e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java @@ -49,13 +49,13 @@ enum State { /** * Quiesce the mailbox. In this state, the mailbox supports only take operations and all pending and future put - * operations will throw {@link MailboxStateException}. + * operations will throw {@link IllegalStateException}. */ void quiesce(); /** * Close the mailbox. In this state, all pending and future put operations and all pending and future take - * operations will throw {@link MailboxStateException}. Returns all mails that were still enqueued. + * operations will throw {@link IllegalStateException}. Returns all mails that were still enqueued. * * @return list with all mails that where enqueued in the mailbox at the time of closing. */ diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java index 276ded4d7fdf..672d677d2a1e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java @@ -83,7 +83,7 @@ public boolean hasMail() { } @Override - public Optional tryTake(int priority) throws MailboxStateException { + public Optional tryTake(int priority) { final ReentrantLock lock = this.lock; lock.lock(); try { @@ -94,7 +94,7 @@ public Optional tryTake(int priority) throws MailboxStateException { } @Override - public @Nonnull Mail take(int priorty) throws InterruptedException, MailboxStateException { + public @Nonnull Mail take(int priorty) throws InterruptedException { final ReentrantLock lock = this.lock; lock.lockInterruptibly(); try { @@ -111,7 +111,7 @@ public Optional tryTake(int priority) throws MailboxStateException { //------------------------------------------------------------------------------------------------------------------ @Override - public void put(@Nonnull Mail mail) throws MailboxStateException { + public void put(@Nonnull Mail mail) { final ReentrantLock lock = this.lock; lock.lock(); try { @@ -124,7 +124,7 @@ public void put(@Nonnull Mail mail) throws MailboxStateException { //------------------------------------------------------------------------------------------------------------------ @Override - public void putFirst(@Nonnull Mail mail) throws MailboxStateException { + public void putFirst(@Nonnull Mail mail) { final ReentrantLock lock = this.lock; lock.lock(); try { @@ -136,7 +136,7 @@ public void putFirst(@Nonnull Mail mail) throws MailboxStateException { //------------------------------------------------------------------------------------------------------------------ - private void putHeadInternal(Mail newHead) throws MailboxStateException { + private void putHeadInternal(Mail newHead) { assert lock.isHeldByCurrentThread(); checkPutStateConditions(); queue.addFirst(newHead); @@ -144,7 +144,7 @@ private void putHeadInternal(Mail newHead) throws MailboxStateException { notEmpty.signal(); } - private void putTailInternal(Mail newTail) throws MailboxStateException { + private void putTailInternal(Mail newTail) { assert lock.isHeldByCurrentThread(); checkPutStateConditions(); queue.addLast(newTail); @@ -157,7 +157,7 @@ private void incrementCountAndCheckOverflow() { } @Nullable - private Mail takeHeadInternal(int priority) throws MailboxStateException { + private Mail takeHeadInternal(int priority) { assert lock.isHeldByCurrentThread(); checkTakeStateConditions(); Iterator iterator = queue.iterator(); @@ -193,18 +193,18 @@ private boolean isTakeAbleState() { return state != State.CLOSED; } - private void checkPutStateConditions() throws MailboxStateException { + private void checkPutStateConditions() { final State state = this.state; if (!isPutAbleState()) { - throw new MailboxStateException("Mailbox is in state " + state + ", but is required to be in state " + + throw new IllegalStateException("Mailbox is in state " + state + ", but is required to be in state " + State.OPEN + " for put operations."); } } - private void checkTakeStateConditions() throws MailboxStateException { + private void checkTakeStateConditions() { final State state = this.state; if (!isTakeAbleState()) { - throw new MailboxStateException("Mailbox is in state " + state + ", but is required to be in state " + + throw new IllegalStateException("Mailbox is in state " + state + ", but is required to be in state " + State.OPEN + " or " + State.QUIESCED + " for take operations."); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutor.java index 4b54eceefee5..e47cd6bf3b04 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutor.java @@ -176,7 +176,7 @@ default void execute(@Nonnull Runnable command, String description) { * @throws InterruptedException on interruption. * @throws IllegalStateException if the mailbox is closed and can no longer supply runnables for yielding. */ - void yield() throws InterruptedException, IllegalStateException; + void yield() throws InterruptedException; /** * This methods attempts to run the command at the head of the mailbox. This is intended to be used by the mailbox @@ -187,7 +187,7 @@ default void execute(@Nonnull Runnable command, String description) { * @return true on successful yielding to another command, false if there was no command to yield to. * @throws IllegalStateException if the mailbox is closed and can no longer supply runnables for yielding. */ - boolean tryYield() throws IllegalStateException; + boolean tryYield(); /** * Check if the current thread is the mailbox thread. diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java index d85d178b52d7..19544177caac 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java @@ -20,7 +20,6 @@ import org.apache.flink.streaming.runtime.tasks.mailbox.Mail; import org.apache.flink.streaming.runtime.tasks.mailbox.Mailbox; -import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxStateException; import javax.annotation.Nonnull; @@ -59,7 +58,7 @@ public void execute( final Object... descriptionArgs) { try { mailbox.put(new Mail(command, priority, descriptionFormat, descriptionArgs)); - } catch (MailboxStateException mbex) { + } catch (IllegalStateException mbex) { throw new RejectedExecutionException(mbex); } } @@ -71,34 +70,26 @@ public void executeFirst( final Object... descriptionArgs) { try { mailbox.putFirst(new Mail(command, priority, descriptionFormat, descriptionArgs)); - } catch (MailboxStateException mbex) { + } catch (IllegalStateException mbex) { throw new RejectedExecutionException(mbex); } } @Override - public void yield() throws InterruptedException, IllegalStateException { + public void yield() throws InterruptedException { checkIsMailboxThread(); - try { - mailbox.take(priority).run(); - } catch (MailboxStateException e) { - throw new IllegalStateException("Mailbox can no longer supply runnables for yielding.", e); - } + mailbox.take(priority).run(); } @Override - public boolean tryYield() throws IllegalStateException { + public boolean tryYield() { checkIsMailboxThread(); - try { - Optional optionalMail = mailbox.tryTake(priority); - if (optionalMail.isPresent()) { - optionalMail.get().run(); - return true; - } else { - return false; - } - } catch (MailboxStateException e) { - throw new IllegalStateException("Mailbox can no longer supply runnables for yielding.", e); + Optional optionalMail = mailbox.tryTake(priority); + if (optionalMail.isPresent()) { + optionalMail.get().run(); + return true; + } else { + return false; } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java index 20ca2c9b21bf..fc5f047cdb12 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java @@ -20,7 +20,6 @@ import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.streaming.runtime.tasks.mailbox.Mail; -import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxStateException; import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox; import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailboxImpl; import org.apache.flink.util.Preconditions; @@ -188,7 +187,7 @@ private void sendPriorityMail(Runnable priorityMail, String descriptionFormat, O * changes. This keeps the hot path in {@link #runMailboxLoop()} free from any other flag checking, at the cost * that all flag changes must make sure that the mailbox signals mailbox#hasMail. */ - private boolean processMail(TaskMailbox mailbox) throws MailboxStateException, InterruptedException { + private boolean processMail(TaskMailbox mailbox) throws InterruptedException { // Doing this check is an optimization to only have a volatile read in the expected hot path, locks are only // acquired after this point. diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImplTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImplTest.java index a4cd996eba77..814bbee4fd70 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImplTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImplTest.java @@ -18,7 +18,6 @@ package org.apache.flink.streaming.runtime.tasks.mailbox; -import org.apache.flink.util.function.FunctionWithException; import org.apache.flink.util.function.RunnableWithException; import org.apache.flink.util.function.ThrowingRunnable; @@ -60,7 +59,7 @@ public void tearDown() { } @Test - public void testPutAsHead() throws Exception { + public void testPutAsHead() throws InterruptedException { Mail mailA = new Mail(() -> {}, MAX_PRIORITY, "mailA"); Mail mailB = new Mail(() -> {}, MAX_PRIORITY, "mailB"); @@ -81,7 +80,7 @@ public void testPutAsHead() throws Exception { } @Test - public void testContracts() throws Exception { + public void testContracts() throws InterruptedException { final Queue testObjects = new LinkedList<>(); Assert.assertFalse(taskMailbox.hasMail()); @@ -102,7 +101,7 @@ public void testContracts() throws Exception { * Test the producer-consumer pattern using the blocking methods on the mailbox. */ @Test - public void testConcurrentPutTakeBlocking() throws Exception { + public void testConcurrentPutTakeBlocking() throws InterruptedException { testPutTake(mailbox -> mailbox.take(DEFAULT_PRIORITY)); } @@ -110,7 +109,7 @@ public void testConcurrentPutTakeBlocking() throws Exception { * Test the producer-consumer pattern using the non-blocking methods & waits on the mailbox. */ @Test - public void testConcurrentPutTakeNonBlockingAndWait() throws Exception { + public void testConcurrentPutTakeNonBlockingAndWait() throws InterruptedException { testPutTake((mailbox -> { Optional optionalMail = mailbox.tryTake(DEFAULT_PRIORITY); while (!optionalMail.isPresent()) { @@ -134,12 +133,12 @@ public void testCloseUnblocks() throws InterruptedException { * Test that silencing the mailbox unblocks pending accesses with correct exceptions. */ @Test - public void testQuiesceUnblocks() throws Exception { + public void testQuiesceUnblocks() throws InterruptedException { testAllPuttingUnblocksInternal(TaskMailbox::quiesce); } @Test - public void testLifeCycleQuiesce() throws Exception { + public void testLifeCycleQuiesce() throws InterruptedException { taskMailbox.put(new Mail(NO_OP, DEFAULT_PRIORITY, "NO_OP, DEFAULT_PRIORITY")); taskMailbox.put(new Mail(NO_OP, DEFAULT_PRIORITY, "NO_OP, DEFAULT_PRIORITY")); taskMailbox.quiesce(); @@ -150,33 +149,33 @@ public void testLifeCycleQuiesce() throws Exception { } @Test - public void testLifeCycleClose() throws Exception { + public void testLifeCycleClose() throws InterruptedException { taskMailbox.close(); testLifecyclePuttingInternal(); try { taskMailbox.take(DEFAULT_PRIORITY); Assert.fail(); - } catch (MailboxStateException ignore) { + } catch (IllegalStateException ignore) { } try { taskMailbox.tryTake(DEFAULT_PRIORITY); Assert.fail(); - } catch (MailboxStateException ignore) { + } catch (IllegalStateException ignore) { } } - private void testLifecyclePuttingInternal() throws Exception { + private void testLifecyclePuttingInternal() { try { taskMailbox.put(new Mail(NO_OP, DEFAULT_PRIORITY, "NO_OP, DEFAULT_PRIORITY")); Assert.fail(); - } catch (MailboxStateException ignore) { + } catch (IllegalStateException ignore) { } try { taskMailbox.putFirst(new Mail(NO_OP, MAX_PRIORITY, "NO_OP")); Assert.fail(); - } catch (MailboxStateException ignore) { + } catch (IllegalStateException ignore) { } } @@ -220,7 +219,7 @@ private void testUnblocksInternal( } for (Exception exception : exceptions) { - Assert.assertEquals(MailboxStateException.class, exception.getClass()); + Assert.assertEquals(IllegalStateException.class, exception.getClass()); } } @@ -228,7 +227,8 @@ private void testUnblocksInternal( /** * Test producer-consumer pattern through the mailbox in a concurrent setting (n-writer / 1-reader). */ - private void testPutTake(FunctionWithException takeMethod) throws Exception { + private void testPutTake(FunctionWithException takeMethod) + throws InterruptedException { final int numThreads = 10; final int numMailsPerThread = 1000; final int[] results = new int[numThreads]; @@ -267,7 +267,7 @@ private void testPutTake(FunctionWithException takeMet } @Test - public void testPutAsHeadWithPriority() throws Exception { + public void testPutAsHeadWithPriority() throws InterruptedException { Mail mailA = new Mail(() -> {}, 2, "mailA"); Mail mailB = new Mail(() -> {}, 2, "mailB"); @@ -290,7 +290,7 @@ public void testPutAsHeadWithPriority() throws Exception { } @Test - public void testPutWithPriorityAndReadingFromMainMailbox() throws Exception { + public void testPutWithPriorityAndReadingFromMainMailbox() throws InterruptedException { Mail mailA = new Mail(() -> {}, 2, "mailA"); Mail mailB = new Mail(() -> {}, 2, "mailB"); From 14fde468f9e8ce709b6e6c05c7d0f7910e515b01 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Sun, 27 Oct 2019 20:12:43 +0100 Subject: [PATCH 474/746] [hotfix][runtime] Added TaskMailbox#drain and using it to implement MailboxProcessor. Also changed return types of TaskMailbox#close to Mail for unification. --- .../streaming/runtime/tasks/StreamTask.java | 6 ++--- .../runtime/tasks/mailbox/TaskMailbox.java | 9 ++++++- .../tasks/mailbox/TaskMailboxImpl.java | 25 +++++++++++-------- .../mailbox/execution/MailboxProcessor.java | 20 +++++++++++++-- .../execution/MailboxExecutorImplTest.java | 6 +++-- 5 files changed, 47 insertions(+), 19 deletions(-) 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 c276d488fb44..351f11f9f27b 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 @@ -449,10 +449,8 @@ public final void invoke() throws Exception { // See FLINK-7430 isRunning = false; } - MailboxExecutor mainMailboxExecutor = mailboxProcessor.getMainMailboxExecutor(); - while (mainMailboxExecutor.tryYield()) { - // Run until we have processed all remaining letters. - } + // processes the remaining mails; no new mails can be enqueued + mailboxProcessor.drain(); // make sure all timers finish timerService.awaitPendingAfterQuiesce(); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java index 358628e4678e..e32e8a1bb2b7 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java @@ -47,6 +47,13 @@ enum State { OPEN, QUIESCED, CLOSED } + /** + * Drains the mailbox and returns all mails that were still enqueued. + * + * @return list with all mails that where enqueued in the mailbox. + */ + List drain(); + /** * Quiesce the mailbox. In this state, the mailbox supports only take operations and all pending and future put * operations will throw {@link IllegalStateException}. @@ -60,7 +67,7 @@ enum State { * @return list with all mails that where enqueued in the mailbox at the time of closing. */ @Nonnull - List close(); + List close(); /** * Returns the current state of the mailbox as defined by the lifecycle enum {@link State}. diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java index 672d677d2a1e..bb6d28f7f0c3 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java @@ -172,13 +172,17 @@ private Mail takeHeadInternal(int priority) { return null; } - private void drainAllMails(List drainInto) { - assert lock.isHeldByCurrentThread(); - for (Mail mail : queue) { - drainInto.add(mail.getRunnable()); + @Override + public List drain() { + final ReentrantLock lock = this.lock; + lock.lock(); + try { + List drainedMails = new ArrayList<>(queue); + queue.clear(); + return drainedMails; + } finally { + lock.unlock(); } - queue.clear(); - count = 0; } private boolean isEmpty() { @@ -211,26 +215,27 @@ private void checkTakeStateConditions() { @Override public void quiesce() { + final ReentrantLock lock = this.lock; lock.lock(); try { if (state == State.OPEN) { state = State.QUIESCED; } } finally { - lock.unlock(); + this.lock.unlock(); } } @Nonnull @Override - public List close() { + public List close() { + final ReentrantLock lock = this.lock; lock.lock(); try { if (state == State.CLOSED) { return Collections.emptyList(); } - List droppedMails = new ArrayList<>(count); - drainAllMails(droppedMails); + List droppedMails = drain(); state = State.CLOSED; // to unblock all notEmpty.signalAll(); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java index fc5f047cdb12..cb2e85dbff73 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java @@ -28,6 +28,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.ArrayList; import java.util.List; import java.util.Optional; @@ -121,10 +122,25 @@ public void prepareClose() { * {@link java.util.concurrent.RunnableFuture} that are still contained in the mailbox. */ public void close() { - List droppedMails = mailbox.close(); + List droppedMails = mailbox.close(); if (!droppedMails.isEmpty()) { LOG.debug("Closing the mailbox dropped mails {}.", droppedMails); - FutureUtils.cancelRunnableFutures(droppedMails); + List runnables = new ArrayList<>(); + for (Mail droppedMail : droppedMails) { + Runnable runnable = droppedMail.getRunnable(); + runnables.add(runnable); + } + FutureUtils.cancelRunnableFutures(runnables); + } + } + + /** + * Finishes running all mails in the mailbox. If no concurrent write operations occurred, the mailbox must be + * empty after this method. + */ + public void drain() { + for (final Mail mail : mailbox.drain()) { + mail.run(); } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImplTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImplTest.java index 023c56ad5e64..519f63629364 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImplTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImplTest.java @@ -19,6 +19,7 @@ package org.apache.flink.streaming.runtime.tasks.mailbox.execution; import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.streaming.runtime.tasks.mailbox.Mail; import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailboxImpl; import org.apache.flink.util.Preconditions; @@ -34,6 +35,7 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -93,10 +95,10 @@ public void testOperations() throws Exception { Assert.assertEquals(Thread.currentThread(), yieldRun.wasExecutedBy()); assertFalse(leftoverFuture.isDone()); - List leftoverTasks = mailbox.close(); + List leftoverTasks = mailbox.close(); Assert.assertEquals(1, leftoverTasks.size()); assertFalse(leftoverFuture.isCancelled()); - FutureUtils.cancelRunnableFutures(leftoverTasks); + FutureUtils.cancelRunnableFutures(leftoverTasks.stream().map(Mail::getRunnable).collect(Collectors.toList())); assertTrue(leftoverFuture.isCancelled()); try { From f4e78efdec5996406a2245ab2b60d7383114cb3f Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Sun, 27 Oct 2019 20:18:04 +0100 Subject: [PATCH 475/746] [hotfix][runtime] Simplified and tweaked TaskMailboxImpl. In benchmarks, a separate count field or state imposed (small) additional costs. --- .../tasks/mailbox/TaskMailboxImpl.java | 82 +++++++------------ .../mailbox/execution/MailboxProcessor.java | 25 +++--- 2 files changed, 43 insertions(+), 64 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java index bb6d28f7f0c3..5fb4fce782ea 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java @@ -18,68 +18,61 @@ package org.apache.flink.streaming.runtime.tasks.mailbox; -import org.apache.flink.util.Preconditions; - import javax.annotation.Nonnull; import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; import javax.annotation.concurrent.ThreadSafe; +import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Collections; +import java.util.Deque; import java.util.Iterator; -import java.util.LinkedList; import java.util.List; import java.util.Optional; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; +import static org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox.State.*; + /** * Implementation of {@link TaskMailbox} in a {@link java.util.concurrent.BlockingQueue} fashion and tailored towards - * our use case with multiple writers, single reader and volatile reads. + * our use case with multiple writers and single reader. */ @ThreadSafe public class TaskMailboxImpl implements TaskMailbox { /** * Lock for all concurrent ops. */ - private final ReentrantLock lock; + private final ReentrantLock lock = new ReentrantLock(); /** * Internal queue of mails. */ @GuardedBy("lock") - private final LinkedList queue; + private final Deque queue = new ArrayDeque<>(); /** * Condition that is triggered when the mailbox is no longer empty. */ @GuardedBy("lock") - private final Condition notEmpty; - - /** - * Number of mails in the mailbox. We track it separately from the queue#size to avoid locking on {@link #hasMail()}. - */ - @GuardedBy("lock") - private volatile int count; + private final Condition notEmpty = lock.newCondition(); /** * The state of the mailbox in the lifecycle of open, quiesced, and closed. */ @GuardedBy("lock") - private volatile State state; - - public TaskMailboxImpl() { - this.lock = new ReentrantLock(); - this.notEmpty = lock.newCondition(); - this.state = State.OPEN; - this.queue = new LinkedList<>(); - this.count = 0; - } + private State state = OPEN; @Override public boolean hasMail() { - return !isEmpty(); + final ReentrantLock lock = this.lock; + lock.lock(); + try { + return !queue.isEmpty(); + } finally { + lock.unlock(); + } } @Override @@ -140,7 +133,6 @@ private void putHeadInternal(Mail newHead) { assert lock.isHeldByCurrentThread(); checkPutStateConditions(); queue.addFirst(newHead); - incrementCountAndCheckOverflow(); notEmpty.signal(); } @@ -148,14 +140,9 @@ private void putTailInternal(Mail newTail) { assert lock.isHeldByCurrentThread(); checkPutStateConditions(); queue.addLast(newTail); - incrementCountAndCheckOverflow(); notEmpty.signal(); } - private void incrementCountAndCheckOverflow() { - Preconditions.checkState(++count > 0, "Mailbox overflow."); - } - @Nullable private Mail takeHeadInternal(int priority) { assert lock.isHeldByCurrentThread(); @@ -164,7 +151,6 @@ private Mail takeHeadInternal(int priority) { while (iterator.hasNext()) { Mail mail = iterator.next(); if (mail.getPriority() >= priority) { - --count; iterator.remove(); return mail; } @@ -185,31 +171,19 @@ public List drain() { } } - private boolean isEmpty() { - return count == 0; - } - - private boolean isPutAbleState() { - return state == State.OPEN; - } - - private boolean isTakeAbleState() { - return state != State.CLOSED; - } - private void checkPutStateConditions() { final State state = this.state; - if (!isPutAbleState()) { + if (this.state != OPEN) { throw new IllegalStateException("Mailbox is in state " + state + ", but is required to be in state " + - State.OPEN + " for put operations."); + OPEN + " for put operations."); } } private void checkTakeStateConditions() { final State state = this.state; - if (!isTakeAbleState()) { + if (this.state == CLOSED) { throw new IllegalStateException("Mailbox is in state " + state + ", but is required to be in state " + - State.OPEN + " or " + State.QUIESCED + " for take operations."); + OPEN + " or " + QUIESCED + " for take operations."); } } @@ -218,8 +192,8 @@ public void quiesce() { final ReentrantLock lock = this.lock; lock.lock(); try { - if (state == State.OPEN) { - state = State.QUIESCED; + if (state == OPEN) { + state = QUIESCED; } } finally { this.lock.unlock(); @@ -232,11 +206,11 @@ public List close() { final ReentrantLock lock = this.lock; lock.lock(); try { - if (state == State.CLOSED) { + if (state == CLOSED) { return Collections.emptyList(); } List droppedMails = drain(); - state = State.CLOSED; + state = CLOSED; // to unblock all notEmpty.signalAll(); return droppedMails; @@ -248,7 +222,13 @@ public List close() { @Nonnull @Override public State getState() { - return state; + final ReentrantLock lock = this.lock; + lock.lock(); + try { + return state; + } finally { + lock.unlock(); + } } @Override diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java index cb2e85dbff73..9ea3bc378ea7 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java @@ -35,23 +35,22 @@ import static org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox.MIN_PRIORITY; /** - * This class encapsulates the logic of the mailbox-based execution model. At the core of this model - * {@link #runMailboxLoop()} that continuously executes the provided {@link MailboxDefaultAction} in a loop. On each - * iteration, the method also checks if there are pending actions in the mailbox and executes such actions. This model - * ensures single-threaded execution between the default action (e.g. record processing) and mailbox actions (e.g. - * checkpoint trigger, timer firing, ...). + * This class encapsulates the logic of the mailbox-based execution model. At the core of this model {@link + * #runMailboxLoop()} that continuously executes the provided {@link MailboxDefaultAction} in a loop. On each iteration, + * the method also checks if there are pending actions in the mailbox and executes such actions. This model ensures + * single-threaded execution between the default action (e.g. record processing) and mailbox actions (e.g. checkpoint + * trigger, timer firing, ...). * *

    The {@link MailboxDefaultAction} interacts with this class through the {@link MailboxDefaultActionContext} to - * communicate control flow changes to the mailbox loop, e.g. that invocations of the default action are temporarily - * or permanently exhausted. + * communicate control flow changes to the mailbox loop, e.g. that invocations of the default action are temporarily or + * permanently exhausted. * *

    The design of {@link #runMailboxLoop()} is centered around the idea of keeping the expected hot path - * (default action, no mail) as fast as possible, with just a single volatile read per iteration in - * {@link TaskMailbox#hasMail}. This means that all checking of mail and other control flags (mailboxLoopRunning, - * suspendedDefaultAction) are always connected to #hasMail indicating true. This means that control flag changes in - * the mailbox thread can be done directly, but we must ensure that there is at least one action in the mailbox so that - * the change is picked up. For control flag changes by all other threads, that must happen through mailbox actions, - * this is automatically the case. + * (default action, no mail) as fast as possible. This means that all checking of mail and other control flags + * (mailboxLoopRunning, suspendedDefaultAction) are always connected to #hasMail indicating true. This means that + * control flag changes in the mailbox thread can be done directly, but we must ensure that there is at least one action + * in the mailbox so that the change is picked up. For control flag changes by all other threads, that must happen + * through mailbox actions, this is automatically the case. * *

    This class has a open-prepareClose-close lifecycle that is connected with and maps to the lifecycle of the * encapsulated {@link TaskMailbox} (which is open-quiesce-close). From 05c691543aaa12b1b034ee637ff384b606b98d32 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Sun, 27 Oct 2019 21:04:46 +0100 Subject: [PATCH 476/746] [FLINK-14304][runtime] Adding batch capabilities to TaskMailbox, moved mailbox thread ownership into mailbox to sharpen the threading model, and simplified hierarchy. A batch is a local copy of mails that avoid task starvation and also reduces the amount of synchronization (as batch is thread-local). While the performance improvement is modest (5-10%), it allows a more predictable interleaving of mails and input. Also clarified threading model in javadoc. --- .../runtime/tasks/mailbox/Mailbox.java | 69 -------- .../runtime/tasks/mailbox/TaskMailbox.java | 143 ++++++++++++++-- .../tasks/mailbox/TaskMailboxImpl.java | 116 ++++++++++--- .../mailbox/execution/MailboxExecutor.java | 14 +- .../execution/MailboxExecutorImpl.java | 39 +---- .../mailbox/execution/MailboxProcessor.java | 29 ++-- .../operators/MailboxOperatorTest.java | 160 ++++++++++++++++++ .../tasks/mailbox/TaskMailboxImplTest.java | 78 +++++++-- .../execution/TaskMailboxProcessorTest.java | 36 ++++ 9 files changed, 503 insertions(+), 181 deletions(-) delete mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/Mailbox.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/MailboxOperatorTest.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/Mailbox.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/Mailbox.java deleted file mode 100644 index ae89ca16c201..000000000000 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/Mailbox.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.streaming.runtime.tasks.mailbox; - -import javax.annotation.Nonnull; - -import java.util.Optional; - -/** - * A mailbox is basically a queue for inter-thread message exchange in form of {@link Runnable} objects between multiple - * producer threads and a single consumer. - * - *

    This interface combines the {@link Mailbox} and {@link Mailbox} side without life-cycle methods. - * - * @see TaskMailbox - */ -public interface Mailbox { - /** - * Returns an optional with either the oldest mail from the mailbox (head of queue) if the mailbox is not empty or - * an empty optional otherwise. - * - * @return an optional with either the oldest mail from the mailbox (head of queue) if the mailbox is not empty or - * an empty optional otherwise. - * @throws IllegalStateException if mailbox is already closed. - */ - Optional tryTake(int priority); - - /** - * This method returns the oldest mail from the mailbox (head of queue) or blocks until a mail is available. - * - * @return the oldest mail from the mailbox (head of queue). - * @throws InterruptedException on interruption. - * @throws IllegalStateException if mailbox is already closed. - */ - @Nonnull - Mail take(int priority) throws InterruptedException; - - /** - * Enqueues the given mail to the mailbox and blocks until there is capacity for a successful put. - * - * @param mail the mail to enqueue. - * @throws IllegalStateException if the mailbox is quiesced or closed. - */ - void put(Mail mail); - - /** - * Adds the given action to the head of the mailbox. - * - * @param mail the mail to enqueue. - * @throws IllegalStateException if the mailbox is quiesced or closed. - */ - void putFirst(Mail mail); -} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java index e32e8a1bb2b7..102ff80370e1 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java @@ -18,18 +18,48 @@ package org.apache.flink.streaming.runtime.tasks.mailbox; +import org.apache.flink.annotation.Internal; + import javax.annotation.Nonnull; import java.util.List; +import java.util.Optional; /** - * A task mailbox wraps the basic {@link Mailbox} interface with a lifecycle of open -> (quiesced) -> closed. - * In the open state, the mailbox supports put and take operations. - * In the quiesced state, the mailbox supports only take operations. + * A task mailbox provides read and write access to a mailbox and has a lifecycle of open -> (quiesced) -> closed. Mails + * have a priority that can be used to retrieve only relevant letters. + * + *

    Threading model

    + * The mailbox is bound to a mailbox thread passed during creation. Most operations may only occur through that thread. + * Write operations ({@link #put(Mail)}, {@link #putFirst(Mail)}) can be executed by any thread. All other methods can + * only be invoked by the mailbox thread, which is passed upon construction. To verify that the current thread is + * allowed to take any mail, use {@link #isMailboxThread()}, but all methods will perform the check themselves and fail + * accordingly if called from another thread. + * + *

    Life cycle

    + * In the open state, the mailbox supports put and take operations. In the quiesced state, the mailbox supports only + * take operations. + * + *

    Batch

    + * A batch is a local view on the mailbox that does not contain simultaneously added mails similar to iterators of + * copy-on-write collections. * - *

    Additionally, mails have a priority that can be used to retrieve only relevant mails. + *

    A batch serves two purposes: it reduces synchronization if more than one mail is processable at the time of the + * creation of a batch. Furthermore, it allows to divide the work of a mailbox in smaller logical chunks, such that the + * task threads cannot be blocked by a mail that enqueues itself and thus provides input starvation. + * + *

    A batch is created with {@link #createBatch()} and consumed with {@link #tryTakeFromBatch()}. Note that there is + * no blocking {@code takeFromBatch} as batches can only be created and consumed from the * mailbox thread. + * + *

    Also note that a batch can only be created in the {@link MailboxProcessor#runMailboxLoop()}. A batch must not + * be extended in any of the consuming methods as we may run into task input starvation again. Assume a case where + * the mailbox loop handles a timeout event that produces a record. That record is subsequently handled downstream, + * where it may lead to a {@link MailboxExecutor#yield()} triggering another consumption method. If we extend the + * batch in any way during that processing, we effectively lose the bounded processing guarantee of mails inside the + * mailbox loop. */ -public interface TaskMailbox extends Mailbox { +@Internal +public interface TaskMailbox { /** * The minimal priority for mails. The priority is used when no operator is associated with the mail. */ @@ -40,6 +70,104 @@ public interface TaskMailbox extends Mailbox { */ int MAX_PRIORITY = Integer.MAX_VALUE; + /** + * Check if the current thread is the mailbox thread. + * + *

    Read operations will fail if they are called from another thread. + * + * @return only true if called from the mailbox thread. + */ + boolean isMailboxThread(); + + /** + * Returns true if the mailbox contains mail. + * + *

    Must be called from the mailbox thread ({@link #isMailboxThread()}. + */ + boolean hasMail(); + + /** + * Returns an optional with either the oldest mail from the mailbox (head of queue) if the mailbox is not empty or + * an empty optional otherwise. + * + *

    Must be called from the mailbox thread ({@link #isMailboxThread()}. + * + * @return an optional with either the oldest mail from the mailbox (head of queue) if the mailbox is not empty or + * an empty optional otherwise. + * @throws IllegalStateException if mailbox is already closed. + */ + Optional tryTake(int priority); + + /** + * This method returns the oldest mail from the mailbox (head of queue) or blocks until a mail is available. + * + *

    Must be called from the mailbox thread ({@link #isMailboxThread()}. + * + * @return the oldest mail from the mailbox (head of queue). + * @throws InterruptedException on interruption. + * @throws IllegalStateException if mailbox is already closed. + */ + @Nonnull + Mail take(int priority) throws InterruptedException; + + // --- Batch + + /** + * Creates a batch of mails that can be taken with {@link #tryTakeFromBatch()}. The batch does not affect + * {@link #tryTake(int)} and {@link #take(int)}; that is, they return the same mails even if no batch would have + * been created. + * + *

    The default batch is empty. Thus, this method must be invoked once before {@link #tryTakeFromBatch()}. + * + *

    If a batch is not completely consumed by {@link #tryTakeFromBatch()}, its elements are carried over to the + * new batch. + * + *

    Must be called from the mailbox thread ({@link #isMailboxThread()}. + * + * @return true if there is at least one element in the batch; that is, if there is any mail at all at the time + * of the invocation. + */ + boolean createBatch(); + + /** + * Returns an optional with either the oldest mail from the batch (head of queue) if the batch is not empty or an + * empty optional otherwise. + * + *

    Must be called from the mailbox thread ({@link #isMailboxThread()}. + * + *

    Note that there is no blocking {@code takeFromBatch} as batches can only be created and consumed from the + * mailbox thread. + * + * @return an optional with either the oldest mail from the batch (head of queue) if the batch is not empty or an + * empty optional otherwise. + * @throws IllegalStateException if mailbox is already closed. + */ + Optional tryTakeFromBatch(); + + // --- Write methods + + /** + * Enqueues the given mail to the mailbox and blocks until there is capacity for a successful put. + * + *

    Mails can be added from any thread. + * + * @param mail the mail to enqueue. + * @throws IllegalStateException if the mailbox is quiesced or closed. + */ + void put(Mail mail); + + /** + * Adds the given action to the head of the mailbox. + * + *

    Mails can be added from any thread. + * + * @param mail the mail to enqueue. + * @throws IllegalStateException if the mailbox is quiesced or closed. + */ + void putFirst(Mail mail); + + // --- Lifecycle methods + /** * This enum represents the states of the mailbox lifecycle. */ @@ -77,11 +205,6 @@ enum State { @Nonnull State getState(); - /** - * Returns true if the mailbox contains mail. - */ - boolean hasMail(); - /** * Runs the given code exclusively on this mailbox. No synchronized operations can be run concurrently to the * given runnable (e.g., {@link #put(Mail)} or modifying lifecycle methods). diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java index 5fb4fce782ea..dce262c4f347 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImpl.java @@ -18,6 +18,8 @@ package org.apache.flink.streaming.runtime.tasks.mailbox; +import org.apache.flink.annotation.VisibleForTesting; + import javax.annotation.Nonnull; import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; @@ -33,7 +35,9 @@ import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; -import static org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox.State.*; +import static org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox.State.CLOSED; +import static org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox.State.OPEN; +import static org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox.State.QUIESCED; /** * Implementation of {@link TaskMailbox} in a {@link java.util.concurrent.BlockingQueue} fashion and tailored towards @@ -64,8 +68,39 @@ public class TaskMailboxImpl implements TaskMailbox { @GuardedBy("lock") private State state = OPEN; + /** + * Reference to the thread that executes the mailbox mails. + */ + @Nonnull + private final Thread taskMailboxThread; + + /** + * The current batch of mails. A new batch can be created with {@link #createBatch()} and consumed with {@link + * #tryTakeFromBatch()}. + */ + private final Deque batch = new ArrayDeque<>(); + + public TaskMailboxImpl(@Nonnull final Thread taskMailboxThread) { + this.taskMailboxThread = taskMailboxThread; + } + + @VisibleForTesting + public TaskMailboxImpl() { + this(Thread.currentThread()); + } + + @Override + public boolean isMailboxThread() { + return Thread.currentThread() == taskMailboxThread; + } + @Override public boolean hasMail() { + checkIsMailboxThread(); + if (!batch.isEmpty()) { + return true; + } + final ReentrantLock lock = this.lock; lock.lock(); try { @@ -77,22 +112,30 @@ public boolean hasMail() { @Override public Optional tryTake(int priority) { + Optional head = tryTakeFromBatch(); + if (head.isPresent()) { + return head; + } final ReentrantLock lock = this.lock; lock.lock(); try { - return Optional.ofNullable(takeHeadInternal(priority)); + return Optional.ofNullable(takeOrNull(queue, priority)); } finally { lock.unlock(); } } @Override - public @Nonnull Mail take(int priorty) throws InterruptedException { + public @Nonnull Mail take(int priorty) throws InterruptedException, IllegalStateException { + Optional head = tryTakeFromBatch(); + if (head.isPresent()) { + return head.get(); + } final ReentrantLock lock = this.lock; lock.lockInterruptibly(); try { Mail headMail; - while ((headMail = takeHeadInternal(priorty)) == null) { + while ((headMail = takeOrNull(queue, priorty)) == null) { notEmpty.await(); } return headMail; @@ -104,48 +147,64 @@ public Optional tryTake(int priority) { //------------------------------------------------------------------------------------------------------------------ @Override - public void put(@Nonnull Mail mail) { + public boolean createBatch() { + checkIsMailboxThread(); final ReentrantLock lock = this.lock; lock.lock(); try { - putTailInternal(mail); + Mail mail; + while ((mail = queue.pollFirst()) != null) { + batch.addLast(mail); + } + return !batch.isEmpty(); } finally { lock.unlock(); } } + @Override + public Optional tryTakeFromBatch() { + checkIsMailboxThread(); + return Optional.ofNullable(takeOrNull(batch, MIN_PRIORITY)); + } + //------------------------------------------------------------------------------------------------------------------ @Override - public void putFirst(@Nonnull Mail mail) { + public void put(@Nonnull Mail mail) { final ReentrantLock lock = this.lock; lock.lock(); try { - putHeadInternal(mail); + checkPutStateConditions(); + queue.addLast(mail); + notEmpty.signal(); } finally { lock.unlock(); } } - //------------------------------------------------------------------------------------------------------------------ - - private void putHeadInternal(Mail newHead) { - assert lock.isHeldByCurrentThread(); - checkPutStateConditions(); - queue.addFirst(newHead); - notEmpty.signal(); + @Override + public void putFirst(@Nonnull Mail mail) { + if (isMailboxThread()) { + checkPutStateConditions(); + batch.addFirst(mail); + } else { + final ReentrantLock lock = this.lock; + lock.lock(); + try { + checkPutStateConditions(); + queue.addFirst(mail); + notEmpty.signal(); + } finally { + lock.unlock(); + } + } } - private void putTailInternal(Mail newTail) { - assert lock.isHeldByCurrentThread(); - checkPutStateConditions(); - queue.addLast(newTail); - notEmpty.signal(); - } + //------------------------------------------------------------------------------------------------------------------ @Nullable - private Mail takeHeadInternal(int priority) { - assert lock.isHeldByCurrentThread(); + private Mail takeOrNull(Deque queue, int priority) { checkTakeStateConditions(); Iterator iterator = queue.iterator(); while (iterator.hasNext()) { @@ -160,10 +219,12 @@ private Mail takeHeadInternal(int priority) { @Override public List drain() { + List drainedMails = new ArrayList<>(batch); + batch.clear(); final ReentrantLock lock = this.lock; lock.lock(); try { - List drainedMails = new ArrayList<>(queue); + drainedMails.addAll(queue); queue.clear(); return drainedMails; } finally { @@ -171,6 +232,13 @@ public List drain() { } } + private void checkIsMailboxThread() { + if (!isMailboxThread()) { + throw new IllegalStateException( + "Illegal thread detected. This method must be called from inside the mailbox thread!"); + } + } + private void checkPutStateConditions() { final State state = this.state; if (this.state != OPEN) { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutor.java index e47cd6bf3b04..4ebcb4182a90 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutor.java @@ -18,7 +18,7 @@ package org.apache.flink.streaming.runtime.tasks.mailbox.execution; -import org.apache.flink.streaming.runtime.tasks.mailbox.Mailbox; +import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox; import javax.annotation.Nonnull; @@ -30,7 +30,10 @@ import java.util.concurrent.RejectedExecutionException; /** - * Interface for an {@link Executor} build around a {@link Mailbox}-based execution model. + * Interface for an {@link Executor} build around a mailbox-based execution model (see {@link TaskMailbox}). + * + *

    All submission functions can be called from any thread and will enqueue the action for further processing in a + * FIFO fashion. */ public interface MailboxExecutor { /** @@ -189,13 +192,6 @@ default void execute(@Nonnull Runnable command, String description) { */ boolean tryYield(); - /** - * Check if the current thread is the mailbox thread. - * - * @return only true if called from the mailbox thread. - */ - boolean isMailboxThread(); - /** * Provides an {@link Executor} view on this {@code MailboxExecutor}, where submitted tasks will receive the * given description. The {@link Executor} can be used with {@link java.util.concurrent.CompletableFuture}. diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java index 19544177caac..f7be8b21f004 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java @@ -19,7 +19,7 @@ package org.apache.flink.streaming.runtime.tasks.mailbox.execution; import org.apache.flink.streaming.runtime.tasks.mailbox.Mail; -import org.apache.flink.streaming.runtime.tasks.mailbox.Mailbox; +import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox; import javax.annotation.Nonnull; @@ -31,23 +31,14 @@ */ public final class MailboxExecutorImpl implements MailboxExecutor { - /** Reference to the thread that executes the mailbox mails. */ - @Nonnull - private final Thread taskMailboxThread; - /** The mailbox that manages the submitted runnable objects. */ @Nonnull - private final Mailbox mailbox; + private final TaskMailbox mailbox; private final int priority; - public MailboxExecutorImpl(@Nonnull Mailbox mailbox, int priority) { - this(mailbox, Thread.currentThread(), priority); - } - - public MailboxExecutorImpl(@Nonnull Mailbox mailbox, @Nonnull Thread taskMailboxThread, int priority) { + public MailboxExecutorImpl(@Nonnull TaskMailbox mailbox, int priority) { this.mailbox = mailbox; - this.taskMailboxThread = taskMailboxThread; this.priority = priority; } @@ -77,13 +68,11 @@ public void executeFirst( @Override public void yield() throws InterruptedException { - checkIsMailboxThread(); mailbox.take(priority).run(); } @Override public boolean tryYield() { - checkIsMailboxThread(); Optional optionalMail = mailbox.tryTake(priority); if (optionalMail.isPresent()) { optionalMail.get().run(); @@ -92,26 +81,4 @@ public boolean tryYield() { return false; } } - - @Override - public boolean isMailboxThread() { - return Thread.currentThread() == taskMailboxThread; - } - - /** - * Returns the mailbox that manages the execution order. - * - * @return the mailbox. - */ - @Nonnull - public Mailbox getMailbox() { - return mailbox; - } - - private void checkIsMailboxThread() { - if (!isMailboxThread()) { - throw new IllegalStateException( - "Illegal thread detected. This method must be called from inside the mailbox thread!"); - } - } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java index 9ea3bc378ea7..40cdbc801ef9 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java @@ -65,9 +65,6 @@ public class MailboxProcessor { /** Action that is repeatedly executed if no action request is in the mailbox. Typically record processing. */ private final MailboxDefaultAction mailboxDefaultAction; - /** The thread that executes the mailbox mails. */ - private final Thread mailboxThread; - /** A pre-created instance of mailbox executor that executes all mails. */ private final MailboxExecutor mainMailboxExecutor; @@ -86,9 +83,8 @@ public class MailboxProcessor { public MailboxProcessor(MailboxDefaultAction mailboxDefaultAction) { this.mailboxDefaultAction = Preconditions.checkNotNull(mailboxDefaultAction); - this.mailbox = new TaskMailboxImpl(); - this.mailboxThread = Thread.currentThread(); - this.mainMailboxExecutor = new MailboxExecutorImpl(mailbox, mailboxThread, TaskMailbox.MIN_PRIORITY); + this.mailbox = new TaskMailboxImpl(Thread.currentThread()); + this.mainMailboxExecutor = new MailboxExecutorImpl(mailbox, TaskMailbox.MIN_PRIORITY); this.mailboxPoisonMail = () -> mailboxLoopRunning = false; this.mailboxLoopRunning = true; this.suspendedDefaultAction = null; @@ -106,7 +102,7 @@ public MailboxExecutor getMainMailboxExecutor() { * @param priority */ public MailboxExecutor getMailboxExecutor(int priority) { - return new MailboxExecutorImpl(mailbox, mailboxThread, priority); + return new MailboxExecutorImpl(mailbox, priority); } /** @@ -143,21 +139,17 @@ public void drain() { } } - private boolean isMailboxThread() { - return Thread.currentThread() == mailboxThread; - } - /** * Runs the mailbox processing loop. This is where the main work is done. */ public void runMailboxLoop() throws Exception { + final TaskMailbox localMailbox = mailbox; + Preconditions.checkState( - isMailboxThread(), + localMailbox.isMailboxThread(), "Method must be executed by declared mailbox thread!"); - final TaskMailbox localMailbox = mailbox; - assert localMailbox.getState() == TaskMailbox.State.OPEN : "Mailbox must be opened!"; final MailboxDefaultActionContext defaultActionContext = new MailboxDefaultActionContext(this); @@ -206,16 +198,15 @@ private boolean processMail(TaskMailbox mailbox) throws InterruptedException { // Doing this check is an optimization to only have a volatile read in the expected hot path, locks are only // acquired after this point. - if (!mailbox.hasMail()) { + if (!mailbox.createBatch()) { // We can also directly return true because all changes to #isMailboxLoopRunning must be connected to // mailbox.hasMail() == true. return true; } - // TODO consider batched draining into list and/or limit number of executed mails // Take mails in a non-blockingly and execute them. Optional maybeMail; - while (isMailboxLoopRunning() && (maybeMail = mailbox.tryTake(MIN_PRIORITY)).isPresent()) { + while (isMailboxLoopRunning() && (maybeMail = mailbox.tryTakeFromBatch()).isPresent()) { maybeMail.get().run(); } @@ -234,7 +225,7 @@ private boolean processMail(TaskMailbox mailbox) throws InterruptedException { */ private SuspendedMailboxDefaultAction suspendDefaultAction() { - Preconditions.checkState(isMailboxThread(), "Suspending must only be called from the mailbox thread!"); + Preconditions.checkState(mailbox.isMailboxThread(), "Suspending must only be called from the mailbox thread!"); if (suspendedDefaultAction == null) { suspendedDefaultAction = new SuspendDefaultActionRunnable(); @@ -292,7 +283,7 @@ private final class SuspendDefaultActionRunnable implements SuspendedMailboxDefa @Override public void resume() { - if (isMailboxThread()) { + if (mailbox.isMailboxThread()) { resumeInternal(); } else { sendPriorityMail(this::resumeInternal, "resume default action"); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/MailboxOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/MailboxOperatorTest.java new file mode 100644 index 000000000000..9e8ff5da32bf --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/MailboxOperatorTest.java @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.operators; + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.runtime.jobgraph.OperatorID; +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.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.YieldingOperatorFactory; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask; +import org.apache.flink.streaming.runtime.tasks.OneInputStreamTaskTestHarness; +import org.apache.flink.streaming.runtime.tasks.StreamTask; +import org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxExecutor; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.RejectedExecutionException; +import java.util.stream.Collectors; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; + +/** + * Test to verify that timer triggers are run according to operator precedence (combined with yield() at operator + * level). + */ +public class MailboxOperatorTest extends TestLogger { + + @Test + public void testAvoidTaskStarvation() throws Exception { + final OneInputStreamTaskTestHarness testHarness = new OneInputStreamTaskTestHarness<>( + OneInputStreamTask::new, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO); + + testHarness.setupOperatorChain(new OperatorID(), new ReplicatingMailOperatorFactory()) + .chain(new OperatorID(), new ReplicatingMailOperatorFactory(), IntSerializer.INSTANCE) + .finish(); + + testHarness.invoke(); + testHarness.waitForTaskRunning(); + + testHarness.processElement(new StreamRecord<>(0)); + testHarness.processElement(new StreamRecord<>(0)); + testHarness.processElement(new StreamRecord<>(0)); + + testHarness.endInput(); + testHarness.waitForTaskCompletion(); + + // with each input two mails should be processed, one of each operator in the chain + List numMailsProcessed = testHarness.getOutput().stream() + .map(element -> ((StreamRecord) element).getValue()) + .collect(Collectors.toList()); + assertThat(numMailsProcessed, is(Arrays.asList(0, 2, 4))); + } + + private static class ReplicatingMailOperatorFactory implements OneInputStreamOperatorFactory, + YieldingOperatorFactory { + private MailboxExecutor mailboxExecutor; + + @Override + public void setMailboxExecutor(MailboxExecutor mailboxExecutor) { + this.mailboxExecutor = mailboxExecutor; + } + + @Override + public > Operator createStreamOperator( + StreamTask containingTask, + StreamConfig config, + Output> output) { + ReplicatingMailOperator operator = new ReplicatingMailOperator(mailboxExecutor); + operator.setup(containingTask, config, output); + return (Operator) operator; + } + + @Override + public void setChainingStrategy(ChainingStrategy strategy) { + } + + @Override + public ChainingStrategy getChainingStrategy() { + return ChainingStrategy.ALWAYS; + } + + @Override + public Class getStreamOperatorClass(ClassLoader classLoader) { + return ReplicatingMailOperator.class; + } + } + + private static class ReplicatingMailOperator extends AbstractStreamOperator + implements OneInputStreamOperator { + private final ReplicatingMail replicatingMail; + + ReplicatingMailOperator(final MailboxExecutor mailboxExecutor) { + replicatingMail = new ReplicatingMail(mailboxExecutor); + } + + @Override + public void processElement(StreamRecord upstreamMailCount) throws Exception { + // for the very first element, enqueue one mail that replicates itself + if (!replicatingMail.hasBeenEnqueued()) { + replicatingMail.run(); + } + // output how many mails have been processed so far (from upstream and this operator) + output.collect(new StreamRecord<>(replicatingMail.getMailCount() + upstreamMailCount.getValue())); + } + } + + private static class ReplicatingMail implements Runnable { + private int mailCount = -1; + private final MailboxExecutor mailboxExecutor; + + ReplicatingMail(final MailboxExecutor mailboxExecutor) { + this.mailboxExecutor = mailboxExecutor; + } + + @Override + public void run() { + try { + mailboxExecutor.execute(this, "Blocking mail" + ++mailCount); + } catch (RejectedExecutionException e) { + // during shutdown the executor will reject new mails, which is fine for us. + } + } + + boolean hasBeenEnqueued() { + return mailCount > -1; + } + + int getMailCount() { + return mailCount; + } + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImplTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImplTest.java index 814bbee4fd70..023f750e31fe 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImplTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxImplTest.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.runtime.tasks.mailbox; +import org.apache.flink.util.function.FunctionWithException; import org.apache.flink.util.function.RunnableWithException; import org.apache.flink.util.function.ThrowingRunnable; @@ -26,11 +27,16 @@ import org.junit.Before; import org.junit.Test; +import java.util.Arrays; +import java.util.Collections; import java.util.LinkedList; +import java.util.List; import java.util.Optional; import java.util.Queue; import java.util.concurrent.CountDownLatch; import java.util.function.Consumer; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import static org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox.MAX_PRIORITY; import static org.junit.Assert.assertEquals; @@ -92,8 +98,8 @@ public void testContracts() throws InterruptedException { } while (!testObjects.isEmpty()) { - Assert.assertEquals(testObjects.remove(), taskMailbox.take(DEFAULT_PRIORITY)); - Assert.assertEquals(!testObjects.isEmpty(), taskMailbox.hasMail()); + assertEquals(testObjects.remove(), taskMailbox.take(DEFAULT_PRIORITY)); + assertEquals(!testObjects.isEmpty(), taskMailbox.hasMail()); } } @@ -219,7 +225,7 @@ private void testUnblocksInternal( } for (Exception exception : exceptions) { - Assert.assertEquals(IllegalStateException.class, exception.getClass()); + assertEquals(IllegalStateException.class, exception.getClass()); } } @@ -227,20 +233,13 @@ private void testUnblocksInternal( /** * Test producer-consumer pattern through the mailbox in a concurrent setting (n-writer / 1-reader). */ - private void testPutTake(FunctionWithException takeMethod) + private void testPutTake(FunctionWithException takeMethod) throws InterruptedException { final int numThreads = 10; final int numMailsPerThread = 1000; final int[] results = new int[numThreads]; Thread[] writerThreads = new Thread[numThreads]; - Thread readerThread = new Thread(ThrowingRunnable.unchecked(() -> { - Mail mail; - while ((mail = takeMethod.apply(taskMailbox)).getRunnable() != POISON_MAIL) { - mail.run(); - } - })); - - readerThread.start(); + for (int i = 0; i < writerThreads.length; ++i) { final int threadId = i; writerThreads[i] = new Thread(ThrowingRunnable.unchecked(() -> { @@ -260,9 +259,12 @@ private void testPutTake(FunctionWithExceptionBoth {@link TaskMailbox#take(int)} and {@link TaskMailbox#tryTake(int)} consume the batch but once drained + * will fetch elements from the remaining mails. + * + *

    In contrast, {@link TaskMailbox#tryTakeFromBatch()} will not return any mail once the batch is drained. + */ + @Test + public void testBatchAndNonBatchTake() throws InterruptedException { + final List mails = IntStream.range(0, 6).mapToObj(i -> new Mail( + NO_OP, + DEFAULT_PRIORITY, + String.valueOf(i))).collect(Collectors.toList()); + + // create a batch with 3 mails + mails.subList(0, 3).forEach(taskMailbox::put); + Assert.assertTrue(taskMailbox.createBatch()); + // add 3 more mails after the batch + mails.subList(3, 6).forEach(taskMailbox::put); + + // now take all mails in the batch with all available methods + assertEquals(Optional.ofNullable(mails.get(0)), taskMailbox.tryTakeFromBatch()); + assertEquals(Optional.ofNullable(mails.get(1)), taskMailbox.tryTake(DEFAULT_PRIORITY)); + assertEquals(mails.get(2), taskMailbox.take(DEFAULT_PRIORITY)); + + // batch empty, so only regular methods work + assertEquals(Optional.empty(), taskMailbox.tryTakeFromBatch()); + assertEquals(Optional.ofNullable(mails.get(3)), taskMailbox.tryTake(DEFAULT_PRIORITY)); + assertEquals(mails.get(4), taskMailbox.take(DEFAULT_PRIORITY)); + + // one unprocessed mail left + assertEquals(Collections.singletonList(mails.get(5)), taskMailbox.close()); + } + + @Test + public void testBatchDrain() throws Exception { + + Mail mailA = new Mail(() -> {}, MAX_PRIORITY, "mailA"); + Mail mailB = new Mail(() -> {}, MAX_PRIORITY, "mailB"); + + taskMailbox.put(mailA); + Assert.assertTrue(taskMailbox.createBatch()); + taskMailbox.put(mailB); + + assertEquals(Arrays.asList(mailA, mailB), taskMailbox.drain()); + } + /** * Testing that we cannot close while running exclusively. */ diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/TaskMailboxProcessorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/TaskMailboxProcessorTest.java index 9cac93457ec2..506b466dda5f 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/TaskMailboxProcessorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/TaskMailboxProcessorTest.java @@ -218,6 +218,42 @@ private static MailboxProcessor start(MailboxThread mailboxThread) { return mailboxProcessor; } + /** + * FLINK-14304: Avoid newly spawned letters to prevent input processing from ever happening. + */ + @Test + public void testAvoidStarvation() throws Exception { + + final int expectedInvocations = 3; + final AtomicInteger counter = new AtomicInteger(0); + MailboxThread mailboxThread = new MailboxThread() { + @Override + public void runDefaultAction(DefaultActionContext context) { + if (counter.incrementAndGet() == expectedInvocations) { + context.allActionsCompleted(); + } + } + }; + + mailboxThread.start(); + final MailboxProcessor mailboxProcessor = mailboxThread.getMailboxProcessor(); + final MailboxExecutor mailboxExecutor = mailboxProcessor.getMailboxExecutor(DEFAULT_PRIORITY); + AtomicInteger index = new AtomicInteger(); + mailboxExecutor.execute( + new Runnable() { + @Override + public void run() { + mailboxExecutor.execute(this, "Blocking mail" + index.incrementAndGet()); + } + }, + "Blocking mail" + index); + + mailboxThread.signalStart(); + stop(mailboxThread); + Assert.assertEquals(expectedInvocations, counter.get()); + Assert.assertEquals(expectedInvocations, index.get()); + } + private static void stop(MailboxThread mailboxThread) throws Exception { mailboxThread.join(); MailboxProcessor mailboxProcessor = mailboxThread.getMailboxProcessor(); From 7111507ad7f336caf94b46471bd794fc0399986a Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Sun, 27 Oct 2019 21:16:22 +0100 Subject: [PATCH 477/746] [hotfix][runtime/core] Cleaning up MailProcessor and adding utility method to WrappingRuntimeException. --- .../flink/util/WrappingRuntimeException.java | 13 +++++++++++++ .../mailbox/execution/MailboxProcessor.java | 19 ++++++++----------- 2 files changed, 21 insertions(+), 11 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/util/WrappingRuntimeException.java b/flink-core/src/main/java/org/apache/flink/util/WrappingRuntimeException.java index 7b230be4cef3..e882f9d36aff 100644 --- a/flink-core/src/main/java/org/apache/flink/util/WrappingRuntimeException.java +++ b/flink-core/src/main/java/org/apache/flink/util/WrappingRuntimeException.java @@ -51,4 +51,17 @@ public Throwable unwrap() { Throwable cause = getCause(); return (cause instanceof WrappingRuntimeException) ? ((WrappingRuntimeException) cause).unwrap() : cause; } + + /** + * Ensures that any throwable can be thrown as a checked exception by potentially wrapping it. + * + * @return a runtime exception wrapping the throwable if checked or by returning the throwable if it's a runtime + * exception. + */ + public static RuntimeException wrapIfNecessary(Throwable throwable) { + if (throwable instanceof RuntimeException) { + return (RuntimeException) throwable; + } + return new WrappingRuntimeException(throwable); + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java index 40cdbc801ef9..801f899ad89c 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java @@ -78,16 +78,12 @@ public class MailboxProcessor { */ private SuspendedMailboxDefaultAction suspendedDefaultAction; - /** Special action that is used to terminate the mailbox loop. */ - private final Runnable mailboxPoisonMail; - public MailboxProcessor(MailboxDefaultAction mailboxDefaultAction) { this.mailboxDefaultAction = Preconditions.checkNotNull(mailboxDefaultAction); - this.mailbox = new TaskMailboxImpl(Thread.currentThread()); - this.mainMailboxExecutor = new MailboxExecutorImpl(mailbox, TaskMailbox.MIN_PRIORITY); - this.mailboxPoisonMail = () -> mailboxLoopRunning = false; - this.mailboxLoopRunning = true; - this.suspendedDefaultAction = null; + mailbox = new TaskMailboxImpl(Thread.currentThread()); + mainMailboxExecutor = new MailboxExecutorImpl(mailbox, TaskMailbox.MIN_PRIORITY); + mailboxLoopRunning = true; + suspendedDefaultAction = null; } /** @@ -99,7 +95,8 @@ public MailboxExecutor getMainMailboxExecutor() { /** * Returns an executor service facade to submit actions to the mailbox. - * @param priority + * + * @param priority the priority of the {@link MailboxExecutor}. */ public MailboxExecutor getMailboxExecutor(int priority) { return new MailboxExecutorImpl(mailbox, priority); @@ -166,7 +163,7 @@ public void runMailboxLoop() throws Exception { public void reportThrowable(Throwable throwable) { sendPriorityMail( () -> { - throw new WrappingRuntimeException(throwable); + throw WrappingRuntimeException.wrapIfNecessary(throwable); }, "Report throwable %s", throwable); } @@ -179,7 +176,7 @@ public void allActionsCompleted() { // keep state check and poison mail enqueuing atomic, such that no intermediate #close may cause a // MailboxStateException in #sendPriorityMail. if (mailbox.getState() == TaskMailbox.State.OPEN) { - sendPriorityMail(mailboxPoisonMail, "poison mail"); + sendPriorityMail(() -> mailboxLoopRunning = false, "poison mail"); } }); } From f62d98e1d8440b216bf7af7940f4a6818a111a2a Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Tue, 5 Nov 2019 11:34:38 +0100 Subject: [PATCH 478/746] [hotfix][runtime] Consolidating javadoc and package structure. --- .../operators}/MailboxExecutor.java | 56 +++++++++++++++---- .../operators/StreamOperatorFactoryUtil.java | 3 +- .../operators/YieldingOperatorFactory.java | 2 - .../operators/async/AsyncWaitOperator.java | 2 +- .../async/AsyncWaitOperatorFactory.java | 2 +- .../runtime/tasks/OperatorChain.java | 2 +- .../runtime/tasks/SourceStreamTask.java | 2 +- .../runtime/tasks/StreamIterationHead.java | 2 +- .../streaming/runtime/tasks/StreamTask.java | 10 ++-- .../{execution => }/DefaultActionContext.java | 20 ++++--- .../{execution => }/MailboxDefaultAction.java | 17 +++--- .../MailboxExecutorFactory.java | 6 +- .../{execution => }/MailboxExecutorImpl.java | 22 ++++---- .../{execution => }/MailboxProcessor.java | 23 ++++---- .../SuspendedMailboxDefaultAction.java | 32 +++++++++++ .../runtime/tasks/mailbox/TaskMailbox.java | 1 + .../SuspendedMailboxDefaultAction.java | 30 ---------- .../operators/MailboxOperatorTest.java | 2 +- .../StreamTaskOperatorTimerTest.java | 2 +- .../tasks/StreamTaskSelectiveReadingTest.java | 2 +- .../tasks/StreamTaskTerminationTest.java | 2 +- .../runtime/tasks/StreamTaskTest.java | 2 +- .../tasks/SynchronousCheckpointITCase.java | 2 +- .../tasks/SynchronousCheckpointTest.java | 2 +- .../tasks/TaskCheckpointingBehaviourTest.java | 2 +- .../MailboxExecutorImplTest.java | 20 +++---- .../TaskMailboxProcessorTest.java | 18 +++--- .../flink/streaming/util/MockStreamTask.java | 2 +- 28 files changed, 163 insertions(+), 125 deletions(-) rename flink-streaming-java/src/main/java/org/apache/flink/streaming/{runtime/tasks/mailbox/execution => api/operators}/MailboxExecutor.java (79%) rename flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/{execution => }/DefaultActionContext.java (65%) rename flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/{execution => }/MailboxDefaultAction.java (63%) rename flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/{execution => }/MailboxExecutorFactory.java (87%) rename flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/{execution => }/MailboxExecutorImpl.java (73%) rename flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/{execution => }/MailboxProcessor.java (93%) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/SuspendedMailboxDefaultAction.java delete mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/SuspendedMailboxDefaultAction.java rename flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/{execution => }/MailboxExecutorImplTest.java (88%) rename flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/{execution => }/TaskMailboxProcessorTest.java (93%) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/MailboxExecutor.java similarity index 79% rename from flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutor.java rename to flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/MailboxExecutor.java index 4ebcb4182a90..d54182db250e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/MailboxExecutor.java @@ -1,13 +1,12 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this 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,8 +15,10 @@ * limitations under the License. */ -package org.apache.flink.streaming.runtime.tasks.mailbox.execution; +package org.apache.flink.streaming.api.operators; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.streaming.runtime.tasks.mailbox.Mail; import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox; import javax.annotation.Nonnull; @@ -30,11 +31,46 @@ import java.util.concurrent.RejectedExecutionException; /** - * Interface for an {@link Executor} build around a mailbox-based execution model (see {@link TaskMailbox}). + * Interface for an {@link Executor} build around a mailbox-based execution model (see {@link TaskMailbox}). {@code + * MailboxExecutor} can also execute downstream messages of a mailbox by yielding control from the task thread. * *

    All submission functions can be called from any thread and will enqueue the action for further processing in a * FIFO fashion. + * + *

    The yielding functions avoid the following situation: One operator cannot fully process an input record and + * blocks the task thread until some resources are available. However, since the introduction of the mailbox model + * blocking the task thread will not only block new inputs but also all events from being processed. If the resources + * depend on downstream operators being able to process such events (e.g., timers), then we may easily arrive at some + * livelocks. + * + *

    The yielding functions will only process events from the operator itself and any downstream operator. Events of upstream + * operators are only processed when the input has been fully processed or if they yield themselves. This method avoid + * congestion and potential deadlocks, but will process {@link Mail}s slightly out-of-order, effectively creating a view + * on the mailbox that contains no message from upstream operators. + * + *

    All yielding functions must be called in the mailbox thread (see {@link TaskMailbox#isMailboxThread()}) to not + * violate the single-threaded execution model. There are two typical cases, both waiting until the resource is + * available. The main difference is if the resource becomes available through a mailbox message itself or not. + * + *

    If the resource becomes available through a mailbox mail, we can effectively block the task thread. + * Implicitly, this requires the mail to be enqueued by a different thread. + *

    {@code
    + * while (resource not available) {
    + *     mailboxExecutor.yield();
    + * }
    + * }
    + * + *

    If the resource becomes available through an external mechanism or the corresponding mail needs to be enqueued + * in the task thread, we cannot block. + *

    {@code
    + * while (resource not available) {
    + *     if (!mailboxExecutor.tryYield()) {
    + *         do stuff or sleep for a small amount of time
    + *     }
    + * }
    + * }
    */ +@PublicEvolving public interface MailboxExecutor { /** * A constant for empty args to save on object allocation. diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorFactoryUtil.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorFactoryUtil.java index 6c469d96a8b3..6653b3d62ecf 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorFactoryUtil.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorFactoryUtil.java @@ -20,8 +20,7 @@ import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.StreamTask; -import org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxExecutor; -import org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxExecutorFactory; +import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxExecutorFactory; /** * A utility to instantiate new operators with a given factory. diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/YieldingOperatorFactory.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/YieldingOperatorFactory.java index 207136424b63..054ad8d51d22 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/YieldingOperatorFactory.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/YieldingOperatorFactory.java @@ -17,8 +17,6 @@ package org.apache.flink.streaming.api.operators; -import org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxExecutor; - /** * An operator that needs access to the {@link MailboxExecutor} to yield to downstream operators needs to be created * through a factory implementing this interface. diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java index 7d6d5ab00e30..84a3e883fa8f 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java @@ -31,6 +31,7 @@ import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator; import org.apache.flink.streaming.api.operators.ChainingStrategy; +import org.apache.flink.streaming.api.operators.MailboxExecutor; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.TimestampedCollector; @@ -42,7 +43,6 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.StreamTask; -import org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxExecutor; import org.apache.flink.util.Preconditions; import javax.annotation.Nonnull; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorFactory.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorFactory.java index addd67c1ce4a..6f393645d3ae 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorFactory.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorFactory.java @@ -21,12 +21,12 @@ import org.apache.flink.streaming.api.functions.async.AsyncFunction; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.ChainingStrategy; +import org.apache.flink.streaming.api.operators.MailboxExecutor; import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.YieldingOperatorFactory; import org.apache.flink.streaming.runtime.tasks.StreamTask; -import org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxExecutor; /** * The factory of {@link AsyncWaitOperator}. diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java index fa7faf23678f..9acf8cd8e4c3 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java @@ -54,7 +54,7 @@ import org.apache.flink.streaming.runtime.streamstatus.StreamStatus; import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; import org.apache.flink.streaming.runtime.streamstatus.StreamStatusProvider; -import org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxExecutorFactory; +import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxExecutorFactory; import org.apache.flink.util.OutputTag; import org.apache.flink.util.XORShiftRandom; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java index 3dc3dd2a8d79..1ed3fb4ba566 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java @@ -25,7 +25,7 @@ import org.apache.flink.streaming.api.checkpoint.ExternallyInducedSource; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.operators.StreamSource; -import org.apache.flink.streaming.runtime.tasks.mailbox.execution.DefaultActionContext; +import org.apache.flink.streaming.runtime.tasks.mailbox.DefaultActionContext; import org.apache.flink.util.FlinkException; import java.util.Optional; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java index 68db112cb202..8732f0310aad 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java @@ -24,7 +24,7 @@ import org.apache.flink.streaming.runtime.io.BlockingQueueBroker; import org.apache.flink.streaming.runtime.io.RecordWriterOutput; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.mailbox.execution.DefaultActionContext; +import org.apache.flink.streaming.runtime.tasks.mailbox.DefaultActionContext; import org.apache.flink.util.FlinkRuntimeException; import org.slf4j.Logger; 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 351f11f9f27b..b297288042e4 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 @@ -53,6 +53,7 @@ import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.graph.StreamEdge; +import org.apache.flink.streaming.api.operators.MailboxExecutor; import org.apache.flink.streaming.api.operators.OperatorSnapshotFinalizer; import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures; import org.apache.flink.streaming.api.operators.StreamOperator; @@ -65,12 +66,11 @@ import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; +import org.apache.flink.streaming.runtime.tasks.mailbox.DefaultActionContext; +import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxExecutorFactory; +import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor; +import org.apache.flink.streaming.runtime.tasks.mailbox.SuspendedMailboxDefaultAction; import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox; -import org.apache.flink.streaming.runtime.tasks.mailbox.execution.DefaultActionContext; -import org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxExecutor; -import org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxExecutorFactory; -import org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxProcessor; -import org.apache.flink.streaming.runtime.tasks.mailbox.execution.SuspendedMailboxDefaultAction; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/DefaultActionContext.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/DefaultActionContext.java similarity index 65% rename from flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/DefaultActionContext.java rename to flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/DefaultActionContext.java index ba933e98f9af..dee148dd61bb 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/DefaultActionContext.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/DefaultActionContext.java @@ -1,13 +1,12 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this 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,12 +15,15 @@ * limitations under the License. */ -package org.apache.flink.streaming.runtime.tasks.mailbox.execution; +package org.apache.flink.streaming.runtime.tasks.mailbox; + +import org.apache.flink.annotation.Internal; /** * This context is a feedback interface for the default action to interact with the mailbox execution. In particular * it offers ways to signal that the execution of the default action should be finished or temporarily suspended. */ +@Internal public interface DefaultActionContext { /** diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxDefaultAction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxDefaultAction.java similarity index 63% rename from flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxDefaultAction.java rename to flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxDefaultAction.java index 21fc19101a6d..10d3732b1a06 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxDefaultAction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxDefaultAction.java @@ -1,13 +1,12 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this 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,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.runtime.tasks.mailbox.execution; +package org.apache.flink.streaming.runtime.tasks.mailbox; import org.apache.flink.annotation.Internal; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorFactory.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxExecutorFactory.java similarity index 87% rename from flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorFactory.java rename to flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxExecutorFactory.java index 1b9420c80879..465d71cbe1dd 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorFactory.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxExecutorFactory.java @@ -15,12 +15,16 @@ * limitations under the License. */ -package org.apache.flink.streaming.runtime.tasks.mailbox.execution; +package org.apache.flink.streaming.runtime.tasks.mailbox; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.api.operators.MailboxExecutor; /** * A factory for creating mailbox executors with a given priority. The factory is usually bound to a specific task. */ @FunctionalInterface +@Internal public interface MailboxExecutorFactory { /** diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxExecutorImpl.java similarity index 73% rename from flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java rename to flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxExecutorImpl.java index f7be8b21f004..301de3fb5ec9 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxExecutorImpl.java @@ -1,13 +1,12 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this 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,10 +15,10 @@ * limitations under the License. */ -package org.apache.flink.streaming.runtime.tasks.mailbox.execution; +package org.apache.flink.streaming.runtime.tasks.mailbox; -import org.apache.flink.streaming.runtime.tasks.mailbox.Mail; -import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox; +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.api.operators.MailboxExecutor; import javax.annotation.Nonnull; @@ -29,6 +28,7 @@ /** * Implementation of an executor service build around a mailbox-based execution model. */ +@Internal public final class MailboxExecutorImpl implements MailboxExecutor { /** The mailbox that manages the submitted runnable objects. */ diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxProcessor.java similarity index 93% rename from flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java rename to flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxProcessor.java index 801f899ad89c..a3415c991415 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxProcessor.java @@ -1,13 +1,12 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this 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,12 +15,11 @@ * limitations under the License. */ -package org.apache.flink.streaming.runtime.tasks.mailbox.execution; +package org.apache.flink.streaming.runtime.tasks.mailbox; +import org.apache.flink.annotation.Internal; import org.apache.flink.runtime.concurrent.FutureUtils; -import org.apache.flink.streaming.runtime.tasks.mailbox.Mail; -import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox; -import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailboxImpl; +import org.apache.flink.streaming.api.operators.MailboxExecutor; import org.apache.flink.util.Preconditions; import org.apache.flink.util.WrappingRuntimeException; @@ -55,6 +53,7 @@ *

    This class has a open-prepareClose-close lifecycle that is connected with and maps to the lifecycle of the * encapsulated {@link TaskMailbox} (which is open-quiesce-close). */ +@Internal public class MailboxProcessor { private static final Logger LOG = LoggerFactory.getLogger(MailboxProcessor.class); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/SuspendedMailboxDefaultAction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/SuspendedMailboxDefaultAction.java new file mode 100644 index 000000000000..7bdad3bccbfb --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/SuspendedMailboxDefaultAction.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.runtime.tasks.mailbox; + +import org.apache.flink.annotation.Internal; + +/** + * Represents the suspended state of a {@link MailboxDefaultAction}, ready to resume. + */ +@Internal +public interface SuspendedMailboxDefaultAction { + + /** + * Resume execution of the default action. + */ + void resume(); +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java index 102ff80370e1..5e9fb85e642a 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailbox.java @@ -19,6 +19,7 @@ package org.apache.flink.streaming.runtime.tasks.mailbox; import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.api.operators.MailboxExecutor; import javax.annotation.Nonnull; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/SuspendedMailboxDefaultAction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/SuspendedMailboxDefaultAction.java deleted file mode 100644 index 6b3f32cd73f2..000000000000 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/SuspendedMailboxDefaultAction.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.streaming.runtime.tasks.mailbox.execution; - -/** - * Represents the suspended state of a {@link MailboxDefaultAction}, ready to resume. - */ -public interface SuspendedMailboxDefaultAction { - - /** - * Resume execution of the default action. - */ - void resume(); -} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/MailboxOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/MailboxOperatorTest.java index 9e8ff5da32bf..95f1c74b0ee3 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/MailboxOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/MailboxOperatorTest.java @@ -23,6 +23,7 @@ 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.MailboxExecutor; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.streaming.api.operators.Output; @@ -32,7 +33,6 @@ import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask; import org.apache.flink.streaming.runtime.tasks.OneInputStreamTaskTestHarness; import org.apache.flink.streaming.runtime.tasks.StreamTask; -import org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxExecutor; import org.apache.flink.util.TestLogger; import org.junit.Test; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskOperatorTimerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskOperatorTimerTest.java index 12e0c347bc58..2bdbcecd0003 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskOperatorTimerTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskOperatorTimerTest.java @@ -24,6 +24,7 @@ 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.MailboxExecutor; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.streaming.api.operators.Output; @@ -34,7 +35,6 @@ import org.apache.flink.streaming.runtime.tasks.OneInputStreamTaskTestHarness; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.streaming.runtime.tasks.StreamTask; -import org.apache.flink.streaming.runtime.tasks.mailbox.execution.MailboxExecutor; import org.apache.flink.util.TestLogger; import org.junit.Test; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskSelectiveReadingTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskSelectiveReadingTest.java index 8caeda7bb00d..3088860b7092 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskSelectiveReadingTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskSelectiveReadingTest.java @@ -28,7 +28,7 @@ import org.apache.flink.streaming.api.operators.InputSelection; import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.mailbox.execution.DefaultActionContext; +import org.apache.flink.streaming.runtime.tasks.mailbox.DefaultActionContext; import org.apache.flink.streaming.util.TestAnyModeReadingStreamOperator; import org.apache.flink.streaming.util.TestHarnessUtil; import org.apache.flink.streaming.util.TestSequentialReadingStreamOperator; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java index 3919f4e61349..057d88233a89 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java @@ -79,7 +79,7 @@ import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.StreamOperator; -import org.apache.flink.streaming.runtime.tasks.mailbox.execution.DefaultActionContext; +import org.apache.flink.streaming.runtime.tasks.mailbox.DefaultActionContext; import org.apache.flink.util.FlinkException; import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TestLogger; 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 9eb220c37c2b..1978299f91e7 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 @@ -117,7 +117,7 @@ import org.apache.flink.streaming.runtime.io.StreamInputProcessor; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; -import org.apache.flink.streaming.runtime.tasks.mailbox.execution.DefaultActionContext; +import org.apache.flink.streaming.runtime.tasks.mailbox.DefaultActionContext; import org.apache.flink.streaming.util.TestSequentialReadingStreamOperator; import org.apache.flink.util.CloseableIterable; import org.apache.flink.util.ExceptionUtils; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointITCase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointITCase.java index 37cd1108db22..0c7db7c26850 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointITCase.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointITCase.java @@ -61,7 +61,7 @@ import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.taskmanager.TaskManagerActions; import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo; -import org.apache.flink.streaming.runtime.tasks.mailbox.execution.DefaultActionContext; +import org.apache.flink.streaming.runtime.tasks.mailbox.DefaultActionContext; import org.apache.flink.util.SerializedValue; import org.junit.Rule; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointTest.java index da21554f3259..4cd378fabe15 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointTest.java @@ -26,7 +26,7 @@ import org.apache.flink.runtime.operators.testutils.DummyEnvironment; import org.apache.flink.runtime.state.CheckpointStorageLocationReference; import org.apache.flink.streaming.runtime.tasks.StreamTaskTest.NoOpStreamTask; -import org.apache.flink.streaming.runtime.tasks.mailbox.execution.DefaultActionContext; +import org.apache.flink.streaming.runtime.tasks.mailbox.DefaultActionContext; import org.junit.Before; import org.junit.Test; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java index 9b74c01e45d7..7ce3a9d16a52 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java @@ -82,7 +82,7 @@ import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.StreamFilter; import org.apache.flink.streaming.api.operators.StreamOperator; -import org.apache.flink.streaming.runtime.tasks.mailbox.execution.DefaultActionContext; +import org.apache.flink.streaming.runtime.tasks.mailbox.DefaultActionContext; import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TestLogger; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImplTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxExecutorImplTest.java similarity index 88% rename from flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImplTest.java rename to flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxExecutorImplTest.java index 519f63629364..54d5e7e1d298 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/MailboxExecutorImplTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxExecutorImplTest.java @@ -1,13 +1,12 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this 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,11 +15,10 @@ * limitations under the License. */ -package org.apache.flink.streaming.runtime.tasks.mailbox.execution; +package org.apache.flink.streaming.runtime.tasks.mailbox; import org.apache.flink.runtime.concurrent.FutureUtils; -import org.apache.flink.streaming.runtime.tasks.mailbox.Mail; -import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailboxImpl; +import org.apache.flink.streaming.api.operators.MailboxExecutor; import org.apache.flink.util.Preconditions; import org.junit.After; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/TaskMailboxProcessorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxProcessorTest.java similarity index 93% rename from flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/TaskMailboxProcessorTest.java rename to flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxProcessorTest.java index 506b466dda5f..d1513c6dde89 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/execution/TaskMailboxProcessorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxProcessorTest.java @@ -1,13 +1,12 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this 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,9 +15,10 @@ * limitations under the License. */ -package org.apache.flink.streaming.runtime.tasks.mailbox.execution; +package org.apache.flink.streaming.runtime.tasks.mailbox; import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.streaming.api.operators.MailboxExecutor; import org.junit.Assert; import org.junit.Test; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTask.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTask.java index 9685ee17e9d6..cae70f05cf68 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTask.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTask.java @@ -30,7 +30,7 @@ import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.streaming.runtime.tasks.TimerService; -import org.apache.flink.streaming.runtime.tasks.mailbox.execution.DefaultActionContext; +import org.apache.flink.streaming.runtime.tasks.mailbox.DefaultActionContext; import java.util.Map; import java.util.function.BiConsumer; From 809533e5b5c686e2d21b64361d22178ccb92ec26 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Tue, 5 Nov 2019 14:34:07 +0100 Subject: [PATCH 479/746] [hotfix][runtime] Moving interfaces closely related to mailbox default action into MailboxDefaultAction. Renaming DefaultActionContext to MailboxDefaultAction.Controller. --- .../state/api/output/BoundedStreamTask.java | 6 +-- .../runtime/tasks/SourceStreamTask.java | 6 +-- .../runtime/tasks/StreamIterationHead.java | 6 +-- .../streaming/runtime/tasks/StreamTask.java | 11 ++--- .../tasks/mailbox/DefaultActionContext.java | 40 ---------------- .../tasks/mailbox/MailboxDefaultAction.java | 36 ++++++++++++++- .../tasks/mailbox/MailboxProcessor.java | 20 ++++---- .../SuspendedMailboxDefaultAction.java | 32 ------------- .../tasks/StreamTaskSelectiveReadingTest.java | 6 +-- .../tasks/StreamTaskTerminationTest.java | 6 +-- .../runtime/tasks/StreamTaskTest.java | 46 +++++++++---------- .../tasks/SynchronousCheckpointITCase.java | 8 ++-- .../tasks/SynchronousCheckpointTest.java | 6 +-- .../tasks/TaskCheckpointingBehaviourTest.java | 6 +-- .../mailbox/TaskMailboxProcessorTest.java | 42 ++++++++--------- .../flink/streaming/util/MockStreamTask.java | 6 +-- .../JobMasterStopWithSavepointIT.java | 10 ++-- 17 files changed, 126 insertions(+), 167 deletions(-) delete mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/DefaultActionContext.java delete mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/SuspendedMailboxDefaultAction.java diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/BoundedStreamTask.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/BoundedStreamTask.java index e114de88bed0..e97ff0a55364 100644 --- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/BoundedStreamTask.java +++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/BoundedStreamTask.java @@ -29,7 +29,7 @@ import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.StreamTask; -import org.apache.flink.streaming.runtime.tasks.mailbox.execution.DefaultActionContext; +import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; import org.apache.flink.util.Collector; import org.apache.flink.util.OutputTag; import org.apache.flink.util.Preconditions; @@ -82,14 +82,14 @@ protected void init() throws Exception { } @Override - protected void processInput(DefaultActionContext context) throws Exception { + protected void processInput(MailboxDefaultAction.Controller controller) throws Exception { if (input.hasNext()) { reuse.replace(input.next()); headOperator.setKeyContextElement1(reuse); headOperator.processElement(reuse); } else { headOperator.endInput(); - context.allActionsCompleted(); + controller.allActionsCompleted(); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java index 1ed3fb4ba566..85e285e96e91 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java @@ -25,7 +25,7 @@ import org.apache.flink.streaming.api.checkpoint.ExternallyInducedSource; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.operators.StreamSource; -import org.apache.flink.streaming.runtime.tasks.mailbox.DefaultActionContext; +import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; import org.apache.flink.util.FlinkException; import java.util.Optional; @@ -113,9 +113,9 @@ protected void cleanup() { } @Override - protected void processInput(DefaultActionContext context) throws Exception { + protected void processInput(MailboxDefaultAction.Controller controller) throws Exception { - context.suspendDefaultAction(); + controller.suspendDefaultAction(); // Against the usual contract of this method, this implementation is not step-wise but blocking instead for // compatibility reasons with the current source interface (source functions run as a loop, not in steps). diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java index 8732f0310aad..ec1e62c00026 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java @@ -24,7 +24,7 @@ import org.apache.flink.streaming.runtime.io.BlockingQueueBroker; import org.apache.flink.streaming.runtime.io.RecordWriterOutput; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.mailbox.DefaultActionContext; +import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; import org.apache.flink.util.FlinkRuntimeException; import org.slf4j.Logger; @@ -67,7 +67,7 @@ public StreamIterationHead(Environment env) { // ------------------------------------------------------------------------ @Override - protected void processInput(DefaultActionContext context) throws Exception { + protected void processInput(MailboxDefaultAction.Controller controller) throws Exception { StreamRecord nextRecord = shouldWait ? dataChannel.poll(iterationWaitTime, TimeUnit.MILLISECONDS) : dataChannel.take(); @@ -79,7 +79,7 @@ protected void processInput(DefaultActionContext context) throws Exception { } } } else { - context.allActionsCompleted(); + controller.allActionsCompleted(); } } 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 b297288042e4..62c56170715f 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 @@ -66,10 +66,9 @@ import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; -import org.apache.flink.streaming.runtime.tasks.mailbox.DefaultActionContext; +import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxExecutorFactory; import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor; -import org.apache.flink.streaming.runtime.tasks.mailbox.SuspendedMailboxDefaultAction; import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; @@ -273,16 +272,16 @@ protected void cleanup() throws Exception { * This method implements the default action of the task (e.g. processing one event from the input). Implementations * should (in general) be non-blocking. * - * @param context context object for collaborative interaction between the action and the stream task. + * @param controller controller object for collaborative interaction between the action and the stream task. * @throws Exception on any problems in the action. */ - protected void processInput(DefaultActionContext context) throws Exception { + protected void processInput(MailboxDefaultAction.Controller controller) throws Exception { InputStatus status = inputProcessor.processInput(); if (status == InputStatus.END_OF_INPUT) { - context.allActionsCompleted(); + controller.allActionsCompleted(); } else if (status == InputStatus.NOTHING_AVAILABLE) { - SuspendedMailboxDefaultAction suspendedDefaultAction = context.suspendDefaultAction(); + MailboxDefaultAction.Suspension suspendedDefaultAction = controller.suspendDefaultAction(); inputProcessor.isAvailable().thenRun(suspendedDefaultAction::resume); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/DefaultActionContext.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/DefaultActionContext.java deleted file mode 100644 index dee148dd61bb..000000000000 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/DefaultActionContext.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.streaming.runtime.tasks.mailbox; - -import org.apache.flink.annotation.Internal; - -/** - * This context is a feedback interface for the default action to interact with the mailbox execution. In particular - * it offers ways to signal that the execution of the default action should be finished or temporarily suspended. - */ -@Internal -public interface DefaultActionContext { - - /** - * This method must be called to end the stream task when all actions for the tasks have been performed. This - * method can be invoked from any thread. - */ - void allActionsCompleted(); - - /** - * Calling this method signals that the mailbox-thread should (temporarily) stop invoking the default action, - * e.g. because there is currently no input available. This method must be invoked from the mailbox-thread only! - */ - SuspendedMailboxDefaultAction suspendDefaultAction(); -} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxDefaultAction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxDefaultAction.java index 10d3732b1a06..d118ae6f9bbc 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxDefaultAction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxDefaultAction.java @@ -29,8 +29,40 @@ public interface MailboxDefaultAction { * This method implements the default action of the mailbox loop (e.g. processing one event from the input). * Implementations should (in general) be non-blocking. * - * @param context context object for collaborative interaction between the default action and the mailbox loop. + * @param controller controller object for collaborative interaction between the default action and the mailbox loop. * @throws Exception on any problems in the action. */ - void runDefaultAction(DefaultActionContext context) throws Exception; + void runDefaultAction(Controller controller) throws Exception; + + /** + * Represents the suspended state of a {@link MailboxDefaultAction}, ready to resume. + */ + @Internal + interface Suspension { + + /** + * Resume execution of the default action. + */ + void resume(); + } + + /** + * This context is a feedback interface for the default action to interact with the mailbox execution. In particular + * it offers ways to signal that the execution of the default action should be finished or temporarily suspended. + */ + @Internal + interface Controller { + + /** + * This method must be called to end the stream task when all actions for the tasks have been performed. This + * method can be invoked from any thread. + */ + void allActionsCompleted(); + + /** + * Calling this method signals that the mailbox-thread should (temporarily) stop invoking the default action, + * e.g. because there is currently no input available. This method must be invoked from the mailbox-thread only! + */ + Suspension suspendDefaultAction(); + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxProcessor.java index a3415c991415..def6cfc10a36 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxProcessor.java @@ -39,7 +39,7 @@ * single-threaded execution between the default action (e.g. record processing) and mailbox actions (e.g. checkpoint * trigger, timer firing, ...). * - *

    The {@link MailboxDefaultAction} interacts with this class through the {@link MailboxDefaultActionContext} to + *

    The {@link MailboxDefaultAction} interacts with this class through the {@link MailboxController} to * communicate control flow changes to the mailbox loop, e.g. that invocations of the default action are temporarily or * permanently exhausted. * @@ -75,7 +75,7 @@ public class MailboxProcessor { * default action (suspended if not-null) and to reuse the object as return value in consecutive suspend attempts. * Must only be accessed from mailbox thread. */ - private SuspendedMailboxDefaultAction suspendedDefaultAction; + private MailboxDefaultAction.Suspension suspendedDefaultAction; public MailboxProcessor(MailboxDefaultAction mailboxDefaultAction) { this.mailboxDefaultAction = Preconditions.checkNotNull(mailboxDefaultAction); @@ -148,7 +148,7 @@ public void runMailboxLoop() throws Exception { assert localMailbox.getState() == TaskMailbox.State.OPEN : "Mailbox must be opened!"; - final MailboxDefaultActionContext defaultActionContext = new MailboxDefaultActionContext(this); + final MailboxController defaultActionContext = new MailboxController(this); while (processMail(localMailbox)) { mailboxDefaultAction.runDefaultAction(defaultActionContext); @@ -219,12 +219,12 @@ private boolean processMail(TaskMailbox mailbox) throws InterruptedException { * Calling this method signals that the mailbox-thread should (temporarily) stop invoking the default action, * e.g. because there is currently no input available. */ - private SuspendedMailboxDefaultAction suspendDefaultAction() { + private MailboxDefaultAction.Suspension suspendDefaultAction() { Preconditions.checkState(mailbox.isMailboxThread(), "Suspending must only be called from the mailbox thread!"); if (suspendedDefaultAction == null) { - suspendedDefaultAction = new SuspendDefaultActionRunnable(); + suspendedDefaultAction = new DefaultActionSuspension(); ensureControlFlowSignalCheck(); } @@ -250,14 +250,14 @@ private void ensureControlFlowSignalCheck() { } /** - * Implementation of {@link DefaultActionContext} that is connected to a {@link MailboxProcessor} + * Implementation of {@link MailboxDefaultAction.Controller} that is connected to a {@link MailboxProcessor} * instance. */ - private static final class MailboxDefaultActionContext implements DefaultActionContext { + private static final class MailboxController implements MailboxDefaultAction.Controller { private final MailboxProcessor mailboxProcessor; - private MailboxDefaultActionContext(MailboxProcessor mailboxProcessor) { + private MailboxController(MailboxProcessor mailboxProcessor) { this.mailboxProcessor = mailboxProcessor; } @@ -267,7 +267,7 @@ public void allActionsCompleted() { } @Override - public SuspendedMailboxDefaultAction suspendDefaultAction() { + public MailboxDefaultAction.Suspension suspendDefaultAction() { return mailboxProcessor.suspendDefaultAction(); } } @@ -275,7 +275,7 @@ public SuspendedMailboxDefaultAction suspendDefaultAction() { /** * Represents the suspended state of the default action and offers an idempotent method to resume execution. */ - private final class SuspendDefaultActionRunnable implements SuspendedMailboxDefaultAction { + private final class DefaultActionSuspension implements MailboxDefaultAction.Suspension { @Override public void resume() { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/SuspendedMailboxDefaultAction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/SuspendedMailboxDefaultAction.java deleted file mode 100644 index 7bdad3bccbfb..000000000000 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/SuspendedMailboxDefaultAction.java +++ /dev/null @@ -1,32 +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.mailbox; - -import org.apache.flink.annotation.Internal; - -/** - * Represents the suspended state of a {@link MailboxDefaultAction}, ready to resume. - */ -@Internal -public interface SuspendedMailboxDefaultAction { - - /** - * Resume execution of the default action. - */ - void resume(); -} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskSelectiveReadingTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskSelectiveReadingTest.java index 3088860b7092..94c8d849d292 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskSelectiveReadingTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskSelectiveReadingTest.java @@ -28,7 +28,7 @@ import org.apache.flink.streaming.api.operators.InputSelection; import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.mailbox.DefaultActionContext; +import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; import org.apache.flink.streaming.util.TestAnyModeReadingStreamOperator; import org.apache.flink.streaming.util.TestHarnessUtil; import org.apache.flink.streaming.util.TestSequentialReadingStreamOperator; @@ -194,14 +194,14 @@ private static class TestSelectiveReadingTask extends TwoInputStr } @Override - protected void processInput(DefaultActionContext context) throws Exception { + protected void processInput(MailboxDefaultAction.Controller controller) throws Exception { if (!started) { synchronized (this) { this.wait(); } } - super.processInput(context); + super.processInput(controller); } public void startProcessing() { diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java index 057d88233a89..e29d58df5abd 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java @@ -79,7 +79,7 @@ import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.StreamOperator; -import org.apache.flink.streaming.runtime.tasks.mailbox.DefaultActionContext; +import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; import org.apache.flink.util.FlinkException; import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TestLogger; @@ -234,14 +234,14 @@ protected void init() { } @Override - protected void processInput(DefaultActionContext context) throws Exception { + protected void processInput(MailboxDefaultAction.Controller controller) throws Exception { if (!isRunning) { isRunning = true; RUN_LATCH.trigger(); } // wait until we have started an asynchronous checkpoint if (isCanceled() || SNAPSHOT_HAS_STARTED.get()) { - context.allActionsCompleted(); + controller.allActionsCompleted(); } } 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 1978299f91e7..e88b3a20ea5a 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 @@ -117,7 +117,7 @@ import org.apache.flink.streaming.runtime.io.StreamInputProcessor; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; -import org.apache.flink.streaming.runtime.tasks.mailbox.DefaultActionContext; +import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; import org.apache.flink.streaming.util.TestSequentialReadingStreamOperator; import org.apache.flink.util.CloseableIterable; import org.apache.flink.util.ExceptionUtils; @@ -778,9 +778,9 @@ protected void init() throws Exception { } @Override - protected void processInput(DefaultActionContext context) throws Exception { + protected void processInput(MailboxDefaultAction.Controller controller) throws Exception { mailboxProcessor.getMailboxExecutor(0).execute(latch::trigger, "trigger"); - context.allActionsCompleted(); + controller.allActionsCompleted(); } }); latch.await(); @@ -1245,11 +1245,11 @@ protected void init() throws Exception { } @Override - protected void processInput(DefaultActionContext context) throws Exception { + protected void processInput(MailboxDefaultAction.Controller controller) throws Exception { if (fail) { throw new RuntimeException(); } - context.allActionsCompleted(); + controller.allActionsCompleted(); } @Override @@ -1260,7 +1260,7 @@ public StreamTaskStateInitializer createStreamTaskStateInitializer() { final StreamTaskStateInitializer streamTaskStateManager = super.createStreamTaskStateInitializer(); return (operatorID, operatorClassName, processingTimeService, keyContext, keySerializer, closeableRegistry, metricGroup) -> { - final StreamOperatorStateContext context = streamTaskStateManager.streamOperatorStateContext( + final StreamOperatorStateContext controller = streamTaskStateManager.streamOperatorStateContext( operatorID, operatorClassName, processingTimeService, @@ -1272,33 +1272,33 @@ public StreamTaskStateInitializer createStreamTaskStateInitializer() { return new StreamOperatorStateContext() { @Override public boolean isRestored() { - return context.isRestored(); + return controller.isRestored(); } @Override public OperatorStateBackend operatorStateBackend() { - return context.operatorStateBackend(); + return controller.operatorStateBackend(); } @Override public AbstractKeyedStateBackend keyedStateBackend() { - return context.keyedStateBackend(); + return controller.keyedStateBackend(); } @Override public InternalTimeServiceManager internalTimerServiceManager() { - InternalTimeServiceManager timeServiceManager = context.internalTimerServiceManager(); + InternalTimeServiceManager timeServiceManager = controller.internalTimerServiceManager(); return timeServiceManager != null ? spy(timeServiceManager) : null; } @Override public CloseableIterable rawOperatorStateInputs() { - return replaceWithSpy(context.rawOperatorStateInputs()); + return replaceWithSpy(controller.rawOperatorStateInputs()); } @Override public CloseableIterable rawKeyedStateInputs() { - return replaceWithSpy(context.rawKeyedStateInputs()); + return replaceWithSpy(controller.rawKeyedStateInputs()); } public T replaceWithSpy(T closeable) { @@ -1334,7 +1334,7 @@ public CancelLockingTask(Environment env) { protected void init() {} @Override - protected void processInput(DefaultActionContext context) throws Exception { + protected void processInput(MailboxDefaultAction.Controller controller) throws Exception { holder = new LockHolder(getCheckpointLock(), latch); holder.start(); latch.await(); @@ -1349,7 +1349,7 @@ protected void processInput(DefaultActionContext context) throws Exception { // restore interruption state Thread.currentThread().interrupt(); } - context.allActionsCompleted(); + controller.allActionsCompleted(); } @Override @@ -1379,7 +1379,7 @@ public CancelFailingTask(Environment env) { protected void init() {} @Override - protected void processInput(DefaultActionContext context) throws Exception { + protected void processInput(MailboxDefaultAction.Controller controller) throws Exception { final OneShotLatch latch = new OneShotLatch(); final Object lock = new Object(); @@ -1405,7 +1405,7 @@ protected void processInput(DefaultActionContext context) throws Exception { finally { holder.close(); } - context.allActionsCompleted(); + controller.allActionsCompleted(); } @Override @@ -1453,10 +1453,10 @@ public void onProcessingTime(long timestamp) throws Exception { } @Override - protected void processInput(DefaultActionContext context) throws Exception { + protected void processInput(MailboxDefaultAction.Controller controller) throws Exception { checkTaskThreadInfo(); if (hasTimerTriggered) { - context.allActionsCompleted(); + controller.allActionsCompleted(); } } @@ -1472,7 +1472,7 @@ private void checkTaskThreadInfo() { "Task's method was called in non task thread."); Preconditions.checkState( taskClassLoader == currentThread.getContextClassLoader(), - "Task's context class loader has been changed during invocation."); + "Task's controller class loader has been changed during invocation."); } } @@ -1541,14 +1541,14 @@ public TestStreamSource(SRC sourceFunction) { } @Override - public void initializeState(StateInitializationContext context) throws Exception { + public void initializeState(StateInitializationContext controller) throws Exception { keyedStateBackend = (AbstractKeyedStateBackend) getKeyedStateBackend(); operatorStateBackend = getOperatorStateBackend(); rawOperatorStateInputs = - (CloseableIterable) context.getRawOperatorStateInputs(); + (CloseableIterable) controller.getRawOperatorStateInputs(); rawKeyedStateInputs = - (CloseableIterable) context.getRawKeyedStateInputs(); - super.initializeState(context); + (CloseableIterable) controller.getRawKeyedStateInputs(); + super.initializeState(controller); } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointITCase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointITCase.java index 0c7db7c26850..f5f54f313aa6 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointITCase.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointITCase.java @@ -61,7 +61,7 @@ import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.taskmanager.TaskManagerActions; import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo; -import org.apache.flink.streaming.runtime.tasks.mailbox.DefaultActionContext; +import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; import org.apache.flink.util.SerializedValue; import org.junit.Rule; @@ -139,15 +139,15 @@ public SynchronousCheckpointTestingTask(Environment environment) { } @Override - protected void processInput(DefaultActionContext context) throws Exception { + protected void processInput(MailboxDefaultAction.Controller controller) throws Exception { if (!isRunning) { isRunning = true; eventQueue.put(Event.TASK_IS_RUNNING); } if (isCanceled()) { - context.allActionsCompleted(); + controller.allActionsCompleted(); } else { - context.suspendDefaultAction(); + controller.suspendDefaultAction(); } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointTest.java index 4cd378fabe15..c5acee0e7337 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointTest.java @@ -26,7 +26,7 @@ import org.apache.flink.runtime.operators.testutils.DummyEnvironment; import org.apache.flink.runtime.state.CheckpointStorageLocationReference; import org.apache.flink.streaming.runtime.tasks.StreamTaskTest.NoOpStreamTask; -import org.apache.flink.streaming.runtime.tasks.mailbox.DefaultActionContext; +import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; import org.junit.Before; import org.junit.Test; @@ -156,9 +156,9 @@ protected void init() { } @Override - protected void processInput(DefaultActionContext context) throws Exception { + protected void processInput(MailboxDefaultAction.Controller controller) throws Exception { if (stopped || isCanceled()) { - context.allActionsCompleted(); + controller.allActionsCompleted(); } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java index 7ce3a9d16a52..6606e2b4d988 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java @@ -82,7 +82,7 @@ import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.StreamFilter; import org.apache.flink.streaming.api.operators.StreamOperator; -import org.apache.flink.streaming.runtime.tasks.mailbox.DefaultActionContext; +import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TestLogger; @@ -475,7 +475,7 @@ public TestStreamTask(Environment env) { public void init() {} @Override - protected void processInput(DefaultActionContext context) throws Exception { + protected void processInput(MailboxDefaultAction.Controller controller) throws Exception { triggerCheckpointOnBarrier( new CheckpointMetaData( 11L, @@ -486,7 +486,7 @@ protected void processInput(DefaultActionContext context) throws Exception { while (isRunning()) { Thread.sleep(1L); } - context.allActionsCompleted(); + controller.allActionsCompleted(); } @Override diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxProcessorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxProcessorTest.java index d1513c6dde89..b4887a2fceda 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxProcessorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/TaskMailboxProcessorTest.java @@ -38,7 +38,7 @@ public class TaskMailboxProcessorTest { @Test public void testRejectIfNotOpen() { - MailboxProcessor mailboxProcessor = new MailboxProcessor((ctx) -> {}); + MailboxProcessor mailboxProcessor = new MailboxProcessor(controller -> {}); mailboxProcessor.prepareClose(); try { mailboxProcessor.getMailboxExecutor(DEFAULT_PRIORITY).execute(() -> {}, "dummy"); @@ -49,7 +49,7 @@ public void testRejectIfNotOpen() { @Test public void testShutdown() { - MailboxProcessor mailboxProcessor = new MailboxProcessor((ctx) -> {}); + MailboxProcessor mailboxProcessor = new MailboxProcessor(controller -> {}); FutureTask testRunnableFuture = new FutureTask<>(() -> {}, null); mailboxProcessor.getMailboxExecutor(DEFAULT_PRIORITY).execute(testRunnableFuture, "testRunnableFuture"); mailboxProcessor.prepareClose(); @@ -71,9 +71,9 @@ public void testRunDefaultActionAndMails() throws Exception { AtomicBoolean stop = new AtomicBoolean(false); MailboxThread mailboxThread = new MailboxThread() { @Override - public void runDefaultAction(DefaultActionContext context) throws Exception { + public void runDefaultAction(Controller controller) throws Exception { if (stop.get()) { - context.allActionsCompleted(); + controller.allActionsCompleted(); } else { Thread.sleep(10L); } @@ -92,9 +92,9 @@ public void testRunDefaultAction() throws Exception { final AtomicInteger counter = new AtomicInteger(0); MailboxThread mailboxThread = new MailboxThread() { @Override - public void runDefaultAction(DefaultActionContext context) { + public void runDefaultAction(Controller controller) { if (counter.incrementAndGet() == expectedInvocations) { - context.allActionsCompleted(); + controller.allActionsCompleted(); } } }; @@ -108,19 +108,19 @@ public void runDefaultAction(DefaultActionContext context) { public void testSignalUnAvailable() throws Exception { final AtomicInteger counter = new AtomicInteger(0); - final AtomicReference suspendedActionRef = new AtomicReference<>(); + final AtomicReference suspendedActionRef = new AtomicReference<>(); final OneShotLatch actionSuspendedLatch = new OneShotLatch(); final int blockAfterInvocations = 3; final int totalInvocations = blockAfterInvocations * 2; MailboxThread mailboxThread = new MailboxThread() { @Override - public void runDefaultAction(DefaultActionContext context) { + public void runDefaultAction(Controller controller) { if (counter.incrementAndGet() == blockAfterInvocations) { - suspendedActionRef.set(context.suspendDefaultAction()); + suspendedActionRef.set(controller.suspendDefaultAction()); actionSuspendedLatch.trigger(); } else if (counter.get() == totalInvocations) { - context.allActionsCompleted(); + controller.allActionsCompleted(); } } }; @@ -129,29 +129,29 @@ public void runDefaultAction(DefaultActionContext context) { actionSuspendedLatch.await(); Assert.assertEquals(blockAfterInvocations, counter.get()); - SuspendedMailboxDefaultAction suspendedMailboxDefaultAction = suspendedActionRef.get(); - mailboxProcessor.getMailboxExecutor(DEFAULT_PRIORITY).execute(suspendedMailboxDefaultAction::resume, "resume"); + MailboxDefaultAction.Suspension suspension = suspendedActionRef.get(); + mailboxProcessor.getMailboxExecutor(DEFAULT_PRIORITY).execute(suspension::resume, "resume"); stop(mailboxThread); Assert.assertEquals(totalInvocations, counter.get()); } @Test public void testSignalUnAvailablePingPong() throws Exception { - final AtomicReference suspendedActionRef = new AtomicReference<>(); + final AtomicReference suspendedActionRef = new AtomicReference<>(); final int totalSwitches = 10000; final MailboxThread mailboxThread = new MailboxThread() { int count = 0; @Override - public void runDefaultAction(DefaultActionContext context) { + public void runDefaultAction(Controller controller) { // If this is violated, it means that the default action was invoked while we assumed suspension - Assert.assertTrue(suspendedActionRef.compareAndSet(null, context.suspendDefaultAction())); + Assert.assertTrue(suspendedActionRef.compareAndSet(null, controller.suspendDefaultAction())); ++count; if (count == totalSwitches) { - context.allActionsCompleted(); + controller.allActionsCompleted(); } else if (count % 1000 == 0) { try { Thread.sleep(1L); @@ -169,7 +169,7 @@ public void runDefaultAction(DefaultActionContext context) { int count = 0; while (!Thread.currentThread().isInterrupted()) { - final SuspendedMailboxDefaultAction resume = + final MailboxDefaultAction.Suspension resume = suspendedActionRef.getAndSet(null); if (resume != null) { mailboxProcessor.getMailboxExecutor(DEFAULT_PRIORITY).execute(resume::resume, "resume"); @@ -228,9 +228,9 @@ public void testAvoidStarvation() throws Exception { final AtomicInteger counter = new AtomicInteger(0); MailboxThread mailboxThread = new MailboxThread() { @Override - public void runDefaultAction(DefaultActionContext context) { + public void runDefaultAction(Controller controller) { if (counter.incrementAndGet() == expectedInvocations) { - context.allActionsCompleted(); + controller.allActionsCompleted(); } } }; @@ -282,8 +282,8 @@ public final void run() { } @Override - public void runDefaultAction(DefaultActionContext context) throws Exception { - context.allActionsCompleted(); + public void runDefaultAction(Controller controller) throws Exception { + controller.allActionsCompleted(); } final MailboxProcessor getMailboxProcessor() { diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTask.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTask.java index cae70f05cf68..1a54625e018c 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTask.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamTask.java @@ -30,7 +30,7 @@ import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.streaming.runtime.tasks.TimerService; -import org.apache.flink.streaming.runtime.tasks.mailbox.DefaultActionContext; +import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; import java.util.Map; import java.util.function.BiConsumer; @@ -85,8 +85,8 @@ public void init() { } @Override - protected void processInput(DefaultActionContext context) throws Exception { - context.allActionsCompleted(); + protected void processInput(MailboxDefaultAction.Controller controller) throws Exception { + controller.allActionsCompleted(); } @Override diff --git a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointIT.java b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointIT.java index 982643abe396..d585bf1aa586 100644 --- a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointIT.java +++ b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointIT.java @@ -38,7 +38,7 @@ import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.streaming.runtime.tasks.StreamTaskTest.NoOpStreamTask; -import org.apache.flink.streaming.runtime.tasks.mailbox.execution.DefaultActionContext; +import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; import org.apache.flink.test.util.AbstractTestBase; import org.junit.Assume; @@ -288,14 +288,14 @@ public ExceptionOnCallbackStreamTask(final Environment environment) { } @Override - protected void processInput(DefaultActionContext context) throws Exception { + protected void processInput(MailboxDefaultAction.Controller controller) throws Exception { final long taskIndex = getEnvironment().getTaskInfo().getIndexOfThisSubtask(); if (taskIndex == 0) { numberOfRestarts.countDown(); } invokeLatch.countDown(); finishLatch.await(); - context.allActionsCompleted(); + controller.allActionsCompleted(); } @Override @@ -345,10 +345,10 @@ public NoOpBlockingStreamTask(final Environment environment) { } @Override - protected void processInput(DefaultActionContext context) throws Exception { + protected void processInput(MailboxDefaultAction.Controller controller) throws Exception { invokeLatch.countDown(); finishLatch.await(); - context.allActionsCompleted(); + controller.allActionsCompleted(); } @Override From ced28c9572743a4b3f4c2c925d349a4470fd1a91 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Fri, 1 Nov 2019 14:56:13 +0800 Subject: [PATCH 480/746] [hotfix][metrics] Define the default time span for MeterView and add a constructor to use it MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Currently in production, MeterView is always created with a 60 s time span. Rather than specify that number everywhere, it’s better to make 60 the default time span value. --- .../main/java/org/apache/flink/metrics/MeterView.java | 7 +++++++ .../java/org/apache/flink/metrics/MeterViewTest.java | 6 +++--- .../io/network/metrics/InputChannelMetrics.java | 2 +- .../runtime/metrics/groups/OperatorIOMetricGroup.java | 4 ++-- .../runtime/metrics/groups/TaskIOMetricGroup.java | 10 +++++----- .../table/runtime/operators/window/WindowOperator.java | 2 +- 6 files changed, 19 insertions(+), 12 deletions(-) diff --git a/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/MeterView.java b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/MeterView.java index 6d53325edafe..0d9b564d20c0 100644 --- a/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/MeterView.java +++ b/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/MeterView.java @@ -33,6 +33,9 @@ *

    The events are counted by a {@link Counter}. */ public class MeterView implements Meter, View { + + private static final int DEFAULT_TIME_SPAN_IN_SECONDS = 60; + /** The underlying counter maintaining the count. */ private final Counter counter; /** The time-span over which the average is calculated. */ @@ -48,6 +51,10 @@ public MeterView(int timeSpanInSeconds) { this(new SimpleCounter(), timeSpanInSeconds); } + public MeterView(Counter counter) { + this(counter, DEFAULT_TIME_SPAN_IN_SECONDS); + } + public MeterView(Counter counter, int timeSpanInSeconds) { this.counter = counter; // the time-span must be larger than the update-interval as otherwise the array has a size of 1, diff --git a/flink-metrics/flink-metrics-core/src/test/java/org/apache/flink/metrics/MeterViewTest.java b/flink-metrics/flink-metrics-core/src/test/java/org/apache/flink/metrics/MeterViewTest.java index c074894d98ec..7beb6bb3be8e 100644 --- a/flink-metrics/flink-metrics-core/src/test/java/org/apache/flink/metrics/MeterViewTest.java +++ b/flink-metrics/flink-metrics-core/src/test/java/org/apache/flink/metrics/MeterViewTest.java @@ -33,7 +33,7 @@ public class MeterViewTest extends TestLogger { public void testGetCount() { Counter c = new SimpleCounter(); c.inc(5); - Meter m = new MeterView(c, 60); + Meter m = new MeterView(c); assertEquals(5, m.getCount()); } @@ -41,7 +41,7 @@ public void testGetCount() { @Test public void testMarkEvent() { Counter c = new SimpleCounter(); - Meter m = new MeterView(c, 60); + Meter m = new MeterView(c); assertEquals(0, m.getCount()); m.markEvent(); @@ -53,7 +53,7 @@ public void testMarkEvent() { @Test public void testGetRate() { Counter c = new SimpleCounter(); - MeterView m = new MeterView(c, 60); + MeterView m = new MeterView(c); // values = [0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0] for (int x = 0; x < 12; x++) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/metrics/InputChannelMetrics.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/metrics/InputChannelMetrics.java index b48a6649160e..a95602288098 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/metrics/InputChannelMetrics.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/metrics/InputChannelMetrics.java @@ -52,7 +52,7 @@ private static Counter createCounter(String name, MetricGroup ... parents) { Counter[] counters = new Counter[parents.length]; for (int i = 0; i < parents.length; i++) { counters[i] = parents[i].counter(name); - parents[i].meter(name + MetricNames.SUFFIX_RATE, new MeterView(counters[i], 60)); + parents[i].meter(name + MetricNames.SUFFIX_RATE, new MeterView(counters[i])); } return new MultiCounterWrapper(counters); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/OperatorIOMetricGroup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/OperatorIOMetricGroup.java index 6bf3c6c8a23e..7ee2413e3410 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/OperatorIOMetricGroup.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/OperatorIOMetricGroup.java @@ -39,8 +39,8 @@ public OperatorIOMetricGroup(OperatorMetricGroup parentMetricGroup) { super(parentMetricGroup); numRecordsIn = parentMetricGroup.counter(MetricNames.IO_NUM_RECORDS_IN); numRecordsOut = parentMetricGroup.counter(MetricNames.IO_NUM_RECORDS_OUT); - numRecordsInRate = parentMetricGroup.meter(MetricNames.IO_NUM_RECORDS_IN_RATE, new MeterView(numRecordsIn, 60)); - numRecordsOutRate = parentMetricGroup.meter(MetricNames.IO_NUM_RECORDS_OUT_RATE, new MeterView(numRecordsOut, 60)); + numRecordsInRate = parentMetricGroup.meter(MetricNames.IO_NUM_RECORDS_IN_RATE, new MeterView(numRecordsIn)); + numRecordsOutRate = parentMetricGroup.meter(MetricNames.IO_NUM_RECORDS_OUT_RATE, new MeterView(numRecordsOut)); } public Counter getNumRecordsInCounter() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TaskIOMetricGroup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TaskIOMetricGroup.java index ebf447ada7e4..d3fa82991592 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TaskIOMetricGroup.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/TaskIOMetricGroup.java @@ -51,16 +51,16 @@ public TaskIOMetricGroup(TaskMetricGroup parent) { this.numBytesIn = counter(MetricNames.IO_NUM_BYTES_IN); this.numBytesOut = counter(MetricNames.IO_NUM_BYTES_OUT); - this.numBytesInRate = meter(MetricNames.IO_NUM_BYTES_IN_RATE, new MeterView(numBytesIn, 60)); - this.numBytesOutRate = meter(MetricNames.IO_NUM_BYTES_OUT_RATE, new MeterView(numBytesOut, 60)); + this.numBytesInRate = meter(MetricNames.IO_NUM_BYTES_IN_RATE, new MeterView(numBytesIn)); + this.numBytesOutRate = meter(MetricNames.IO_NUM_BYTES_OUT_RATE, new MeterView(numBytesOut)); this.numRecordsIn = counter(MetricNames.IO_NUM_RECORDS_IN, new SumCounter()); this.numRecordsOut = counter(MetricNames.IO_NUM_RECORDS_OUT, new SumCounter()); - this.numRecordsInRate = meter(MetricNames.IO_NUM_RECORDS_IN_RATE, new MeterView(numRecordsIn, 60)); - this.numRecordsOutRate = meter(MetricNames.IO_NUM_RECORDS_OUT_RATE, new MeterView(numRecordsOut, 60)); + this.numRecordsInRate = meter(MetricNames.IO_NUM_RECORDS_IN_RATE, new MeterView(numRecordsIn)); + this.numRecordsOutRate = meter(MetricNames.IO_NUM_RECORDS_OUT_RATE, new MeterView(numRecordsOut)); this.numBuffersOut = counter(MetricNames.IO_NUM_BUFFERS_OUT); - this.numBuffersOutRate = meter(MetricNames.IO_NUM_BUFFERS_OUT_RATE, new MeterView(numBuffersOut, 60)); + this.numBuffersOutRate = meter(MetricNames.IO_NUM_BUFFERS_OUT_RATE, new MeterView(numBuffersOut)); } public IOMetrics createSnapshot() { diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/WindowOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/WindowOperator.java index 8774493f117c..d713a03cf3d8 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/WindowOperator.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/WindowOperator.java @@ -286,7 +286,7 @@ public void open() throws Exception { this.numLateRecordsDropped = metrics.counter(LATE_ELEMENTS_DROPPED_METRIC_NAME); this.lateRecordsDroppedRate = metrics.meter( LATE_ELEMENTS_DROPPED_RATE_METRIC_NAME, - new MeterView(numLateRecordsDropped, 60)); + new MeterView(numLateRecordsDropped)); this.watermarkLatency = metrics.gauge(WATERMARK_LATENCY_METRIC_NAME, () -> { long watermark = internalTimerService.currentWatermark(); if (watermark < 0) { From 282f785469c0e1fa642d44166f3ddbb187ed40d5 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Thu, 7 Nov 2019 18:10:55 +0800 Subject: [PATCH 481/746] [hotfix][runtime] Remove the redundant handleGlobalFailure() definition from SchedulerBase --- .../java/org/apache/flink/runtime/scheduler/SchedulerBase.java | 3 --- 1 file changed, 3 deletions(-) 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 eaa688da4887..d9d4a2974049 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 @@ -412,9 +412,6 @@ public CompletableFuture getTerminationFuture() { return executionGraph.getTerminationFuture().thenApply(FunctionUtils.nullFn()); } - @Override - public abstract void handleGlobalFailure(final Throwable cause); - @Override public final boolean updateTaskExecutionState(final TaskExecutionState taskExecutionState) { final Optional executionVertexId = getExecutionVertexId(taskExecutionState.getID()); From 4f0d6c170937c57934faa23141777f2304072ee4 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Thu, 7 Nov 2019 18:12:15 +0800 Subject: [PATCH 482/746] [FLINK-14164][runtime] Add a 'numberOfRestarts' metric to exhibit number of restarts --- .../failover/flip1/ExecutionFailureHandler.java | 15 ++++++++++++--- .../apache/flink/runtime/metrics/MetricNames.java | 2 ++ .../flink/runtime/scheduler/DefaultScheduler.java | 5 +++++ .../flink/runtime/scheduler/LegacyScheduler.java | 5 +++++ .../flink/runtime/scheduler/SchedulerBase.java | 5 +++++ .../flip1/ExecutionFailureHandlerTest.java | 3 +++ 6 files changed, 32 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandler.java index 3a6753ef5fb3..a766a3e5e82a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandler.java @@ -44,6 +44,9 @@ public class ExecutionFailureHandler { /** Strategy to judge whether and when a restarting should be done. */ private final RestartBackoffTimeStrategy restartBackoffTimeStrategy; + /** Number of all restarts happened since this job is submitted. */ + private long numberOfRestarts; + /** * Creates the handler to deal with task failures. * @@ -52,9 +55,9 @@ public class ExecutionFailureHandler { * @param restartBackoffTimeStrategy helps to decide whether to restart failed tasks and the restarting delay */ public ExecutionFailureHandler( - FailoverTopology failoverTopology, - FailoverStrategy failoverStrategy, - RestartBackoffTimeStrategy restartBackoffTimeStrategy) { + final FailoverTopology failoverTopology, + final FailoverStrategy failoverStrategy, + final RestartBackoffTimeStrategy restartBackoffTimeStrategy) { this.failoverTopology = checkNotNull(failoverTopology); this.failoverStrategy = checkNotNull(failoverStrategy); @@ -98,6 +101,8 @@ private FailureHandlingResult handleFailure( restartBackoffTimeStrategy.notifyFailure(cause); if (restartBackoffTimeStrategy.canRestart()) { + numberOfRestarts++; + return FailureHandlingResult.restartable( verticesToRestart, restartBackoffTimeStrategy.getBackoffTime()); @@ -113,4 +118,8 @@ static boolean isUnrecoverableError(Throwable cause) { cause, ThrowableType.NonRecoverableError); return unrecoverableError.isPresent(); } + + public long getNumberOfRestarts() { + return numberOfRestarts; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java index 6ab3c55a6127..d6f539d2bb27 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java @@ -51,6 +51,8 @@ private MetricNames() { public static final String TASK_SLOTS_TOTAL = "taskSlotsTotal"; public static final String NUM_REGISTERED_TASK_MANAGERS = "numRegisteredTaskManagers"; + public static final String NUMBER_OF_RESTARTS = "numberOfRestarts"; + public static final String MEMORY_USED = "Used"; public static final String MEMORY_COMMITTED = "Committed"; public static final String MEMORY_MAX = "Max"; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java index ccb59eaed514..f139edcecf1b 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 @@ -151,6 +151,11 @@ public DefaultScheduler( // SchedulerNG // ------------------------------------------------------------------------ + @Override + protected long getNumberOfRestarts() { + return executionFailureHandler.getNumberOfRestarts(); + } + @Override protected void startSchedulingInternal() { log.debug("Starting scheduling with scheduling strategy [{}]", schedulingStrategy.getClass().getName()); 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 e227dba1f58c..28d92d467ccc 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 @@ -81,6 +81,11 @@ public LegacyScheduler( partitionTracker); } + @Override + protected long getNumberOfRestarts() { + return getExecutionGraph().getNumberOfRestarts(); + } + @Override protected void startSchedulingInternal() { final ExecutionGraph executionGraph = getExecutionGraph(); 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 d9d4a2974049..573ccea67730 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 @@ -99,6 +99,7 @@ import java.util.concurrent.Executor; import java.util.concurrent.ScheduledExecutorService; +import static org.apache.flink.runtime.metrics.MetricNames.NUMBER_OF_RESTARTS; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkState; @@ -194,6 +195,8 @@ public SchedulerBase( this.failoverTopology = executionGraph.getFailoverTopology(); this.inputsLocationsRetriever = new ExecutionGraphToInputsLocationsRetrieverAdapter(executionGraph); + + jobManagerJobMetricGroup.gauge(NUMBER_OF_RESTARTS, this::getNumberOfRestarts); } private ExecutionGraph createAndRestoreExecutionGraph( @@ -372,6 +375,8 @@ protected JobGraph getJobGraph() { return jobGraph; } + protected abstract long getNumberOfRestarts(); + // ------------------------------------------------------------------------ // SchedulerNG // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandlerTest.java index bf4feec3df1f..e0c274d282c1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/ExecutionFailureHandlerTest.java @@ -87,6 +87,7 @@ public void testNormalFailureHandling() { } catch (IllegalStateException ex) { // expected } + assertEquals(1, executionFailureHandler.getNumberOfRestarts()); } /** @@ -118,6 +119,7 @@ public void testRestartingSuppressedFailureHandlingResult() { } catch (IllegalStateException ex) { // expected } + assertEquals(0, executionFailureHandler.getNumberOfRestarts()); } /** @@ -146,6 +148,7 @@ public void testNonRecoverableFailureHandlingResult() { } catch (IllegalStateException ex) { // expected } + assertEquals(0, executionFailureHandler.getNumberOfRestarts()); } /** From ba7c279a147c14270d842a475a7135ab2c403b8f Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Thu, 7 Nov 2019 18:12:42 +0800 Subject: [PATCH 483/746] [FLINK-14164][runtime] Add docs for 'numberOfRestarts' metric This closes #10082. --- docs/monitoring/metrics.md | 5 +++++ docs/monitoring/metrics.zh.md | 5 +++++ 2 files changed, 10 insertions(+) diff --git a/docs/monitoring/metrics.md b/docs/monitoring/metrics.md index 948d193f9403..87fea3dbc873 100644 --- a/docs/monitoring/metrics.md +++ b/docs/monitoring/metrics.md @@ -1279,6 +1279,11 @@ Metrics related to data exchange between task executors using netty network comm The total number of full restarts since this job was submitted. Gauge + + numberOfRestarts + The total number of restarts since this job was submitted, including full restarts and fine grained restarts. + Gauge + diff --git a/docs/monitoring/metrics.zh.md b/docs/monitoring/metrics.zh.md index 798cbab516fb..fa1db8e7b6e9 100644 --- a/docs/monitoring/metrics.zh.md +++ b/docs/monitoring/metrics.zh.md @@ -1279,6 +1279,11 @@ Metrics related to data exchange between task executors using netty network comm The total number of full restarts since this job was submitted. Gauge + + numberOfRestarts + The total number of restarts since this job was submitted, including full restarts and fine grained restarts. + Gauge + From e933a5cbc263be5041bd3723cd2400c871739733 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Thu, 7 Nov 2019 18:18:43 +0800 Subject: [PATCH 484/746] [FLINK-14373][tests] Fix ZooKeeperHighAvailabilityITCase to pass with NG scheduler Also annotate it with category AlsoRunWithSchedulerNG. This closes #10085. --- .../ZooKeeperHighAvailabilityITCase.java | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java index 4a68c914a06e..7d2e3b258b69 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java @@ -32,14 +32,15 @@ import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Metric; import org.apache.flink.metrics.MetricConfig; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.metrics.reporter.MetricReporter; import org.apache.flink.runtime.concurrent.FutureUtils; -import org.apache.flink.runtime.executiongraph.metrics.NumberOfFullRestartsGauge; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.metrics.MetricNames; import org.apache.flink.runtime.state.CheckpointListener; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; @@ -53,6 +54,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.testutils.junit.category.AlsoRunWithSchedulerNG; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; import org.apache.flink.util.TestLogger; @@ -62,6 +64,7 @@ import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.junit.rules.TemporaryFolder; import java.io.File; @@ -93,6 +96,7 @@ /** * Integration tests for {@link org.apache.flink.runtime.checkpoint.ZooKeeperCompletedCheckpointStore}. */ +@Category(AlsoRunWithSchedulerNG.class) public class ZooKeeperHighAvailabilityITCase extends TestLogger { private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10000L); @@ -424,10 +428,10 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { } /** - * Reporter that exposes the {@link NumberOfFullRestartsGauge} metric. + * Reporter that exposes the {@code numberOfRestarts} metric. */ public static class RestartReporter implements MetricReporter { - static volatile NumberOfFullRestartsGauge numRestarts = null; + static volatile Gauge numRestarts = null; @Override public void open(MetricConfig metricConfig) { @@ -438,9 +442,9 @@ public void close() { } @Override - public void notifyOfAddedMetric(Metric metric, String s, MetricGroup metricGroup) { - if (metric instanceof NumberOfFullRestartsGauge) { - numRestarts = (NumberOfFullRestartsGauge) metric; + public void notifyOfAddedMetric(Metric metric, String name, MetricGroup metricGroup) { + if (name.equals(MetricNames.NUMBER_OF_RESTARTS)) { + numRestarts = (Gauge) metric; } } From 525f3c9c45f8534f1782986781140473909116a8 Mon Sep 17 00:00:00 2001 From: Yu Li Date: Tue, 20 Aug 2019 16:12:01 +0200 Subject: [PATCH 485/746] [FLINK-12697][state backends] Support on-disk state storage for spill-able heap backend --- .../memory/ByteArrayInputStreamWithPos.java | 97 +- .../flink/core/memory/MemorySegment.java | 17 + .../MemorySegmentInputStreamWithPos.java | 114 ++ .../ByteArrayInputStreamWithPosTest.java | 48 +- .../core/memory/MemorySegmentTestBase.java | 23 + .../flink-statebackend-heap-spillable/pom.xml | 19 +- .../heap/CopyOnWriteSkipListStateMap.java | 1582 +++++++++++++++++ .../CopyOnWriteSkipListStateMapSnapshot.java | 229 +++ .../runtime/state/heap/LevelIndexHeader.java | 55 + .../flink/runtime/state/heap/NodeStatus.java | 48 + .../state/heap/OnHeapLevelIndexHeader.java | 118 ++ .../state/heap/SkipListKeyComparator.java | 80 + .../state/heap/SkipListKeySerializer.java | 165 ++ .../runtime/state/heap/SkipListUtils.java | 784 ++++++++ .../state/heap/SkipListValueSerializer.java | 75 + .../runtime/state/heap/space/Allocator.java | 51 + .../flink/runtime/state/heap/space/Chunk.java | 64 + .../runtime/state/heap/space/Constants.java | 29 + .../runtime/state/heap/space/SpaceUtils.java | 36 + ...opyOnWriteSkipListStateMapBasicOpTest.java | 548 ++++++ ...yOnWriteSkipListStateMapComplexOpTest.java | 930 ++++++++++ .../CopyOnWriteSkipListStateMapTestUtils.java | 259 +++ .../heap/OnHeapLevelIndexHeaderTest.java | 169 ++ .../state/heap/SkipListKeyComparatorTest.java | 299 ++++ .../state/heap/SkipListSerializerTest.java | 110 ++ .../runtime/state/heap/SkipListUtilsTest.java | 170 ++ .../runtime/state/heap/TestAllocator.java | 166 ++ 27 files changed, 6185 insertions(+), 100 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentInputStreamWithPos.java create mode 100644 flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteSkipListStateMap.java create mode 100644 flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteSkipListStateMapSnapshot.java create mode 100644 flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/LevelIndexHeader.java create mode 100644 flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/NodeStatus.java create mode 100644 flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/OnHeapLevelIndexHeader.java create mode 100644 flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/SkipListKeyComparator.java create mode 100644 flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/SkipListKeySerializer.java create mode 100644 flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/SkipListUtils.java create mode 100644 flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/SkipListValueSerializer.java create mode 100644 flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/space/Allocator.java create mode 100644 flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/space/Chunk.java create mode 100644 flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/space/Constants.java create mode 100644 flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/space/SpaceUtils.java create mode 100644 flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteSkipListStateMapBasicOpTest.java create mode 100644 flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteSkipListStateMapComplexOpTest.java create mode 100644 flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteSkipListStateMapTestUtils.java create mode 100644 flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/OnHeapLevelIndexHeaderTest.java create mode 100644 flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/SkipListKeyComparatorTest.java create mode 100644 flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/SkipListSerializerTest.java create mode 100644 flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/SkipListUtilsTest.java create mode 100644 flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/TestAllocator.java diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/ByteArrayInputStreamWithPos.java b/flink-core/src/main/java/org/apache/flink/core/memory/ByteArrayInputStreamWithPos.java index bc81593dea45..46377916d634 100644 --- a/flink-core/src/main/java/org/apache/flink/core/memory/ByteArrayInputStreamWithPos.java +++ b/flink-core/src/main/java/org/apache/flink/core/memory/ByteArrayInputStreamWithPos.java @@ -19,24 +19,15 @@ package org.apache.flink.core.memory; import org.apache.flink.annotation.Internal; -import org.apache.flink.util.Preconditions; - -import java.io.IOException; -import java.io.InputStream; /** * Un-synchronized stream similar to Java's ByteArrayInputStream that also exposes the current position. */ @Internal -public class ByteArrayInputStreamWithPos extends InputStream { +public class ByteArrayInputStreamWithPos extends MemorySegmentInputStreamWithPos { private static final byte[] EMPTY = new byte[0]; - protected byte[] buffer; - protected int position; - protected int count; - protected int mark = 0; - public ByteArrayInputStreamWithPos() { this(EMPTY); } @@ -46,90 +37,10 @@ public ByteArrayInputStreamWithPos(byte[] buffer) { } public ByteArrayInputStreamWithPos(byte[] buffer, int offset, int length) { - setBuffer(buffer, offset, length); - } - - @Override - public int read() { - return (position < count) ? 0xFF & (buffer[position++]) : -1; - } - - @Override - public int read(byte[] b, int off, int len) { - Preconditions.checkNotNull(b); - - if (off < 0 || len < 0 || len > b.length - off) { - throw new IndexOutOfBoundsException(); - } - - if (position >= count) { - return -1; // signal EOF - } - - int available = count - position; - - if (len > available) { - len = available; - } - - if (len <= 0) { - return 0; - } - - System.arraycopy(buffer, position, b, off, len); - position += len; - return len; - } - - @Override - public long skip(long toSkip) { - long remain = count - position; - - if (toSkip < remain) { - remain = toSkip < 0 ? 0 : toSkip; - } - - position += remain; - return remain; - } - - @Override - public boolean markSupported() { - return true; - } - - @Override - public void mark(int readAheadLimit) { - mark = position; - } - - @Override - public void reset() { - position = mark; - } - - @Override - public int available() { - return count - position; - } - - @Override - public void close() throws IOException { - } - - public int getPosition() { - return position; - } - - public void setPosition(int pos) { - Preconditions.checkArgument(pos >= 0 && pos <= count, "Position out of bounds."); - this.position = pos; + super(MemorySegmentFactory.wrap(buffer), offset, length); } - public void setBuffer(byte[] buffer, int offset, int length) { - this.count = Math.min(buffer.length, offset + length); - setPosition(offset); - this.buffer = buffer; - this.mark = offset; + public void setBuffer(byte[] buffer, int off, int len) { + setSegment(MemorySegmentFactory.wrap(buffer), off, len); } } diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java index d66b75d4d9df..82dbd56ef5bf 100644 --- a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java +++ b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java @@ -1356,6 +1356,23 @@ public final int compare(MemorySegment seg2, int offset1, int offset2, int len) return 0; } + /** + * Compares two memory segment regions with different length. + * + * @param seg2 Segment to compare this segment with + * @param offset1 Offset of this segment to start comparing + * @param offset2 Offset of seg2 to start comparing + * @param len1 Length of this memory region to compare + * @param len2 Length of seg2 to compare + * + * @return 0 if equal, -1 if seg1 < seg2, 1 otherwise + */ + public final int compare(MemorySegment seg2, int offset1, int offset2, int len1, int len2) { + final int minLength = Math.min(len1, len2); + int c = compare(seg2, offset1, offset2, minLength); + return c == 0 ? (len1 - len2) : c; + } + /** * Swaps bytes between two memory segments, using the given auxiliary buffer. * diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentInputStreamWithPos.java b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentInputStreamWithPos.java new file mode 100644 index 000000000000..8b39ff73188b --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentInputStreamWithPos.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.core.memory; + +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nonnull; + +import java.io.InputStream; + +/** + * Un-synchronized input stream using the given memory segment. + */ +public class MemorySegmentInputStreamWithPos extends InputStream { + + private MemorySegment segment; + private int position; + private int count; + private int mark; + + public MemorySegmentInputStreamWithPos(MemorySegment segment, int offset, int length) { + setSegment(segment, offset, length); + } + + @Override + public int read() { + return (position < count) ? 0xFF & (segment.get(position++)) : -1; + } + + @Override + public int read(@Nonnull byte[] b, int off, int len) { + if (position >= count) { + return -1; // signal EOF + } + if (len <= 0) { + return 0; + } + + final int numBytes = Math.min(count - position, len); + + segment.get(position, b, off, numBytes); + position += numBytes; + + return numBytes; + } + + @Override + public long skip(long toSkip) { + long remain = count - position; + + if (toSkip < remain) { + remain = toSkip < 0 ? 0 : toSkip; + } + + position += remain; + return remain; + } + + @Override + public boolean markSupported() { + return true; + } + + @Override + public void mark(int readAheadLimit) { + mark = position; + } + + @Override + public void reset() { + position = mark; + } + + @Override + public int available() { + return count - position; + } + + @Override + public void close() { + } + + public int getPosition() { + return position; + } + + public void setPosition(int pos) { + Preconditions.checkArgument(pos >= 0 && pos <= count, "Position out of bounds."); + this.position = pos; + } + + public void setSegment(MemorySegment segment, int offset, int length) { + this.count = Math.min(segment.size(), offset + length); + setPosition(offset); + this.segment = segment; + this.mark = offset; + } +} diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/ByteArrayInputStreamWithPosTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/ByteArrayInputStreamWithPosTest.java index 62ad911ab250..6d3439e15697 100644 --- a/flink-core/src/test/java/org/apache/flink/core/memory/ByteArrayInputStreamWithPosTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/memory/ByteArrayInputStreamWithPosTest.java @@ -19,11 +19,12 @@ package org.apache.flink.core.memory; import org.junit.Assert; -import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import static org.junit.Assert.assertEquals; + /** * Tests for {@link ByteArrayInputStreamWithPos}. */ @@ -32,13 +33,40 @@ public class ByteArrayInputStreamWithPosTest { @Rule public ExpectedException thrown = ExpectedException.none(); - private byte[] data = new byte[] {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9'}; + private final byte[] data = new byte[] {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9'}; - private ByteArrayInputStreamWithPos stream; + private final ByteArrayInputStreamWithPos stream = new ByteArrayInputStreamWithPos(data); - @Before - public void setup() { - stream = new ByteArrayInputStreamWithPos(data); + @Test + public void testGetWithNullArray() { + thrown.expect(NullPointerException.class); + stream.read(null, 0, 1); + } + + @Test + public void testGetWithNegativeLength() { + int read = stream.read(new byte[0], 0, -1); + assertEquals(0, read); + } + + @Test + public void testGetWithTargetArrayOverflow() { + thrown.expect(IndexOutOfBoundsException.class); + stream.read(new byte[0], 0, 2); + } + + @Test + public void testGetWithEOF() { + drainStream(stream); + int read = stream.read(new byte[1], 0, 1); + assertEquals(-1, read); + } + + @Test + public void testGetMoreThanAvailable() { + int read = stream.read(new byte[20], 0, 20); + assertEquals(10, read); + assertEquals(-1, stream.read()); // exhausted now } /** @@ -95,4 +123,12 @@ public void testSetBuffer() { } Assert.assertEquals(-1, in.read()); } + + private static int drainStream(ByteArrayInputStreamWithPos stream) { + int skipped = 0; + while (stream.read() != -1) { + skipped++; + } + return skipped; + } } diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentTestBase.java b/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentTestBase.java index b618cc2fccec..dcfcb75bd05b 100644 --- a/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentTestBase.java +++ b/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentTestBase.java @@ -37,11 +37,15 @@ import java.util.Collection; import java.util.Random; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.lessThan; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -2282,6 +2286,25 @@ public void testCompareBytes() { } } + @Test + public void testCompareBytesWithDifferentLength() { + final byte[] bytes1 = new byte[] {'a', 'b', 'c'}; + final byte[] bytes2 = new byte[] {'a', 'b', 'c', 'd'}; + + MemorySegment seg1 = createSegment(4); + MemorySegment seg2 = createSegment(4); + seg1.put(0, bytes1); + seg2.put(0, bytes2); + + assertThat(seg1.compare(seg2, 0, 0, 3, 4), lessThan(0)); + assertThat(seg1.compare(seg2, 0, 0, 3, 3), equalTo(0)); + assertThat(seg1.compare(seg2, 0, 0, 3, 2), greaterThan(0)); + // test non-zero offset + assertThat(seg1.compare(seg2, 1, 1, 2, 3), lessThan(0)); + assertThat(seg1.compare(seg2, 1, 1, 2, 2), equalTo(0)); + assertThat(seg1.compare(seg2, 1, 1, 2, 1), greaterThan(0)); + } + @Test public void testSwapBytes() { final int halfPageSize = pageSize / 2; diff --git a/flink-state-backends/flink-statebackend-heap-spillable/pom.xml b/flink-state-backends/flink-statebackend-heap-spillable/pom.xml index 327dca096968..78cd7134b7ba 100644 --- a/flink-state-backends/flink-statebackend-heap-spillable/pom.xml +++ b/flink-state-backends/flink-statebackend-heap-spillable/pom.xml @@ -31,11 +31,28 @@ under the License. .. - flink-statebackend-heap-spillable + flink-statebackend-heap-spillable_${scala.binary.version} flink-statebackend-heap-spillable jar + + + + org.apache.flink + flink-runtime_${scala.binary.version} + ${project.version} + provided + + + + + + org.apache.flink + flink-test-utils-junit + ${project.version} + test + diff --git a/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteSkipListStateMap.java b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteSkipListStateMap.java new file mode 100644 index 000000000000..106939247d01 --- /dev/null +++ b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteSkipListStateMap.java @@ -0,0 +1,1582 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.state.heap; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.core.memory.MemorySegmentFactory; +import org.apache.flink.runtime.state.StateEntry; +import org.apache.flink.runtime.state.StateTransformationFunction; +import org.apache.flink.runtime.state.heap.space.Allocator; +import org.apache.flink.runtime.state.heap.space.Chunk; +import org.apache.flink.runtime.state.heap.space.SpaceUtils; +import org.apache.flink.runtime.state.internal.InternalKvState; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.ResourceGuard; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.Spliterators; +import java.util.TreeSet; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.BiFunction; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; + +import static org.apache.flink.runtime.state.heap.SkipListUtils.HEAD_NODE; +import static org.apache.flink.runtime.state.heap.SkipListUtils.NIL_NODE; +import static org.apache.flink.runtime.state.heap.SkipListUtils.NIL_VALUE_POINTER; + +/** + * Implementation of state map which is based on skip list with copy-on-write support. states will + * be serialized to bytes and stored in the space allocated with the given allocator. + * + * @param type of key + * @param type of namespace + * @param type of state + */ +public final class CopyOnWriteSkipListStateMap extends StateMap implements AutoCloseable { + + private static final Logger LOG = LoggerFactory.getLogger(CopyOnWriteSkipListStateMap.class); + + /** + * Default max number of logically-removed keys to delete one time. + */ + static final int DEFAULT_MAX_KEYS_TO_DELETE_ONE_TIME = 3; + + /** + * Default ratio of the logically-removed keys to trigger deletion when snapshot. + */ + static final float DEFAULT_LOGICAL_REMOVED_KEYS_RATIO = 0.2f; + + /** + * The serializer used to serialize the key and namespace to bytes stored in skip list. + */ + private final SkipListKeySerializer skipListKeySerializer; + + /** + * The serializer used to serialize the state to bytes stored in skip list. + */ + private final SkipListValueSerializer skipListValueSerializer; + + /** + * Space allocator. + */ + private final Allocator spaceAllocator; + + /** + * The level index header. + */ + private final LevelIndexHeader levelIndexHeader; + + /** + * Seed to generate random index level. + */ + private int randomSeed; + + /** + * The current version of this map. Used for copy-on-write mechanics. + */ + private volatile int stateMapVersion; + + /** + * Any version less than this value is still required by some unreleased snapshot. 0 means no snapshot ongoing. + */ + private volatile int highestRequiredSnapshotVersionPlusOne; + + /** + * Snapshots no more than this version must have been finished, but there may be some + * snapshots more than this version are still running. + */ + private volatile int highestFinishedSnapshotVersion; + + /** + * Maintains an ordered set of version ids that are still used by unreleased snapshots. + */ + private final TreeSet snapshotVersions; + + /** + * The size of skip list which includes the logical removed keys. + */ + private int totalSize; + + /** + * Number of requests for this skip list. + */ + private int requestCount; + + /** + * Set of logical removed nodes. + */ + private final Set logicallyRemovedNodes; + + /** + * Number of keys to remove physically one time. + */ + private int numKeysToDeleteOneTime; + + /** + * Ratio of the logically-removed keys to trigger deletion when snapshot. + */ + private float logicalRemovedKeysRatio; + + /** + * Set of nodes whose values are being pruned by snapshots. + */ + private final Set pruningValueNodes; + + /** + * Whether this map has been closed. + */ + private final AtomicBoolean closed; + + /** + * Guards for the free of space when state map is closed. This is mainly + * used to synchronize with snapshots. + */ + private final ResourceGuard resourceGuard; + + public CopyOnWriteSkipListStateMap( + @Nonnull TypeSerializer keySerializer, + @Nonnull TypeSerializer namespaceSerializer, + @Nonnull TypeSerializer stateSerializer, + @Nonnull Allocator spaceAllocator, + int numKeysToDeleteOneTime, + float logicalRemovedKeysRatio) { + this.skipListKeySerializer = new SkipListKeySerializer<>(keySerializer, namespaceSerializer); + this.skipListValueSerializer = new SkipListValueSerializer<>(stateSerializer); + this.spaceAllocator = spaceAllocator; + Preconditions.checkArgument(numKeysToDeleteOneTime >= 0, + "numKeysToDeleteOneTime should be non-negative, but is " + numKeysToDeleteOneTime); + this.numKeysToDeleteOneTime = numKeysToDeleteOneTime; + Preconditions.checkArgument(logicalRemovedKeysRatio >= 0 && logicalRemovedKeysRatio <= 1, + "logicalRemovedKeysRatio should be in [0, 1], but is " + logicalRemovedKeysRatio); + this.logicalRemovedKeysRatio = logicalRemovedKeysRatio; + + this.levelIndexHeader = new OnHeapLevelIndexHeader(); + // Refers to JDK implementation of Xor-shift random number generator, 0x0100 is to ensure non-zero seed. + // See https://github.com/openjdk-mirror/jdk7u-jdk/blob/master/src/share/classes/java/util/concurrent/ConcurrentSkipListMap.java#L373 + this.randomSeed = ThreadLocalRandom.current().nextInt() | 0x0100; + + this.stateMapVersion = 0; + this.highestRequiredSnapshotVersionPlusOne = 0; + this.highestFinishedSnapshotVersion = 0; + this.snapshotVersions = new TreeSet<>(); + + this.totalSize = 0; + this.requestCount = 0; + + this.logicallyRemovedNodes = new HashSet<>(); + this.pruningValueNodes = ConcurrentHashMap.newKeySet(); + + this.closed = new AtomicBoolean(false); + this.resourceGuard = new ResourceGuard(); + } + + @Override + public int size() { + return totalSize - logicallyRemovedNodes.size(); + } + + /** + * Returns total size of this map, including logically removed state. + */ + int totalSize() { + return totalSize; + } + + public int getRequestCount() { + return requestCount; + } + + @Override + public S get(K key, N namespace) { + updateStat(); + + return getNodeInternal(key, namespace); + } + + @Override + public boolean containsKey(K key, N namespace) { + updateStat(); + + S node = getNodeInternal(key, namespace); + + return node != null; + } + + @Override + public void put(K key, N namespace, S state) { + updateStat(); + MemorySegment keySegment = getKeySegment(key, namespace); + int keyLen = keySegment.size(); + byte[] value = skipListValueSerializer.serialize(state); + + putValue(keySegment, 0, keyLen, value, false); + } + + @Override + public S putAndGetOld(K key, N namespace, S state) { + updateStat(); + MemorySegment keySegment = getKeySegment(key, namespace); + int keyLen = keySegment.size(); + byte[] value = skipListValueSerializer.serialize(state); + + return putValue(keySegment, 0, keyLen, value, true); + } + + @Override + public void remove(K key, N namespace) { + updateStat(); + MemorySegment keySegment = getKeySegment(key, namespace); + int keyLen = keySegment.size(); + + removeNode(keySegment, 0, keyLen, false); + } + + @Override + public S removeAndGetOld(K key, N namespace) { + updateStat(); + MemorySegment keySegment = getKeySegment(key, namespace); + int keyLen = keySegment.size(); + + return removeNode(keySegment, 0, keyLen, true); + } + + @Override + public void transform( + K key, + N namespace, + T value, + StateTransformationFunction transformation) throws Exception { + updateStat(); + MemorySegment keySegment = getKeySegment(key, namespace); + int keyLen = keySegment.size(); + + S oldState = getNode(keySegment, 0, keyLen); + S newState = transformation.apply(oldState, value); + byte[] stateBytes = skipListValueSerializer.serialize(newState); + putValue(keySegment, 0, keyLen, stateBytes, false); + } + + // Detail implementation methods --------------------------------------------------------------- + + /** + * Find the node containing the given key. + * + * @param keySegment memory segment storing the key. + * @param keyOffset offset of the key. + * @param keyLen length of the key. + * @return the state. Null will be returned if key does not exist. + */ + @VisibleForTesting + @Nullable S getNode(MemorySegment keySegment, int keyOffset, int keyLen) { + SkipListIterateAndProcessResult result = iterateAndProcess(keySegment, keyOffset, keyLen, + (pointers, isRemoved) -> { + long currentNode = pointers.currentNode; + return isRemoved ? null : getNodeStateHelper(currentNode); + }); + return result.isKeyFound ? result.state : null; + } + + /** + * Put the key into the skip list. If the key does not exist before, a new node will be created. + * If the key exists before, return the old state or null depending on {@code returnOldState}. + * + * @param keySegment memory segment storing the key. + * @param keyOffset offset of the key. + * @param keyLen length of the key. + * @param value the value. + * @param returnOldState whether to return old state. + * @return the old state. Null will be returned if key does not exist or returnOldState is false. + */ + @VisibleForTesting + S putValue(MemorySegment keySegment, int keyOffset, int keyLen, byte[] value, boolean returnOldState) { + SkipListIterateAndProcessResult result = iterateAndProcess(keySegment, keyOffset, keyLen, + (pointers, isLogicallyRemoved) -> putValue(pointers.currentNode, value, returnOldState)); + + if (result.isKeyFound) { + return result.state; + } + + long prevNode = result.prevNode; + long currentNode = result.currentNode; + + int level = getRandomIndexLevel(); + levelIndexHeader.updateLevel(level); + + int totalMetaKeyLen = SkipListUtils.getKeyMetaLen(level) + keyLen; + long node = allocateSpace(totalMetaKeyLen); + + int totalValueLen = SkipListUtils.getValueMetaLen() + value.length; + long valuePointer = allocateSpace(totalValueLen); + + doWriteKey(node, level, keySegment, keyOffset, keyLen, valuePointer, currentNode); + doWriteValue(valuePointer, value, stateMapVersion, node, NIL_VALUE_POINTER); + + helpSetNextNode(prevNode, node, 0); + + if (level > 0) { + SkipListUtils.buildLevelIndex(node, level, keySegment, keyOffset, levelIndexHeader, spaceAllocator); + } + + totalSize++; + + return null; + } + + /** + * Update or insert the value for the given node. + * + * @param currentNode the node to put value for. + * @param value the value to put. + * @param returnOldState whether to return the old state. + * @return the old state if it exists and {@code returnOldState} is true, or else null. + */ + private S putValue(long currentNode, byte[] value, boolean returnOldState) { + int version = SkipListUtils.helpGetNodeLatestVersion(currentNode, spaceAllocator); + boolean needCopyOnWrite = version < highestRequiredSnapshotVersionPlusOne; + long oldValuePointer; + + if (needCopyOnWrite) { + oldValuePointer = updateValueWithCopyOnWrite(currentNode, value); + } else { + oldValuePointer = updateValueWithReplace(currentNode, value); + } + + NodeStatus oldStatus = helpSetNodeStatus(currentNode, NodeStatus.PUT); + if (oldStatus == NodeStatus.REMOVE) { + logicallyRemovedNodes.remove(currentNode); + } + + S oldState = null; + if (returnOldState) { + oldState = helpGetState(oldValuePointer); + } + + // for the replace, old value space need to free + if (!needCopyOnWrite) { + spaceAllocator.free(oldValuePointer); + } + + return oldState; + } + + /** + * Remove the key from the skip list. The key can be removed logically or physically. + * Logical remove means put a null value whose size is 0. If the key exists before, + * the old value state will be returned. + * + * @param keySegment memory segment storing the key. + * @param keyOffset offset of the key. + * @param keyLen length of the key. + * @param returnOldState whether to return old state. + * @return the old state. Null will be returned if key does not exist or returnOldState is false. + */ + private S removeNode(MemorySegment keySegment, int keyOffset, int keyLen, boolean returnOldState) { + SkipListIterateAndProcessResult result = iterateAndProcess(keySegment, keyOffset, keyLen, + (pointers, isLogicallyRemoved) -> removeNode(pointers, isLogicallyRemoved, returnOldState)); + return result.isKeyFound ? result.state : null; + } + + /** + * Remove the given node indicated by {@link SkipListNodePointers#currentNode}. + * + * @param pointers pointers of the node to remove and its prev/next node. + * @param isLogicallyRemoved whether the node to remove is already logically removed. + * @param returnOldState whether to return the old state after removal. + * @return the old state if {@code returnOldState} is true, or else return null. + */ + private S removeNode(SkipListNodePointers pointers, Boolean isLogicallyRemoved, boolean returnOldState) { + long prevNode = pointers.prevNode; + long currentNode = pointers.currentNode; + long nextNode = pointers.nextNode; + // if the node has been logically removed, and can not be physically + // removed here, just return null + if (isLogicallyRemoved && highestRequiredSnapshotVersionPlusOne != 0) { + return null; + } + + long oldValuePointer; + boolean oldValueNeedFree; + + if (highestRequiredSnapshotVersionPlusOne == 0) { + // do physically remove only when there is no snapshot running + oldValuePointer = doPhysicalRemoveAndGetValue(currentNode, prevNode, nextNode); + // the node has been logically removed, and remove it from the set + if (isLogicallyRemoved) { + logicallyRemovedNodes.remove(currentNode); + } + oldValueNeedFree = true; + } else { + int version = SkipListUtils.helpGetNodeLatestVersion(currentNode, spaceAllocator); + if (version < highestRequiredSnapshotVersionPlusOne) { + // the newest-version value may be used by snapshots, and update it with copy-on-write + oldValuePointer = updateValueWithCopyOnWrite(currentNode, null); + oldValueNeedFree = false; + } else { + // replace the newest-version value. + oldValuePointer = updateValueWithReplace(currentNode, null); + oldValueNeedFree = true; + } + + helpSetNodeStatus(currentNode, NodeStatus.REMOVE); + logicallyRemovedNodes.add(currentNode); + } + + S oldState = null; + if (returnOldState) { + oldState = helpGetState(oldValuePointer); + } + + if (oldValueNeedFree) { + spaceAllocator.free(oldValuePointer); + } + + return oldState; + } + + private long allocateSpace(int size) { + try { + return spaceAllocator.allocate(size); + } catch (Exception e) { + throw new FlinkRuntimeException("Failed to allocate space in CopyOnWriteSkipListStateMap", e); + } + } + + /** + * Iterate the skip list and perform given function. + * + * @param keySegment memory segment storing the key. + * @param keyOffset offset of the key. + * @param keyLen length of the key. + * @param function the function to apply when the skip list contains the given key, which accepts two parameters: + * an encapsulation of [previous_node, current_node, next_node] and a boolean indicating + * whether the node with same key has been logically removed, and returns a state. + * @return the iterate and processing result + */ + private SkipListIterateAndProcessResult iterateAndProcess( + MemorySegment keySegment, + int keyOffset, + int keyLen, + BiFunction function) { + int deleteCount = 0; + long prevNode = findPredecessor(keySegment, keyOffset, 1); + long currentNode = helpGetNextNode(prevNode, 0); + long nextNode; + + int c; + while (currentNode != NIL_NODE) { + nextNode = helpGetNextNode(currentNode, 0); + + // Check whether the current code is already logically removed to save some comparisons on key, + // with the cost of an additional remove-then-add operation if the to-be-removed node has the same key + // with the to-be-put one. + boolean isRemoved = isNodeRemoved(currentNode); + if (isRemoved && highestRequiredSnapshotVersionPlusOne == 0 && deleteCount < numKeysToDeleteOneTime) { + doPhysicalRemove(currentNode, prevNode, nextNode); + logicallyRemovedNodes.remove(currentNode); + currentNode = nextNode; + deleteCount++; + continue; + } + + c = compareSegmentAndNode(keySegment, keyOffset, keyLen, currentNode); + + if (c < 0) { + // The given key is less than the current node, break the loop + break; + } else if (c > 0) { + // The given key is larger than the current node, continue + prevNode = currentNode; + currentNode = nextNode; + } else { + // The given key is equal to the current node, apply the function + S state = function.apply(new SkipListNodePointers(prevNode, currentNode, nextNode), isRemoved); + return new SkipListIterateAndProcessResult(prevNode, currentNode, true, state); + } + } + return new SkipListIterateAndProcessResult(prevNode, currentNode, false, null); + } + + /** + * Find the predecessor node for the given key at the given level. + * The key is in the memory segment positioning at the given offset. + * + * @param keySegment memory segment which contains the key. + * @param keyOffset offset of the key in the memory segment. + * @param level the level. + * @return node id before the key at the given level. + */ + private long findPredecessor(MemorySegment keySegment, int keyOffset, int level) { + return SkipListUtils.findPredecessor(keySegment, keyOffset, level, levelIndexHeader, spaceAllocator); + } + + /** + * Compare the first skip list key in the given memory segment with the second skip list key in the given node. + * + * @param keySegment memory segment storing the first key. + * @param keyOffset offset of the first key in memory segment. + * @param keyLen length of the first key. + * @param targetNode the node storing the second key. + * @return Returns a negative integer, zero, or a positive integer as the first key is less than, + * equal to, or greater than the second. + */ + private int compareSegmentAndNode(MemorySegment keySegment, int keyOffset, int keyLen, long targetNode) { + return SkipListUtils.compareSegmentAndNode(keySegment, keyOffset, targetNode, spaceAllocator); + } + + /** + * Compare the first namespace in the given memory segment with the second namespace in the given node. + * + * @param namespaceSegment memory segment storing the first namespace. + * @param namespaceOffset offset of the first namespace in memory segment. + * @param namespaceLen length of the first namespace. + * @param targetNode the node storing the second namespace. + * @return Returns a negative integer, zero, or a positive integer as the first key is less than, + * equal to, or greater than the second. + */ + private int compareNamespaceAndNode(MemorySegment namespaceSegment, int namespaceOffset, int namespaceLen, long targetNode) { + Node nodeStorage = getNodeSegmentAndOffset(targetNode); + MemorySegment targetSegment = nodeStorage.nodeSegment; + int offsetInSegment = nodeStorage.nodeOffset; + + int level = SkipListUtils.getLevel(targetSegment, offsetInSegment); + int targetKeyOffset = offsetInSegment + SkipListUtils.getKeyDataOffset(level); + + return SkipListKeyComparator.compareNamespaceAndNode(namespaceSegment, namespaceOffset, namespaceLen, + targetSegment, targetKeyOffset); + } + + /** + * Update the value of the node with copy-on-write mode. The old value will + * be linked after the new value, and can be still accessed. + * + * @param node the node to update. + * @param value the value. + * @return the old value pointer. + */ + private long updateValueWithCopyOnWrite(long node, byte[] value) { + // a null value indicates this is a removed node + int valueSize = value == null ? 0 : value.length; + int totalValueLen = SkipListUtils.getValueMetaLen() + valueSize; + long valuePointer = allocateSpace(totalValueLen); + + Node nodeStorage = getNodeSegmentAndOffset(node); + MemorySegment nodeSegment = nodeStorage.nodeSegment; + int offsetInNodeSegment = nodeStorage.nodeOffset; + long oldValuePointer = SkipListUtils.getValuePointer(nodeSegment, offsetInNodeSegment); + + doWriteValue(valuePointer, value, stateMapVersion, node, oldValuePointer); + + // update value pointer in node after the new value has points the older value so that + // old value can be accessed concurrently + SkipListUtils.putValuePointer(nodeSegment, offsetInNodeSegment, valuePointer); + + return oldValuePointer; + } + + /** + * Update the value of the node with replace mode. The old value will be unlinked and replaced + * by the new value, and can not be accessed later. Note that the space of the old value + * is not freed here, and the caller of this method should be responsible for the space management. + * + * @param node the node whose value will be replaced. + * @param value the value. + * @return the old value pointer. + */ + private long updateValueWithReplace(long node, byte[] value) { + // a null value indicates this is a removed node + int valueSize = value == null ? 0 : value.length; + int totalValueLen = SkipListUtils.getValueMetaLen() + valueSize; + long valuePointer = allocateSpace(totalValueLen); + + Node nodeStorage = getNodeSegmentAndOffset(node); + MemorySegment nodeSegment = nodeStorage.nodeSegment; + int offsetInNodeSegment = nodeStorage.nodeOffset; + + long oldValuePointer = SkipListUtils.getValuePointer(nodeSegment, offsetInNodeSegment); + long nextValuePointer = SkipListUtils.helpGetNextValuePointer(oldValuePointer, spaceAllocator); + + doWriteValue(valuePointer, value, stateMapVersion, node, nextValuePointer); + + // update value pointer in node after the new value has points the older value so that + // old value can be accessed concurrently + SkipListUtils.putValuePointer(nodeSegment, offsetInNodeSegment, valuePointer); + + return oldValuePointer; + } + + /** + * Removes the node physically, and free all space used by the key and value. + * + * @param node node to remove. + * @param prevNode previous node at the level 0. + * @param nextNode next node at the level 0. + */ + private void doPhysicalRemove(long node, long prevNode, long nextNode) { + // free space used by key and level index + long valuePointer = deleteNodeMeta(node, prevNode, nextNode); + // free space used by value + SkipListUtils.removeAllValues(valuePointer, spaceAllocator); + } + + /** + * Removes the node physically, and return the newest-version value pointer. + * Space used by key and value will be freed here, but the space of newest-version + * value will not be freed, and the caller should be responsible for the free + * of space. + * + * @param node node to remove. + * @param prevNode previous node at the level 0. + * @param nextNode next node at the level 0. + * @return newest-version value pointer. + */ + private long doPhysicalRemoveAndGetValue(long node, long prevNode, long nextNode) { + // free space used by key and level index + long valuePointer = deleteNodeMeta(node, prevNode, nextNode); + // free space used by values except for the newest-version + long nextValuePointer = SkipListUtils.helpGetNextValuePointer(valuePointer, spaceAllocator); + SkipListUtils.removeAllValues(nextValuePointer, spaceAllocator); + + return valuePointer; + } + + /** + * Physically delte the meta of the node, including the node level index, the node key, and reduce the total size of + * the skip list. + * + * @param node node to remove. + * @param prevNode previous node at the level 0. + * @param nextNode next node at the level 0. + * @return value pointer of the node. + */ + private long deleteNodeMeta(long node, long prevNode, long nextNode) { + // set next node of prevNode at level 0 to nextNode + helpSetNextNode(prevNode, nextNode, 0); + + // remove the level index for the node + SkipListUtils.removeLevelIndex(node, spaceAllocator, levelIndexHeader); + + // free space used by key + long valuePointer = SkipListUtils.helpGetValuePointer(node, spaceAllocator); + this.spaceAllocator.free(node); + + // reduce total size of the skip list + // note that we regard the node to be removed once its meta is deleted + totalSize--; + + return valuePointer; + } + + /** + * Return a random level for new node. + *

    + * The implementation refers to the {@code randomLevel} method of JDK7's ConcurrentSkipListMap. See + * https://github.com/openjdk-mirror/jdk7u-jdk/blob/master/src/share/classes/java/util/concurrent/ConcurrentSkipListMap.java#L899 + */ + private int getRandomIndexLevel() { + int x = randomSeed; + x ^= x << 13; + x ^= x >>> 17; + x ^= x << 5; + randomSeed = x; + // test highest and lowest bits + if ((x & 0x8001) != 0) { + return 0; + } + int level = 1; + int curMax = levelIndexHeader.getLevel(); + x >>>= 1; + while ((x & 1) != 0) { + ++level; + x >>>= 1; + // the level only be increased by step + if (level > curMax) { + break; + } + } + return level; + } + + /** + * Write the meta and data for the key to the given node. + * + * @param node the node for the key to write. + * @param level level of this node. + * @param keySegment memory segment storing the key. + * @param keyOffset offset of key in memory segment. + * @param keyLen length of the key. + * @param valuePointer pointer to value. + * @param nextNode next node on level 0. + */ + private void doWriteKey( + long node, + int level, + MemorySegment keySegment, + int keyOffset, + int keyLen, + long valuePointer, + long nextNode) { + Node nodeStorage = getNodeSegmentAndOffset(node); + MemorySegment segment = nodeStorage.nodeSegment; + int offsetInSegment = nodeStorage.nodeOffset; + + SkipListUtils.putLevelAndNodeStatus(segment, offsetInSegment, level, NodeStatus.PUT); + SkipListUtils.putKeyLen(segment, offsetInSegment, keyLen); + SkipListUtils.putValuePointer(segment, offsetInSegment, valuePointer); + SkipListUtils.putNextKeyPointer(segment, offsetInSegment, nextNode); + SkipListUtils.putKeyData(segment, offsetInSegment, keySegment, keyOffset, keyLen, level); + } + + /** + * Write the meta and data for the value to the space where the value pointer points. + * + * @param valuePointer pointer to the space where the meta and data is written. + * @param value data of the value. + * @param version version of this value. + * @param keyPointer pointer to the key. + * @param nextValuePointer pointer to the next value. + */ + private void doWriteValue( + long valuePointer, + byte[] value, + int version, + long keyPointer, + long nextValuePointer) { + Node node = getNodeSegmentAndOffset(valuePointer); + MemorySegment segment = node.nodeSegment; + int offsetInSegment = node.nodeOffset; + + SkipListUtils.putValueVersion(segment, offsetInSegment, version); + SkipListUtils.putKeyPointer(segment, offsetInSegment, keyPointer); + SkipListUtils.putNextValuePointer(segment, offsetInSegment, nextValuePointer); + SkipListUtils.putValueLen(segment, offsetInSegment, value == null ? 0 : value.length); + if (value != null) { + SkipListUtils.putValueData(segment, offsetInSegment, value); + } + } + + /** + * Find the first node with the given namespace at level 0. + * + * @param namespaceSegment memory segment storing the namespace. + * @param namespaceOffset offset of the namespace. + * @param namespaceLen length of the namespace. + * @return the first node with the given namespace. + * NIL_NODE will be returned if not exist. + */ + private long getFirstNodeWithNamespace(MemorySegment namespaceSegment, int namespaceOffset, int namespaceLen) { + int currentLevel = levelIndexHeader.getLevel(); + long prevNode = HEAD_NODE; + long currentNode = helpGetNextNode(prevNode, currentLevel); + + int c; + // find the predecessor node at level 0. + for ( ; ; ) { + if (currentNode != NIL_NODE) { + c = compareNamespaceAndNode(namespaceSegment, namespaceOffset, namespaceLen, currentNode); + if (c > 0) { + prevNode = currentNode; + currentNode = helpGetNextNode(prevNode, currentLevel); + continue; + } + } + + currentLevel--; + if (currentLevel < 0) { + break; + } + currentNode = helpGetNextNode(prevNode, currentLevel); + } + + // find the first node that has not been logically removed + while (currentNode != NIL_NODE) { + if (isNodeRemoved(currentNode)) { + currentNode = helpGetNextNode(currentNode, 0); + continue; + } + + c = compareNamespaceAndNode(namespaceSegment, namespaceOffset, namespaceLen, currentNode); + if (c == 0) { + return currentNode; + } + + if (c < 0) { + break; + } + } + + return NIL_NODE; + } + + /** + * Try to delete some nodes that has been logically removed. + */ + private void tryToDeleteNodesPhysically() { + if (highestRequiredSnapshotVersionPlusOne != 0) { + return; + } + + int threshold = (int) (totalSize * logicalRemovedKeysRatio); + int size = logicallyRemovedNodes.size(); + if (size > threshold) { + deleteLogicallyRemovedNodes(size - threshold); + } + } + + private void deleteLogicallyRemovedNodes(int maxNodes) { + int count = 0; + Iterator nodeIterator = logicallyRemovedNodes.iterator(); + while (count < maxNodes && nodeIterator.hasNext()) { + deleteNode(nodeIterator.next()); + nodeIterator.remove(); + count++; + } + } + + private void deleteNode(long node) { + long prevNode = SkipListUtils.findPredecessor(node, 1, levelIndexHeader, spaceAllocator); + long currentNode = helpGetNextNode(prevNode, 0); + while (currentNode != node) { + prevNode = currentNode; + currentNode = helpGetNextNode(prevNode, 0); + } + + long nextNode = helpGetNextNode(currentNode, 0); + doPhysicalRemove(currentNode, prevNode, nextNode); + } + + /** + * Release all resource used by the map. + */ + private void releaseAllResource() { + long node = levelIndexHeader.getNextNode(0); + while (node != NIL_NODE) { + long nextNode = helpGetNextNode(node, 0); + long valuePointer = SkipListUtils.helpGetValuePointer(node, spaceAllocator); + spaceAllocator.free(node); + SkipListUtils.removeAllValues(valuePointer, spaceAllocator); + node = nextNode; + } + totalSize = 0; + logicallyRemovedNodes.clear(); + } + + /** + * Returns the value pointer used by the snapshot of the given version. + * + * @param snapshotVersion version of snapshot. + * @return the value pointer of the version used by the given snapshot. NIL_VALUE_POINTER + * will be returned if there is no value for this snapshot. + */ + long getValueForSnapshot(long node, int snapshotVersion) { + long snapshotValuePointer = NIL_VALUE_POINTER; + ValueVersionIterator versionIterator = new ValueVersionIterator(node); + long valuePointer; + while (versionIterator.hasNext()) { + valuePointer = versionIterator.getValuePointer(); + int version = versionIterator.next(); + // the first value whose version is less than snapshotVersion + if (version < snapshotVersion) { + snapshotValuePointer = valuePointer; + break; + } + } + + return snapshotValuePointer; + } + + /** + * Returns the value pointer used by the snapshot of the given version, + * and useless version values will be pruned. + * + * @param snapshotVersion version of snapshot. + * @return the value pointer of the version used by the given snapshot. NIL_VALUE_POINTER + * will be returned if there is no value for this snapshot. + */ + long getAndPruneValueForSnapshot(long node, int snapshotVersion) { + // whether the node is being pruned by some snapshot + boolean isPruning = pruningValueNodes.add(node); + try { + long snapshotValuePointer = NIL_VALUE_POINTER; + long valuePointer; + ValueVersionIterator versionIterator = new ValueVersionIterator(node); + while (versionIterator.hasNext()) { + valuePointer = versionIterator.getValuePointer(); + int version = versionIterator.next(); + // find the first value whose version is less than snapshotVersion + if (snapshotValuePointer == NIL_VALUE_POINTER && version < snapshotVersion) { + snapshotValuePointer = valuePointer; + if (!isPruning) { + break; + } + } + + // if the version of the value is no more than highestFinishedSnapshotVersion, + // snapshots that is running and to be run will not use the values who are + // older than this version, so these values can be safely removed. + if (highestFinishedSnapshotVersion >= version) { + long nextValuePointer = SkipListUtils.helpGetNextValuePointer(valuePointer, spaceAllocator); + if (nextValuePointer != NIL_VALUE_POINTER) { + SkipListUtils.helpSetNextValuePointer(valuePointer, NIL_VALUE_POINTER, spaceAllocator); + SkipListUtils.removeAllValues(nextValuePointer, spaceAllocator); + } + break; + } + } + + return snapshotValuePointer; + } finally { + // only remove the node from the set when this snapshot has pruned values + if (isPruning) { + pruningValueNodes.remove(node); + } + } + } + + /** + * Update some statistics. + */ + private void updateStat() { + requestCount++; + } + + /** + * Find the node containing the given key. + * + * @param key the key. + * @param namespace the namespace. + * @return id of the node. NIL_NODE will be returned if key does no exist. + */ + private S getNodeInternal(K key, N namespace) { + MemorySegment keySegment = getKeySegment(key, namespace); + int keyLen = keySegment.size(); + + return getNode(keySegment, 0, keyLen); + } + + /** + * Get the {@link MemorySegment} wrapping up the serialized key bytes. + * + * @param key the key. + * @param namespace the namespace. + * @return the {@link MemorySegment} wrapping up the serialized key bytes. + */ + private MemorySegment getKeySegment(K key, N namespace) { + byte[] keyBytes = skipListKeySerializer.serialize(key, namespace); + return MemorySegmentFactory.wrap(keyBytes); + } + + // Help methods --------------------------------------------------------------- + + /** + * Whether the node has been logically removed. + */ + private boolean isNodeRemoved(long node) { + return SkipListUtils.isNodeRemoved(node, spaceAllocator); + } + + /** + * Set the next node of the given node at the given level. + */ + private void helpSetNextNode(long node, long nextNode, int level) { + SkipListUtils.helpSetNextNode(node, nextNode, level, levelIndexHeader, spaceAllocator); + } + + /** + * Return the next of the given node at the given level. + */ + long helpGetNextNode(long node, int level) { + return SkipListUtils.helpGetNextNode(node, level, this.levelIndexHeader, this.spaceAllocator); + } + + /** + * Returns the length of the value. + */ + int helpGetValueLen(long valuePointer) { + return SkipListUtils.helpGetValueLen(valuePointer, spaceAllocator); + } + + /** + * Set node status to the given new status, and return old status. + */ + private NodeStatus helpSetNodeStatus(long node, NodeStatus newStatus) { + Node nodeStorage = getNodeSegmentAndOffset(node); + MemorySegment segment = nodeStorage.nodeSegment; + int offsetInSegment = nodeStorage.nodeOffset; + NodeStatus oldStatus = SkipListUtils.getNodeStatus(segment, offsetInSegment); + if (oldStatus != newStatus) { + int level = SkipListUtils.getLevel(segment, offsetInSegment); + SkipListUtils.putLevelAndNodeStatus(segment, offsetInSegment, level, newStatus); + } + + return oldStatus; + } + + /** + * Return the state of the node. null will be returned if the node is removed. + */ + private S getNodeStateHelper(long node) { + Node nodeStorage = getNodeSegmentAndOffset(node); + MemorySegment segment = nodeStorage.nodeSegment; + int offsetInSegment = nodeStorage.nodeOffset; + long valuePointer = SkipListUtils.getValuePointer(segment, offsetInSegment); + + return helpGetState(valuePointer); + } + + /** + * Returns the byte arrays of serialized key and namespace. + * + * @param node the node. + * @return a tuple of byte arrays of serialized key and namespace + */ + Tuple2 helpGetBytesForKeyAndNamespace(long node) { + Node nodeStorage = getNodeSegmentAndOffset(node); + MemorySegment segment = nodeStorage.nodeSegment; + int offsetInSegment = nodeStorage.nodeOffset; + + int level = SkipListUtils.getLevel(segment, offsetInSegment); + int keyDataOffset = offsetInSegment + SkipListUtils.getKeyDataOffset(level); + + return skipListKeySerializer.getSerializedKeyAndNamespace(segment, keyDataOffset); + } + + /** + * Returns the byte array of serialized state. + * + * @param valuePointer pointer to value. + * @return byte array of serialized value. + */ + byte[] helpGetBytesForState(long valuePointer) { + Node node = getNodeSegmentAndOffset(valuePointer); + MemorySegment segment = node.nodeSegment; + int offsetInSegment = node.nodeOffset; + + int valueLen = SkipListUtils.getValueLen(segment, offsetInSegment); + MemorySegment valueSegment = MemorySegmentFactory.allocateUnpooledSegment(valueLen); + segment.copyTo(offsetInSegment + SkipListUtils.getValueMetaLen(), valueSegment, 0 , valueLen); + + return valueSegment.getArray(); + } + + /** + * Returns the key of the node. + */ + private K helpGetKey(long node) { + Node nodeStorage = getNodeSegmentAndOffset(node); + MemorySegment segment = nodeStorage.nodeSegment; + int offsetInSegment = nodeStorage.nodeOffset; + + int level = SkipListUtils.getLevel(segment, offsetInSegment); + int keyDataLen = SkipListUtils.getKeyLen(segment, offsetInSegment); + int keyDataOffset = offsetInSegment + SkipListUtils.getKeyDataOffset(level); + + return skipListKeySerializer.deserializeKey(segment, keyDataOffset, keyDataLen); + } + + /** + * Return the state pointed by the given pointer. The value will be de-serialized + * with the given serializer. + */ + S helpGetState(long valuePointer, SkipListValueSerializer serializer) { + if (valuePointer == NIL_VALUE_POINTER) { + return null; + } + + Node node = getNodeSegmentAndOffset(valuePointer); + MemorySegment segment = node.nodeSegment; + int offsetInSegment = node.nodeOffset; + + int valueLen = SkipListUtils.getValueLen(segment, offsetInSegment); + if (valueLen == 0) { + // it is a removed key + return null; + } + + return serializer.deserializeState(segment, + offsetInSegment + SkipListUtils.getValueMetaLen(), valueLen); + } + + /** + * Return the state pointed by the given pointer. The serializer used is the + * {@link #skipListValueSerializer}. Because serializer is not thread safe, so + * this method should only be called in the state map synchronously. + */ + S helpGetState(long valuePointer) { + return helpGetState(valuePointer, skipListValueSerializer); + } + + /** + * Returns the state entry of the node. + */ + private StateEntry helpGetStateEntry(long node) { + Node nodeStorage = getNodeSegmentAndOffset(node); + MemorySegment segment = nodeStorage.nodeSegment; + int offsetInSegment = nodeStorage.nodeOffset; + + int level = SkipListUtils.getLevel(segment, offsetInSegment); + int keyDataLen = SkipListUtils.getKeyLen(segment, offsetInSegment); + int keyDataOffset = offsetInSegment + SkipListUtils.getKeyDataOffset(level); + + K key = skipListKeySerializer.deserializeKey(segment, keyDataOffset, keyDataLen); + N namespace = skipListKeySerializer.deserializeNamespace(segment, keyDataOffset, keyDataLen); + long valuePointer = SkipListUtils.getValuePointer(segment, offsetInSegment); + S state = helpGetState(valuePointer); + + return new StateEntry.SimpleStateEntry<>(key, namespace, state); + } + + // ---------------------------------------------------------------------------------- + + @Override + public Stream getKeys(N namespace) { + updateStat(); + byte[] namespaceBytes = skipListKeySerializer.serializeNamespace(namespace); + MemorySegment namespaceSegment = MemorySegmentFactory.wrap(namespaceBytes); + Iterator nodeIter = new NamespaceNodeIterator(namespaceSegment, 0, namespaceBytes.length); + return StreamSupport.stream(Spliterators.spliteratorUnknownSize(nodeIter, 0), false) + .map(this::helpGetKey); + } + + @SuppressWarnings("unchecked") + @Override + public int sizeOfNamespace(Object namespace) { + updateStat(); + byte[] namespaceBytes = skipListKeySerializer.serializeNamespace((N) namespace); + MemorySegment namespaceSegment = MemorySegmentFactory.wrap(namespaceBytes); + Iterator nodeIter = new NamespaceNodeIterator(namespaceSegment, 0, namespaceBytes.length); + int size = 0; + while (nodeIter.hasNext()) { + nodeIter.next(); + size++; + } + + return size; + } + + @Nonnull + @Override + public Iterator> iterator() { + updateStat(); + final Iterator nodeIter = new NodeIterator(); + return new Iterator>() { + @Override + public boolean hasNext() { + return nodeIter.hasNext(); + } + + @Override + public StateEntry next() { + return helpGetStateEntry(nodeIter.next()); + } + }; + } + + @Override + public InternalKvState.StateIncrementalVisitor getStateIncrementalVisitor(int recommendedMaxNumberOfReturnedRecords) { + return new StateIncrementalVisitor(recommendedMaxNumberOfReturnedRecords); + } + + @Nonnull + @Override + public CopyOnWriteSkipListStateMapSnapshot stateSnapshot() { + tryToDeleteNodesPhysically(); + + ResourceGuard.Lease lease; + try { + lease = resourceGuard.acquireResource(); + } catch (Exception e) { + throw new RuntimeException("Acquire resource failed, and can't make snapshot of state map", e); + } + + synchronized (snapshotVersions) { + // increase the map version for copy-on-write and register the snapshot + if (++stateMapVersion < 0) { + // this is just a safety net against overflows, but should never happen in practice (i.e., only after 2^31 snapshots) + throw new IllegalStateException("Version count overflow. Enforcing restart."); + } + + highestRequiredSnapshotVersionPlusOne = stateMapVersion; + snapshotVersions.add(highestRequiredSnapshotVersionPlusOne); + } + + return new CopyOnWriteSkipListStateMapSnapshot<>(this, lease); + } + + @SuppressWarnings("unchecked") + @Override + public void releaseSnapshot(StateMapSnapshot> snapshotToRelease) { + CopyOnWriteSkipListStateMapSnapshot snapshot = (CopyOnWriteSkipListStateMapSnapshot) snapshotToRelease; + int snapshotVersion = snapshot.getSnapshotVersion(); + + Preconditions.checkArgument(snapshot.isOwner(this), + "Cannot release snapshot which is owned by a different state map."); + + synchronized (snapshotVersions) { + Preconditions.checkState(snapshotVersions.remove(snapshotVersion), "Attempt to release unknown snapshot version"); + highestRequiredSnapshotVersionPlusOne = snapshotVersions.isEmpty() ? 0 : snapshotVersions.last(); + highestFinishedSnapshotVersion = snapshotVersions.isEmpty() ? stateMapVersion : snapshotVersions.first() - 1; + } + } + + LevelIndexHeader getLevelIndexHeader() { + return levelIndexHeader; + } + + int getStateMapVersion() { + return stateMapVersion; + } + + @VisibleForTesting + int getHighestRequiredSnapshotVersionPlusOne() { + return highestRequiredSnapshotVersionPlusOne; + } + + @VisibleForTesting + int getHighestFinishedSnapshotVersion() { + return highestFinishedSnapshotVersion; + } + + @VisibleForTesting + Set getSnapshotVersions() { + return snapshotVersions; + } + + @VisibleForTesting + Set getLogicallyRemovedNodes() { + return logicallyRemovedNodes; + } + + @VisibleForTesting + Set getPruningValueNodes() { + return pruningValueNodes; + } + + @VisibleForTesting + ResourceGuard getResourceGuard() { + return resourceGuard; + } + + boolean isClosed() { + return closed.get(); + } + + @Override + public void close() { + if (!closed.compareAndSet(false, true)) { + LOG.warn("State map has been closed"); + return; + } + + // wait for all running snapshots finished + resourceGuard.close(); + + releaseAllResource(); + } + + /** + * Iterates all nodes in the skip list. + */ + class NodeIterator implements Iterator { + + private long nextNode; + + NodeIterator() { + this.nextNode = getNextNode(HEAD_NODE); + } + + private long getNextNode(long node) { + long n = helpGetNextNode(node, 0); + while (n != NIL_NODE && isNodeRemoved(n)) { + n = helpGetNextNode(n, 0); + } + + return n; + } + + @Override + public boolean hasNext() { + return nextNode != NIL_NODE; + } + + @Override + public Long next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + long node = nextNode; + nextNode = getNextNode(node); + + return node; + } + } + + private Node getNodeSegmentAndOffset(long node) { + Chunk nodeChunk = spaceAllocator.getChunkById(SpaceUtils.getChunkIdByAddress(node)); + int offsetInNodeChunk = SpaceUtils.getChunkOffsetByAddress(node); + MemorySegment nodeSegment = nodeChunk.getMemorySegment(offsetInNodeChunk); + int offsetInNodeSegment = nodeChunk.getOffsetInSegment(offsetInNodeChunk); + return new Node(nodeSegment, offsetInNodeSegment); + } + + /** + * Iterates nodes with the given namespace. + */ + class NamespaceNodeIterator implements Iterator { + + private final MemorySegment namespaceSegment; + private final int namespaceOffset; + private final int namespaceLen; + private long nextNode; + + NamespaceNodeIterator(MemorySegment namespaceSegment, int namespaceOffset, int namespaceLen) { + this.namespaceSegment = namespaceSegment; + this.namespaceOffset = namespaceOffset; + this.namespaceLen = namespaceLen; + this.nextNode = getFirstNodeWithNamespace(namespaceSegment, namespaceOffset, namespaceLen); + } + + private long getNextNode(long node) { + long n = helpGetNextNode(node, 0); + while (n != NIL_NODE && isNodeRemoved(n)) { + n = helpGetNextNode(n, 0); + } + + if (n != NIL_NODE && + compareNamespaceAndNode(namespaceSegment, namespaceOffset, namespaceLen, n) == 0) { + return n; + } + + return NIL_NODE; + } + + @Override + public boolean hasNext() { + return nextNode != NIL_NODE; + } + + @Override + public Long next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + long node = nextNode; + nextNode = getNextNode(node); + + return node; + } + } + + class StateIncrementalVisitor implements InternalKvState.StateIncrementalVisitor { + + private final int recommendedMaxNumberOfReturnedRecords; + private MemorySegment nextKeySegment; + private int nextKeyOffset; + private final Collection> entryToReturn = new ArrayList<>(5); + + StateIncrementalVisitor(int recommendedMaxNumberOfReturnedRecords) { + this.recommendedMaxNumberOfReturnedRecords = recommendedMaxNumberOfReturnedRecords; + init(); + } + + private void init() { + long node = getNextNode(HEAD_NODE); + if (node != NIL_NODE) { + setKeySegment(node); + } + } + + private long findNextNode(MemorySegment segment, int offset) { + long node = findPredecessor(segment, offset, 0); + return getNextNode(node); + } + + private long getNextNode(long node) { + long n = helpGetNextNode(node, 0); + while (n != NIL_NODE && isNodeRemoved(n)) { + n = helpGetNextNode(n, 0); + } + + return n; + } + + private void updateNextKeySegment(long node) { + if (node != NIL_NODE) { + node = getNextNode(node); + if (node != NIL_NODE) { + setKeySegment(node); + return; + } + } + nextKeySegment = null; + } + + private void setKeySegment(long node) { + Node nodeStorage = getNodeSegmentAndOffset(node); + MemorySegment segment = nodeStorage.nodeSegment; + int offsetInSegment = nodeStorage.nodeOffset; + + int level = SkipListUtils.getLevel(segment, offsetInSegment); + int keyLen = SkipListUtils.getKeyLen(segment, offsetInSegment); + int keyDataOffset = offsetInSegment + SkipListUtils.getKeyDataOffset(level); + + MemorySegment nextKeySegment = MemorySegmentFactory.allocateUnpooledSegment(keyLen); + segment.copyTo(keyDataOffset, nextKeySegment, 0, keyLen); + this.nextKeySegment = nextKeySegment; + nextKeyOffset = 0; + } + + @Override + public boolean hasNext() { + // visitor may be held by the external for a long time, and the map + // can be closed between two nextEntries(), so check the status of map + return !isClosed() && nextKeySegment != null && + findNextNode(nextKeySegment, nextKeyOffset) != NIL_NODE; + } + + @Override + public Collection> nextEntries() { + if (nextKeySegment == null) { + return Collections.emptyList(); + } + + long node = findNextNode(nextKeySegment, nextKeyOffset); + if (node == NIL_NODE) { + nextKeySegment = null; + return Collections.emptyList(); + } + + entryToReturn.clear(); + entryToReturn.add(helpGetStateEntry(node)); + int n = 1; + while (n < recommendedMaxNumberOfReturnedRecords) { + node = getNextNode(node); + if (node == NIL_NODE) { + break; + } + entryToReturn.add(helpGetStateEntry(node)); + n++; + } + + updateNextKeySegment(node); + + return entryToReturn; + } + + @Override + public void remove(StateEntry stateEntry) { + CopyOnWriteSkipListStateMap.this.remove(stateEntry.getKey(), stateEntry.getNamespace()); + } + + @Override + public void update(StateEntry stateEntry, S newValue) { + CopyOnWriteSkipListStateMap.this.put(stateEntry.getKey(), stateEntry.getNamespace(), newValue); + } + } + + /** + * Iterate versions of the given node. + */ + private class ValueVersionIterator implements Iterator { + private long valuePointer; + + ValueVersionIterator(long node) { + valuePointer = SkipListUtils.helpGetValuePointer(node, spaceAllocator); + } + + @Override + public boolean hasNext() { + return valuePointer != NIL_VALUE_POINTER; + } + + @Override + public Integer next() { + int version = SkipListUtils.helpGetValueVersion(valuePointer, spaceAllocator); + valuePointer = SkipListUtils.helpGetNextValuePointer(valuePointer, spaceAllocator); + return version; + } + + long getValuePointer() { + return valuePointer; + } + } + + /** + * Encapsulation of skip list iterate and process result. + */ + private class SkipListIterateAndProcessResult { + long prevNode; + long currentNode; + boolean isKeyFound; + S state; + SkipListIterateAndProcessResult(long prevNode, long currentNode, boolean isKeyFound, S state) { + this.prevNode = prevNode; + this.currentNode = currentNode; + this.isKeyFound = isKeyFound; + this.state = state; + } + } + + /** + * Encapsulation of skip list node pointers. + */ + private class SkipListNodePointers { + long prevNode; + long currentNode; + long nextNode; + SkipListNodePointers(long prevNode, long currentNode, long nextNode) { + this.prevNode = prevNode; + this.currentNode = currentNode; + this.nextNode = nextNode; + } + } + + /** + * Encapsulation of the storage of the node. + */ + private class Node { + MemorySegment nodeSegment; + int nodeOffset; + Node(MemorySegment nodeSegment, int nodeOffset) { + this.nodeSegment = nodeSegment; + this.nodeOffset = nodeOffset; + } + } + +} diff --git a/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteSkipListStateMapSnapshot.java b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteSkipListStateMapSnapshot.java new file mode 100644 index 000000000000..994c234b9d2d --- /dev/null +++ b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteSkipListStateMapSnapshot.java @@ -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. + */ + +package org.apache.flink.runtime.state.heap; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.runtime.state.StateSnapshotTransformer; +import org.apache.flink.util.ResourceGuard; + +import javax.annotation.Nonnegative; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.Iterator; +import java.util.NoSuchElementException; + +import static org.apache.flink.runtime.state.heap.SkipListUtils.HEAD_NODE; +import static org.apache.flink.runtime.state.heap.SkipListUtils.NIL_NODE; +import static org.apache.flink.runtime.state.heap.SkipListUtils.NIL_VALUE_POINTER; + +/** + * This class represents the snapshot of a {@link CopyOnWriteSkipListStateMap}. + * + * @param type of key + * @param type of namespace + * @param type of state + */ +public class CopyOnWriteSkipListStateMapSnapshot + extends StateMapSnapshot> { + + /** + * Version of the {@link CopyOnWriteSkipListStateMap} when this snapshot was created. This can be used to release the snapshot. + */ + private final int snapshotVersion; + + /** The number of (non-null) entries in snapshotData. */ + @Nonnegative + private final int numberOfEntriesInSnapshotData; + + /** + * This lease protects the state map resources. + */ + private final ResourceGuard.Lease lease; + + /** + * Creates a new {@link CopyOnWriteSkipListStateMap}. + * + * @param owningStateMap the {@link CopyOnWriteSkipListStateMap} for which this object represents a snapshot. + * @param lease the lease protects the state map resources. + */ + CopyOnWriteSkipListStateMapSnapshot( + CopyOnWriteSkipListStateMap owningStateMap, + ResourceGuard.Lease lease) { + super(owningStateMap); + + this.snapshotVersion = owningStateMap.getStateMapVersion(); + this.numberOfEntriesInSnapshotData = owningStateMap.size(); + this.lease = lease; + } + + /** + * Returns the internal version of the when this snapshot was created. + */ + int getSnapshotVersion() { + return snapshotVersion; + } + + @Override + public void release() { + owningStateMap.releaseSnapshot(this); + lease.close(); + } + + @Override + public void writeState( + TypeSerializer keySerializer, + TypeSerializer namespaceSerializer, + TypeSerializer stateSerializer, + @Nonnull DataOutputView dov, + @Nullable StateSnapshotTransformer stateSnapshotTransformer) throws IOException { + if (stateSnapshotTransformer == null) { + writeStateWithNoTransform(dov); + } else { + writeStateWithTransform(stateSerializer, dov, stateSnapshotTransformer); + } + } + + private void writeStateWithNoTransform(@Nonnull DataOutputView dov) throws IOException { + dov.writeInt(numberOfEntriesInSnapshotData); + SnapshotNodeIterator nodeIterator = new SnapshotNodeIterator(true); + while (nodeIterator.hasNext()) { + Tuple2 tuple = nodeIterator.next(); + writeKeyAndNamespace(tuple.f0, dov); + writeValue(tuple.f1, dov); + } + } + + private void writeStateWithTransform( + TypeSerializer stateSerializer, + @Nonnull DataOutputView dov, + @Nonnull StateSnapshotTransformer stateSnapshotTransformer) throws IOException { + SkipListValueSerializer skipListValueSerializer = + new SkipListValueSerializer<>(stateSerializer); + + // 1. iterates nodes to get size after transform + SnapshotNodeIterator transformNodeIterator = new SnapshotNodeIterator(true); + int size = 0; + while (transformNodeIterator.hasNext()) { + Tuple2 tuple = transformNodeIterator.next(); + S oldState = owningStateMap.helpGetState(tuple.f1, skipListValueSerializer); + S newState = stateSnapshotTransformer.filterOrTransform(oldState); + if (newState != null) { + size++; + } + } + + dov.writeInt(size); + + // 2. iterates nodes again to write them to output, and there is no need to prune + SnapshotNodeIterator writeNodeIterator = new SnapshotNodeIterator(false); + while (writeNodeIterator.hasNext()) { + Tuple2 tuple = writeNodeIterator.next(); + S oldState = owningStateMap.helpGetState(tuple.f1, skipListValueSerializer); + S newState = stateSnapshotTransformer.filterOrTransform(oldState); + if (newState != null) { + writeKeyAndNamespace(tuple.f0, dov); + stateSerializer.serialize(newState, dov); + } + } + } + + /** + * Write key and namespace from bytes. + */ + private void writeKeyAndNamespace(long nodeId, DataOutputView outputView) throws IOException { + // tuple of byte arrays for key and namespace + Tuple2 tuple = owningStateMap.helpGetBytesForKeyAndNamespace(nodeId); + // write namespace first + outputView.write(tuple.f1); + outputView.write(tuple.f0); + } + + /** + * Write value from bytes. + */ + private void writeValue(long valuePointer, DataOutputView outputView) throws IOException { + outputView.write(owningStateMap.helpGetBytesForState(valuePointer)); + } + + /** + * Iterates over all nodes used by this snapshot. The iterator will return + * a tuple, and f0 is the node and f1 is the value pointer. + */ + class SnapshotNodeIterator implements Iterator> { + + /** + * Whether to prune values during iteration. + */ + private boolean isPrune; + private long nextNode; + private long nextValuePointer; + + SnapshotNodeIterator(boolean isPrune) { + this.isPrune = isPrune; + this.nextNode = HEAD_NODE; + advance(); + } + + private void advance() { + if (nextNode == NIL_NODE) { + return; + } + + long node = owningStateMap.helpGetNextNode(nextNode, 0); + long valuePointer = NIL_VALUE_POINTER; + while (node != NIL_NODE) { + valuePointer = isPrune ? + owningStateMap.getAndPruneValueForSnapshot(node, snapshotVersion) : + owningStateMap.getValueForSnapshot(node, snapshotVersion); + int valueLen = valuePointer == NIL_VALUE_POINTER ? 0 : + owningStateMap.helpGetValueLen(valuePointer); + // for a logically removed node, it's value length will be 0 + if (valueLen != 0) { + break; + } + node = owningStateMap.helpGetNextNode(node, 0); + } + + nextNode = node; + nextValuePointer = valuePointer; + } + + @Override + public boolean hasNext() { + return nextNode != NIL_NODE; + } + + @Override + public Tuple2 next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + long node = nextNode; + long valuePointer = nextValuePointer; + advance(); + + return Tuple2.of(node, valuePointer); + } + } +} diff --git a/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/LevelIndexHeader.java b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/LevelIndexHeader.java new file mode 100644 index 000000000000..35de64508ac5 --- /dev/null +++ b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/LevelIndexHeader.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.state.heap; + +/** + * Head level index for skip list. + */ +public interface LevelIndexHeader { + + /** + * Returns the top level of skip list. + * + * @return the top level of skip list. + */ + int getLevel(); + + /** + * Updates the top level of skip list to the given level. + * + * @param level the level which top level of skip list updates to. + */ + void updateLevel(int level); + + /** + * Returns the next node in the given level. + * + * @param level the level whose next node is returned. + * @return id of the next node. + */ + long getNextNode(int level); + + /** + * Updates the next node in the given level to the specified node id. + * + * @param level the level whose next node is updated. + * @param newNodeId the id of the next node. + */ + void updateNextNode(int level, long newNodeId); +} diff --git a/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/NodeStatus.java b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/NodeStatus.java new file mode 100644 index 000000000000..2c315020f8b6 --- /dev/null +++ b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/NodeStatus.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.state.heap; + +/** + * Status of the node. + */ +public enum NodeStatus { + + PUT((byte) 0), REMOVE((byte) 1); + + private final byte value; + + NodeStatus(byte value) { + this.value = value; + } + + public byte getValue() { + return value; + } + + public static NodeStatus valueOf(byte value) { + switch (value) { + case 0: + return PUT; + case 1: + return REMOVE; + default: + throw new IllegalArgumentException("Unknown type: " + value); + } + } +} diff --git a/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/OnHeapLevelIndexHeader.java b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/OnHeapLevelIndexHeader.java new file mode 100644 index 000000000000..108d50c8de7c --- /dev/null +++ b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/OnHeapLevelIndexHeader.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.runtime.state.heap; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.util.Preconditions; + +import static org.apache.flink.runtime.state.heap.SkipListUtils.DEFAULT_LEVEL; +import static org.apache.flink.runtime.state.heap.SkipListUtils.MAX_LEVEL; +import static org.apache.flink.runtime.state.heap.SkipListUtils.NIL_NODE; + +/** + * Implementation of {@link LevelIndexHeader} which stores index on heap. + */ +public class OnHeapLevelIndexHeader implements LevelIndexHeader { + + /** + * The level index array where each position stores the next node id of the level. + */ + private volatile long[] levelIndex; + + /** + * The topmost level currently. + */ + private volatile int topLevel; + + /** + * Next node at level 0. + */ + private volatile long nextNode; + + OnHeapLevelIndexHeader() { + this(DEFAULT_LEVEL); + } + + private OnHeapLevelIndexHeader(int maxLevel) { + Preconditions.checkArgument(maxLevel >= 1 && maxLevel <= MAX_LEVEL, + "maxLevel(" + maxLevel + ") must be non-negative and no more than " + MAX_LEVEL); + this.topLevel = 1; + this.nextNode = NIL_NODE; + this.levelIndex = new long[maxLevel]; + initLevelIndex(levelIndex); + } + + private void initLevelIndex(long[] levelIndex) { + for (int i = 0; i < levelIndex.length; i++) { + levelIndex[i] = NIL_NODE; + } + } + + @Override + public int getLevel() { + return topLevel; + } + + @Override + public void updateLevel(int level) { + Preconditions.checkArgument(level >= 0 && level <= MAX_LEVEL, + "level(" + level + ") must be non-negative and no more than " + MAX_LEVEL); + Preconditions.checkArgument(level <= this.topLevel + 1, + "top level " + topLevel + " must be updated level by level, but new level is " + level); + + if (levelIndex.length < level) { + long[] newLevelIndex = new long[this.levelIndex.length * 2]; + initLevelIndex(newLevelIndex); + System.arraycopy(this.levelIndex, 0, newLevelIndex, 0, this.levelIndex.length); + this.levelIndex = newLevelIndex; + } + + if (topLevel < level) { + topLevel = level; + } + } + + @Override + public long getNextNode(int level) { + Preconditions.checkArgument(level >= 0 && level <= topLevel, + "invalid level " + level + " current top level is " + topLevel); + + if (level == 0) { + return nextNode; + } + return levelIndex[level - 1]; + } + + @Override + public void updateNextNode(int level, long node) { + Preconditions.checkArgument(level >= 0 && level <= topLevel, + "invalid level " + level + " current top level is " + topLevel); + + if (level == 0) { + nextNode = node; + } else { + levelIndex[level - 1] = node; + } + } + + @VisibleForTesting + long[] getLevelIndex() { + return levelIndex; + } +} diff --git a/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/SkipListKeyComparator.java b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/SkipListKeyComparator.java new file mode 100644 index 000000000000..110638defdc2 --- /dev/null +++ b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/SkipListKeyComparator.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.state.heap; + +import org.apache.flink.core.memory.MemorySegment; + +/** + * Comparator used for skip list key. + */ +class SkipListKeyComparator { + + /** + * Compares for order. Returns a negative integer, zero, or a positive integer + * as the first node is less than, equal to, or greater than the second. + * + * @param left left skip list key's ByteBuffer + * @param leftOffset left skip list key's ByteBuffer's offset + * @param right right skip list key's ByteBuffer + * @param rightOffset right skip list key's ByteBuffer's offset + * @return An integer result of the comparison. + */ + static int compareTo(MemorySegment left, int leftOffset, MemorySegment right, int rightOffset) { + // compare namespace + int leftNamespaceLen = left.getInt(leftOffset); + int rightNamespaceLen = right.getInt(rightOffset); + + int c = left.compare(right, leftOffset + Integer.BYTES, rightOffset + Integer.BYTES, + leftNamespaceLen, rightNamespaceLen); + + if (c != 0) { + return c; + } + + // compare key + int leftKeyOffset = leftOffset + Integer.BYTES + leftNamespaceLen; + int rightKeyOffset = rightOffset + Integer.BYTES + rightNamespaceLen; + int leftKeyLen = left.getInt(leftKeyOffset); + int rightKeyLen = right.getInt(rightKeyOffset); + + return left.compare(right, leftKeyOffset + Integer.BYTES, rightKeyOffset + Integer.BYTES, + leftKeyLen, rightKeyLen); + } + + /** + * Compares the namespace in the memory segment with the namespace in the node . + * Returns a negative integer, zero, or a positive integer as the first node is + * less than, equal to, or greater than the second. + * + * @param namespaceSegment memory segment to store the namespace. + * @param namespaceOffset offset of namespace in the memory segment. + * @param namespaceLen length of namespace. + * @param nodeSegment memory segment to store the node key. + * @param nodeKeyOffset offset of node key in the memory segment. + * @return An integer result of the comparison. + */ + static int compareNamespaceAndNode( + MemorySegment namespaceSegment, int namespaceOffset, int namespaceLen, + MemorySegment nodeSegment, int nodeKeyOffset) { + + int nodeNamespaceLen = nodeSegment.getInt(nodeKeyOffset); + return namespaceSegment.compare(nodeSegment, namespaceOffset, nodeKeyOffset + Integer.BYTES, + namespaceLen, nodeNamespaceLen); + } +} diff --git a/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/SkipListKeySerializer.java b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/SkipListKeySerializer.java new file mode 100644 index 000000000000..56a09b6389f3 --- /dev/null +++ b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/SkipListKeySerializer.java @@ -0,0 +1,165 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.state.heap; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.core.memory.MemorySegmentFactory; +import org.apache.flink.core.memory.MemorySegmentInputStreamWithPos; + +import java.io.IOException; + +/** + * Serializer/deserializer used for conversion between key/namespace and skip list key. + * It is not thread safe. + * + * @param The type of the key. + * @param The type of the namespace. + */ +class SkipListKeySerializer { + + private final TypeSerializer keySerializer; + private final TypeSerializer namespaceSerializer; + private final ByteArrayOutputStreamWithPos outputStream; + private final DataOutputViewStreamWrapper outputView; + + SkipListKeySerializer( + TypeSerializer keySerializer, + TypeSerializer namespaceSerializer) { + this.keySerializer = keySerializer; + this.namespaceSerializer = namespaceSerializer; + this.outputStream = new ByteArrayOutputStreamWithPos(); + this.outputView = new DataOutputViewStreamWrapper(outputStream); + } + + /** + * Serialize the key and namespace to bytes. The format is + * - int: length of serialized namespace + * - byte[]: serialized namespace + * - int: length of serialized key + * - byte[]: serialized key + */ + byte[] serialize(K key, N namespace) { + outputStream.reset(); + try { + // serialize namespace + outputStream.setPosition(Integer.BYTES); + namespaceSerializer.serialize(namespace, outputView); + } catch (IOException e) { + throw new RuntimeException("Failed to serialize namespace", e); + } + + int keyStartPos = outputStream.getPosition(); + try { + // serialize key + outputStream.setPosition(keyStartPos + Integer.BYTES); + keySerializer.serialize(key, outputView); + } catch (IOException e) { + throw new RuntimeException("Failed to serialize key", e); + } + + byte[] result = outputStream.toByteArray(); + // set length of namespace and key + int namespaceLen = keyStartPos - Integer.BYTES; + int keyLen = result.length - keyStartPos - Integer.BYTES; + MemorySegment segment = MemorySegmentFactory.wrap(result); + segment.putInt(0, namespaceLen); + segment.putInt(keyStartPos, keyLen); + + return result; + } + + /** + * Deserialize the namespace from the byte buffer which stores skip list key. + * + * @param memorySegment the memory segment which stores the skip list key. + * @param offset the start position of the skip list key in the byte buffer. + * @param len length of the skip list key. + */ + N deserializeNamespace(MemorySegment memorySegment, int offset, int len) { + MemorySegmentInputStreamWithPos inputStream = + new MemorySegmentInputStreamWithPos(memorySegment, offset, len); + DataInputViewStreamWrapper inputView = new DataInputViewStreamWrapper(inputStream); + inputStream.setPosition(offset + Integer.BYTES); + try { + return namespaceSerializer.deserialize(inputView); + } catch (IOException e) { + throw new RuntimeException("deserialize namespace failed", e); + } + } + + /** + * Deserialize the partition key from the byte buffer which stores skip list key. + * + * @param memorySegment the memory segment which stores the skip list key. + * @param offset the start position of the skip list key in the byte buffer. + * @param len length of the skip list key. + */ + K deserializeKey(MemorySegment memorySegment, int offset, int len) { + MemorySegmentInputStreamWithPos inputStream = + new MemorySegmentInputStreamWithPos(memorySegment, offset, len); + DataInputViewStreamWrapper inputView = new DataInputViewStreamWrapper(inputStream); + int namespaceLen = memorySegment.getInt(offset); + inputStream.setPosition(offset + Integer.BYTES + namespaceLen + Integer.BYTES); + try { + return keySerializer.deserialize(inputView); + } catch (IOException e) { + throw new RuntimeException("deserialize key failed", e); + } + } + + /** + * Gets serialized key and namespace from the byte buffer. + * + * @param memorySegment the memory segment which stores the skip list key. + * @param offset the start position of the skip list key in the byte buffer. + * @return tuple of serialized key and namespace. + */ + Tuple2 getSerializedKeyAndNamespace(MemorySegment memorySegment, int offset) { + // read namespace + int namespaceLen = memorySegment.getInt(offset); + MemorySegment namespaceSegment = MemorySegmentFactory.allocateUnpooledSegment(namespaceLen); + memorySegment.copyTo(offset + Integer.BYTES, namespaceSegment, 0, namespaceLen); + + // read key + int keyOffset = offset + Integer.BYTES + namespaceLen; + int keyLen = memorySegment.getInt(keyOffset); + MemorySegment keySegment = MemorySegmentFactory.allocateUnpooledSegment(keyLen); + memorySegment.copyTo(keyOffset + Integer.BYTES, keySegment, 0, keyLen); + + return Tuple2.of(keySegment.getArray(), namespaceSegment.getArray()); + } + + /** + * Serialize the namespace to bytes. + */ + byte[] serializeNamespace(N namespace) { + outputStream.reset(); + try { + namespaceSerializer.serialize(namespace, outputView); + } catch (IOException e) { + throw new RuntimeException("serialize namespace failed", e); + } + return outputStream.toByteArray(); + } +} diff --git a/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/SkipListUtils.java b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/SkipListUtils.java new file mode 100644 index 000000000000..150076ffb2e5 --- /dev/null +++ b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/SkipListUtils.java @@ -0,0 +1,784 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.state.heap; + +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.core.memory.MemorySegmentFactory; +import org.apache.flink.runtime.state.heap.space.Allocator; +import org.apache.flink.runtime.state.heap.space.Chunk; +import org.apache.flink.runtime.state.heap.space.SpaceUtils; +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nonnull; + +/** + * Utilities for skip list. + */ +@SuppressWarnings("WeakerAccess") +public class SkipListUtils { + static final long NIL_NODE = -1; + static final long HEAD_NODE = -2; + static final long NIL_VALUE_POINTER = -1; + static final int MAX_LEVEL = 255; + static final int DEFAULT_LEVEL = 32; + static final int BYTE_MASK = 0xFF; + + /** + * Key space schema. + * - key meta + * -- int: level & status + * -- byte 0: level of node in skip list + * -- byte 1: node status + * -- byte 2: preserve + * -- byte 3: preserve + * -- int: length of key + * -- long: pointer to the newest value + * -- long: pointer to next node on level 0 + * -- long[]: array of pointers to next node on different levels excluding level 0 + * -- long[]: array of pointers to previous node on different levels excluding level 0 + * - byte[]: data of key + */ + static final int KEY_META_OFFSET = 0; + static final int KEY_LEN_OFFSET = KEY_META_OFFSET + Integer.BYTES; + static final int VALUE_POINTER_OFFSET = KEY_LEN_OFFSET + Integer.BYTES; + static final int NEXT_KEY_POINTER_OFFSET = VALUE_POINTER_OFFSET + Long.BYTES; + static final int LEVEL_INDEX_OFFSET = NEXT_KEY_POINTER_OFFSET + Long.BYTES; + + + /** + * Pre-compute the offset of index for different levels to dismiss the duplicated + * computation at runtime. + */ + private static final int[] INDEX_NEXT_OFFSET_BY_LEVEL_ARRAY = new int[MAX_LEVEL + 1]; + + /** + * Pre-compute the length of key meta for different levels to dismiss the duplicated + * computation at runtime. + */ + private static final int[] KEY_META_LEN_BY_LEVEL_ARRAY = new int[MAX_LEVEL + 1]; + + static { + for (int i = 1; i < INDEX_NEXT_OFFSET_BY_LEVEL_ARRAY.length; i++) { + INDEX_NEXT_OFFSET_BY_LEVEL_ARRAY[i] = LEVEL_INDEX_OFFSET + (i - 1) * Long.BYTES; + } + + for (int i = 0; i < KEY_META_LEN_BY_LEVEL_ARRAY.length; i++) { + KEY_META_LEN_BY_LEVEL_ARRAY[i] = LEVEL_INDEX_OFFSET + 2 * i * Long.BYTES; + } + } + + /** + * Returns the level of the node. + * + * @param memorySegment memory segment for key space. + * @param offset offset of key space in the memory segment. + */ + public static int getLevel(MemorySegment memorySegment, int offset) { + return memorySegment.getInt(offset + KEY_META_OFFSET) & BYTE_MASK; + } + + /** + * Returns the status of the node. + * + * @param memorySegment memory segment for key space. + * @param offset offset of key space in the memory segment. + */ + public static NodeStatus getNodeStatus(MemorySegment memorySegment, int offset) { + byte status = (byte) ((memorySegment.getInt(offset + KEY_META_OFFSET) >>> 8) & BYTE_MASK); + return NodeStatus.valueOf(status); + } + + /** + * Puts the level and status to the key space. + * + * @param memorySegment memory segment for key space. + * @param offset offset of key space in the memory segment. + * @param level the level. + * @param status the status. + */ + public static void putLevelAndNodeStatus(MemorySegment memorySegment, int offset, int level, NodeStatus status) { + int data = ((status.getValue() & BYTE_MASK) << 8) | level; + memorySegment.putInt(offset + SkipListUtils.KEY_META_OFFSET, data); + } + + /** + * Returns the length of the key. + * + * @param memorySegment memory segment for key space. + * @param offset offset of key space in the memory segment. + */ + public static int getKeyLen(MemorySegment memorySegment, int offset) { + return memorySegment.getInt(offset + KEY_LEN_OFFSET); + } + + /** + * Puts the length of key to the key space. + * + * @param memorySegment memory segment for key space. + * @param offset offset of key space in the memory segment. + * @param keyLen length of key. + */ + public static void putKeyLen(MemorySegment memorySegment, int offset, int keyLen) { + memorySegment.putInt(offset + KEY_LEN_OFFSET, keyLen); + } + + /** + * Returns the value pointer. + * + * @param memorySegment memory segment for key space. + * @param offset offset of key space in the memory segment. + */ + public static long getValuePointer(MemorySegment memorySegment, int offset) { + return memorySegment.getLong(offset + VALUE_POINTER_OFFSET); + } + + /** + * Puts the value pointer to key space. + * + * @param memorySegment memory segment for key space. + * @param offset offset of key space in the memory segment. + * @param valuePointer the value pointer. + */ + public static void putValuePointer(MemorySegment memorySegment, int offset, long valuePointer) { + memorySegment.putLong(offset + VALUE_POINTER_OFFSET, valuePointer); + } + + /** + * Returns the next key pointer on level 0. + * + * @param memorySegment memory segment for key space. + * @param offset offset of key space in the memory segment. + */ + public static long getNextKeyPointer(MemorySegment memorySegment, int offset) { + return memorySegment.getLong(offset + NEXT_KEY_POINTER_OFFSET); + } + + /** + * Puts the next key pointer on level 0 to key space. + * + * @param memorySegment memory segment for key space. + * @param offset offset of key space in the memory segment. + * @param nextKeyPointer next key pointer on level 0. + */ + public static void putNextKeyPointer(MemorySegment memorySegment, int offset, long nextKeyPointer) { + memorySegment.putLong(offset + NEXT_KEY_POINTER_OFFSET, nextKeyPointer); + } + + /** + * Returns next key pointer on the given index level. + * + * @param memorySegment memory segment for key space. + * @param offset offset of key space in the memory segment. + * @param level level of index. + */ + public static long getNextIndexNode(MemorySegment memorySegment, int offset, int level) { + return memorySegment.getLong(offset + INDEX_NEXT_OFFSET_BY_LEVEL_ARRAY[level]); + } + + /** + * Puts next key pointer on the given index level to key space. + * + * @param memorySegment memory segment for key space. + * @param offset offset of key space in the memory segment. + * @param level level of index. + * @param nextKeyPointer next key pointer on the given level. + */ + public static void putNextIndexNode(MemorySegment memorySegment, int offset, int level, long nextKeyPointer) { + memorySegment.putLong(offset + INDEX_NEXT_OFFSET_BY_LEVEL_ARRAY[level], nextKeyPointer); + } + + /** + * Returns previous key pointer on the given index level. + * + * @param memorySegment memory segment for key space. + * @param offset offset of key space in the memory segment. + * @param totalLevel the level of the node. + * @param level on which level to get the previous key pointer of the node. + */ + public static long getPrevIndexNode(MemorySegment memorySegment, int offset, int totalLevel, int level) { + int of = getIndexOffset(offset, totalLevel, level); + return memorySegment.getLong(of); + } + + private static int getIndexOffset(int offset, int totalLevel, int level) { + return offset + INDEX_NEXT_OFFSET_BY_LEVEL_ARRAY[totalLevel] + level * Long.BYTES; + } + + /** + * Puts previous key pointer on the given index level to key space. + * + * @param memorySegment memory segment for key space. + * @param offset offset of key space in the memory segment. + * @param totalLevel top level of the key. + * @param level level of index. + * @param prevKeyPointer previous key pointer on the given level. + */ + public static void putPrevIndexNode( + MemorySegment memorySegment, int offset, int totalLevel, int level, long prevKeyPointer) { + int of = getIndexOffset(offset, totalLevel, level); + memorySegment.putLong(of, prevKeyPointer); + } + + /** + * Returns the length of key meta with the given level. + * + * @param level level of the key. + */ + public static int getKeyMetaLen(int level) { + Preconditions.checkArgument(level >= 0 && level < KEY_META_LEN_BY_LEVEL_ARRAY.length, + "level " + level + " out of range [0, " + KEY_META_LEN_BY_LEVEL_ARRAY.length + ")"); + return KEY_META_LEN_BY_LEVEL_ARRAY[level]; + } + + /** + * Returns the offset of key data in the key space. + * + * @param level level of the key. + */ + public static int getKeyDataOffset(int level) { + return SkipListUtils.getKeyMetaLen(level); + } + + /** + * Puts the key data into key space. + * + * @param segment memory segment for key space. + * @param offset offset of key space in memory segment. + * @param keySegment memory segment for key data. + * @param keyOffset offset of key data in memory segment. + * @param keyLen length of key data. + * @param level level of the key. + */ + public static void putKeyData( + MemorySegment segment, int offset, MemorySegment keySegment, int keyOffset, int keyLen, int level) { + keySegment.copyTo(keyOffset, segment, offset + getKeyDataOffset(level), keyLen); + } + + /** + * Value space schema. + * - value meta + * -- int: version of this value to support copy on write + * -- long: pointer to the key space + * -- long: pointer to next older value + * -- int: length of data + * - byte[] data of value + */ + static final int VALUE_META_OFFSET = 0; + static final int VALUE_VERSION_OFFSET = VALUE_META_OFFSET; + static final int KEY_POINTER_OFFSET = VALUE_VERSION_OFFSET + Integer.BYTES; + static final int NEXT_VALUE_POINTER_OFFSET = KEY_POINTER_OFFSET + Long.BYTES; + static final int VALUE_LEN_OFFSET = NEXT_VALUE_POINTER_OFFSET + Long.BYTES; + static final int VALUE_DATA_OFFSET = VALUE_LEN_OFFSET + Integer.BYTES; + + /** + * Returns the version of value. + * + * @param memorySegment memory segment for value space. + * @param offset offset of value space in memory segment. + */ + public static int getValueVersion(MemorySegment memorySegment, int offset) { + return memorySegment.getInt(offset + VALUE_VERSION_OFFSET); + } + + /** + * Puts the version of value to value space. + * + * @param memorySegment memory segment for value space. + * @param offset offset of value space in memory segment. + * @param version version of value. + */ + public static void putValueVersion(MemorySegment memorySegment, int offset, int version) { + memorySegment.putInt(offset + VALUE_VERSION_OFFSET, version); + } + + /** + * Return the pointer to key space. + * + * @param memorySegment memory segment for value space. + * @param offset offset of value space in memory segment. + */ + public static long getKeyPointer(MemorySegment memorySegment, int offset) { + return memorySegment.getLong(offset + KEY_POINTER_OFFSET); + } + + /** + * Puts the pointer of key space. + * + * @param memorySegment memory segment for value space. + * @param offset offset of value space in memory segment. + * @param keyPointer pointer to key space. + */ + public static void putKeyPointer(MemorySegment memorySegment, int offset, long keyPointer) { + memorySegment.putLong(offset + KEY_POINTER_OFFSET, keyPointer); + } + + /** + * Return the pointer to next value space. + * + * @param memorySegment memory segment for value space. + * @param offset offset of value space in memory segment. + */ + public static long getNextValuePointer(MemorySegment memorySegment, int offset) { + return memorySegment.getLong(offset + NEXT_VALUE_POINTER_OFFSET); + } + + /** + * Puts the pointer of next value space. + * + * @param memorySegment memory segment for value space. + * @param offset offset of value space in memory segment. + * @param nextValuePointer pointer to next value space. + */ + public static void putNextValuePointer(MemorySegment memorySegment, int offset, long nextValuePointer) { + memorySegment.putLong(offset + NEXT_VALUE_POINTER_OFFSET, nextValuePointer); + } + + /** + * Return the length of value data. + * + * @param memorySegment memory segment for value space. + * @param offset offset of value space in memory segment. + */ + public static int getValueLen(MemorySegment memorySegment, int offset) { + return memorySegment.getInt(offset + VALUE_LEN_OFFSET); + } + + /** + * Puts the length of value data. + * + * @param memorySegment memory segment for value space. + * @param offset offset of value space in memory segment. + * @param valueLen length of value data. + */ + public static void putValueLen(MemorySegment memorySegment, int offset, int valueLen) { + memorySegment.putInt(offset + VALUE_LEN_OFFSET, valueLen); + } + + /** + * Returns the length of value meta. + */ + public static int getValueMetaLen() { + return VALUE_DATA_OFFSET; + } + + /** + * Puts the value data into value space. + * + * @param memorySegment memory segment for value space. + * @param offset offset of value space in memory segment. + * @param value value data. + */ + public static void putValueData(MemorySegment memorySegment, int offset, byte[] value) { + MemorySegment valueSegment = MemorySegmentFactory.wrap(value); + valueSegment.copyTo(0, memorySegment, offset + getValueMetaLen(), value.length); + } + + /** + * Set the next node of the given node at the given level. + * + * @param node the node. + * @param nextNode the next node to set. + * @param level the level to find the next node. + * @param levelIndexHeader the header of the level index. + * @param spaceAllocator the space allocator. + */ + static void helpSetNextNode( + long node, + long nextNode, + int level, + LevelIndexHeader levelIndexHeader, + Allocator spaceAllocator) { + if (node == HEAD_NODE) { + levelIndexHeader.updateNextNode(level, nextNode); + return; + } + + Chunk chunk = spaceAllocator.getChunkById(SpaceUtils.getChunkIdByAddress(node)); + int offsetInChunk = SpaceUtils.getChunkOffsetByAddress(node); + MemorySegment segment = chunk.getMemorySegment(offsetInChunk); + int offsetInByteBuffer = chunk.getOffsetInSegment(offsetInChunk); + + if (level == 0) { + putNextKeyPointer(segment, offsetInByteBuffer, nextNode); + } else { + putNextIndexNode(segment, offsetInByteBuffer, level, nextNode); + } + } + + /** + * Return the next of the given node at the given level. + * + * @param node the node to find the next node for. + * @param level the level to find the next node. + * @param levelIndexHeader the header of the level index. + * @param spaceAllocator the space allocator. + * @return the pointer to the next node of the given node at the given level. + */ + static long helpGetNextNode( + long node, + int level, + LevelIndexHeader levelIndexHeader, + Allocator spaceAllocator) { + if (node == HEAD_NODE) { + return levelIndexHeader.getNextNode(level); + } + + Chunk chunk = spaceAllocator.getChunkById(SpaceUtils.getChunkIdByAddress(node)); + int offsetInChunk = SpaceUtils.getChunkOffsetByAddress(node); + MemorySegment segment = chunk.getMemorySegment(offsetInChunk); + int offsetInByteBuffer = chunk.getOffsetInSegment(offsetInChunk); + + return level == 0 ? getNextKeyPointer(segment, offsetInByteBuffer) + : getNextIndexNode(segment, offsetInByteBuffer, level); + } + + /** + * Set the previous node of the given node at the given level. The level must be positive. + * + * @param node the node. + * @param prevNode the previous node to set. + * @param level the level to find the next node. + * @param spaceAllocator the space allocator. + */ + static void helpSetPrevNode(long node, long prevNode, int level, Allocator spaceAllocator) { + Preconditions.checkArgument(level > 0, "only index level have previous node"); + + if (node == HEAD_NODE || node == NIL_NODE) { + return; + } + + Chunk chunk = spaceAllocator.getChunkById(SpaceUtils.getChunkIdByAddress(node)); + int offsetInChunk = SpaceUtils.getChunkOffsetByAddress(node); + MemorySegment segment = chunk.getMemorySegment(offsetInChunk); + int offsetInByteBuffer = chunk.getOffsetInSegment(offsetInChunk); + + int topLevel = getLevel(segment, offsetInByteBuffer); + + putPrevIndexNode(segment, offsetInByteBuffer, topLevel, level, prevNode); + } + + /** + * Set the previous node and the next node of the given node at the given level. + * The level must be positive. + * @param node the node. + * @param prevNode the previous node to set. + * @param nextNode the next node to set. + * @param level the level to find the next node. + * @param spaceAllocator the space allocator. + */ + static void helpSetPrevAndNextNode( + long node, + long prevNode, + long nextNode, + int level, + Allocator spaceAllocator) { + Preconditions.checkArgument(node != HEAD_NODE, "head node does not have previous node"); + Preconditions.checkArgument(level > 0, "only index level have previous node"); + + Chunk chunk = spaceAllocator.getChunkById(SpaceUtils.getChunkIdByAddress(node)); + int offsetInChunk = SpaceUtils.getChunkOffsetByAddress(node); + MemorySegment segment = chunk.getMemorySegment(offsetInChunk); + int offsetInByteBuffer = chunk.getOffsetInSegment(offsetInChunk); + + int topLevel = getLevel(segment, offsetInByteBuffer); + + putNextIndexNode(segment, offsetInByteBuffer, level, nextNode); + putPrevIndexNode(segment, offsetInByteBuffer, topLevel, level, prevNode); + } + + /** + * Whether the node has been logically removed. + * + * @param node the node to check against + * @param spaceAllocator the space allocator + * @return true if the node has been logically removed. + */ + static boolean isNodeRemoved(long node, Allocator spaceAllocator) { + if (node == NIL_NODE) { + return false; + } + + Chunk chunk = spaceAllocator.getChunkById(SpaceUtils.getChunkIdByAddress(node)); + int offsetInChunk = SpaceUtils.getChunkOffsetByAddress(node); + MemorySegment segment = chunk.getMemorySegment(offsetInChunk); + int offsetInByteBuffer = chunk.getOffsetInSegment(offsetInChunk); + return getNodeStatus(segment, offsetInByteBuffer) == NodeStatus.REMOVE; + } + + /** + * Compare the first skip list key in the given memory segment with the second skip list key in the given node. + * + * @param keySegment memory segment storing the first key. + * @param keyOffset offset of the first key in memory segment. + * @param targetNode the node storing the second key. + * @param spaceAllocator the space allocator. + * @return Returns a negative integer, zero, or a positive integer as the first key is less than, + * equal to, or greater than the second. + */ + static int compareSegmentAndNode( + MemorySegment keySegment, + int keyOffset, + long targetNode, + @Nonnull Allocator spaceAllocator) { + Chunk chunk = spaceAllocator.getChunkById(SpaceUtils.getChunkIdByAddress(targetNode)); + int offsetInChunk = SpaceUtils.getChunkOffsetByAddress(targetNode); + MemorySegment targetKeySegment = chunk.getMemorySegment(offsetInChunk); + int offsetInByteBuffer = chunk.getOffsetInSegment(offsetInChunk); + + int level = getLevel(targetKeySegment, offsetInByteBuffer); + int targetKeyOffset = offsetInByteBuffer + getKeyDataOffset(level); + + return SkipListKeyComparator.compareTo(keySegment, keyOffset, targetKeySegment, targetKeyOffset); + } + + /** + * Find the predecessor node for the given node at the given level. + * + * @param node the node. + * @param level the level. + * @param levelIndexHeader the head level index. + * @param spaceAllocator the space allocator. + * @return node id before the key at the given level. + */ + static long findPredecessor( + long node, + int level, + LevelIndexHeader levelIndexHeader, + @Nonnull Allocator spaceAllocator) { + Chunk chunk = spaceAllocator.getChunkById(SpaceUtils.getChunkIdByAddress(node)); + int offsetInChunk = SpaceUtils.getChunkOffsetByAddress(node); + MemorySegment keySegment = chunk.getMemorySegment(offsetInChunk); + int offsetInByteBuffer = chunk.getOffsetInSegment(offsetInChunk); + + int keyLevel = getLevel(keySegment, offsetInByteBuffer); + int keyOffset = offsetInByteBuffer + getKeyDataOffset(keyLevel); + + return findPredecessor(keySegment, keyOffset, level, levelIndexHeader, spaceAllocator); + } + + /** + * Find the predecessor node for the given key at the given level. + * The key is in the memory segment positioning at the given offset. + * + * @param keySegment memory segment which contains the key. + * @param keyOffset offset of the key in the memory segment. + * @param level the level. + * @param levelIndexHeader the head level index. + * @param spaceAllocator the space allocator. + * @return node id before the key at the given level. + */ + static long findPredecessor( + MemorySegment keySegment, + int keyOffset, + int level, + @Nonnull LevelIndexHeader levelIndexHeader, + Allocator spaceAllocator) { + int currentLevel = levelIndexHeader.getLevel(); + long currentNode = HEAD_NODE; + long nextNode = levelIndexHeader.getNextNode(currentLevel); + + for ( ; ; ) { + if (nextNode != NIL_NODE) { + int c = compareSegmentAndNode(keySegment, keyOffset, nextNode, spaceAllocator); + if (c > 0) { + currentNode = nextNode; + nextNode = helpGetNextNode(currentNode, currentLevel, levelIndexHeader, spaceAllocator); + continue; + } + } + + if (currentLevel <= level) { + return currentNode; + } + + currentLevel--; + nextNode = helpGetNextNode(currentNode, currentLevel, levelIndexHeader, spaceAllocator); + } + } + + /** + * Returns the next value pointer of the value. + * + * @param valuePointer the value pointer of current value. + * @param spaceAllocator the space allocator. + */ + static long helpGetNextValuePointer(long valuePointer, Allocator spaceAllocator) { + Chunk chunk = spaceAllocator.getChunkById(SpaceUtils.getChunkIdByAddress(valuePointer)); + int offsetInChunk = SpaceUtils.getChunkOffsetByAddress(valuePointer); + MemorySegment segment = chunk.getMemorySegment(offsetInChunk); + int offsetInByteBuffer = chunk.getOffsetInSegment(offsetInChunk); + + return getNextValuePointer(segment, offsetInByteBuffer); + } + + /** + * Sets the next value pointer of the value. + * + * @param valuePointer the value pointer. + * @param nextValuePointer the next value pointer to set. + * @param spaceAllocator the space allocator. + */ + static void helpSetNextValuePointer(long valuePointer, long nextValuePointer, Allocator spaceAllocator) { + Chunk chunk = spaceAllocator.getChunkById(SpaceUtils.getChunkIdByAddress(valuePointer)); + int offsetInChunk = SpaceUtils.getChunkOffsetByAddress(valuePointer); + MemorySegment segment = chunk.getMemorySegment(offsetInChunk); + int offsetInByteBuffer = chunk.getOffsetInSegment(offsetInChunk); + + putNextValuePointer(segment, offsetInByteBuffer, nextValuePointer); + } + + /** + * Build the level index for the given node. + * + * @param node the node. + * @param level level of the node. + * @param keySegment memory segment of the key in the node. + * @param keyOffset offset of the key in memory segment. + * @param levelIndexHeader the head level index. + * @param spaceAllocator the space allocator. + */ + static void buildLevelIndex(long node, int level, MemorySegment keySegment, int keyOffset, LevelIndexHeader levelIndexHeader, Allocator spaceAllocator) { + int currLevel = level; + long prevNode = findPredecessor(keySegment, keyOffset, currLevel, levelIndexHeader, spaceAllocator); + long currentNode = helpGetNextNode(prevNode, currLevel, levelIndexHeader, spaceAllocator); + + for (; ; ) { + if (currentNode != NIL_NODE) { + int c = compareSegmentAndNode(keySegment, keyOffset, currentNode, spaceAllocator); + if (c > 0) { + prevNode = currentNode; + currentNode = helpGetNextNode(currentNode, currLevel, levelIndexHeader, spaceAllocator); + continue; + } + } + + helpSetPrevAndNextNode(node, prevNode, currentNode, currLevel, spaceAllocator); + helpSetNextNode(prevNode, node, currLevel, levelIndexHeader, spaceAllocator); + helpSetPrevNode(currentNode, node, currLevel, spaceAllocator); + + currLevel--; + if (currLevel == 0) { + break; + } + + currentNode = helpGetNextNode(prevNode, currLevel, levelIndexHeader, spaceAllocator); + } + } + + /** + * Remove the level index for the node from the skip list. + * + * @param node the node. + * @param spaceAllocator the space allocator. + * @param levelIndexHeader the head level index. + */ + static void removeLevelIndex(long node, Allocator spaceAllocator, LevelIndexHeader levelIndexHeader) { + Chunk chunk = spaceAllocator.getChunkById(SpaceUtils.getChunkIdByAddress(node)); + int offsetInChunk = SpaceUtils.getChunkOffsetByAddress(node); + MemorySegment segment = chunk.getMemorySegment(offsetInChunk); + int offsetInByteBuffer = chunk.getOffsetInSegment(offsetInChunk); + + int level = getLevel(segment, offsetInByteBuffer); + + for (int i = 1; i <= level; i++) { + long prevNode = getPrevIndexNode(segment, offsetInByteBuffer, level, i); + long nextNode = getNextIndexNode(segment, offsetInByteBuffer, i); + helpSetNextNode(prevNode, nextNode, i, levelIndexHeader, spaceAllocator); + helpSetPrevNode(nextNode, prevNode, i, spaceAllocator); + } + } + + /** + * Free the space of the linked values, and the head value + * is pointed by the given pointer. + * + * @param valuePointer the pointer of the value to start removing. + * @param spaceAllocator the space allocator. + */ + static void removeAllValues(long valuePointer, Allocator spaceAllocator) { + long nextValuePointer; + while (valuePointer != NIL_VALUE_POINTER) { + nextValuePointer = helpGetNextValuePointer(valuePointer, spaceAllocator); + spaceAllocator.free(valuePointer); + valuePointer = nextValuePointer; + } + } + + /** + * Returns the value pointer of the node. + * + * @param node the node. + * @param spaceAllocator the space allocator. + */ + static long helpGetValuePointer(long node, Allocator spaceAllocator) { + Chunk chunk = spaceAllocator.getChunkById(SpaceUtils.getChunkIdByAddress(node)); + int offsetInChunk = SpaceUtils.getChunkOffsetByAddress(node); + MemorySegment segment = chunk.getMemorySegment(offsetInChunk); + int offsetInByteBuffer = chunk.getOffsetInSegment(offsetInChunk); + + return getValuePointer(segment, offsetInByteBuffer); + } + + /** + * Returns the version of the value. + * + * @param valuePointer the pointer to the value. + * @param spaceAllocator the space allocator. + */ + static int helpGetValueVersion(long valuePointer, Allocator spaceAllocator) { + Chunk chunk = spaceAllocator.getChunkById(SpaceUtils.getChunkIdByAddress(valuePointer)); + int offsetInChunk = SpaceUtils.getChunkOffsetByAddress(valuePointer); + MemorySegment segment = chunk.getMemorySegment(offsetInChunk); + int offsetInByteBuffer = chunk.getOffsetInSegment(offsetInChunk); + + return getValueVersion(segment, offsetInByteBuffer); + } + + /** + * Returns the length of the value. + * + * @param valuePointer the pointer to the value. + * @param spaceAllocator the space allocator. + */ + static int helpGetValueLen(long valuePointer, Allocator spaceAllocator) { + Chunk chunk = spaceAllocator.getChunkById(SpaceUtils.getChunkIdByAddress(valuePointer)); + int offsetInChunk = SpaceUtils.getChunkOffsetByAddress(valuePointer); + MemorySegment segment = chunk.getMemorySegment(offsetInChunk); + int offsetInByteBuffer = chunk.getOffsetInSegment(offsetInChunk); + + return getValueLen(segment, offsetInByteBuffer); + } + + /** + * Return of the newest version of value for the node. + * + * @param node the node. + * @param spaceAllocator the space allocator. + */ + static int helpGetNodeLatestVersion(long node, Allocator spaceAllocator) { + Chunk chunk = spaceAllocator.getChunkById(SpaceUtils.getChunkIdByAddress(node)); + int offsetInChunk = SpaceUtils.getChunkOffsetByAddress(node); + MemorySegment segment = chunk.getMemorySegment(offsetInChunk); + int offsetInByteBuffer = chunk.getOffsetInSegment(offsetInChunk); + long valuePointer = getValuePointer(segment, offsetInByteBuffer); + + return helpGetValueVersion(valuePointer, spaceAllocator); + } + +} diff --git a/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/SkipListValueSerializer.java b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/SkipListValueSerializer.java new file mode 100644 index 000000000000..c1f05567549d --- /dev/null +++ b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/SkipListValueSerializer.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.runtime.state.heap; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.core.memory.MemorySegmentInputStreamWithPos; + +import java.io.IOException; + +/** + * Serializer/deserializer used for conversion between state and skip list value. + * It is not thread safe. + * + * @param type of state. + */ +class SkipListValueSerializer { + + private final TypeSerializer stateSerializer; + + /** The reusable output serialization buffer. */ + private final DataOutputSerializer dos; + + SkipListValueSerializer(TypeSerializer stateSerializer) { + this.stateSerializer = stateSerializer; + this.dos = new DataOutputSerializer(16); + } + + byte[] serialize(S state) { + try { + stateSerializer.serialize(state, dos); + } catch (IOException e) { + throw new RuntimeException("serialize key and namespace failed", e); + } + byte[] ret = dos.getCopyOfBuffer(); + dos.clear(); + return ret; + } + + /** + * Deserialize the state from the byte buffer which stores skip list value. + * + * @param memorySegment the memory segment which stores the skip list value. + * @param offset the start position of the skip list value in the byte buffer. + * @param len length of the skip list value. + */ + S deserializeState(MemorySegment memorySegment, int offset, int len) { + final MemorySegmentInputStreamWithPos src = new MemorySegmentInputStreamWithPos(memorySegment, offset, len); + final DataInputViewStreamWrapper in = new DataInputViewStreamWrapper(src); + + try { + return stateSerializer.deserialize(in); + } catch (IOException e) { + throw new RuntimeException("deserialize state failed", e); + } + } +} diff --git a/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/space/Allocator.java b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/space/Allocator.java new file mode 100644 index 000000000000..cd03de3fb6ee --- /dev/null +++ b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/space/Allocator.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.runtime.state.heap.space; + +import java.io.Closeable; + +/** + * Implementations are responsible for allocate space. + */ +public interface Allocator extends Closeable { + + /** + * Allocate space with the given size. + * + * @param size size of space to allocate. + * @return address of the allocated space. + * @throws Exception This method will throw exception if failed to allocate space. + */ + long allocate(int size) throws Exception; + + /** + * Free the space with the given address. + * + * @param address address of the space to free. + */ + void free(long address); + + /** + * Returns the chunk with the given chunk id. + * + * @param chunkId id of the chunk. + * @return chunk with the given id. + */ + Chunk getChunkById(int chunkId); +} diff --git a/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/space/Chunk.java b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/space/Chunk.java new file mode 100644 index 000000000000..7b314d52114c --- /dev/null +++ b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/space/Chunk.java @@ -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.runtime.state.heap.space; + +import org.apache.flink.core.memory.MemorySegment; + +import java.nio.ByteBuffer; + +/** + * Chunk is a logically contiguous space backed by one or multiple {@link ByteBuffer}. + *

    + * For example: a Chunk of 1G size may be backed by one {@link java.nio.MappedByteBuffer} from a memory-mapped 1G file, + * or multiple {@link java.nio.HeapByteBuffer}/{@link java.nio.DirectByteBuffer}. + */ +public interface Chunk { + /** + * Try to allocate size bytes from the chunk. + * + * @param len size of bytes to allocate. + * @return the offset of the successful allocation, or -1 to indicate not-enough-space + */ + int allocate(int len); + + /** + * release the space addressed by interChunkOffset. + * + * @param interChunkOffset offset of the chunk + */ + void free(int interChunkOffset); + + /** + * @return Id of this Chunk + */ + int getChunkId(); + + int getChunkCapacity(); + + /** + * @return This chunk's backing MemorySegment described by chunkOffset. + */ + MemorySegment getMemorySegment(int chunkOffset); + + /** + * @param offsetInChunk virtual and global address in chunk + * @return chunk maybe compose of multi {@link MemorySegment}s, return the offset in certain one. + */ + int getOffsetInSegment(int offsetInChunk); +} diff --git a/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/space/Constants.java b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/space/Constants.java new file mode 100644 index 000000000000..c07e9b293970 --- /dev/null +++ b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/space/Constants.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.state.heap.space; + +/** + * constants. + */ +public class Constants { + public static final int NO_SPACE = -1; + public static final int BUCKET_SIZE = 1024 * 1024; + public static final int FOUR_BYTES_BITS = 32; + public static final long FOUR_BYTES_MARK = 0xFFFFFFFFL; +} diff --git a/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/space/SpaceUtils.java b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/space/SpaceUtils.java new file mode 100644 index 000000000000..cdac45897ad6 --- /dev/null +++ b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/space/SpaceUtils.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.state.heap.space; + +import static org.apache.flink.runtime.state.heap.space.Constants.FOUR_BYTES_BITS; +import static org.apache.flink.runtime.state.heap.space.Constants.FOUR_BYTES_MARK; + +/** + * Utils. + */ +public class SpaceUtils { + + public static int getChunkIdByAddress(long offset) { + return (int) ((offset >>> FOUR_BYTES_BITS) & FOUR_BYTES_MARK); + } + + public static int getChunkOffsetByAddress(long offset) { + return (int) (offset & FOUR_BYTES_MARK); + } +} diff --git a/flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteSkipListStateMapBasicOpTest.java b/flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteSkipListStateMapBasicOpTest.java new file mode 100644 index 000000000000..591a807278b0 --- /dev/null +++ b/flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteSkipListStateMapBasicOpTest.java @@ -0,0 +1,548 @@ +/* + * + * * Licensed to the Apache Software Foundation (ASF) under one + * * or more contributor license agreements. See the NOTICE file + * * distributed with this work for additional information + * * regarding copyright ownership. The ASF licenses this file + * * to you under the Apache License, Version 2.0 (the + * * "License"); you may not use this file except in compliance + * * with the License. You may obtain a copy of the License at + * * + * * http://www.apache.org/licenses/LICENSE-2.0 + * * + * * Unless required by applicable law or agreed to in writing, software + * * distributed under the License is distributed on an "AS IS" BASIS, + * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * * See the License for the specific language governing permissions and + * * limitations under the License. + * + */ + +package org.apache.flink.runtime.state.heap; + +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.core.memory.MemorySegmentFactory; +import org.apache.flink.runtime.state.StateEntry; +import org.apache.flink.runtime.state.StateTransformationFunction; +import org.apache.flink.runtime.state.heap.space.Allocator; +import org.apache.flink.runtime.state.heap.space.Chunk; +import org.apache.flink.runtime.state.internal.InternalKvState; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.IOUtils; +import org.apache.flink.util.TestLogger; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import javax.annotation.Nullable; + +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.NoSuchElementException; + +import static org.apache.flink.runtime.state.heap.CopyOnWriteSkipListStateMapTestUtils.addToReferenceState; +import static org.apache.flink.runtime.state.heap.CopyOnWriteSkipListStateMapTestUtils.createEmptyStateMap; +import static org.apache.flink.runtime.state.heap.CopyOnWriteSkipListStateMapTestUtils.removeFromReferenceState; +import static org.apache.flink.runtime.state.heap.CopyOnWriteSkipListStateMapTestUtils.verifyState; +import static org.hamcrest.Matchers.contains; +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.assertNull; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * Test basic operations of {@link CopyOnWriteSkipListStateMap}. + */ +public class CopyOnWriteSkipListStateMapBasicOpTest extends TestLogger { + private final long namespace = 1L; + private TestAllocator allocator; + private CopyOnWriteSkipListStateMap stateMap; + + @Before + public void setUp() { + allocator = new TestAllocator(256); + stateMap = createEmptyStateMap(0, 0.0f, allocator); + } + + @After + public void tearDown() { + stateMap.close(); + IOUtils.closeQuietly(allocator); + } + + /** + * Test initialization of state map. + */ + @Test + public void testInitStateMap() { + assertTrue(stateMap.isEmpty()); + assertEquals(0, stateMap.size()); + assertEquals(0, stateMap.totalSize()); + assertEquals(0, stateMap.getRequestCount()); + assertTrue(stateMap.getLogicallyRemovedNodes().isEmpty()); + assertEquals(0, stateMap.getHighestRequiredSnapshotVersionPlusOne()); + assertEquals(0, stateMap.getHighestFinishedSnapshotVersion()); + assertTrue(stateMap.getSnapshotVersions().isEmpty()); + assertTrue(stateMap.getPruningValueNodes().isEmpty()); + assertEquals(0, stateMap.getResourceGuard().getLeaseCount()); + assertFalse(stateMap.getResourceGuard().isClosed()); + assertFalse(stateMap.isClosed()); + + assertNull(stateMap.get(0, 0L)); + assertFalse(stateMap.containsKey(1, 2L)); + assertNull(stateMap.removeAndGetOld(3, 4L)); + assertFalse(stateMap.getKeys(-92L).iterator().hasNext()); + assertEquals(0, stateMap.sizeOfNamespace(8L)); + assertFalse(stateMap.iterator().hasNext()); + assertFalse(stateMap.getStateIncrementalVisitor(100).hasNext()); + } + + /** + * Test state put and get. + */ + @Test + public void testPutAndGetState() { + int totalSize = 0; + // test put empty skip list and get + totalSize = putAndVerify(2, "2", totalSize, true); + // put state at tail of the skip list (new key is bigger than all existing keys) and get + totalSize = putAndVerify(4, "4", totalSize, true); + // put state at head of the skip list (new key is smaller than all existing keys) and get + totalSize = putAndVerify(1, "1", totalSize, true); + // put state in the middle and get + putAndVerify(3, "3", totalSize, true); + } + + private int putAndVerify(int key, String state, int initTotalSize, boolean isNewKey) { + stateMap.put(key, namespace, state); + int totalSize = isNewKey ? initTotalSize + 1 : initTotalSize; + assertThat(stateMap.get(key, namespace), is(state)); + assertThat(stateMap.size(), is(totalSize)); + assertThat(stateMap.totalSize(), is(totalSize)); + assertThat(allocator.getTotalSpaceNumber(), is(totalSize * 2)); + return totalSize; + } + + /** + * Test state update (put existing key). + */ + @Test + public void testUpdateState() { + int totalSize = 3; + for (int i = 1; i <= totalSize; i++) { + stateMap.put(i, namespace, String.valueOf(i)); + } + // update state at tail of the skip list (new key is bigger than all existing keys) + totalSize = putAndVerify(3, "33", totalSize, false); + // update state at head of the skip list (new key is smaller than all existing keys) + totalSize = putAndVerify(1, "11", totalSize, false); + // update state in the middle + putAndVerify(2, "22", totalSize, false); + } + + /** + * Test state putAndGetOld. + */ + @Test + public void testPutAndGetOldState() { + int totalSize = 0; + // test put empty skip list and get + totalSize = putAndGetOldVerify(2, "2", totalSize); + // put state at tail of the skip list (new key is bigger than all existing keys) and get + totalSize = putAndGetOldVerify(4, "4", totalSize); + // put state at head of the skip list (new key is smaller than all existing keys) and get + totalSize = putAndGetOldVerify(1, "1", totalSize); + // put state in the middle and get + putAndGetOldVerify(3, "3", totalSize); + } + + private int putAndGetOldVerify(int key, String state, int initTotalSize) { + int totalSize = initTotalSize + 1; + String oldState = stateMap.get(key, namespace); + assertThat(stateMap.putAndGetOld(key, namespace, state), is(oldState)); + assertThat(stateMap.get(key, namespace), is(state)); + assertThat(stateMap.size(), is(totalSize)); + assertThat(stateMap.totalSize(), is(totalSize)); + assertThat(allocator.getTotalSpaceNumber(), is(totalSize * 2)); + return totalSize; + } + + /** + * Test state remove. + */ + @Test + public void testRemoveState() { + int totalSize = 4; + for (int i = 1; i <= totalSize; i++) { + stateMap.put(i, namespace, String.valueOf(i)); + } + // test remove state in the middle + totalSize = removeAndVerify(2, totalSize, true); + // test remove state at tail + totalSize = removeAndVerify(4, totalSize, true); + // test remove state at head + totalSize = removeAndVerify(1, totalSize, true); + // test remove the single state + totalSize = removeAndVerify(3, totalSize, true); + // test remove none-existing state + removeAndVerify(3, totalSize, false); + } + + private int removeAndVerify(int key, int initTotalSize, boolean keyExists) { + stateMap.remove(key, namespace); + int totalSize = keyExists ? initTotalSize - 1 : initTotalSize; + assertThat(stateMap.get(key, namespace), nullValue()); + assertThat(stateMap.size(), is(totalSize)); + assertThat(stateMap.totalSize(), is(totalSize)); + assertThat(allocator.getTotalSpaceNumber(), is(totalSize * 2)); + return totalSize; + } + + /** + * Test state removeAndGetOld. + */ + @Test + public void testRemoveAndGetOldState() { + int totalSize = 4; + for (int i = 1; i <= totalSize; i++) { + stateMap.put(i, namespace, String.valueOf(i)); + } + // test remove state in the middle + totalSize = removeAndGetOldVerify(2, totalSize); + // test remove state at tail + totalSize = removeAndGetOldVerify(4, totalSize); + // test remove state at head + totalSize = removeAndGetOldVerify(1, totalSize); + // test remove the single state + removeAndGetOldVerify(3, totalSize); + } + + private int removeAndGetOldVerify(int key, int initTotalSize) { + int totalSize = initTotalSize - 1; + String oldState = stateMap.get(key, namespace); + assertThat(stateMap.removeAndGetOld(key, namespace), is(oldState)); + assertThat(stateMap.get(key, namespace), nullValue()); + assertThat(stateMap.size(), is(totalSize)); + assertThat(stateMap.totalSize(), is(totalSize)); + assertThat(allocator.getTotalSpaceNumber(), is(totalSize * 2)); + return totalSize; + } + + /** + * Test state transform with existing key. + */ + @Test + public void testTransformExistingState() throws Exception { + final int key = 1; + final String oldState = "1"; + final int delta = 1; + StateTransformationFunction function = + (String prevState, Integer value) -> prevState == null ? String.valueOf(value) : prevState + value; + stateMap.put(key, namespace, oldState); + String expectedState = function.apply(oldState, delta); + stateMap.transform(key, namespace, delta, function); + assertThat(stateMap.get(key, namespace), is(expectedState)); + assertThat(stateMap.size(), is(1)); + assertThat(stateMap.totalSize(), is(1)); + assertThat(allocator.getTotalSpaceNumber(), is(2)); + } + + /** + * Test state transform with new key. + */ + @Test + public void testTransformAbsentState() throws Exception { + final int key = 1; + final int delta = 1; + StateTransformationFunction function = + (String prevState, Integer value) -> prevState == null ? String.valueOf(value) : prevState + value; + String expectedState = function.apply(null, delta); + stateMap.transform(key, namespace, delta, function); + assertThat(stateMap.get(key, namespace), is(expectedState)); + assertThat(stateMap.size(), is(1)); + assertThat(stateMap.totalSize(), is(1)); + assertThat(allocator.getTotalSpaceNumber(), is(2)); + } + + /** + * Test close of state map. + */ + @Test + public void testCloseStateMap() { + stateMap.close(); + assertTrue(stateMap.isClosed()); + assertThat(stateMap.size(), is(0)); + assertThat(stateMap.totalSize(), is(0)); + assertThat(allocator.getTotalSpaceNumber(), is(0)); + // make sure double close won't cause problem + stateMap.close(); + } + + /** + * Make sure exception will be thrown with allocation failure rather than swallowed. + */ + @Test + public void testPutWithAllocationFailure() { + Allocator exceptionalAllocator = new Allocator() { + @Override + public long allocate(int size) { + throw new RuntimeException("Exception on purpose"); + } + + @Override + public void free(long address) { + + } + + @Override + @Nullable + public Chunk getChunkById(int chunkId) { + return null; + } + + @Override + public void close() { + + } + }; + try (CopyOnWriteSkipListStateMap stateMap = + createEmptyStateMap(0, 0.0f, exceptionalAllocator)) { + stateMap.put(1, 1L, "test-value"); + fail("Should have thrown exception when space allocation fails"); + } catch (FlinkRuntimeException e) { + // expected + } + } + + /** + * This tests the internal capability of using partial {@link ByteBuffer}, making sure the internal methods + * works when put/get state with a key stored at a none-zero offset of a ByteBuffer. + */ + @Test + public void testPutAndGetNodeWithNoneZeroOffset() { + final int key = 10; + final long namespace = 0L; + final String valueString = "test"; + SkipListKeySerializer skipListKeySerializer = + new SkipListKeySerializer<>(IntSerializer.INSTANCE, LongSerializer.INSTANCE); + SkipListValueSerializer skipListValueSerializer = + new SkipListValueSerializer<>(StringSerializer.INSTANCE); + byte[] keyBytes = skipListKeySerializer.serialize(key, namespace); + byte[] constructedKeyBytes = new byte[keyBytes.length + 1]; + System.arraycopy(keyBytes, 0, constructedKeyBytes, 1, keyBytes.length); + MemorySegment keySegment = MemorySegmentFactory.wrap(constructedKeyBytes); + int keyLen = keyBytes.length; + byte[] value = skipListValueSerializer.serialize(valueString); + stateMap.putValue(keySegment, 1, keyLen, value, false); + String state = stateMap.getNode(keySegment, 1, keyLen); + assertThat(state, is(valueString)); + } + + /** + * Test next/update/remove during global iteration of StateIncrementalVisitor. + */ + @Test + public void testStateIncrementalVisitor() { + // map to store expected states, namespace -> key -> state + Map> referenceStates = new HashMap<>(); + + // put some states + for (long namespace = 0; namespace < 15; namespace++) { + for (int key = 0; key < 20; key++) { + String state = String.valueOf(namespace * key); + stateMap.put(key, namespace, state); + addToReferenceState(referenceStates, key, namespace, state); + } + } + verifyState(referenceStates, stateMap); + + InternalKvState.StateIncrementalVisitor visitor = + stateMap.getStateIncrementalVisitor(5); + int op = 0; + while (visitor.hasNext()) { + for (StateEntry stateEntry : visitor.nextEntries()) { + int key = stateEntry.getKey(); + long namespace = stateEntry.getNamespace(); + String state = stateEntry.getState(); + assertEquals(state, stateMap.get(key, namespace)); + switch (op) { + case 0: + visitor.remove(stateEntry); + removeFromReferenceState(referenceStates, key, namespace); + op = 1; + break; + case 1: + String newState = state + "-update"; + visitor.update(stateEntry, newState); + addToReferenceState(referenceStates, key, namespace, newState); + op = 2; + break; + default: + op = 0; + break; + } + } + } + verifyState(referenceStates, stateMap); + } + + /** + * Test StateIncrementalVisitor with closed state map. + */ + @Test + public void testStateIncrementalVisitorWithClosedStateMap() { + CopyOnWriteSkipListStateMap stateMap = createEmptyStateMap(); + // put some states + for (long namespace = 0; namespace < 15; namespace++) { + for (int key = 0; key < 20; key++) { + String state = String.valueOf(namespace * key); + stateMap.put(key, namespace, state); + } + } + InternalKvState.StateIncrementalVisitor closedVisitor = + stateMap.getStateIncrementalVisitor(5); + assertTrue(closedVisitor.hasNext()); + + stateMap.close(); + // the visitor will be invalid after state map is closed + assertFalse(closedVisitor.hasNext()); + } + + /** + * Test basic snapshot correctness. + */ + @Test + public void testBasicSnapshot() { + // take an empty snapshot + CopyOnWriteSkipListStateMapSnapshot snapshot = stateMap.stateSnapshot(); + snapshot.release(); + // put some states + stateMap.put(1, namespace, "1"); + // take the 2nd snapshot with data + CopyOnWriteSkipListStateMapSnapshot snapshot2 = stateMap.stateSnapshot(); + assertEquals(2, stateMap.getStateMapVersion()); + assertEquals(2, stateMap.getHighestRequiredSnapshotVersionPlusOne()); + assertEquals(1, stateMap.getSnapshotVersions().size()); + assertThat(stateMap.getSnapshotVersions(), contains(2)); + assertEquals(1, stateMap.getResourceGuard().getLeaseCount()); + snapshot2.release(); + stateMap.close(); + } + + /** + * Test snapshot empty state map. + */ + @Test + public void testSnapshotEmptyStateMap() { + // take snapshot on an empty state map + CopyOnWriteSkipListStateMapSnapshot snapshot = stateMap.stateSnapshot(); + assertEquals(1, stateMap.getHighestRequiredSnapshotVersionPlusOne()); + assertEquals(1, stateMap.getSnapshotVersions().size()); + assertThat(stateMap.getSnapshotVersions(), contains(1)); + assertEquals(1, stateMap.getResourceGuard().getLeaseCount()); + snapshot.release(); + } + + /** + * Test snapshot empty state map. + */ + @Test + public void testSnapshotClosedStateMap() { + // close state map + stateMap.close(); + try { + stateMap.stateSnapshot(); + fail("Should have thrown exception when trying to snapshot an already closed state map."); + } catch (Exception e) { + // expected + } + } + + /** + * Test snapshot release. + */ + @Test + public void testReleaseSnapshot() { + int expectedSnapshotVersion = 0; + int round = 10; + for (int i = 0; i < round; i++) { + assertEquals(expectedSnapshotVersion, stateMap.getStateMapVersion()); + assertEquals(expectedSnapshotVersion, stateMap.getHighestFinishedSnapshotVersion()); + CopyOnWriteSkipListStateMapSnapshot snapshot = stateMap.stateSnapshot(); + expectedSnapshotVersion++; + snapshot.release(); + assertEquals(0, stateMap.getHighestRequiredSnapshotVersionPlusOne()); + assertTrue(stateMap.getSnapshotVersions().isEmpty()); + assertEquals(0, stateMap.getResourceGuard().getLeaseCount()); + } + } + + /** + * Test state map iterator illegal next call. + */ + @Test + public void testStateMapIteratorIllegalNextInvocation() { + Iterator iterator = stateMap.iterator(); + while (iterator.hasNext()) { + iterator.next(); + } + try { + iterator.next(); + fail("Should have thrown NoSuchElementException."); + } catch (NoSuchElementException e) { + // expected + } + } + + /** + * Test state map iterator illegal next call. + */ + @Test + public void testNamespaceNodeIteratorIllegalNextInvocation() { + SkipListKeySerializer skipListKeySerializer = + new SkipListKeySerializer<>(IntSerializer.INSTANCE, LongSerializer.INSTANCE); + byte[] namespaceBytes = skipListKeySerializer.serializeNamespace(namespace); + MemorySegment namespaceSegment = MemorySegmentFactory.wrap(namespaceBytes); + Iterator iterator = stateMap.new NamespaceNodeIterator(namespaceSegment, 0, namespaceBytes.length); + while (iterator.hasNext()) { + iterator.next(); + } + try { + iterator.next(); + fail("Should have thrown NoSuchElementException."); + } catch (NoSuchElementException e) { + // expected + } + } + + /** + * Test snapshot node iterator illegal next call. + */ + @Test + public void testSnapshotNodeIteratorIllegalNextInvocation() { + CopyOnWriteSkipListStateMapSnapshot snapshot = stateMap.stateSnapshot(); + CopyOnWriteSkipListStateMapSnapshot.SnapshotNodeIterator iterator = snapshot.new SnapshotNodeIterator(false); + while (iterator.hasNext()) { + iterator.next(); + } + try { + iterator.next(); + fail("Should have thrown NoSuchElementException."); + } catch (NoSuchElementException e) { + // expected + } finally { + snapshot.release(); + } + } + +} diff --git a/flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteSkipListStateMapComplexOpTest.java b/flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteSkipListStateMapComplexOpTest.java new file mode 100644 index 000000000000..456fe49197aa --- /dev/null +++ b/flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteSkipListStateMapComplexOpTest.java @@ -0,0 +1,930 @@ +/* + * + * * Licensed to the Apache Software Foundation (ASF) under one + * * or more contributor license agreements. See the NOTICE file + * * distributed with this work for additional information + * * regarding copyright ownership. The ASF licenses this file + * * to you under the Apache License, Version 2.0 (the + * * "License"); you may not use this file except in compliance + * * with the License. You may obtain a copy of the License at + * * + * * http://www.apache.org/licenses/LICENSE-2.0 + * * + * * Unless required by applicable law or agreed to in writing, software + * * distributed under the License is distributed on an "AS IS" BASIS, + * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * * See the License for the specific language governing permissions and + * * limitations under the License. + * + */ + +package org.apache.flink.runtime.state.heap; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.runtime.state.StateSnapshotTransformer; +import org.apache.flink.util.IOUtils; +import org.apache.flink.util.TestLogger; +import org.apache.flink.util.function.TriFunction; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; +import java.util.function.Consumer; + +import static org.apache.flink.runtime.state.heap.CopyOnWriteSkipListStateMap.DEFAULT_MAX_KEYS_TO_DELETE_ONE_TIME; +import static org.apache.flink.runtime.state.heap.CopyOnWriteSkipListStateMapTestUtils.addToReferenceState; +import static org.apache.flink.runtime.state.heap.CopyOnWriteSkipListStateMapTestUtils.createEmptyStateMap; +import static org.apache.flink.runtime.state.heap.CopyOnWriteSkipListStateMapTestUtils.createStateMapForTesting; +import static org.apache.flink.runtime.state.heap.CopyOnWriteSkipListStateMapTestUtils.getAllValuesOfNode; +import static org.apache.flink.runtime.state.heap.CopyOnWriteSkipListStateMapTestUtils.removeFromReferenceState; +import static org.apache.flink.runtime.state.heap.CopyOnWriteSkipListStateMapTestUtils.snapshotReferenceStates; +import static org.apache.flink.runtime.state.heap.CopyOnWriteSkipListStateMapTestUtils.verifySnapshotWithTransform; +import static org.apache.flink.runtime.state.heap.CopyOnWriteSkipListStateMapTestUtils.verifySnapshotWithoutTransform; +import static org.apache.flink.runtime.state.heap.CopyOnWriteSkipListStateMapTestUtils.verifyState; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; + +/** + * Tests for {@link CopyOnWriteSkipListStateMap}. + */ +public class CopyOnWriteSkipListStateMapComplexOpTest extends TestLogger { + + private final TypeSerializer keySerializer = IntSerializer.INSTANCE; + private final TypeSerializer namespaceSerializer = LongSerializer.INSTANCE; + private final TypeSerializer stateSerializer = StringSerializer.INSTANCE; + private final int initNamespaceNumber = 10; + private final int initKeyNumber = 100; + private final int initTotalSize = initNamespaceNumber * initKeyNumber; + private final StateSnapshotTransformer transformer = new StateSnapshotTransformer() { + @Nullable + @Override + public String filterOrTransform(@Nullable String value) { + if (value == null) { + return null; + } + int op = value.hashCode() % 3; + switch (op) { + case 0: + return null; + case 1: + return value + "-transform"; + default: + return value; + } + } + }; + + /** + * We always create a space allocator and state map with some states. Note that Any test case need to test + * a state map from empty state should not use these pre-created instances. + */ + private TestAllocator spaceAllocator; + private CopyOnWriteSkipListStateMap stateMapWithStates; + private Map> referenceStates; + + @Before + public void setUp() { + int maxAllocateSize = 256; + spaceAllocator = new TestAllocator(maxAllocateSize); + // do not remove states physically when get, put, remove and snapshot + stateMapWithStates = createStateMapForTesting(0, 1.0f, spaceAllocator); + referenceStates = new HashMap<>(); + // put some states + for (long namespace = 0; namespace < initNamespaceNumber; namespace++) { + for (int key = 0; key < initKeyNumber; key++) { + String state = String.valueOf(key * namespace); + stateMapWithStates.put(key, namespace, state); + if (referenceStates != null) { + referenceStates.computeIfAbsent(namespace, (none) -> new HashMap<>()).put(key, state); + } + } + } + } + + @After + public void tearDown() { + stateMapWithStates.close(); + referenceStates.clear(); + IOUtils.closeQuietly(spaceAllocator); + } + + /** + * Test remove namespace. + */ + @Test + public void testPurgeNamespace() { + verify(purgeNamespace(false)); + } + + /** + * Test remove namespace. + */ + @Test + public void testPurgeNamespaceWithSnapshot() { + CopyOnWriteSkipListStateMapSnapshot snapshot = stateMapWithStates.stateSnapshot(); + TestExecutionResult result = purgeNamespace(true); + snapshot.release(); + verify(result); + } + + @Nonnull + private TestExecutionResult purgeNamespace(boolean withSnapshot) { + int totalSize = initTotalSize; + int totalSizeIncludingLogicalRemove = totalSize; + int totalSpaceSize = totalSize * 2; + // empty half of the namespaces + Set removedNamespaces = new HashSet<>(); + int cnt = 0; + for (Map.Entry> namespaceEntry : referenceStates.entrySet()) { + if (cnt++ % 2 != 0) { + continue; + } + long namespace = namespaceEntry.getKey(); + removedNamespaces.add(namespace); + for (Map.Entry keyEntry : namespaceEntry.getValue().entrySet()) { + int key = keyEntry.getKey(); + stateMapWithStates.remove(key, namespace); + totalSize--; + if (withSnapshot) { + // logical remove with copy-on-write + totalSpaceSize++; + } else { + // physical remove + totalSizeIncludingLogicalRemove--; + totalSpaceSize -= 2; + } + } + } + + for (long namespace : removedNamespaces) { + referenceStates.remove(namespace); + // verify namespace related stuff. + assertEquals(0, stateMapWithStates.sizeOfNamespace(namespace)); + assertFalse(stateMapWithStates.getKeys(namespace).iterator().hasNext()); + } + + return new TestExecutionResult(totalSize, totalSizeIncludingLogicalRemove, totalSpaceSize); + } + + /** + * Test put -> put during snapshot, the first put should trigger copy-on-write and the second shouldn't. + */ + @Test + public void testPutAndPutWithSnapshot() { + CopyOnWriteSkipListStateMapSnapshot snapshot = stateMapWithStates.stateSnapshot(); + TestExecutionResult result = testPutAndPut(); + snapshot.release(); + verify(result); + } + + @Nonnull + private TestExecutionResult testPutAndPut() { + final int key = 1; + final long namespace = 1L; + final String value = "11"; + final String newValue = "111"; + int totalSize = initTotalSize; + // put (key 1, namespace 1) + int totalSpaceNumber = + putExistingKeyWithSnapshot(totalSize, totalSize * 2, key, namespace, value); + + // put (key 1, namespace 1) again, old value should be replaced and space will not increase + assertEquals("11", stateMapWithStates.putAndGetOld(key, namespace, newValue)); + addToReferenceState(referenceStates, key, namespace, newValue); + assertEquals("111", stateMapWithStates.get(key, namespace)); + assertTrue(stateMapWithStates.containsKey(key, namespace)); + return new TestExecutionResult(totalSize, totalSize, totalSpaceNumber); + } + + /** + * Test put -> remove during snapshot, put should trigger copy-on-write and remove shouldn't. + */ + @Test + public void testPutAndRemoveWithSnapshot() { + CopyOnWriteSkipListStateMapSnapshot snapshot = stateMapWithStates.stateSnapshot(); + TestExecutionResult result = testPutAndRemove(); + snapshot.release(); + verify(result); + } + + @Nonnull + private TestExecutionResult testPutAndRemove() { + final int key = 6; + final long namespace = 6L; + final String value = "66"; + int totalSize = initTotalSize; + int totalLogicallyRemovedKey = 0; + int totalSizeIncludingLogicalRemovedKey = totalSize; + CopyOnWriteSkipListStateMapSnapshot snapshot = null; + + // put (key 6, namespace 6) + int totalSpaceNumber = + putExistingKeyWithSnapshot(totalSize, totalSize * 2, key, namespace, value); + + // remove (key 6, namespace 6), and it should be logically removed + assertEquals(value, stateMapWithStates.removeAndGetOld(key, namespace)); + removeFromReferenceState(referenceStates, key, namespace); + totalSize--; + // with snapshot, it should be a logical remove + totalLogicallyRemovedKey++; + assertNull(stateMapWithStates.get(key, namespace)); + assertFalse(stateMapWithStates.containsKey(key, namespace)); + assertEquals(totalLogicallyRemovedKey, stateMapWithStates.getLogicallyRemovedNodes().size()); + + return new TestExecutionResult(totalSize, totalSizeIncludingLogicalRemovedKey, totalSpaceNumber); + } + + private int putExistingKeyWithSnapshot( + int totalSize, + int initTotalSpaceSize, + int key, + long namespace, + String value) { + + int totalSpaceNumber = initTotalSpaceSize; + stateMapWithStates.put(key, namespace, value); + addToReferenceState(referenceStates, key, namespace, value); + // copy-on-write should happen, a space for new value should be allocated + totalSpaceNumber += 1; + assertEquals(totalSpaceNumber, spaceAllocator.getTotalSpaceNumber()); + assertEquals(value, stateMapWithStates.get(key, namespace)); + assertTrue(stateMapWithStates.containsKey(key, namespace)); + assertThat(stateMapWithStates.size(), is(totalSize)); + verifyState(referenceStates, stateMapWithStates); + return totalSpaceNumber; + } + + /** + * Test remove -> put during snapshot, remove should trigger copy-on-write and put shouldn't. + */ + @Test + public void testRemoveAndPutWithSnapshot() { + CopyOnWriteSkipListStateMapSnapshot snapshot = stateMapWithStates.stateSnapshot(); + TestExecutionResult result = testRemoveAndPut(); + snapshot.release(); + verify(result); + } + + @Nonnull + private TestExecutionResult testRemoveAndPut() { + final int key = 8; + final long namespace = 8L; + final String value = "64"; + int totalSize = initTotalSize; + // remove (key 8, namespace 8) + RemoveResult result = + removeExistingKeyWithSnapshot(totalSize, stateMapWithStates, referenceStates, + totalSize * 2, key, namespace, value); + totalSize--; + int totalLogicallyRemovedKey = result.totalLogicallyRemovedKey; + int totalSizeIncludingLogicalRemovedKey = result.totalSizeIncludingLogicalRemovedKey; + int totalSpaceNumber = result.totalSpaceNumber; + + // put (key 8, namespace 8) again + assertNull(stateMapWithStates.putAndGetOld(key, namespace, value)); + addToReferenceState(referenceStates, key, namespace, value); + totalSize++; + totalLogicallyRemovedKey--; + assertEquals(value, stateMapWithStates.get(key, namespace)); + assertTrue(stateMapWithStates.containsKey(key, namespace)); + assertEquals(totalLogicallyRemovedKey, stateMapWithStates.getLogicallyRemovedNodes().size()); + return new TestExecutionResult(totalSize, totalSizeIncludingLogicalRemovedKey, totalSpaceNumber); + } + + /** + * Test remove -> remove during snapshot, the first remove should trigger copy-on-write and the second shouldn't. + */ + @Test + public void testRemoveAndRemoveWithSnapshot() { + final int key = 4; + final long namespace = 4L; + final String value = "16"; + int totalSize = initTotalSize; + CopyOnWriteSkipListStateMapSnapshot snapshot = stateMapWithStates.stateSnapshot(); + + // remove (key 4, namespace 4) + RemoveResult result = + removeExistingKeyWithSnapshot(totalSize, stateMapWithStates, referenceStates, + totalSize * 2, key, namespace, value); + totalSize--; + int totalLogicallyRemovedKey = result.totalLogicallyRemovedKey; + int totalSizeIncludingLogicalRemovedKey = result.totalSizeIncludingLogicalRemovedKey; + int totalSpaceNumber = result.totalSpaceNumber; + + // remove (key 4, namespace 4) again, and nothing should happen + assertNull(stateMapWithStates.removeAndGetOld(key, namespace)); + assertEquals(totalLogicallyRemovedKey, stateMapWithStates.getLogicallyRemovedNodes().size()); + + snapshot.release(); + + // remove (key 4, namespace 4) again after snapshot released, should be physically removed + assertNull(stateMapWithStates.removeAndGetOld(key, namespace)); + totalLogicallyRemovedKey--; + totalSizeIncludingLogicalRemovedKey--; + totalSpaceNumber -= 3; // key, value and one copy-on-write by remove + assertEquals(totalLogicallyRemovedKey, stateMapWithStates.getLogicallyRemovedNodes().size()); + + verify(new TestExecutionResult(totalSize, totalSizeIncludingLogicalRemovedKey, totalSpaceNumber)); + } + + @Nonnull + private RemoveResult removeExistingKeyWithSnapshot( + int totalSize, + @Nonnull CopyOnWriteSkipListStateMap stateMap, + Map> referenceStates, + int initTotalSpaceSize, + int key, + long namespace, + String value) { + + int totalLogicallyRemovedKey = 0; + int totalSpaceNumber = initTotalSpaceSize; + int totalSizeIncludingLogicalRemovedKey = totalSize; + assertEquals(value, stateMap.removeAndGetOld(key, namespace)); + removeFromReferenceState(referenceStates, key, namespace); + totalSize--; + // with snapshot, it should be a logical remove with copy-on-write + totalLogicallyRemovedKey++; + totalSpaceNumber += 1; + assertNull(stateMap.get(key, namespace)); + assertFalse(stateMap.containsKey(key, namespace)); + assertThat(stateMap.size(), is(totalSize)); + assertEquals(totalLogicallyRemovedKey, stateMap.getLogicallyRemovedNodes().size()); + assertEquals(totalSpaceNumber, spaceAllocator.getTotalSpaceNumber()); + assertEquals(totalSizeIncludingLogicalRemovedKey, stateMap.totalSize()); + verifyState(referenceStates, stateMap); + return new RemoveResult(totalLogicallyRemovedKey, totalSizeIncludingLogicalRemovedKey, totalSpaceNumber); + } + + /** + * Test snapshot unmodifiable. + */ + @Test + public void testSnapshotUnmodifiable() throws IOException { + final int key = 1; + final int newKey = 101; + final long namespace = 1L; + final long newNamespace = initNamespaceNumber + 1; + final String newValue = "11"; + CopyOnWriteSkipListStateMapSnapshot snapshot = stateMapWithStates.stateSnapshot(); + Map> expectedSnapshot = snapshotReferenceStates(referenceStates); + // make sure update existing key won't change snapshot + processAndVerifySnapshot(expectedSnapshot, snapshot, stateMapWithStates, map -> map.put(key, namespace, newValue)); + // make sure insert new key won't change snapshot + processAndVerifySnapshot(expectedSnapshot, snapshot, stateMapWithStates, map -> map.put(newKey, newNamespace, newValue)); + // make sure remove existing key won't change snapshot + processAndVerifySnapshot(expectedSnapshot, snapshot, stateMapWithStates, map -> map.remove(key, namespace)); + snapshot.release(); + stateMapWithStates.close(); + } + + private void processAndVerifySnapshot( + Map> expectedSnapshot, + CopyOnWriteSkipListStateMapSnapshot snapshot, + CopyOnWriteSkipListStateMap stateMap, + @Nonnull Consumer> consumer) throws IOException { + consumer.accept(stateMap); + verifySnapshotWithoutTransform( + expectedSnapshot, snapshot, keySerializer, namespaceSerializer, stateSerializer); + } + + /** + * Tests that remove states physically when get is invoked. + */ + @Test + public void testPhysicallyRemoveWithGet() throws IOException { + testPhysicallyRemoveWithFunction( + (map, reference, i) -> { + map.get(i, (long) i); + return 0; + }); + } + + /** + * Tests that remove states physically when contains is invoked. + */ + @Test + public void testPhysicallyRemoveWithContains() throws IOException { + testPhysicallyRemoveWithFunction( + (map, reference, i) -> { + assertFalse(map.containsKey(i, (long) i)); + return 0; + }); + } + + /** + * Tests that remove states physically when remove is invoked. + */ + @Test + public void testPhysicallyRemoveWithRemove() throws IOException { + testPhysicallyRemoveWithFunction( + (map, reference, i) -> { + map.remove(i, (long) i); + return 0; + }); + } + + /** + * Tests that remove states physically when removeAndGetOld is invoked. + */ + @Test + public void testPhysicallyRemoveWithRemoveAndGetOld() throws IOException { + testPhysicallyRemoveWithFunction( + (map, reference, i) -> { + assertNull(map.removeAndGetOld(i, (long) i)); + return 0; + }); + } + + /** + * Tests that remove states physically when put is invoked. + */ + @Test + public void testPhysicallyRemoveWithPut() throws IOException { + testPhysicallyRemoveWithFunction( + (map, reference, i) -> { + map.put(i, (long) i, String.valueOf(i)); + addToReferenceState(reference, i, (long) i, String.valueOf(i)); + return 1; + }); + } + + /** + * Tests that remove states physically when putAndGetOld is invoked. + */ + @Test + public void testPhysicallyRemoveWithPutAndGetOld() throws IOException { + testPhysicallyRemoveWithFunction( + (map, reference, i) -> { + assertNull(map.putAndGetOld(i, (long) i, String.valueOf(i))); + addToReferenceState(reference, i, (long) i, String.valueOf(i)); + return 1; + }); + } + + /** + * Tests remove states physically when the given function is applied. + * + * @param f the function to apply for each test iteration, with [stateMap, referenceStates, testRoundIndex] as input + * and returns the delta size caused by applying function. + * @throws IOException if unexpected error occurs. + */ + private void testPhysicallyRemoveWithFunction( + TriFunction< + CopyOnWriteSkipListStateMap, + Map>, + Integer, + Integer> f) throws IOException { + + TestAllocator spaceAllocator = new TestAllocator(256); + CopyOnWriteSkipListStateMap stateMap = + createEmptyStateMap(2, 1.0f, spaceAllocator); + // map to store expected states, namespace -> key -> state + Map> referenceStates = new HashMap<>(); + + // here we use a trick that put all odd namespace to state map, and get/put/remove even namespace + // so that all logically removed nodes can be accessed + prepareLogicallyRemovedStates( + referenceStates, stateMap, keySerializer, namespaceSerializer, stateSerializer); + int expectedSize = 0; + for (int i = 0; i <= 100; i += 2) { + expectedSize += f.apply(stateMap, referenceStates, i); + } + assertEquals(expectedSize, stateMap.size()); + assertEquals(expectedSize, stateMap.totalSize()); + assertEquals(expectedSize * 2, spaceAllocator.getTotalSpaceNumber()); + verifyState(referenceStates, stateMap); + stateMap.close(); + } + + /** + * Tests that remove states physically during sync part of snapshot. + */ + @Test + public void testPhysicallyRemoveDuringSyncPartOfSnapshot() throws IOException { + TestAllocator spaceAllocator = new TestAllocator(256); + // set logicalRemovedKeysRatio to 0 so that all logically removed states will be deleted when snapshot + CopyOnWriteSkipListStateMap stateMap = + createEmptyStateMap(0, 0.0f, spaceAllocator); + + // map to store expected states, namespace -> key -> state + Map> referenceStates = new HashMap<>(); + int totalStateSize = 0; + int totalSizeIncludingLogicallyRemovedStates = 0; + + // put some states + for (int i = 1; i <= 100; i++) { + totalStateSize++; + totalSizeIncludingLogicallyRemovedStates++; + stateMap.put(i, (long) i, String.valueOf(i)); + addToReferenceState(referenceStates, i, (long) i, String.valueOf(i)); + } + verifyState(referenceStates, stateMap); + + assertEquals(totalStateSize * 2, spaceAllocator.getTotalSpaceNumber()); + + Map> expectedSnapshot1 = snapshotReferenceStates(referenceStates); + CopyOnWriteSkipListStateMapSnapshot snapshot1 = stateMap.stateSnapshot(); + + // remove all states logically + for (int i = 1; i <= 100; i++) { + totalStateSize--; + stateMap.remove(i, (long) i); + removeFromReferenceState(referenceStates, i, (long) i); + } + assertEquals(totalSizeIncludingLogicallyRemovedStates * 3, spaceAllocator.getTotalSpaceNumber()); + assertEquals(0, totalStateSize); + assertEquals(totalStateSize, stateMap.size()); + assertEquals(totalSizeIncludingLogicallyRemovedStates, stateMap.totalSize()); + assertEquals(totalSizeIncludingLogicallyRemovedStates, stateMap.getLogicallyRemovedNodes().size()); + verifyState(referenceStates, stateMap); + + verifySnapshotWithoutTransform( + expectedSnapshot1, snapshot1, keySerializer, namespaceSerializer, stateSerializer); + snapshot1.release(); + + // no spaces should be free + assertEquals(totalSizeIncludingLogicallyRemovedStates * 3, spaceAllocator.getTotalSpaceNumber()); + verifyState(referenceStates, stateMap); + + Map> expectedSnapshot2 = snapshotReferenceStates(referenceStates); + CopyOnWriteSkipListStateMapSnapshot snapshot2 = stateMap.stateSnapshot(); + + // all state should be removed physically + int totalSizeIncludingLogicallyRemovedStatesAfterSecondSnapshot = 0; + assertEquals(totalStateSize, stateMap.totalSize()); + assertEquals(totalSizeIncludingLogicallyRemovedStatesAfterSecondSnapshot, stateMap.totalSize()); + assertEquals(totalSizeIncludingLogicallyRemovedStatesAfterSecondSnapshot, + stateMap.getLogicallyRemovedNodes().size()); + assertEquals(0, spaceAllocator.getTotalSpaceNumber()); + + verifySnapshotWithoutTransform( + expectedSnapshot2, snapshot2, keySerializer, namespaceSerializer, stateSerializer); + snapshot2.release(); + + assertEquals(0, stateMap.size()); + assertEquals(0, stateMap.totalSize()); + assertEquals(0, spaceAllocator.getTotalSpaceNumber()); + assertFalse(stateMap.iterator().hasNext()); + + stateMap.close(); + } + + /** + * Tests that snapshots prune useless values. + */ + @Test + public void testSnapshotPruneValues() throws IOException { + TestAllocator spaceAllocator = new TestAllocator(256); + // set logicalRemovedKeysRatio to 0 so that all logically removed states will be deleted when snapshot + CopyOnWriteSkipListStateMap stateMap = + createEmptyStateMap(DEFAULT_MAX_KEYS_TO_DELETE_ONE_TIME, 0.0f, spaceAllocator); + + // firstly build value chain and snapshots as follows + // ------ ------ ------ ------ + // | v3 | -> | v2 | -> | v1 | -> | v0 | + // ------ ------ ------ ------ + // | | | | + // snapshot4 snapshot3 snapshot2 snapshot1 + // snapshot5 + // snapshot6 + + List referenceValues = new ArrayList<>(); + + // build v0 + stateMap.put(1, 1L, "0"); + referenceValues.add(0, "0"); + // get the pointer to the node + long node = stateMap.getLevelIndexHeader().getNextNode(0); + + // take snapshot1 + CopyOnWriteSkipListStateMapSnapshot snapshot1 = stateMap.stateSnapshot(); + + // build v1 + stateMap.put(1, 1L, "1"); + referenceValues.add(0, "1"); + + // take snapshot2 + CopyOnWriteSkipListStateMapSnapshot snapshot2 = stateMap.stateSnapshot(); + + // build v2 + stateMap.put(1, 1L, "2"); + referenceValues.add(0, "2"); + + // take snapshot3 + CopyOnWriteSkipListStateMapSnapshot snapshot3 = stateMap.stateSnapshot(); + + // build v3 + stateMap.put(1, 1L, "3"); + referenceValues.add(0, "3"); + + // take snapshot4 + CopyOnWriteSkipListStateMapSnapshot snapshot4 = stateMap.stateSnapshot(); + + // take snapshot5 + CopyOnWriteSkipListStateMapSnapshot snapshot5 = stateMap.stateSnapshot(); + + // take snapshot6 + CopyOnWriteSkipListStateMapSnapshot snapshot6 = stateMap.stateSnapshot(); + + assertEquals(6, stateMap.getStateMapVersion()); + assertEquals(6, stateMap.getHighestRequiredSnapshotVersionPlusOne()); + assertEquals(6, stateMap.getSnapshotVersions().size()); + assertEquals(5, spaceAllocator.getTotalSpaceNumber()); + assertEquals(referenceValues, getAllValuesOfNode(stateMap, spaceAllocator, node)); + + Map> referenceStates = new HashMap<>(); + referenceStates.put(1L, new HashMap<>()); + referenceStates.get(1L).put(1, "0"); + + // complete snapshot 1, and no value will be removed + verifySnapshotWithoutTransform( + referenceStates, snapshot1, keySerializer, namespaceSerializer, stateSerializer); + snapshot1.release(); + assertEquals(referenceValues, getAllValuesOfNode(stateMap, spaceAllocator, node)); + assertEquals(5, spaceAllocator.getTotalSpaceNumber()); + assertEquals(1, stateMap.getHighestFinishedSnapshotVersion()); + + // complete snapshot 3, and v0 will be removed + referenceStates.get(1L).put(1, "2"); + verifySnapshotWithoutTransform( + referenceStates, snapshot3, keySerializer, namespaceSerializer, stateSerializer); + snapshot3.release(); + referenceValues.remove(referenceValues.size() - 1); + assertEquals(referenceValues, getAllValuesOfNode(stateMap, spaceAllocator, node)); + assertEquals(4, spaceAllocator.getTotalSpaceNumber()); + assertEquals(1, stateMap.getHighestFinishedSnapshotVersion()); + + // complete snapshot 2, and no value will be removed + referenceStates.get(1L).put(1, "1"); + verifySnapshotWithoutTransform( + referenceStates, snapshot2, keySerializer, namespaceSerializer, stateSerializer); + snapshot2.release(); + assertEquals(referenceValues, getAllValuesOfNode(stateMap, spaceAllocator, node)); + assertEquals(4, spaceAllocator.getTotalSpaceNumber()); + assertEquals(3, stateMap.getHighestFinishedSnapshotVersion()); + + // add node to pruning set to prevent snapshot4 to prune + stateMap.getPruningValueNodes().add(node); + // complete snapshot 4, and no value will be removed + referenceStates.get(1L).put(1, "3"); + verifySnapshotWithoutTransform( + referenceStates, snapshot4, keySerializer, namespaceSerializer, stateSerializer); + snapshot4.release(); + assertEquals(referenceValues, getAllValuesOfNode(stateMap, spaceAllocator, node)); + assertEquals(4, spaceAllocator.getTotalSpaceNumber()); + assertEquals(4, stateMap.getHighestFinishedSnapshotVersion()); + + stateMap.getPruningValueNodes().remove(node); + + // complete snapshot 5, v1 and v2 will be removed + verifySnapshotWithoutTransform( + referenceStates, snapshot5, keySerializer, namespaceSerializer, stateSerializer); + snapshot5.release(); + referenceValues.remove(referenceValues.size() - 1); + referenceValues.remove(referenceValues.size() - 1); + assertEquals(referenceValues, getAllValuesOfNode(stateMap, spaceAllocator, node)); + assertEquals(2, spaceAllocator.getTotalSpaceNumber()); + assertEquals(5, stateMap.getHighestFinishedSnapshotVersion()); + + // complete snapshot 6, no value will be removed + verifySnapshotWithoutTransform( + referenceStates, snapshot6, keySerializer, namespaceSerializer, stateSerializer); + snapshot6.release(); + assertEquals(referenceValues, getAllValuesOfNode(stateMap, spaceAllocator, node)); + assertEquals(2, spaceAllocator.getTotalSpaceNumber()); + assertEquals(6, stateMap.getHighestFinishedSnapshotVersion()); + + assertEquals("3", stateMap.removeAndGetOld(1, 1L)); + assertEquals(0 , stateMap.size()); + assertEquals(0, spaceAllocator.getTotalSpaceNumber()); + + stateMap.close(); + } + + /** + * Tests concurrent snapshots. + */ + @Test + public void testConcurrentSnapshots() throws IOException { + TestAllocator spaceAllocator = new TestAllocator(256); + // set logicalRemovedKeysRatio to 0 so that all logically removed states will be deleted when snapshot + CopyOnWriteSkipListStateMap stateMap = + createEmptyStateMap(DEFAULT_MAX_KEYS_TO_DELETE_ONE_TIME, 0.0f, spaceAllocator); + + // snapshot order: create snapshot1 -> update states -> create snapshot2 -> update states + // -> create snapshot3 -> update states -> complete snapshot2 -> update states -> complete snapshot1 + // -> create snapshot4 -> update states -> complete snapshot3 -> update states -> complete snapshot4 + // -> create snapshot5 -> complete snapshot5 + + // map to store expected states, namespace -> key -> state + Map> referenceStates = new HashMap<>(); + + // update states + updateStateForConcurrentSnapshots(referenceStates, stateMap, 0); + verifyState(referenceStates, stateMap); + + // create snapshot1 + Map> expectedSnapshot1 = snapshotReferenceStates(referenceStates); + CopyOnWriteSkipListStateMapSnapshot snapshot1 = stateMap.stateSnapshot(); + + // update states + updateStateForConcurrentSnapshots(referenceStates, stateMap, 1); + verifyState(referenceStates, stateMap); + + // create snapshot2 + Map> expectedSnapshot2 = snapshotReferenceStates(referenceStates); + CopyOnWriteSkipListStateMapSnapshot snapshot2 = stateMap.stateSnapshot(); + + // update states + updateStateForConcurrentSnapshots(referenceStates, stateMap, 2); + verifyState(referenceStates, stateMap); + + // create snapshot3 + Map> expectedSnapshot3 = snapshotReferenceStates(referenceStates); + CopyOnWriteSkipListStateMapSnapshot snapshot3 = stateMap.stateSnapshot(); + + // update states + updateStateForConcurrentSnapshots(referenceStates, stateMap, 3); + verifyState(referenceStates, stateMap); + + // complete snapshot2 + verifySnapshotWithTransform( + expectedSnapshot2, snapshot2, transformer, keySerializer, namespaceSerializer, stateSerializer); + snapshot2.release(); + + // update states + updateStateForConcurrentSnapshots(referenceStates, stateMap, 4); + verifyState(referenceStates, stateMap); + + // complete snapshot1 + verifySnapshotWithoutTransform( + expectedSnapshot1, snapshot1, keySerializer, namespaceSerializer, stateSerializer); + snapshot1.release(); + + // create snapshot4 + Map> expectedSnapshot4 = snapshotReferenceStates(referenceStates); + CopyOnWriteSkipListStateMapSnapshot snapshot4 = stateMap.stateSnapshot(); + + // update states + updateStateForConcurrentSnapshots(referenceStates, stateMap, 5); + verifyState(referenceStates, stateMap); + + // complete snapshot3 + verifySnapshotWithTransform( + expectedSnapshot3, snapshot3, transformer, keySerializer, namespaceSerializer, stateSerializer); + snapshot3.release(); + + // update states + updateStateForConcurrentSnapshots(referenceStates, stateMap, 6); + verifyState(referenceStates, stateMap); + + // complete snapshot4 + verifySnapshotWithTransform( + expectedSnapshot4, snapshot4, transformer, keySerializer, namespaceSerializer, stateSerializer); + snapshot4.release(); + + verifyState(referenceStates, stateMap); + + // create snapshot5 + Map> expectedSnapshot5 = snapshotReferenceStates(referenceStates); + CopyOnWriteSkipListStateMapSnapshot snapshot5 = stateMap.stateSnapshot(); + + // complete snapshot5 + verifySnapshotWithTransform( + expectedSnapshot5, snapshot5, transformer, keySerializer, namespaceSerializer, stateSerializer); + snapshot5.release(); + + verifyState(referenceStates, stateMap); + + stateMap.close(); + } + + private void prepareLogicallyRemovedStates( + Map> referenceStates, + CopyOnWriteSkipListStateMap stateMap, + TypeSerializer keySerializer, + TypeSerializer namespaceSerializer, + TypeSerializer stateSerializer) throws IOException { + // put some states + for (int i = 1; i <= 100; i += 2) { + stateMap.put(i, (long) i, String.valueOf(i)); + addToReferenceState(referenceStates, i, (long) i, String.valueOf(i)); + } + verifyState(referenceStates, stateMap); + + Map> expectedSnapshot1 = snapshotReferenceStates(referenceStates); + CopyOnWriteSkipListStateMapSnapshot snapshot1 = stateMap.stateSnapshot(); + + // remove all states logically + for (int i = 1; i <= 100; i += 2) { + stateMap.remove(i, (long) i); + removeFromReferenceState(referenceStates, i, (long) i); + } + verifyState(referenceStates, stateMap); + + verifySnapshotWithoutTransform( + expectedSnapshot1, snapshot1, keySerializer, namespaceSerializer, stateSerializer); + snapshot1.release(); + } + + private void updateStateForConcurrentSnapshots( + @Nonnull Map> referenceStates, + CopyOnWriteSkipListStateMap stateMap, + int updateCounter) { + ThreadLocalRandom random = ThreadLocalRandom.current(); + // update and remove some states + int op = 0; + Set removedNamespaceSet = new HashSet<>(); + for (Map.Entry> namespaceEntry : referenceStates.entrySet()) { + long namespace = namespaceEntry.getKey(); + for (Map.Entry keyEntry : namespaceEntry.getValue().entrySet()) { + int key = keyEntry.getKey(); + // namespace should be equal to key + assertEquals(namespace, key); + switch (op) { + case 0: + stateMap.remove(key, namespace); + removedNamespaceSet.add(namespace); + op = 1; + break; + case 1: + String state = String.valueOf(random.nextLong()); + stateMap.put(key, namespace, state); + keyEntry.setValue(state); + op = 2; + break; + default: + op = 0; + break; + } + } + } + for (long namespace : removedNamespaceSet) { + referenceStates.remove(namespace); + } + + for (int i = 0; i < 100; i++) { + int key = updateCounter + i * 50; + long namespace = key; + String state = String.valueOf(key * namespace); + stateMap.put(key, namespace, state); + addToReferenceState(referenceStates, key, namespace, state); + } + } + + private void verify(@Nonnull TestExecutionResult result) { + assertEquals(result.totalSize, stateMapWithStates.size()); + assertEquals(result.totalSizeIncludingLogicalRemove, stateMapWithStates.totalSize()); + assertEquals(result.totalSpaceSize, spaceAllocator.getTotalSpaceNumber()); + verifyState(referenceStates, stateMapWithStates); + } + + private class TestExecutionResult { + final int totalSize; + final int totalSizeIncludingLogicalRemove; + final int totalSpaceSize; + + private TestExecutionResult( + int totalSize, + int totalSizeIncludingLogicalRemove, + int totalSpaceSize) { + + this.totalSize = totalSize; + this.totalSizeIncludingLogicalRemove = totalSizeIncludingLogicalRemove; + this.totalSpaceSize = totalSpaceSize; + } + } + + private class RemoveResult { + final int totalLogicallyRemovedKey; + final int totalSizeIncludingLogicalRemovedKey; + final int totalSpaceNumber; + + private RemoveResult(int totalLogicallyRemovedKey, int totalSizeIncludingLogicalRemovedKey, int totalSpaceNumber) { + this.totalLogicallyRemovedKey = totalLogicallyRemovedKey; + this.totalSpaceNumber = totalSpaceNumber; + this.totalSizeIncludingLogicalRemovedKey = totalSizeIncludingLogicalRemovedKey; + } + } + +} diff --git a/flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteSkipListStateMapTestUtils.java b/flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteSkipListStateMapTestUtils.java new file mode 100644 index 000000000000..ba75241825c2 --- /dev/null +++ b/flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteSkipListStateMapTestUtils.java @@ -0,0 +1,259 @@ +/* + * + * * Licensed to the Apache Software Foundation (ASF) under one + * * or more contributor license agreements. See the NOTICE file + * * distributed with this work for additional information + * * regarding copyright ownership. The ASF licenses this file + * * to you under the Apache License, Version 2.0 (the + * * "License"); you may not use this file except in compliance + * * with the License. You may obtain a copy of the License at + * * + * * http://www.apache.org/licenses/LICENSE-2.0 + * * + * * Unless required by applicable law or agreed to in writing, software + * * distributed under the License is distributed on an "AS IS" BASIS, + * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * * See the License for the specific language governing permissions and + * * limitations under the License. + * + */ + +package org.apache.flink.runtime.state.heap; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.core.memory.ByteArrayInputStreamWithPos; +import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.runtime.state.StateEntry; +import org.apache.flink.runtime.state.StateSnapshotTransformer; +import org.apache.flink.runtime.state.heap.space.Allocator; +import org.apache.flink.runtime.state.internal.InternalKvState; + +import javax.annotation.Nonnull; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.apache.flink.runtime.state.heap.CopyOnWriteSkipListStateMap.DEFAULT_LOGICAL_REMOVED_KEYS_RATIO; +import static org.apache.flink.runtime.state.heap.CopyOnWriteSkipListStateMap.DEFAULT_MAX_KEYS_TO_DELETE_ONE_TIME; +import static org.apache.flink.runtime.state.heap.SkipListUtils.NIL_NODE; +import static org.hamcrest.Matchers.nullValue; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; + +/** + * Utils for CopyOnWriteSkipListStateMap test. + */ +class CopyOnWriteSkipListStateMapTestUtils { + @Nonnull + static CopyOnWriteSkipListStateMap createEmptyStateMap( + int keysToDelete, + float logicalKeysRemoveRatio, + Allocator spaceAllocator) { + return createStateMapForTesting( + keysToDelete, + logicalKeysRemoveRatio, + spaceAllocator); + } + + @Nonnull + static CopyOnWriteSkipListStateMap createEmptyStateMap() { + return createStateMapForTesting( + DEFAULT_MAX_KEYS_TO_DELETE_ONE_TIME, + DEFAULT_LOGICAL_REMOVED_KEYS_RATIO, + new TestAllocator(256)); + } + + @Nonnull + static CopyOnWriteSkipListStateMap createStateMapForTesting( + int keysToDelete, + float logicalKeysRemoveRatio, + Allocator spaceAllocator) { + return new CopyOnWriteSkipListStateMap<>( + IntSerializer.INSTANCE, + LongSerializer.INSTANCE, + StringSerializer.INSTANCE, + spaceAllocator, + keysToDelete, + logicalKeysRemoveRatio); + } + + static void verifyState( + @Nonnull Map> referenceStates, + @Nonnull CopyOnWriteSkipListStateMap stateMap) { + + // validates get(K, N) + for (Map.Entry> entry : referenceStates.entrySet()) { + N namespace = entry.getKey(); + for (Map.Entry keyEntry : entry.getValue().entrySet()) { + K key = keyEntry.getKey(); + S state = keyEntry.getValue(); + assertEquals(state, stateMap.get(key, namespace)); + assertTrue(stateMap.containsKey(key, namespace)); + } + } + + // validates getKeys(N) and sizeOfNamespace(N) + for (Map.Entry> entry : referenceStates.entrySet()) { + N namespace = entry.getKey(); + Set expectedKeySet = new HashSet<>(entry.getValue().keySet()); + assertEquals(expectedKeySet.size(), stateMap.sizeOfNamespace(namespace)); + Iterator keyIterator = stateMap.getKeys(namespace).iterator(); + while (keyIterator.hasNext()) { + K key = keyIterator.next(); + assertTrue(expectedKeySet.remove(key)); + } + assertTrue(expectedKeySet.isEmpty()); + } + + // validates iterator() + Map> actualStates = new HashMap<>(); + Iterator> iterator = stateMap.iterator(); + while (iterator.hasNext()) { + StateEntry entry = iterator.next(); + S oldState = actualStates.computeIfAbsent(entry.getNamespace(), (none) -> new HashMap<>()) + .put(entry.getKey(), entry.getState()); + assertNull(oldState); + } + referenceStates.forEach( + (ns, kvMap) -> { + if (kvMap.isEmpty()) { + assertThat(actualStates.get(ns), nullValue()); + } else { + assertEquals(kvMap, actualStates.get(ns)); + } + }); + + // validates getStateIncrementalVisitor() + InternalKvState.StateIncrementalVisitor visitor = + stateMap.getStateIncrementalVisitor(2); + actualStates.clear(); + while (visitor.hasNext()) { + Collection> collection = visitor.nextEntries(); + for (StateEntry entry : collection) { + S oldState = actualStates.computeIfAbsent(entry.getNamespace(), (none) -> new HashMap<>()) + .put(entry.getKey(), entry.getState()); + assertNull(oldState); + } + } + referenceStates.forEach( + (ns, kvMap) -> { + if (kvMap.isEmpty()) { + assertThat(actualStates.get(ns), nullValue()); + } else { + assertEquals(kvMap, actualStates.get(ns)); + } + }); + } + + static void addToReferenceState(@Nonnull Map> referenceStates, K key, N namespace, S state) { + referenceStates.computeIfAbsent(namespace, (none) -> new HashMap<>()).put(key, state); + } + + static void removeFromReferenceState(@Nonnull Map> referenceStates, K key, N namespace) { + Map keyMap = referenceStates.get(namespace); + if (keyMap == null) { + return; + } + keyMap.remove(key); + if (keyMap.isEmpty()) { + referenceStates.remove(namespace); + } + } + + static void verifySnapshotWithoutTransform( + Map> referenceStates, + @Nonnull CopyOnWriteSkipListStateMapSnapshot snapshot, + TypeSerializer keySerializer, + TypeSerializer namespaceSerializer, + TypeSerializer stateSerializer) throws IOException { + ByteArrayOutputStreamWithPos outputStream = new ByteArrayOutputStreamWithPos(); + DataOutputView outputView = new DataOutputViewStreamWrapper(outputStream); + snapshot.writeState(keySerializer, namespaceSerializer, stateSerializer, outputView, null); + + Map> actualStates = readStateFromSnapshot( + outputStream.toByteArray(), keySerializer, namespaceSerializer, stateSerializer); + assertEquals(referenceStates, actualStates); + } + + static void verifySnapshotWithTransform( + @Nonnull Map> referenceStates, + @Nonnull CopyOnWriteSkipListStateMapSnapshot snapshot, + StateSnapshotTransformer transformer, + TypeSerializer keySerializer, + TypeSerializer namespaceSerializer, + TypeSerializer stateSerializer) throws IOException { + ByteArrayOutputStreamWithPos outputStream = new ByteArrayOutputStreamWithPos(); + DataOutputView outputView = new DataOutputViewStreamWrapper(outputStream); + snapshot.writeState(keySerializer, namespaceSerializer, stateSerializer, outputView, transformer); + + Map> transformedStates = new HashMap<>(); + for (Map.Entry> namespaceEntry : referenceStates.entrySet()) { + for (Map.Entry keyEntry : namespaceEntry.getValue().entrySet()) { + S state = transformer.filterOrTransform(keyEntry.getValue()); + if (state != null) { + transformedStates.computeIfAbsent(namespaceEntry.getKey(), (none) -> new HashMap<>()) + .put(keyEntry.getKey(), state); + } + } + } + + Map> actualStates = readStateFromSnapshot( + outputStream.toByteArray(), keySerializer, namespaceSerializer, stateSerializer); + assertEquals(transformedStates, actualStates); + } + + private static Map> readStateFromSnapshot( + byte[] data, + TypeSerializer keySerializer, + TypeSerializer namespaceSerializer, + TypeSerializer stateSerializer) throws IOException { + ByteArrayInputStreamWithPos inputStream = new ByteArrayInputStreamWithPos(data); + DataInputView dataInputView = new DataInputViewStreamWrapper(inputStream); + int size = dataInputView.readInt(); + + Map> states = new HashMap<>(); + for (int i = 0; i < size; i++) { + N namespace = namespaceSerializer.deserialize(dataInputView); + K key = keySerializer.deserialize(dataInputView); + S state = stateSerializer.deserialize(dataInputView); + states.computeIfAbsent(namespace, (none) -> new HashMap<>()).put(key, state); + } + + return states; + } + + static Map> snapshotReferenceStates(@Nonnull Map> referenceStates) { + Map> snapshot = new HashMap<>(); + referenceStates.forEach((namespace, keyMap) -> snapshot.put(namespace, new HashMap<>(keyMap))); + return snapshot; + } + + static List getAllValuesOfNode( + CopyOnWriteSkipListStateMap stateMap, + Allocator spaceAllocator, + long node) { + List values = new ArrayList<>(); + long valuePointer = SkipListUtils.helpGetValuePointer(node, spaceAllocator); + while (valuePointer != NIL_NODE) { + values.add(stateMap.helpGetState(valuePointer)); + valuePointer = SkipListUtils.helpGetNextValuePointer(valuePointer, spaceAllocator); + } + return values; + } +} diff --git a/flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/OnHeapLevelIndexHeaderTest.java b/flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/OnHeapLevelIndexHeaderTest.java new file mode 100644 index 000000000000..ea8faa2c0a85 --- /dev/null +++ b/flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/OnHeapLevelIndexHeaderTest.java @@ -0,0 +1,169 @@ +/* + * + * * Licensed to the Apache Software Foundation (ASF) under one + * * or more contributor license agreements. See the NOTICE file + * * distributed with this work for additional information + * * regarding copyright ownership. The ASF licenses this file + * * to you under the Apache License, Version 2.0 (the + * * "License"); you may not use this file except in compliance + * * with the License. You may obtain a copy of the License at + * * + * * http://www.apache.org/licenses/LICENSE-2.0 + * * + * * Unless required by applicable law or agreed to in writing, software + * * distributed under the License is distributed on an "AS IS" BASIS, + * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * * See the License for the specific language governing permissions and + * * limitations under the License. + * + */ + +package org.apache.flink.runtime.state.heap; + +import org.apache.flink.util.TestLogger; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.concurrent.ThreadLocalRandom; + +import static org.apache.flink.runtime.state.heap.SkipListUtils.DEFAULT_LEVEL; +import static org.apache.flink.runtime.state.heap.SkipListUtils.MAX_LEVEL; +import static org.apache.flink.runtime.state.heap.SkipListUtils.NIL_NODE; + +/** + * Tests for {@link OnHeapLevelIndexHeader}. + */ +public class OnHeapLevelIndexHeaderTest extends TestLogger { + private static final ThreadLocalRandom random = ThreadLocalRandom.current(); + + private OnHeapLevelIndexHeader heapHeadIndex; + + @Before + public void setUp() { + heapHeadIndex = new OnHeapLevelIndexHeader(); + } + + @Test + public void testInitStatus() { + Assert.assertEquals(1, heapHeadIndex.getLevel()); + Assert.assertEquals(DEFAULT_LEVEL, heapHeadIndex.getLevelIndex().length); + for (long node : heapHeadIndex.getLevelIndex()) { + Assert.assertEquals(NIL_NODE, node); + } + for (int level = 0; level <= heapHeadIndex.getLevel(); level++) { + Assert.assertEquals(NIL_NODE, heapHeadIndex.getNextNode(level)); + } + } + + @Test + public void testNormallyUpdateLevel() { + int level = heapHeadIndex.getLevel(); + // update level to no more than init max level + for (; level <= DEFAULT_LEVEL; level++) { + heapHeadIndex.updateLevel(level); + Assert.assertEquals(level, heapHeadIndex.getLevel()); + Assert.assertEquals(DEFAULT_LEVEL, heapHeadIndex.getLevelIndex().length); + } + // update level to trigger scale up + heapHeadIndex.updateLevel(level); + Assert.assertEquals(level, heapHeadIndex.getLevel()); + Assert.assertEquals(DEFAULT_LEVEL * 2, heapHeadIndex.getLevelIndex().length); + } + + /** + * Test update to current level is allowed. + */ + @Test + public void testUpdateToCurrentLevel() { + heapHeadIndex.updateLevel(heapHeadIndex.getLevel()); + } + + /** + * Test update to current level is allowed. + */ + @Test + public void testUpdateLevelToLessThanCurrentLevel() { + int level = heapHeadIndex.getLevel(); + // update level 10 times + for (int i = 0; i < 10; i++) { + heapHeadIndex.updateLevel(++level); + } + // check update level to values less than current top level + for (int i = level - 1; i >= 0; i--) { + heapHeadIndex.updateLevel(i); + Assert.assertEquals(level, heapHeadIndex.getLevel()); + } + } + + /** + * Test once update more than one level is not allowed. + */ + @Test + public void testOnceUpdateMoreThanOneLevel() { + try { + heapHeadIndex.updateLevel(heapHeadIndex.getLevel() + 2); + Assert.fail("Should have thrown exception"); + } catch (Exception e) { + Assert.assertTrue(e instanceof IllegalArgumentException); + } + } + + /** + * Test update to negative level is not allowed. + */ + @Test + public void testUpdateToNegativeLevel() { + try { + heapHeadIndex.updateLevel(-1); + Assert.fail("Should throw exception"); + } catch (Exception e) { + Assert.assertTrue(e instanceof IllegalArgumentException); + } + } + + /** + * Test update to more than max level is not allowed. + */ + @Test + public void testUpdateToMoreThanMaximumAllowed() { + try { + heapHeadIndex.updateLevel(MAX_LEVEL + 1); + Assert.fail("Should throw exception"); + } catch (Exception e) { + Assert.assertTrue(e instanceof IllegalArgumentException); + } + } + + @Test + public void testUpdateNextNode() { + // test update next node of level 0 + int level = 0; + long node1 = random.nextLong(Long.MAX_VALUE); + heapHeadIndex.updateNextNode(level, node1); + Assert.assertEquals(node1, heapHeadIndex.getNextNode(level)); + // Increase one level and make sure everything still works + heapHeadIndex.updateLevel(++level); + long node2 = random.nextLong(Long.MAX_VALUE); + heapHeadIndex.updateNextNode(level, node2); + Assert.assertEquals(node2, heapHeadIndex.getNextNode(level)); + Assert.assertEquals(node1, heapHeadIndex.getNextNode(level - 1)); + } + + @Test + public void testUpdateNextNodeAfterScale() { + int level = 0; + for (; level <= DEFAULT_LEVEL; level++) { + heapHeadIndex.updateLevel(level); + } + heapHeadIndex.updateLevel(level); + long node = random.nextLong(Long.MAX_VALUE); + heapHeadIndex.updateNextNode(level, node); + Assert.assertEquals(node, heapHeadIndex.getNextNode(level)); + for (int i = 0; i < level; i++) { + Assert.assertEquals(NIL_NODE, heapHeadIndex.getNextNode(i)); + } + } + +} diff --git a/flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/SkipListKeyComparatorTest.java b/flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/SkipListKeyComparatorTest.java new file mode 100644 index 000000000000..d105bfaf77dd --- /dev/null +++ b/flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/SkipListKeyComparatorTest.java @@ -0,0 +1,299 @@ +/* + * + * * Licensed to the Apache Software Foundation (ASF) under one + * * or more contributor license agreements. See the NOTICE file + * * distributed with this work for additional information + * * regarding copyright ownership. The ASF licenses this file + * * to you under the Apache License, Version 2.0 (the + * * "License"); you may not use this file except in compliance + * * with the License. You may obtain a copy of the License at + * * + * * http://www.apache.org/licenses/LICENSE-2.0 + * * + * * Unless required by applicable law or agreed to in writing, software + * * distributed under the License is distributed on an "AS IS" BASIS, + * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * * See the License for the specific language governing permissions and + * * limitations under the License. + * + */ + +package org.apache.flink.runtime.state.heap; + +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.core.memory.MemorySegmentFactory; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import javax.annotation.Nonnull; + +import java.io.IOException; + +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThan; +import static org.junit.Assert.assertThat; + +/** + * Tests for {@link SkipListKeyComparator}. + */ +public class SkipListKeyComparatorTest extends TestLogger { + private static final SkipListKeySerializer skipListKeySerializerForPrimitive = + new SkipListKeySerializer<>(LongSerializer.INSTANCE, IntSerializer.INSTANCE); + private static final SkipListKeySerializer skipListKeySerializerForByteArray = + new SkipListKeySerializer<>(ByteArraySerializer.INSTANCE, ByteArraySerializer.INSTANCE); + private static final SkipListKeySerializer skipListKeySerializerForNamespaceCompare = + new SkipListKeySerializer<>(ByteArraySerializer.INSTANCE, ByteArraySerializer.INSTANCE); + + @Test + public void testPrimitiveEqualKeyAndEqualNamespace() { + // verify equal namespace and key + assertThat(compareSkipListKeyOfPrimitive(0L, 0, 0L, 0), is(0)); + } + + @Test + public void testPrimitiveDiffKeyAndEqualNamespace() { + // verify equal namespace and unequal key + assertThat(compareSkipListKeyOfPrimitive(0L, 5, 1L, 5), lessThan(0)); + assertThat(compareSkipListKeyOfPrimitive(192L, 90, 87L, 90), greaterThan(0)); + } + + @Test + public void testPrimitiveEqualKeyAndDiffNamespace() { + // verify unequal namespace and equal key + assertThat(compareSkipListKeyOfPrimitive(8374L, 2, 8374L, 3), lessThan(0)); + assertThat(compareSkipListKeyOfPrimitive(839L, 3, 839L, 2), greaterThan(0)); + } + + @Test + public void testPrimitiveDiffKeyAndDiffNamespace() { + // verify unequal namespace and unequal key + assertThat(compareSkipListKeyOfPrimitive(1L, 2, 3L, 4), lessThan(0)); + assertThat(compareSkipListKeyOfPrimitive(1L, 4, 3L, 2), greaterThan(0)); + assertThat(compareSkipListKeyOfPrimitive(3L, 2, 1L, 4), lessThan(0)); + assertThat(compareSkipListKeyOfPrimitive(3L, 4, 1L, 2), greaterThan(0)); + } + + @Test + public void testByteArrayEqualKeyAndEqualNamespace() { + // verify equal namespace and key + assertThat( + compareSkipListKeyOfByteArray("34", "25", "34", "25"), + is(0)); + } + + @Test + public void testByteArrayEqualKeyAndLargerNamespace() { + // verify larger namespace + assertThat( + compareSkipListKeyOfByteArray("34", "27", "34", "25"), + greaterThan(0)); + assertThat( + compareSkipListKeyOfByteArray("34", "27", "34", "25,34"), + greaterThan(0)); + assertThat( + compareSkipListKeyOfByteArray("34", "27,28", "34", "25"), + greaterThan(0)); + assertThat( + compareSkipListKeyOfByteArray("34", "27,28", "34", "25,34"), + greaterThan(0)); + assertThat( + compareSkipListKeyOfByteArray("34", "27,28", "34", "27,3"), + greaterThan(0)); + } + + @Test + public void testByteArrayEqualKeyAndSmallerNamespace() { + // verify smaller namespace + assertThat( + compareSkipListKeyOfByteArray("34", "25", "34", "27"), + lessThan(0)); + assertThat( + compareSkipListKeyOfByteArray("34", "25", "34", "27,34"), + lessThan(0)); + assertThat( + compareSkipListKeyOfByteArray("34", "25,28", "34", "27"), + lessThan(0)); + assertThat( + compareSkipListKeyOfByteArray("34", "25,28", "34", "27,34"), + lessThan(0)); + assertThat( + compareSkipListKeyOfByteArray("34", "25,28", "34", "25,34"), + lessThan(0)); + } + + @Test + public void testByteArrayLargerKeyAndEqualNamespace() { + // verify larger key + assertThat( + compareSkipListKeyOfByteArray("34", "25", "30", "25"), + greaterThan(0)); + assertThat( + compareSkipListKeyOfByteArray("34", "25", "30,38", "25"), + greaterThan(0)); + assertThat( + compareSkipListKeyOfByteArray("34,22", "25", "30", "25"), + greaterThan(0)); + assertThat( + compareSkipListKeyOfByteArray("34,22", "25", "30,38", "25"), + greaterThan(0)); + assertThat( + compareSkipListKeyOfByteArray("34,82", "25", "34,38", "25"), + greaterThan(0)); + } + + @Test + public void testByteArraySmallerKeyAndEqualNamespace() { + // verify smaller key + assertThat( + compareSkipListKeyOfByteArray("30", "25", "34", "25"), + lessThan(0)); + assertThat( + compareSkipListKeyOfByteArray("30,38", "25", "34", "25"), + lessThan(0)); + assertThat( + compareSkipListKeyOfByteArray("30", "25", "34,22", "25"), + lessThan(0)); + assertThat( + compareSkipListKeyOfByteArray("30,38", "25", "34,22", "25"), + lessThan(0)); + assertThat( + compareSkipListKeyOfByteArray("30,38", "25", "30,72", "25"), + lessThan(0)); + } + + @Test + public void testEqualNamespace() { + // test equal namespace + assertThat(compareNamespace("23", "23"), is(0)); + } + + @Test + public void testSmallerNamespace() { + // test smaller namespace + assertThat(compareNamespace("23", "24"), lessThan(0)); + assertThat(compareNamespace("23", "24,35"), lessThan(0)); + assertThat(compareNamespace("23,25", "24"), lessThan(0)); + assertThat(compareNamespace("23,20", "24,45"), lessThan(0)); + assertThat(compareNamespace("23,20", "23,45"), lessThan(0)); + } + + @Test + public void testLargerNamespace() { + // test larger namespace + assertThat(compareNamespace("26", "14"), greaterThan(0)); + assertThat(compareNamespace("26", "14,73"), greaterThan(0)); + assertThat(compareNamespace("26,25", "14"), greaterThan(0)); + assertThat(compareNamespace("26,20", "14,45"), greaterThan(0)); + assertThat(compareNamespace("26,90", "26,45"), greaterThan(0)); + } + + private int compareSkipListKeyOfByteArray(String key1, String namespace1, String key2, String namespace2) { + return compareSkipListKey(skipListKeySerializerForByteArray, convertStringToByteArray(key1), convertStringToByteArray(namespace1), + convertStringToByteArray(key2), convertStringToByteArray(namespace2)); + } + + private int compareSkipListKeyOfPrimitive(long key1, int namespace1, long key2, int namespace2) { + return compareSkipListKey(skipListKeySerializerForPrimitive, key1, namespace1, key2, namespace2); + } + + private int compareSkipListKey( + @Nonnull SkipListKeySerializer keySerializer, + K key1, N namespace1, K key2, N namespace2) { + MemorySegment b1 = MemorySegmentFactory.wrap(keySerializer.serialize(key1, namespace1)); + MemorySegment b2 = MemorySegmentFactory.wrap(keySerializer.serialize(key2, namespace2)); + return SkipListKeyComparator.compareTo(b1, 0, b2, 0); + } + + private int compareNamespace(String namespace, String targetNamespace) { + final byte[] key = convertStringToByteArray("34"); + byte[] n = skipListKeySerializerForNamespaceCompare.serializeNamespace(convertStringToByteArray(namespace)); + byte[] k = skipListKeySerializerForNamespaceCompare.serialize(key, convertStringToByteArray(targetNamespace)); + return SkipListKeyComparator.compareNamespaceAndNode( + MemorySegmentFactory.wrap(n), 0, n.length, MemorySegmentFactory.wrap(k), 0); + } + + private byte[] convertStringToByteArray(@Nonnull String str) { + String[] subStr = str.split(","); + byte[] value = new byte[subStr.length]; + for (int i = 0; i < subStr.length; i++) { + int v = Integer.valueOf(subStr[i]); + value[i] = (byte) v; + } + return value; + } + + /** + * A serializer for byte array which does not support deserialization. + */ + private static class ByteArraySerializer extends TypeSerializerSingleton { + + private static final byte[] EMPTY = new byte[0]; + + static final ByteArraySerializer INSTANCE = new ByteArraySerializer(); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public byte[] createInstance() { + return EMPTY; + } + + @Override + public byte[] copy(byte[] from) { + byte[] copy = new byte[from.length]; + System.arraycopy(from, 0, copy, 0, from.length); + return copy; + } + + @Override + public byte[] copy(byte[] from, byte[] reuse) { + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(byte[] record, DataOutputView target) throws IOException { + if (record == null) { + throw new IllegalArgumentException("The record must not be null."); + } + + // do not write length of array, so deserialize is not supported + target.write(record); + } + + @Override + public byte[] deserialize(DataInputView source) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public byte[] deserialize(byte[] reuse, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) { + throw new UnsupportedOperationException(); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + throw new UnsupportedOperationException(); + } + } +} diff --git a/flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/SkipListSerializerTest.java b/flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/SkipListSerializerTest.java new file mode 100644 index 000000000000..aef843d01970 --- /dev/null +++ b/flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/SkipListSerializerTest.java @@ -0,0 +1,110 @@ +/* + * + * * Licensed to the Apache Software Foundation (ASF) under one + * * or more contributor license agreements. See the NOTICE file + * * distributed with this work for additional information + * * regarding copyright ownership. The ASF licenses this file + * * to you under the Apache License, Version 2.0 (the + * * "License"); you may not use this file except in compliance + * * with the License. You may obtain a copy of the License at + * * + * * http://www.apache.org/licenses/LICENSE-2.0 + * * + * * Unless required by applicable law or agreed to in writing, software + * * distributed under the License is distributed on an "AS IS" BASIS, + * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * * See the License for the specific language governing permissions and + * * limitations under the License. + * + */ + +package org.apache.flink.runtime.state.heap; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.memory.ByteArrayInputStreamWithPos; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.core.memory.MemorySegmentFactory; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import java.io.IOException; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link SkipListKeySerializer}. + */ +public class SkipListSerializerTest extends TestLogger { + private static final TypeSerializer keySerializer = StringSerializer.INSTANCE; + private static final TypeSerializer namespaceSerializer = StringSerializer.INSTANCE; + private static final SkipListKeySerializer skipListKeySerializer = + new SkipListKeySerializer<>(keySerializer, namespaceSerializer); + private static final TypeSerializer stateSerializer = StringSerializer.INSTANCE; + private static final SkipListValueSerializer skipListValueSerializer = + new SkipListValueSerializer<>(stateSerializer); + + @Test + public void testSkipListKeySerializerBasicOp() throws IOException { + testSkipListKeySerializer(0); + } + + @Test + public void testSkipListKeySerializerStateless() throws IOException { + for (int i = 0; i < 10; i++) { + testSkipListKeySerializer(i); + } + } + + private void testSkipListKeySerializer(int delta) throws IOException { + String key = "key-abcdedg" + delta; + String namespace = "namespace-dfsfdafd" + delta; + + byte[] skipListKey = skipListKeySerializer.serialize(key, namespace); + int offset = 10; + byte[] data = new byte[10 + skipListKey.length]; + System.arraycopy(skipListKey, 0, data, offset, skipListKey.length); + MemorySegment skipListKeySegment = MemorySegmentFactory.wrap(data); + assertEquals(key, skipListKeySerializer.deserializeKey(skipListKeySegment, offset, skipListKey.length)); + assertEquals(namespace, skipListKeySerializer.deserializeNamespace(skipListKeySegment, offset, skipListKey.length)); + + Tuple2 serializedKeyAndNamespace = + skipListKeySerializer.getSerializedKeyAndNamespace(skipListKeySegment, offset); + assertEquals(key, deserialize(keySerializer, serializedKeyAndNamespace.f0)); + assertEquals(namespace, deserialize(namespaceSerializer, serializedKeyAndNamespace.f1)); + + byte[] serializedNamespace = skipListKeySerializer.serializeNamespace(namespace); + assertEquals(namespace, deserialize(namespaceSerializer, serializedNamespace)); + } + + @Test + public void testSkipListValueSerializerBasicOp() throws IOException { + testSkipListValueSerializer(0); + } + + @Test + public void testSkipListValueSerializerStateless() throws IOException { + for (int i = 0; i < 10; i++) { + testSkipListValueSerializer(i); + } + } + + private void testSkipListValueSerializer(int i) throws IOException { + String state = "value-" + i; + byte[] value = skipListValueSerializer.serialize(state); + int offset = 10; + byte[] data = new byte[10 + value.length]; + System.arraycopy(value, 0, data, offset, value.length); + assertEquals(state, deserialize(stateSerializer, value)); + assertEquals(state, skipListValueSerializer.deserializeState(MemorySegmentFactory.wrap(data), offset, value.length)); + } + + private T deserialize(TypeSerializer serializer, byte[] data) throws IOException { + ByteArrayInputStreamWithPos inputStream = new ByteArrayInputStreamWithPos(data); + DataInputViewStreamWrapper inputView = new DataInputViewStreamWrapper(inputStream); + return serializer.deserialize(inputView); + } +} diff --git a/flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/SkipListUtilsTest.java b/flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/SkipListUtilsTest.java new file mode 100644 index 000000000000..3ed2d1850f05 --- /dev/null +++ b/flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/SkipListUtilsTest.java @@ -0,0 +1,170 @@ +/* + * + * * Licensed to the Apache Software Foundation (ASF) under one + * * or more contributor license agreements. See the NOTICE file + * * distributed with this work for additional information + * * regarding copyright ownership. The ASF licenses this file + * * to you under the Apache License, Version 2.0 (the + * * "License"); you may not use this file except in compliance + * * with the License. You may obtain a copy of the License at + * * + * * http://www.apache.org/licenses/LICENSE-2.0 + * * + * * Unless required by applicable law or agreed to in writing, software + * * distributed under the License is distributed on an "AS IS" BASIS, + * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * * See the License for the specific language governing permissions and + * * limitations under the License. + * + */ + +package org.apache.flink.runtime.state.heap; + +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.core.memory.MemorySegmentFactory; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import java.util.concurrent.ThreadLocalRandom; + +import static org.apache.flink.runtime.state.heap.SkipListUtils.MAX_LEVEL; +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link SkipListUtils}. + */ +public class SkipListUtilsTest extends TestLogger { + + @Test + public void testKeySpacePutAndGet() { + for (int level = 0; level <= MAX_LEVEL; level++) { + int keyLen = ThreadLocalRandom.current().nextInt(100) + 1; + KeySpace keySpace = createKeySpace(level, keyLen); + int keyMetaLen = SkipListUtils.getKeyMetaLen(level); + int totalKeySpaceLen = keyMetaLen + keyLen; + int offset = 100; + MemorySegment segment = MemorySegmentFactory.allocateUnpooledSegment(totalKeySpaceLen + offset); + putKeySpace(keySpace, segment, offset); + verifyGetKeySpace(keySpace, segment, offset); + } + } + + @Test + public void testValueSpacePutAndGet() { + for (int i = 0; i < 100; i++) { + int valueLen = ThreadLocalRandom.current().nextInt(100) + 1; + ValueSpace valueSpace = createValueSpace(valueLen); + int valueMetaLen = SkipListUtils.getValueMetaLen(); + int totalValueSpaceLen = valueMetaLen + valueLen; + int offset = 100; + MemorySegment segment = MemorySegmentFactory.allocateUnpooledSegment(totalValueSpaceLen + offset); + putValueSpace(valueSpace, segment, offset); + verifyGetValueSpace(valueSpace, segment, offset); + } + } + + private KeySpace createKeySpace(int level, int keyLen) { + ThreadLocalRandom random = ThreadLocalRandom.current(); + KeySpace keySpace = new KeySpace(); + keySpace.level = level; + keySpace.status = random.nextBoolean() ? NodeStatus.PUT : NodeStatus.REMOVE; + keySpace.valuePointer = random.nextLong(); + keySpace.nextKeyPointer = random.nextLong(); + keySpace.nextIndexNodes = new long[level]; + keySpace.prevIndexNodes = new long[level]; + for (int i = 0; i < level; i++) { + keySpace.nextIndexNodes[i] = random.nextLong(); + keySpace.prevIndexNodes[i] = random.nextLong(); + } + keySpace.keyData = new byte[keyLen]; + random.nextBytes(keySpace.keyData); + return keySpace; + } + + private void putKeySpace(KeySpace keySpace, MemorySegment memorySegment, int offset) { + SkipListUtils.putLevelAndNodeStatus(memorySegment, offset, keySpace.level, keySpace.status); + SkipListUtils.putKeyLen(memorySegment, offset, keySpace.keyData.length); + SkipListUtils.putValuePointer(memorySegment, offset, keySpace.valuePointer); + SkipListUtils.putNextKeyPointer(memorySegment, offset, keySpace.nextKeyPointer); + for (int i = 1; i <= keySpace.nextIndexNodes.length; i++) { + SkipListUtils.putNextIndexNode(memorySegment, offset, i, keySpace.nextIndexNodes[i - 1]); + } + for (int i = 1; i <= keySpace.prevIndexNodes.length; i++) { + SkipListUtils.putPrevIndexNode(memorySegment, offset, keySpace.level, i, keySpace.prevIndexNodes[i - 1]); + } + SkipListUtils.putKeyData(memorySegment, offset, MemorySegmentFactory.wrap(keySpace.keyData), + 0, keySpace.keyData.length, keySpace.level); + } + + private void verifyGetKeySpace(KeySpace keySpace, MemorySegment memorySegment, int offset) { + assertEquals(keySpace.level, SkipListUtils.getLevel(memorySegment, offset)); + assertEquals(keySpace.status, SkipListUtils.getNodeStatus(memorySegment, offset)); + assertEquals(keySpace.keyData.length, SkipListUtils.getKeyLen(memorySegment, offset)); + assertEquals(keySpace.valuePointer, SkipListUtils.getValuePointer(memorySegment, offset)); + assertEquals(keySpace.nextKeyPointer, SkipListUtils.getNextKeyPointer(memorySegment, offset)); + for (int i = 1; i <= keySpace.nextIndexNodes.length; i++) { + assertEquals(keySpace.nextIndexNodes[i - 1], SkipListUtils.getNextIndexNode(memorySegment, offset, i)); + } + for (int i = 1; i <= keySpace.prevIndexNodes.length; i++) { + assertEquals(keySpace.prevIndexNodes[i - 1], + SkipListUtils.getPrevIndexNode(memorySegment, offset, keySpace.level, i)); + } + int keyDataOffset = SkipListUtils.getKeyDataOffset(keySpace.level); + MemorySegment keyDataSegment = MemorySegmentFactory.wrap(keySpace.keyData); + assertEquals(0, memorySegment.compare(keyDataSegment, offset + keyDataOffset, 0, keySpace.keyData.length)); + } + + private ValueSpace createValueSpace(int valueLen) { + ThreadLocalRandom random = ThreadLocalRandom.current(); + ValueSpace valueSpace = new ValueSpace(); + valueSpace.version = random.nextInt(Integer.MAX_VALUE); + valueSpace.keyPointer = random.nextLong(); + valueSpace.nextValuePointer = random.nextLong(); + valueSpace.valueData = new byte[valueLen]; + random.nextBytes(valueSpace.valueData); + return valueSpace; + } + + private void putValueSpace(ValueSpace valueSpace, MemorySegment memorySegment, int offset) { + SkipListUtils.putValueVersion(memorySegment, offset, valueSpace.version); + SkipListUtils.putKeyPointer(memorySegment, offset, valueSpace.keyPointer); + SkipListUtils.putNextValuePointer(memorySegment, offset, valueSpace.nextValuePointer); + SkipListUtils.putValueLen(memorySegment, offset, valueSpace.valueData.length); + SkipListUtils.putValueData(memorySegment, offset, valueSpace.valueData); + } + + private void verifyGetValueSpace(ValueSpace valueSpace, MemorySegment memorySegment, int offset) { + assertEquals(valueSpace.version, SkipListUtils.getValueVersion(memorySegment, offset)); + assertEquals(valueSpace.keyPointer, SkipListUtils.getKeyPointer(memorySegment, offset)); + assertEquals(valueSpace.nextValuePointer, SkipListUtils.getNextValuePointer(memorySegment, offset)); + assertEquals(valueSpace.valueData.length, SkipListUtils.getValueLen(memorySegment, offset)); + int valueDataOffset = SkipListUtils.getValueMetaLen(); + MemorySegment valueDataSegment = MemorySegmentFactory.wrap(valueSpace.valueData); + assertEquals(0, + memorySegment.compare(valueDataSegment, offset + valueDataOffset, 0, valueSpace.valueData.length)); + } + + /** + * Used to test key space. + */ + static class KeySpace { + int level; + NodeStatus status; + long valuePointer; + long nextKeyPointer; + long[] nextIndexNodes; + long[] prevIndexNodes; + byte[] keyData; + } + + /** + * Used to test value space. + */ + static class ValueSpace { + int version; + long keyPointer; + long nextValuePointer; + byte[] valueData; + } +} diff --git a/flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/TestAllocator.java b/flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/TestAllocator.java new file mode 100644 index 000000000000..8dc562b58204 --- /dev/null +++ b/flink-state-backends/flink-statebackend-heap-spillable/src/test/java/org/apache/flink/runtime/state/heap/TestAllocator.java @@ -0,0 +1,166 @@ +/* + * + * * Licensed to the Apache Software Foundation (ASF) under one + * * or more contributor license agreements. See the NOTICE file + * * distributed with this work for additional information + * * regarding copyright ownership. The ASF licenses this file + * * to you under the Apache License, Version 2.0 (the + * * "License"); you may not use this file except in compliance + * * with the License. You may obtain a copy of the License at + * * + * * http://www.apache.org/licenses/LICENSE-2.0 + * * + * * Unless required by applicable law or agreed to in writing, software + * * distributed under the License is distributed on an "AS IS" BASIS, + * * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * * See the License for the specific language governing permissions and + * * limitations under the License. + * + */ + +package org.apache.flink.runtime.state.heap; + +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.core.memory.MemorySegmentFactory; +import org.apache.flink.runtime.state.heap.space.Allocator; +import org.apache.flink.runtime.state.heap.space.Chunk; +import org.apache.flink.runtime.state.heap.space.SpaceUtils; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.TestLogger; + +import java.util.HashMap; +import java.util.Map; + +import static org.apache.flink.runtime.state.heap.space.Constants.FOUR_BYTES_BITS; +import static org.apache.flink.runtime.state.heap.space.Constants.FOUR_BYTES_MARK; + +/** + * Implementation of {@link Allocator} used for test. This allocator + * will create a chunk for each allocation request. Size of a chunk + * is fixed, and only used by one space. + */ +public class TestAllocator extends TestLogger implements Allocator { + + /** + * Max allocate size supported by this allocator. + */ + private final int maxAllocateSize; + + private Map chunkMap; + + private volatile int chunkCounter; + + public TestAllocator() { + this(256); + } + + public TestAllocator(int maxAllocateSize) { + this.maxAllocateSize = maxAllocateSize; + this.chunkMap = new HashMap<>(); + this.chunkCounter = 0; + } + + @Override + public synchronized long allocate(int size) { + Preconditions.checkArgument(size <= maxAllocateSize, + "Can't allocate size of " + size + " larger than maxAllocateSize " + maxAllocateSize); + int chunkId = chunkCounter++; + TestChunk testChunk = new TestChunk(chunkId, maxAllocateSize); + chunkMap.put(chunkId, testChunk); + int offset = testChunk.allocate(size); + return ((chunkId & FOUR_BYTES_MARK) << FOUR_BYTES_BITS) | (offset & FOUR_BYTES_MARK); + } + + @Override + public synchronized void free(long address) { + int chunkId = SpaceUtils.getChunkIdByAddress(address); + int offset = SpaceUtils.getChunkOffsetByAddress(address); + TestChunk chunk = chunkMap.remove(chunkId); + if (chunk != null) { + chunk.free(offset); + } + } + + @Override + public synchronized Chunk getChunkById(int chunkId) { + TestChunk chunk = chunkMap.get(chunkId); + Preconditions.checkNotNull(chunk, "chunk " + chunkId + " doest not exist."); + return chunk; + } + + @Override + public synchronized void close() { + chunkMap.clear(); + } + + public int getMaxAllocateSize() { + return maxAllocateSize; + } + + /** + * Returns total size of used space. + */ + public synchronized int getTotalSpaceSize() { + return chunkMap.size() * maxAllocateSize; + } + + /** + * Returns number of used space. + */ + public synchronized int getTotalSpaceNumber() { + return chunkMap.size(); + } + + /** + * Implementation of {@link Chunk} used for test. A chunk can only be used + * by one space. + */ + public static class TestChunk implements Chunk { + + private final int chunkId; + private final int size; + private final MemorySegment segment; + private final int offset; + private volatile boolean used; + + TestChunk(int chunkId, int size) { + this.offset = 14; + this.chunkId = chunkId; + this.size = size + offset; + this.segment = MemorySegmentFactory.allocateUnpooledSegment(size); + } + + @Override + public synchronized int allocate(int len) { + Preconditions.checkState(!used, "chunk has been allocated."); + Preconditions.checkState(len <= size, "There is no enough size."); + used = true; + return offset; + } + + @Override + public synchronized void free(int interChunkOffset) { + used = false; + } + + @Override + public int getChunkId() { + return chunkId; + } + + @Override + public int getChunkCapacity() { + return size; + } + + @Override + public MemorySegment getMemorySegment(int chunkOffset) { + return segment; + } + + @Override + public int getOffsetInSegment(int offsetInChunk) { + return offsetInChunk; + } + } +} From 9d45ad046742af9d1e8172720ffcd8347bb37ce8 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 7 Nov 2019 14:58:22 +0100 Subject: [PATCH 486/746] [FLINK-12697][state backends] (follow-up) Minor optimization that avoids extra memory segment wrapping. - the SkipListKeySerializer offers methods to directly return a MemorySegment instead of a byte[] to exploit implementations that thus avoid extra wrapping. --- .../heap/CopyOnWriteSkipListStateMap.java | 13 ++++---- .../state/heap/SkipListKeySerializer.java | 30 ++++++++++++++----- 2 files changed, 28 insertions(+), 15 deletions(-) diff --git a/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteSkipListStateMap.java b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteSkipListStateMap.java index 106939247d01..0ecd5e8417fe 100644 --- a/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteSkipListStateMap.java +++ b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteSkipListStateMap.java @@ -996,8 +996,7 @@ private S getNodeInternal(K key, N namespace) { * @return the {@link MemorySegment} wrapping up the serialized key bytes. */ private MemorySegment getKeySegment(K key, N namespace) { - byte[] keyBytes = skipListKeySerializer.serialize(key, namespace); - return MemorySegmentFactory.wrap(keyBytes); + return skipListKeySerializer.serializeToSegment(key, namespace); } // Help methods --------------------------------------------------------------- @@ -1165,9 +1164,8 @@ private StateEntry helpGetStateEntry(long node) { @Override public Stream getKeys(N namespace) { updateStat(); - byte[] namespaceBytes = skipListKeySerializer.serializeNamespace(namespace); - MemorySegment namespaceSegment = MemorySegmentFactory.wrap(namespaceBytes); - Iterator nodeIter = new NamespaceNodeIterator(namespaceSegment, 0, namespaceBytes.length); + MemorySegment namespaceSegment = skipListKeySerializer.serializeNamespaceToSegment(namespace); + Iterator nodeIter = new NamespaceNodeIterator(namespaceSegment, 0, namespaceSegment.size()); return StreamSupport.stream(Spliterators.spliteratorUnknownSize(nodeIter, 0), false) .map(this::helpGetKey); } @@ -1176,9 +1174,8 @@ public Stream getKeys(N namespace) { @Override public int sizeOfNamespace(Object namespace) { updateStat(); - byte[] namespaceBytes = skipListKeySerializer.serializeNamespace((N) namespace); - MemorySegment namespaceSegment = MemorySegmentFactory.wrap(namespaceBytes); - Iterator nodeIter = new NamespaceNodeIterator(namespaceSegment, 0, namespaceBytes.length); + MemorySegment namespaceSegment = skipListKeySerializer.serializeNamespaceToSegment((N) namespace); + Iterator nodeIter = new NamespaceNodeIterator(namespaceSegment, 0, namespaceSegment.size()); int size = 0; while (nodeIter.hasNext()) { nodeIter.next(); diff --git a/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/SkipListKeySerializer.java b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/SkipListKeySerializer.java index 56a09b6389f3..46c5460a7503 100644 --- a/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/SkipListKeySerializer.java +++ b/flink-state-backends/flink-statebackend-heap-spillable/src/main/java/org/apache/flink/runtime/state/heap/SkipListKeySerializer.java @@ -60,6 +60,19 @@ class SkipListKeySerializer { * - byte[]: serialized key */ byte[] serialize(K key, N namespace) { + // we know that the segment contains a byte[], because it is created + // in the method below by wrapping a byte[] + return serializeToSegment(key, namespace).getArray(); + } + + /** + * Serialize the key and namespace to bytes. The format is + * - int: length of serialized namespace + * - byte[]: serialized namespace + * - int: length of serialized key + * - byte[]: serialized key + */ + MemorySegment serializeToSegment(K key, N namespace) { outputStream.reset(); try { // serialize namespace @@ -78,15 +91,14 @@ byte[] serialize(K key, N namespace) { throw new RuntimeException("Failed to serialize key", e); } - byte[] result = outputStream.toByteArray(); + final byte[] result = outputStream.toByteArray(); + final MemorySegment segment = MemorySegmentFactory.wrap(result); + // set length of namespace and key - int namespaceLen = keyStartPos - Integer.BYTES; - int keyLen = result.length - keyStartPos - Integer.BYTES; - MemorySegment segment = MemorySegmentFactory.wrap(result); - segment.putInt(0, namespaceLen); - segment.putInt(keyStartPos, keyLen); + segment.putInt(0, keyStartPos - Integer.BYTES); + segment.putInt(keyStartPos, result.length - keyStartPos - Integer.BYTES); - return result; + return segment; } /** @@ -162,4 +174,8 @@ byte[] serializeNamespace(N namespace) { } return outputStream.toByteArray(); } + + MemorySegment serializeNamespaceToSegment(N namespace) { + return MemorySegmentFactory.wrap(serializeNamespace(namespace)); + } } From 534835b0386c767116708f7986dfb759e377dcaf Mon Sep 17 00:00:00 2001 From: liyafan82 Date: Wed, 17 Apr 2019 14:34:09 +0800 Subject: [PATCH 487/746] [FLINK-12216][runtime] Respect the number of bytes from input parameters in HybridMemorySegment This also needs to move the condition for "isReadOnly" to keep giving consistent error messages between all MemorySegment implementations. This closes #8194 --- .../core/memory/HybridMemorySegment.java | 17 ++++------ .../memory/HybridOnHeapMemorySegmentTest.java | 33 +++++++++++++++++++ 2 files changed, 40 insertions(+), 10 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/HybridMemorySegment.java b/flink-core/src/main/java/org/apache/flink/core/memory/HybridMemorySegment.java index c8889f53aa36..3404bc1b4ef5 100644 --- a/flink-core/src/main/java/org/apache/flink/core/memory/HybridMemorySegment.java +++ b/flink-core/src/main/java/org/apache/flink/core/memory/HybridMemorySegment.java @@ -296,6 +296,9 @@ public final void get(int offset, ByteBuffer target, int numBytes) { if ((offset | numBytes | (offset + numBytes)) < 0) { throw new IndexOutOfBoundsException(); } + if (target.isReadOnly()) { + throw new ReadOnlyBufferException(); + } final int targetOffset = target.position(); final int remaining = target.remaining(); @@ -305,10 +308,6 @@ public final void get(int offset, ByteBuffer target, int numBytes) { } if (target.isDirect()) { - if (target.isReadOnly()) { - throw new ReadOnlyBufferException(); - } - // copy to the target memory directly final long targetPointer = getAddress(target) + targetOffset; final long sourcePointer = address + offset; @@ -333,10 +332,8 @@ else if (target.hasArray()) { target.position(targetOffset + numBytes); } else { - // neither heap buffer nor direct buffer - while (target.hasRemaining()) { - target.put(get(offset++)); - } + // other types of byte buffers + throw new IllegalArgumentException("The target buffer is not direct, and has no array."); } } @@ -379,8 +376,8 @@ else if (source.hasArray()) { source.position(sourceOffset + numBytes); } else { - // neither heap buffer nor direct buffer - while (source.hasRemaining()) { + // other types of byte buffers + for (int i = 0; i < numBytes; i++) { put(offset++, source.get()); } } diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/HybridOnHeapMemorySegmentTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/HybridOnHeapMemorySegmentTest.java index 7cfc3019fcbc..9f725ee87693 100644 --- a/flink-core/src/test/java/org/apache/flink/core/memory/HybridOnHeapMemorySegmentTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/memory/HybridOnHeapMemorySegmentTest.java @@ -77,4 +77,37 @@ public void testHybridHeapSegmentSpecifics() { assertEquals(3, buf2.position()); assertEquals(7, buf2.limit()); } + + @Test + public void testReadOnlyByteBufferPut() { + final byte[] buffer = new byte[100]; + HybridMemorySegment seg = new HybridMemorySegment(buffer, null); + + String content = "hello world"; + ByteBuffer bb = ByteBuffer.allocate(20); + bb.put(content.getBytes()); + bb.rewind(); + + int offset = 10; + int numBytes = 5; + + ByteBuffer readOnlyBuf = bb.asReadOnlyBuffer(); + assertFalse(readOnlyBuf.isDirect()); + assertFalse(readOnlyBuf.hasArray()); + + seg.put(offset, readOnlyBuf, numBytes); + + // verify the area before the written region. + for (int i = 0; i < offset; i++) { + assertEquals(0, buffer[i]); + } + + // verify the region that is written. + assertEquals("hello", new String(buffer, offset, numBytes)); + + // verify the area after the written region. + for (int i = offset + numBytes; i < buffer.length; i++) { + assertEquals(0, buffer[i]); + } + } } From 7ee05e59355c36038a52f790196a098e040f3312 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 7 Nov 2019 16:11:17 +0100 Subject: [PATCH 488/746] [hotfix][core] Cleanups of compiler and inspection warnings for HybridMemorySegment and tests. --- .../apache/flink/core/memory/HybridMemorySegment.java | 3 ++- .../core/memory/HybridOffHeapMemorySegmentTest.java | 10 ++++++---- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/HybridMemorySegment.java b/flink-core/src/main/java/org/apache/flink/core/memory/HybridMemorySegment.java index 3404bc1b4ef5..fbb9837be555 100644 --- a/flink-core/src/main/java/org/apache/flink/core/memory/HybridMemorySegment.java +++ b/flink-core/src/main/java/org/apache/flink/core/memory/HybridMemorySegment.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.io.DataInput; @@ -72,7 +73,7 @@ public final class HybridMemorySegment extends MemorySegment { * @param cleaner optional action to run upon freeing the segment. * @throws IllegalArgumentException Thrown, if the given ByteBuffer is not direct. */ - HybridMemorySegment(ByteBuffer buffer, @Nullable Object owner, @Nullable Runnable cleaner) { + HybridMemorySegment(@Nonnull ByteBuffer buffer, @Nullable Object owner, @Nullable Runnable cleaner) { super(checkBufferAndGetAddress(buffer), buffer.capacity(), owner); this.offHeapBuffer = buffer; this.cleaner = cleaner; diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapMemorySegmentTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapMemorySegmentTest.java index 67c05c307ce8..b1199a2a6ddd 100644 --- a/flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapMemorySegmentTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapMemorySegmentTest.java @@ -24,6 +24,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotSame; +import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -44,7 +46,7 @@ public void testHybridHeapSegmentSpecifics() { assertFalse(seg.isFreed()); assertTrue(seg.isOffHeap()); assertEquals(buffer.capacity(), seg.size()); - assertTrue(buffer == seg.getOffHeapBuffer()); + assertSame(buffer, seg.getOffHeapBuffer()); try { //noinspection ResultOfMethodCallIgnored @@ -58,9 +60,9 @@ public void testHybridHeapSegmentSpecifics() { ByteBuffer buf1 = seg.wrap(1, 2); ByteBuffer buf2 = seg.wrap(3, 4); - assertTrue(buf1 != buffer); - assertTrue(buf2 != buffer); - assertTrue(buf1 != buf2); + assertNotSame(buf1, buffer); + assertNotSame(buf2, buffer); + assertNotSame(buf1, buf2); assertEquals(1, buf1.position()); assertEquals(3, buf1.limit()); assertEquals(3, buf2.position()); From c5b89494914f162cb7a8db898c9a8fd72f3a1694 Mon Sep 17 00:00:00 2001 From: Rui Li Date: Tue, 5 Nov 2019 12:42:27 +0800 Subject: [PATCH 489/746] [FLINK-14605][hive] Use Hive-1.1.0 as the profile to test against 1.1.x Use Hive-1.1.0 as the profile to test against 1.1.x. This closes #10081. --- flink-connectors/flink-connector-hive/pom.xml | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-hive/pom.xml b/flink-connectors/flink-connector-hive/pom.xml index de98156ae407..861453021d6f 100644 --- a/flink-connectors/flink-connector-hive/pom.xml +++ b/flink-connectors/flink-connector-hive/pom.xml @@ -676,9 +676,10 @@ under the License. - hive-1.1.1 + + hive-1.1.0 - 1.1.1 + 1.1.0 2.6.5 3.1.1 From 25a3d9138cd5e39fc786315682586b75d8ac86ea Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 15 Oct 2019 17:37:03 +0200 Subject: [PATCH 490/746] [hotfix] Move TaskManagerSlot to o.a.f.runtime.resourcemanager.slotmanager --- .../resourcemanager/slotmanager/SlotManagerImpl.java | 1 - .../slotmanager}/TaskManagerSlot.java | 8 +++++--- .../resourcemanager/slotmanager/TaskManagerSlotId.java | 1 - .../resourcemanager/slotmanager/SlotManagerImplTest.java | 1 - 4 files changed, 5 insertions(+), 6 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/{clusterframework/types => resourcemanager/slotmanager}/TaskManagerSlot.java (93%) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImpl.java index a8d000325345..b996e5519932 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImpl.java @@ -24,7 +24,6 @@ import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.clusterframework.types.SlotID; -import org.apache.flink.runtime.clusterframework.types.TaskManagerSlot; import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.messages.Acknowledge; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/TaskManagerSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerSlot.java similarity index 93% rename from flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/TaskManagerSlot.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerSlot.java index 633f31a62300..fd861280396d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/TaskManagerSlot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerSlot.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,12 +16,14 @@ * limitations under the License. */ -package org.apache.flink.runtime.clusterframework.types; +package org.apache.flink.runtime.resourcemanager.slotmanager; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.clusterframework.types.SlotID; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection; -import org.apache.flink.runtime.resourcemanager.slotmanager.PendingSlotRequest; import org.apache.flink.util.Preconditions; import javax.annotation.Nullable; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerSlotId.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerSlotId.java index 3084b3e8ed94..4299200d7427 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerSlotId.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerSlotId.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.resourcemanager.slotmanager; -import org.apache.flink.runtime.clusterframework.types.TaskManagerSlot; import org.apache.flink.util.AbstractID; /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImplTest.java index 931c3f30f8b3..31323058a014 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImplTest.java @@ -27,7 +27,6 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.clusterframework.types.SlotID; -import org.apache.flink.runtime.clusterframework.types.TaskManagerSlot; import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.concurrent.ScheduledExecutor; From 740ca1ad6078a51e00202afcee0f185b2c5ba997 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 15 Oct 2019 17:42:42 +0200 Subject: [PATCH 491/746] [hotfix] Introduce TaskManagerSlotInformation TaskManagerSlotInformation is a super interface of the TaskManagerSlot which gives access to the basic information of a registered TaskManagerSlot. --- .../slotmanager/TaskManagerSlot.java | 6 ++- .../TaskManagerSlotInformation.java | 49 +++++++++++++++++++ 2 files changed, 54 insertions(+), 1 deletion(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerSlotInformation.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerSlot.java index fd861280396d..1eeb8d535f58 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerSlot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerSlot.java @@ -36,7 +36,7 @@ * A TaskManagerSlot represents a slot located in a TaskManager. It has a unique identification and * resource profile associated. */ -public class TaskManagerSlot { +public class TaskManagerSlot implements TaskManagerSlotInformation { /** The unique identification of this slot. */ private final SlotID slotId; @@ -76,10 +76,12 @@ public State getState() { return state; } + @Override public SlotID getSlotId() { return slotId; } + @Override public ResourceProfile getResourceProfile() { return resourceProfile; } @@ -101,6 +103,7 @@ public PendingSlotRequest getAssignedSlotRequest() { return assignedSlotRequest; } + @Override public InstanceID getInstanceId() { return taskManagerConnection.getInstanceID(); } @@ -153,6 +156,7 @@ public void updateAllocation(AllocationID allocationId, JobID jobId) { * @param required The required resource profile * @return true if requirement can be matched */ + @Override public boolean isMatchingRequirement(ResourceProfile required) { return resourceProfile.isMatching(required); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerSlotInformation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerSlotInformation.java new file mode 100644 index 000000000000..bb06679b3945 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerSlotInformation.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.resourcemanager.slotmanager; + +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.clusterframework.types.SlotID; +import org.apache.flink.runtime.instance.InstanceID; + +/** + * Basic information about a {@link TaskManagerSlot}. + */ +public interface TaskManagerSlotInformation { + + SlotID getSlotId(); + + InstanceID getInstanceId(); + + /** + * Returns true if the required {@link ResourceProfile} can be fulfilled + * by this slot. + * + * @param required resources + * @return true if the this slot can fulfill the resource requirements + */ + boolean isMatchingRequirement(ResourceProfile required); + + /** + * Get resource profile of this slot. + * + * @return resource profile of this slot + */ + ResourceProfile getResourceProfile(); +} From 5a6aa937253b03f8f626df9175c7118b872e17a5 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 15 Oct 2019 18:13:49 +0200 Subject: [PATCH 492/746] [hotfix] Make SlotManagerImpl#findMatchingRequest and #findMatchingSlot private --- .../runtime/resourcemanager/slotmanager/SlotManagerImpl.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImpl.java index b996e5519932..2856c5462029 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImpl.java @@ -510,7 +510,7 @@ public void setFailUnfulfillableRequest(boolean failUnfulfillableRequest) { * @return A matching slot request which can be deployed in a slot with the given resource * profile. Null if there is no such slot request pending. */ - protected PendingSlotRequest findMatchingRequest(ResourceProfile slotResourceProfile) { + private PendingSlotRequest findMatchingRequest(ResourceProfile slotResourceProfile) { for (PendingSlotRequest pendingSlotRequest : pendingSlotRequests.values()) { if (!pendingSlotRequest.isAssigned() && slotResourceProfile.isMatching(pendingSlotRequest.getResourceProfile())) { @@ -533,7 +533,7 @@ protected PendingSlotRequest findMatchingRequest(ResourceProfile slotResourcePro * @return A matching slot which fulfills the given resource profile. Null if there is no such * slot available. */ - protected TaskManagerSlot findMatchingSlot(ResourceProfile requestResourceProfile) { + private TaskManagerSlot findMatchingSlot(ResourceProfile requestResourceProfile) { Iterator> iterator = freeSlots.entrySet().iterator(); while (iterator.hasNext()) { From 743eae4358cff1c050ac16efa686d72c35aab614 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 15 Oct 2019 18:22:20 +0200 Subject: [PATCH 493/746] [hotfix] Extend OptionalConsumer to accept ThrowingRunnable --- .../src/main/java/org/apache/flink/util/OptionalConsumer.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/flink-core/src/main/java/org/apache/flink/util/OptionalConsumer.java b/flink-core/src/main/java/org/apache/flink/util/OptionalConsumer.java index 6773cdfd0df1..b8d67683896f 100644 --- a/flink-core/src/main/java/org/apache/flink/util/OptionalConsumer.java +++ b/flink-core/src/main/java/org/apache/flink/util/OptionalConsumer.java @@ -18,6 +18,8 @@ package org.apache.flink.util; +import org.apache.flink.util.function.ThrowingRunnable; + import java.util.Optional; import java.util.function.Consumer; @@ -45,7 +47,7 @@ public OptionalConsumer ifPresent(Consumer c) { return this; } - public OptionalConsumer ifNotPresent(Runnable r) { + public OptionalConsumer ifNotPresent(ThrowingRunnable r) throws E { if (!optional.isPresent()) { r.run(); } From b02fe5d210e712cce0ffd21d65e239f817033304 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 15 Oct 2019 18:23:35 +0200 Subject: [PATCH 494/746] [hotfix] Let SlotManagerImpl#findMatchingSlot return Optional --- .../slotmanager/SlotManagerImpl.java | 40 ++++++++++--------- 1 file changed, 21 insertions(+), 19 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImpl.java index 2856c5462029..f8346af03ebb 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImpl.java @@ -38,6 +38,7 @@ import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException; import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException; import org.apache.flink.util.FlinkException; +import org.apache.flink.util.OptionalConsumer; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; @@ -530,10 +531,10 @@ private PendingSlotRequest findMatchingRequest(ResourceProfile slotResourceProfi * request fulfillment, then you should override this method. * * @param requestResourceProfile specifying the resource requirements for the a slot request - * @return A matching slot which fulfills the given resource profile. Null if there is no such - * slot available. + * @return A matching slot which fulfills the given resource profile. {@link Optional#empty()} + * if there is no such slot available. */ - private TaskManagerSlot findMatchingSlot(ResourceProfile requestResourceProfile) { + private Optional findMatchingSlot(ResourceProfile requestResourceProfile) { Iterator> iterator = freeSlots.entrySet().iterator(); while (iterator.hasNext()) { @@ -547,11 +548,11 @@ private TaskManagerSlot findMatchingSlot(ResourceProfile requestResourceProfile) if (taskManagerSlot.getResourceProfile().isMatching(requestResourceProfile)) { iterator.remove(); - return taskManagerSlot; + return Optional.of(taskManagerSlot); } } - return null; + return Optional.empty(); } // --------------------------------------------------------------------------------------------- @@ -746,28 +747,29 @@ private void updateSlotState( */ private void internalRequestSlot(PendingSlotRequest pendingSlotRequest) throws ResourceManagerException { final ResourceProfile resourceProfile = pendingSlotRequest.getResourceProfile(); - TaskManagerSlot taskManagerSlot = findMatchingSlot(resourceProfile); - if (taskManagerSlot != null) { - allocateSlot(taskManagerSlot, pendingSlotRequest); - } else { - Optional pendingTaskManagerSlotOptional = findFreeMatchingPendingTaskManagerSlot(resourceProfile); + OptionalConsumer.of(findMatchingSlot(resourceProfile)) + .ifPresent(taskManagerSlot -> allocateSlot(taskManagerSlot, pendingSlotRequest)) + .ifNotPresent(() -> fulfillPendingSlotRequestWithPendingTaskManagerSlot(pendingSlotRequest)); + } - if (!pendingTaskManagerSlotOptional.isPresent()) { - pendingTaskManagerSlotOptional = allocateResource(resourceProfile); - } + private void fulfillPendingSlotRequestWithPendingTaskManagerSlot(PendingSlotRequest pendingSlotRequest) throws ResourceManagerException { + ResourceProfile resourceProfile = pendingSlotRequest.getResourceProfile(); + Optional pendingTaskManagerSlotOptional = findFreeMatchingPendingTaskManagerSlot(resourceProfile); - if (pendingTaskManagerSlotOptional.isPresent()) { - assignPendingTaskManagerSlot(pendingSlotRequest, pendingTaskManagerSlotOptional.get()); - } - else { + if (!pendingTaskManagerSlotOptional.isPresent()) { + pendingTaskManagerSlotOptional = allocateResource(resourceProfile); + } + + OptionalConsumer.of(pendingTaskManagerSlotOptional) + .ifPresent(pendingTaskManagerSlot -> assignPendingTaskManagerSlot(pendingSlotRequest, pendingTaskManagerSlot)) + .ifNotPresent(() -> { // request can not be fulfilled by any free slot or pending slot that can be allocated, // check whether it can be fulfilled by allocated slots if (failUnfulfillableRequest && !isFulfillableByRegisteredSlots(pendingSlotRequest.getResourceProfile())) { throw new UnfulfillableSlotRequestException(pendingSlotRequest.getAllocationId(), pendingSlotRequest.getResourceProfile()); } - } - } + }); } private Optional findFreeMatchingPendingTaskManagerSlot(ResourceProfile requiredResourceProfile) { From 216d0734c7c0685f25c6f93aa1dfcd9b3c1f0359 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 15 Oct 2019 10:35:53 +0200 Subject: [PATCH 495/746] [FLINK-12122] Introduce SlotMatchingStrategy for SlotManager The SlotMatchingStrategy encapsulates how the SlotManager finds a matching slot for a slot request. At the moment, the only implementation AnyMatchingSlotMatchingStrategy picks any matching slot. --- .../ResourceManagerRuntimeServices.java | 2 + .../AnyMatchingSlotMatchingStrategy.java | 42 +++++++++ .../slotmanager/SlotManagerImpl.java | 22 +++-- .../slotmanager/SlotMatchingStrategy.java | 46 ++++++++++ .../AnyMatchingSlotMatchingStrategyTest.java | 91 +++++++++++++++++++ .../slotmanager/SlotManagerBuilder.java | 8 ++ .../TestingTaskManagerSlotInformation.java | 89 ++++++++++++++++++ 7 files changed, 290 insertions(+), 10 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/AnyMatchingSlotMatchingStrategy.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotMatchingStrategy.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/AnyMatchingSlotMatchingStrategyTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/TestingTaskManagerSlotInformation.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServices.java index f2cf01589850..a70498dcb604 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServices.java @@ -20,6 +20,7 @@ import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.resourcemanager.slotmanager.AnyMatchingSlotMatchingStrategy; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerConfiguration; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerImpl; @@ -56,6 +57,7 @@ public static ResourceManagerRuntimeServices fromConfiguration( final SlotManagerConfiguration slotManagerConfiguration = configuration.getSlotManagerConfiguration(); final SlotManager slotManager = new SlotManagerImpl( + AnyMatchingSlotMatchingStrategy.INSTANCE, scheduledExecutor, slotManagerConfiguration.getTaskManagerRequestTimeout(), slotManagerConfiguration.getSlotRequestTimeout(), diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/AnyMatchingSlotMatchingStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/AnyMatchingSlotMatchingStrategy.java new file mode 100644 index 000000000000..6049220abe73 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/AnyMatchingSlotMatchingStrategy.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.runtime.resourcemanager.slotmanager; + +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.instance.InstanceID; + +import java.util.Collection; +import java.util.Optional; +import java.util.function.Function; + +/** + * {@link SlotMatchingStrategy} which picks the first matching slot. + */ +public enum AnyMatchingSlotMatchingStrategy implements SlotMatchingStrategy { + INSTANCE; + + @Override + public Optional findMatchingSlot( + ResourceProfile requestedProfile, + Collection freeSlots, + Function numberRegisteredSlotsLookup) { + + return freeSlots.stream().filter(slot -> slot.isMatchingRequirement(requestedProfile)).findAny(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImpl.java index f8346af03ebb..f983552382c4 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImpl.java @@ -97,6 +97,8 @@ public class SlotManagerImpl implements SlotManager { private final HashMap pendingSlots; + private final SlotMatchingStrategy slotMatchingStrategy; + /** ResourceManager's id. */ private ResourceManagerId resourceManagerId; @@ -124,12 +126,14 @@ public class SlotManagerImpl implements SlotManager { private boolean failUnfulfillableRequest = true; public SlotManagerImpl( + SlotMatchingStrategy slotMatchingStrategy, ScheduledExecutor scheduledExecutor, Time taskManagerRequestTimeout, Time slotRequestTimeout, Time taskManagerTimeout, boolean waitResultConsumedBeforeRelease) { + this.slotMatchingStrategy = Preconditions.checkNotNull(slotMatchingStrategy); this.scheduledExecutor = Preconditions.checkNotNull(scheduledExecutor); this.taskManagerRequestTimeout = Preconditions.checkNotNull(taskManagerRequestTimeout); this.slotRequestTimeout = Preconditions.checkNotNull(slotRequestTimeout); @@ -535,24 +539,22 @@ private PendingSlotRequest findMatchingRequest(ResourceProfile slotResourceProfi * if there is no such slot available. */ private Optional findMatchingSlot(ResourceProfile requestResourceProfile) { - Iterator> iterator = freeSlots.entrySet().iterator(); - - while (iterator.hasNext()) { - TaskManagerSlot taskManagerSlot = iterator.next().getValue(); + final Optional optionalMatchingSlot = slotMatchingStrategy.findMatchingSlot( + requestResourceProfile, + freeSlots.values(), + this::getNumberRegisteredSlotsOf); + optionalMatchingSlot.ifPresent(taskManagerSlot -> { // sanity check Preconditions.checkState( taskManagerSlot.getState() == TaskManagerSlot.State.FREE, "TaskManagerSlot %s is not in state FREE but %s.", taskManagerSlot.getSlotId(), taskManagerSlot.getState()); - if (taskManagerSlot.getResourceProfile().isMatching(requestResourceProfile)) { - iterator.remove(); - return Optional.of(taskManagerSlot); - } - } + freeSlots.remove(taskManagerSlot.getSlotId()); + }); - return Optional.empty(); + return optionalMatchingSlot; } // --------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotMatchingStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotMatchingStrategy.java new file mode 100644 index 000000000000..4411a188617b --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotMatchingStrategy.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.resourcemanager.slotmanager; + +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.instance.InstanceID; + +import java.util.Collection; +import java.util.Optional; +import java.util.function.Function; + +/** + * Strategy how to find a matching slot. + */ +public interface SlotMatchingStrategy { + + /** + * Finds a matching slot for the requested {@link ResourceProfile} given the + * collection of free slots and the total number of slots per TaskExecutor. + * + * @param requestedProfile to find a matching slot for + * @param freeSlots collection of free slots + * @param numberRegisteredSlotsLookup lookup for the number of registered slots + * @return Returns a matching slots or {@link Optional#empty()} if there is none + */ + Optional findMatchingSlot( + ResourceProfile requestedProfile, + Collection freeSlots, + Function numberRegisteredSlotsLookup); +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/AnyMatchingSlotMatchingStrategyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/AnyMatchingSlotMatchingStrategyTest.java new file mode 100644 index 000000000000..cecccbeae62d --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/AnyMatchingSlotMatchingStrategyTest.java @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.resourcemanager.slotmanager; + +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.util.TestLogger; + +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Optional; +import java.util.function.Function; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * Tests for the {@link AnyMatchingSlotMatchingStrategy}. + */ +public class AnyMatchingSlotMatchingStrategyTest extends TestLogger { + + private final InstanceID instanceId = new InstanceID(); + + private TestingTaskManagerSlotInformation largeTaskManagerSlotInformation = null; + private Collection freeSlots = null; + + @Before + public void setup() { + final ResourceProfile largeResourceProfile = new ResourceProfile(10.2 , 42); + final ResourceProfile smallResourceProfile = new ResourceProfile(1 , 1); + + largeTaskManagerSlotInformation = TestingTaskManagerSlotInformation.newBuilder() + .setInstanceId(instanceId) + .setResourceProfile(largeResourceProfile) + .build(); + + freeSlots = Arrays.asList( + TestingTaskManagerSlotInformation.newBuilder() + .setInstanceId(instanceId) + .setResourceProfile(smallResourceProfile) + .build(), + largeTaskManagerSlotInformation); + } + + @Test + public void findMatchingSlot_withFulfillableRequest_returnsFulfillingSlot() { + final Optional optionalMatchingSlot = AnyMatchingSlotMatchingStrategy.INSTANCE.findMatchingSlot( + largeTaskManagerSlotInformation.getResourceProfile(), + freeSlots, + countSlotsPerInstance(freeSlots)); + + assertTrue(optionalMatchingSlot.isPresent()); + assertThat(optionalMatchingSlot.get().getSlotId(), is(largeTaskManagerSlotInformation.getSlotId())); + } + + @Test + public void findMatchingSlot_withUnfulfillableRequest_returnsEmptyResult() { + final Optional optionalMatchingSlot = AnyMatchingSlotMatchingStrategy.INSTANCE.findMatchingSlot( + new ResourceProfile(Double.MAX_VALUE, Integer.MAX_VALUE), + freeSlots, + countSlotsPerInstance(freeSlots)); + + assertFalse(optionalMatchingSlot.isPresent()); + } + + private Function countSlotsPerInstance(Collection freeSlots) { + return currentInstanceId -> (int) freeSlots.stream().filter(slot -> slot.getInstanceId().equals(currentInstanceId)).count(); + } + +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerBuilder.java index 15b65ed9a0db..97b4bdd1aae8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerBuilder.java @@ -24,6 +24,7 @@ /** Builder for {@link SlotManagerImpl}. */ public class SlotManagerBuilder { + private SlotMatchingStrategy slotMatchingStrategy; private ScheduledExecutor scheduledExecutor; private Time taskManagerRequestTimeout; private Time slotRequestTimeout; @@ -31,6 +32,7 @@ public class SlotManagerBuilder { private boolean waitResultConsumedBeforeRelease; private SlotManagerBuilder() { + this.slotMatchingStrategy = AnyMatchingSlotMatchingStrategy.INSTANCE; this.scheduledExecutor = TestingUtils.defaultScheduledExecutor(); this.taskManagerRequestTimeout = TestingUtils.infiniteTime(); this.slotRequestTimeout = TestingUtils.infiniteTime(); @@ -67,8 +69,14 @@ public SlotManagerBuilder setWaitResultConsumedBeforeRelease(boolean waitResultC return this; } + public SlotManagerBuilder setSlotMatchingStrategy(SlotMatchingStrategy slotMatchingStrategy) { + this.slotMatchingStrategy = slotMatchingStrategy; + return this; + } + public SlotManagerImpl build() { return new SlotManagerImpl( + slotMatchingStrategy, scheduledExecutor, taskManagerRequestTimeout, slotRequestTimeout, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/TestingTaskManagerSlotInformation.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/TestingTaskManagerSlotInformation.java new file mode 100644 index 000000000000..1d311f866fbb --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/TestingTaskManagerSlotInformation.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.resourcemanager.slotmanager; + +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.clusterframework.types.SlotID; +import org.apache.flink.runtime.instance.InstanceID; + +/** + * Testing implementation of {@link TaskManagerSlotInformation}. + */ +public final class TestingTaskManagerSlotInformation implements TaskManagerSlotInformation { + + private final SlotID slotId; + private final InstanceID instanceId; + private final ResourceProfile resourceProfile; + + private TestingTaskManagerSlotInformation(SlotID slotId, InstanceID instanceId, ResourceProfile resourceProfile) { + this.slotId = slotId; + this.instanceId = instanceId; + this.resourceProfile = resourceProfile; + } + + @Override + public SlotID getSlotId() { + return slotId; + } + + @Override + public InstanceID getInstanceId() { + return instanceId; + } + + @Override + public boolean isMatchingRequirement(ResourceProfile required) { + return resourceProfile.isMatching(required); + } + + @Override + public ResourceProfile getResourceProfile() { + return resourceProfile; + } + + public static Builder newBuilder() { + return new Builder(); + } + + static class Builder { + private SlotID slotId = new SlotID(ResourceID.generate(), 0); + private InstanceID instanceId = new InstanceID(); + private ResourceProfile resourceProfile = ResourceProfile.ANY; + + public Builder setInstanceId(InstanceID instanceId) { + this.instanceId = instanceId; + return this; + } + + public Builder setResourceProfile(ResourceProfile resourceProfile) { + this.resourceProfile = resourceProfile; + return this; + } + + public Builder setSlotId(SlotID slotId) { + this.slotId = slotId; + return this; + } + + public TestingTaskManagerSlotInformation build() { + return new TestingTaskManagerSlotInformation(slotId, instanceId, resourceProfile); + } + } +} From 797345321266e16e7262a687832111e2b8563598 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 15 Oct 2019 10:36:59 +0200 Subject: [PATCH 496/746] [FLINK-12122] Add LeastUtilizationSlotMatchingStrategy for spreading slot allocations out The LeastUtilizationSlotMatchingStrategy picks the matching slots which belongs to a TaskExecutor with the least utilization value. That way the SlotManager will spread out slot allocations across all available/registered TaskExecutors. --- .../LeastUtilizationSlotMatchingStrategy.java | 65 +++++++++++++++ ...stUtilizationSlotMatchingStrategyTest.java | 80 +++++++++++++++++++ .../slotmanager/SlotManagerImplTest.java | 55 +++++++++++++ 3 files changed, 200 insertions(+) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/LeastUtilizationSlotMatchingStrategy.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/LeastUtilizationSlotMatchingStrategyTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/LeastUtilizationSlotMatchingStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/LeastUtilizationSlotMatchingStrategy.java new file mode 100644 index 000000000000..dd35c110dc7d --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/LeastUtilizationSlotMatchingStrategy.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.resourcemanager.slotmanager; + +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.util.Preconditions; + +import java.util.Collection; +import java.util.Comparator; +import java.util.Map; +import java.util.Optional; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * {@link SlotMatchingStrategy} which picks a matching slot from a TaskExecutor + * with the least utilization. + */ +public enum LeastUtilizationSlotMatchingStrategy implements SlotMatchingStrategy { + INSTANCE; + + @Override + public Optional findMatchingSlot( + ResourceProfile requestedProfile, + Collection freeSlots, + Function numberRegisteredSlotsLookup) { + final Map numSlotsPerTaskExecutor = freeSlots.stream() + .collect(Collectors.groupingBy( + TaskManagerSlotInformation::getInstanceId, + Collectors.reducing(0, i -> 1, Integer::sum))); + + return freeSlots.stream() + .filter(taskManagerSlot -> taskManagerSlot.isMatchingRequirement(requestedProfile)) + .min(Comparator.comparingDouble(taskManagerSlot -> calculateUtilization(taskManagerSlot.getInstanceId(), numberRegisteredSlotsLookup, numSlotsPerTaskExecutor))); + } + + private static double calculateUtilization(InstanceID instanceId, Function numberRegisteredSlotsLookup, Map numSlotsPerTaskExecutor) { + final int numberRegisteredSlots = numberRegisteredSlotsLookup.apply(instanceId); + + Preconditions.checkArgument(numberRegisteredSlots > 0, "The TaskExecutor %s has no slots registered.", instanceId); + + final int numberFreeSlots = numSlotsPerTaskExecutor.getOrDefault(instanceId, 0); + + Preconditions.checkArgument(numberRegisteredSlots >= numberFreeSlots, "The TaskExecutor %s has fewer registered slots than free slots.", instanceId); + + return (double) (numberRegisteredSlots - numberFreeSlots) / numberRegisteredSlots; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/LeastUtilizationSlotMatchingStrategyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/LeastUtilizationSlotMatchingStrategyTest.java new file mode 100644 index 000000000000..df869fa43fd1 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/LeastUtilizationSlotMatchingStrategyTest.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.resourcemanager.slotmanager; + +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.util.TestLogger; + +import org.apache.flink.shaded.guava18.com.google.common.collect.ImmutableMap; + +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Map; +import java.util.Optional; +import java.util.function.Function; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertTrue; + +/** + * Tests for the {@link LeastUtilizationSlotMatchingStrategy}. + */ +public class LeastUtilizationSlotMatchingStrategyTest extends TestLogger { + + @Test + public void findMatchingSlot_multipleMatchingSlots_returnsSlotWithLeastUtilization() { + final ResourceProfile requestedResourceProfile = new ResourceProfile(2.0, 2); + + final TestingTaskManagerSlotInformation leastUtilizedSlot = TestingTaskManagerSlotInformation.newBuilder() + .setResourceProfile(requestedResourceProfile) + .build(); + final TestingTaskManagerSlotInformation tooSmallSlot = TestingTaskManagerSlotInformation.newBuilder() + .setResourceProfile(new ResourceProfile(1.0, 10)) + .build(); + final TestingTaskManagerSlotInformation alternativeSlot = TestingTaskManagerSlotInformation.newBuilder() + .setResourceProfile(requestedResourceProfile) + .build(); + + final Collection freeSlots = Arrays.asList( + tooSmallSlot, + leastUtilizedSlot, + alternativeSlot); + + Map registeredSlotPerTaskExecutor = ImmutableMap.of( + leastUtilizedSlot.getInstanceId(), 1, + tooSmallSlot.getInstanceId(), 1, + alternativeSlot.getInstanceId(), 2); + + final Optional matchingSlot = LeastUtilizationSlotMatchingStrategy.INSTANCE.findMatchingSlot( + requestedResourceProfile, + freeSlots, + createRegisteredSlotsLookupFunction(registeredSlotPerTaskExecutor)); + + assertTrue(matchingSlot.isPresent()); + assertThat(matchingSlot.get().getSlotId(), is(leastUtilizedSlot.getSlotId())); + } + + private Function createRegisteredSlotsLookupFunction(Map registeredSlotPerTaskExecutor) { + return instanceID -> registeredSlotPerTaskExecutor.getOrDefault(instanceID, 0); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImplTest.java index 31323058a014..ffcd98404050 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImplTest.java @@ -1389,6 +1389,10 @@ public void testPendingTaskManagerSlotCompletion() throws Exception { private TaskExecutorConnection createTaskExecutorConnection() { final TestingTaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway(); + return createTaskExecutorConnection(taskExecutorGateway); + } + + private TaskExecutorConnection createTaskExecutorConnection(TaskExecutorGateway taskExecutorGateway) { return new TaskExecutorConnection(ResourceID.generate(), taskExecutorGateway); } @@ -1498,4 +1502,55 @@ private SlotReport createSingleAllocatedSlotReport(ResourceID resourceID, JobID ResourceProfile.UNKNOWN, (slotId, resourceProfile) -> new SlotStatus(slotId, resourceProfile, jobId, new AllocationID())); } + + /** + * The spread out slot allocation strategy should spread out the allocated + * slots across all available TaskExecutors. See FLINK-12122. + */ + @Test + public void testSpreadOutSlotAllocationStrategy() throws Exception { + try (SlotManagerImpl slotManager = SlotManagerBuilder.newBuilder() + .setSlotMatchingStrategy(LeastUtilizationSlotMatchingStrategy.INSTANCE) + .build()) { + slotManager.start( + ResourceManagerId.generate(), + Executors.directExecutor(), + new TestingResourceActionsBuilder().build()); + + final List> requestSlotFutures = new ArrayList<>(); + + final int numberTaskExecutors = 5; + + // register n TaskExecutors with 2 slots each + for (int i = 0; i < numberTaskExecutors; i++) { + final CompletableFuture requestSlotFuture = new CompletableFuture<>(); + requestSlotFutures.add(requestSlotFuture); + registerTaskExecutorWithTwoSlots(slotManager, requestSlotFuture); + } + + final JobID jobId = new JobID(); + + // request n slots + for (int i = 0; i < numberTaskExecutors; i++) { + assertTrue(slotManager.registerSlotRequest(createSlotRequest(jobId))); + } + + // check that every TaskExecutor has received a slot request + final Set jobIds = new HashSet<>(FutureUtils.combineAll(requestSlotFutures).get(10L, TimeUnit.SECONDS)); + assertThat(jobIds, hasSize(1)); + assertThat(jobIds, containsInAnyOrder(jobId)); + } + } + + private void registerTaskExecutorWithTwoSlots(SlotManagerImpl slotManager, CompletableFuture firstRequestSlotFuture) { + final TestingTaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder() + .setRequestSlotFunction(slotIDJobIDAllocationIDStringResourceManagerIdTuple5 -> { + firstRequestSlotFuture.complete(slotIDJobIDAllocationIDStringResourceManagerIdTuple5.f1); + return CompletableFuture.completedFuture(Acknowledge.get()); + }) + .createTestingTaskExecutorGateway(); + final TaskExecutorConnection firstTaskExecutorConnection = createTaskExecutorConnection(taskExecutorGateway); + final SlotReport firstSlotReport = createSlotReport(firstTaskExecutorConnection.getResourceID(), 2); + slotManager.registerTaskManager(firstTaskExecutorConnection, firstSlotReport); + } } From 0df281e48fbb0e378ec5c8454bc56b8ee7c3f67b Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 15 Oct 2019 18:54:39 +0200 Subject: [PATCH 497/746] [FLINK-12122] Introduce ClusterOptions#EVENLY_SPREAD_OUT_SLOTS_STRATEGY Add config option to enable to evenly spread out slots across all available TaskExecutors. --- .../generated/cluster_configuration.html | 5 +++ .../flink/configuration/ClusterOptions.java | 12 +++++++ .../ResourceManagerRuntimeServices.java | 32 +++++++++++++------ .../slotmanager/SlotManagerConfiguration.java | 19 +++++++++-- .../ResourceManagerHATest.java | 3 +- 5 files changed, 59 insertions(+), 12 deletions(-) diff --git a/docs/_includes/generated/cluster_configuration.html b/docs/_includes/generated/cluster_configuration.html index a24a0d8bf1ad..0762085c3e9a 100644 --- a/docs/_includes/generated/cluster_configuration.html +++ b/docs/_includes/generated/cluster_configuration.html @@ -7,6 +7,11 @@ + +

    cluster.evenly-spread-out-slots
    + false + Enable the slot spread out allocation strategy. This strategy tries to spread out the slots evenly across all available `TaskExecutors`. +
    cluster.registration.error-delay
    10000 diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ClusterOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/ClusterOptions.java index eda46facefc1..b1462a9dbf87 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ClusterOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ClusterOptions.java @@ -19,6 +19,9 @@ package org.apache.flink.configuration; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.description.Description; + +import static org.apache.flink.configuration.description.TextElement.code; /** * Options which control the cluster behaviour. @@ -50,4 +53,13 @@ public class ClusterOptions { .key("cluster.services.shutdown-timeout") .defaultValue(30000L) .withDescription("The shutdown timeout for cluster services like executors in milliseconds."); + + public static final ConfigOption EVENLY_SPREAD_OUT_SLOTS_STRATEGY = ConfigOptions + .key("cluster.evenly-spread-out-slots") + .defaultValue(false) + .withDescription( + Description.builder() + .text("Enable the slot spread out allocation strategy. This strategy tries to spread out " + + "the slots evenly across all available %s.", code("TaskExecutors")) + .build()); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServices.java index a70498dcb604..1d947611ec9f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServices.java @@ -21,9 +21,11 @@ import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.resourcemanager.slotmanager.AnyMatchingSlotMatchingStrategy; +import org.apache.flink.runtime.resourcemanager.slotmanager.LeastUtilizationSlotMatchingStrategy; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerConfiguration; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerImpl; +import org.apache.flink.runtime.resourcemanager.slotmanager.SlotMatchingStrategy; import org.apache.flink.util.Preconditions; /** @@ -54,15 +56,7 @@ public static ResourceManagerRuntimeServices fromConfiguration( HighAvailabilityServices highAvailabilityServices, ScheduledExecutor scheduledExecutor) throws Exception { - final SlotManagerConfiguration slotManagerConfiguration = configuration.getSlotManagerConfiguration(); - - final SlotManager slotManager = new SlotManagerImpl( - AnyMatchingSlotMatchingStrategy.INSTANCE, - scheduledExecutor, - slotManagerConfiguration.getTaskManagerRequestTimeout(), - slotManagerConfiguration.getSlotRequestTimeout(), - slotManagerConfiguration.getTaskManagerTimeout(), - slotManagerConfiguration.isWaitResultConsumedBeforeRelease()); + final SlotManager slotManager = createSlotManager(configuration, scheduledExecutor); final JobLeaderIdService jobLeaderIdService = new JobLeaderIdService( highAvailabilityServices, @@ -71,4 +65,24 @@ public static ResourceManagerRuntimeServices fromConfiguration( return new ResourceManagerRuntimeServices(slotManager, jobLeaderIdService); } + + private static SlotManager createSlotManager(ResourceManagerRuntimeServicesConfiguration configuration, ScheduledExecutor scheduledExecutor) { + final SlotManagerConfiguration slotManagerConfiguration = configuration.getSlotManagerConfiguration(); + + final SlotMatchingStrategy slotMatchingStrategy; + + if (slotManagerConfiguration.evenlySpreadOutSlots()) { + slotMatchingStrategy = LeastUtilizationSlotMatchingStrategy.INSTANCE; + } else { + slotMatchingStrategy = AnyMatchingSlotMatchingStrategy.INSTANCE; + } + + return new SlotManagerImpl( + slotMatchingStrategy, + scheduledExecutor, + slotManagerConfiguration.getTaskManagerRequestTimeout(), + slotManagerConfiguration.getSlotRequestTimeout(), + slotManagerConfiguration.getTaskManagerTimeout(), + slotManagerConfiguration.isWaitResultConsumedBeforeRelease()); + } } 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 c50a9a288216..f75771ca7fe5 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 @@ -20,6 +20,7 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.AkkaOptions; +import org.apache.flink.configuration.ClusterOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.ResourceManagerOptions; @@ -41,17 +42,20 @@ public class SlotManagerConfiguration { private final Time slotRequestTimeout; private final Time taskManagerTimeout; private final boolean waitResultConsumedBeforeRelease; + private final boolean evenlySpreadOutSlots; public SlotManagerConfiguration( Time taskManagerRequestTimeout, Time slotRequestTimeout, Time taskManagerTimeout, - boolean waitResultConsumedBeforeRelease) { + boolean waitResultConsumedBeforeRelease, + boolean evenlySpreadOutSlots) { this.taskManagerRequestTimeout = Preconditions.checkNotNull(taskManagerRequestTimeout); this.slotRequestTimeout = Preconditions.checkNotNull(slotRequestTimeout); this.taskManagerTimeout = Preconditions.checkNotNull(taskManagerTimeout); this.waitResultConsumedBeforeRelease = waitResultConsumedBeforeRelease; + this.evenlySpreadOutSlots = evenlySpreadOutSlots; } public Time getTaskManagerRequestTimeout() { @@ -70,6 +74,10 @@ public boolean isWaitResultConsumedBeforeRelease() { return waitResultConsumedBeforeRelease; } + public boolean evenlySpreadOutSlots() { + return evenlySpreadOutSlots; + } + public static SlotManagerConfiguration fromConfiguration(Configuration configuration) throws ConfigurationException { final Time rpcTimeout; try { @@ -86,7 +94,14 @@ public static SlotManagerConfiguration fromConfiguration(Configuration configura boolean waitResultConsumedBeforeRelease = configuration.getBoolean(ResourceManagerOptions.TASK_MANAGER_RELEASE_WHEN_RESULT_CONSUMED); - return new SlotManagerConfiguration(rpcTimeout, slotRequestTimeout, taskManagerTimeout, waitResultConsumedBeforeRelease); + boolean evenlySpreadOutSlots = configuration.getBoolean(ClusterOptions.EVENLY_SPREAD_OUT_SLOTS_STRATEGY); + + return new SlotManagerConfiguration( + rpcTimeout, + slotRequestTimeout, + taskManagerTimeout, + waitResultConsumedBeforeRelease, + evenlySpreadOutSlots); } private static Time getSlotRequestTimeout(final Configuration configuration) { 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 ff2f1ebc30d3..44c02e2ea4a1 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 @@ -68,7 +68,8 @@ public void confirmLeadership(UUID leaderId, String leaderAddress) { TestingUtils.infiniteTime(), TestingUtils.infiniteTime(), TestingUtils.infiniteTime(), - true)); + true, + false)); ResourceManagerRuntimeServices resourceManagerRuntimeServices = ResourceManagerRuntimeServices.fromConfiguration( resourceManagerRuntimeServicesConfiguration, highAvailabilityServices, From 728c02ded019b6557c818f47b40178eee71b733f Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 16 Oct 2019 13:38:29 +0200 Subject: [PATCH 498/746] [hotfix] Replace SlotInfoAndResource helper constructor with static factory method --- .../flink/runtime/jobmaster/slotpool/SchedulerImpl.java | 2 +- .../runtime/jobmaster/slotpool/SlotSelectionStrategy.java | 8 ++++---- .../types/SlotSelectionStrategyTestBase.java | 8 ++++---- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SchedulerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SchedulerImpl.java index ad318858d019..34b2bc5b9483 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SchedulerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SchedulerImpl.java @@ -283,7 +283,7 @@ private Optional tryAllocateFromAvailable( Collection slotInfoList = slotPool.getAvailableSlotsInformation() .stream() - .map(SlotSelectionStrategy.SlotInfoAndResources::new) + .map(SlotSelectionStrategy.SlotInfoAndResources::fromSingleSlot) .collect(Collectors.toList()); Optional selectedAvailableSlot = diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSelectionStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSelectionStrategy.java index 73941153dd54..7d6972057a0c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSelectionStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSelectionStrategy.java @@ -56,10 +56,6 @@ final class SlotInfoAndResources { @Nonnull private final ResourceProfile remainingResources; - public SlotInfoAndResources(@Nonnull SlotInfo slotInfo) { - this(slotInfo, slotInfo.getResourceProfile()); - } - public SlotInfoAndResources(@Nonnull SlotInfo slotInfo, @Nonnull ResourceProfile remainingResources) { this.slotInfo = slotInfo; this.remainingResources = remainingResources; @@ -74,6 +70,10 @@ public SlotInfo getSlotInfo() { public ResourceProfile getRemainingResources() { return remainingResources; } + + public static SlotInfoAndResources fromSingleSlot(@Nonnull SlotInfo slotInfo) { + return new SlotInfoAndResources(slotInfo, slotInfo.getResourceProfile()); + } } /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/SlotSelectionStrategyTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/SlotSelectionStrategyTestBase.java index e420cf4c7fe7..9658decddc02 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/SlotSelectionStrategyTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/SlotSelectionStrategyTestBase.java @@ -65,10 +65,10 @@ public SlotSelectionStrategyTestBase(SlotSelectionStrategy slotSelectionStrategy private Set createCandidates() { Set candidates = new HashSet<>(4); - candidates.add(new SlotSelectionStrategy.SlotInfoAndResources(ssc1)); - candidates.add(new SlotSelectionStrategy.SlotInfoAndResources(ssc2)); - candidates.add(new SlotSelectionStrategy.SlotInfoAndResources(ssc3)); - candidates.add(new SlotSelectionStrategy.SlotInfoAndResources(ssc4)); + candidates.add(SlotSelectionStrategy.SlotInfoAndResources.fromSingleSlot(ssc1)); + candidates.add(SlotSelectionStrategy.SlotInfoAndResources.fromSingleSlot(ssc2)); + candidates.add(SlotSelectionStrategy.SlotInfoAndResources.fromSingleSlot(ssc3)); + candidates.add(SlotSelectionStrategy.SlotInfoAndResources.fromSingleSlot(ssc4)); return candidates; } From 362d7670593adc2e4b20650c8854398727d8102b Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 16 Oct 2019 15:37:53 +0200 Subject: [PATCH 499/746] [FLINK-12122] Calculate TaskExecutorUtilization when listing available slots When listing available slots stored in the SlotPool and the SlotSharingManager, the system will now also calculate the utilization of the owning TaskExecutor wrt the job. --- .../slotpool/SlotInfoWithUtilization.java | 65 ++++++++++++++ .../runtime/jobmaster/slotpool/SlotPool.java | 7 +- .../jobmaster/slotpool/SlotPoolImpl.java | 23 ++++- .../slotpool/SlotSelectionStrategy.java | 16 +++- .../slotpool/SlotSharingManager.java | 76 ++++++++++++++-- ...onPreferenceSlotSelectionStrategyTest.java | 8 +- ...usAllocationSlotSelectionStrategyTest.java | 8 +- .../types/SlotSelectionStrategyTestBase.java | 25 ++++-- .../runtime/jobmaster/JobMasterTest.java | 8 +- .../jobmaster/slotpool/SlotPoolImplTest.java | 50 +++++++++++ .../slotpool/SlotSharingManagerTest.java | 89 +++++++++++-------- 11 files changed, 305 insertions(+), 70 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotInfoWithUtilization.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotInfoWithUtilization.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotInfoWithUtilization.java new file mode 100644 index 000000000000..31b03e172473 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotInfoWithUtilization.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.jobmaster.slotpool; + +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.jobmaster.SlotInfo; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; + +/** + * Container for {@link SlotInfo} and the task executors utilization (freeSlots / totalOfferedSlots). + */ +public final class SlotInfoWithUtilization implements SlotInfo { + private final SlotInfo slotInfoDelegate; + private final double taskExecutorUtilization; + + private SlotInfoWithUtilization(SlotInfo slotInfo, double taskExecutorUtilization) { + this.slotInfoDelegate = slotInfo; + this.taskExecutorUtilization = taskExecutorUtilization; + } + + double getTaskExecutorUtilization() { + return taskExecutorUtilization; + } + + @Override + public AllocationID getAllocationId() { + return slotInfoDelegate.getAllocationId(); + } + + @Override + public TaskManagerLocation getTaskManagerLocation() { + return slotInfoDelegate.getTaskManagerLocation(); + } + + @Override + public int getPhysicalSlotNumber() { + return slotInfoDelegate.getPhysicalSlotNumber(); + } + + @Override + public ResourceProfile getResourceProfile() { + return slotInfoDelegate.getResourceProfile(); + } + + public static SlotInfoWithUtilization from(SlotInfo slotInfo, double taskExecutorUtilization) { + return new SlotInfoWithUtilization(slotInfo, taskExecutorUtilization); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java index 20b1c68242a3..47046d5b9ed7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java @@ -26,7 +26,6 @@ import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; import org.apache.flink.runtime.jobmaster.AllocatedSlotReport; import org.apache.flink.runtime.jobmaster.JobMasterId; -import org.apache.flink.runtime.jobmaster.SlotInfo; import org.apache.flink.runtime.jobmaster.SlotRequestId; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; @@ -128,13 +127,13 @@ Collection offerSlots( // ------------------------------------------------------------------------ /** - * Returns a list of {@link SlotInfo} objects about all slots that are currently available in the slot + * Returns a list of {@link SlotInfoWithUtilization} objects about all slots that are currently available in the slot * pool. * - * @return a list of {@link SlotInfo} objects about all slots that are currently available in the slot pool. + * @return a list of {@link SlotInfoWithUtilization} objects about all slots that are currently available in the slot pool. */ @Nonnull - Collection getAvailableSlotsInformation(); + Collection getAvailableSlotsInformation(); /** * Allocates the available slot with the given allocation id under the given request id. This method returns diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolImpl.java index a38161367837..03ae4a800fbe 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolImpl.java @@ -451,8 +451,19 @@ public CompletableFuture requestNewAllocatedBatchSlot( @Override @Nonnull - public Collection getAvailableSlotsInformation() { - return availableSlots.listSlotInfo(); + public Collection getAvailableSlotsInformation() { + final Map> availableSlotsByTaskManager = availableSlots.getSlotsByTaskManager(); + final Map> allocatedSlotsSlotsByTaskManager = allocatedSlots.getSlotsByTaskManager(); + + return availableSlotsByTaskManager.entrySet().stream() + .flatMap(entry -> { + final int numberAllocatedSlots = allocatedSlotsSlotsByTaskManager.getOrDefault(entry.getKey(), Collections.emptySet()).size(); + final int numberAvailableSlots = entry.getValue().size(); + final double taskExecutorUtilization = (double) numberAllocatedSlots / (numberAllocatedSlots + numberAvailableSlots); + + return entry.getValue().stream().map(slot -> SlotInfoWithUtilization.from(slot, taskExecutorUtilization)); + }) + .collect(Collectors.toList()); } private void releaseSingleSlot(SlotRequestId slotRequestId, Throwable cause) { @@ -1145,6 +1156,10 @@ Set getSlotsForTaskManager(ResourceID resourceId) { Collection listSlotInfo() { return new ArrayList<>(allocatedSlotsById.values()); } + + Map> getSlotsByTaskManager() { + return Collections.unmodifiableMap(allocatedSlotsByTaskManager); + } } // ------------------------------------------------------------------------ @@ -1309,6 +1324,10 @@ void clear() { Set getSlotsForTaskManager(ResourceID resourceId) { return availableSlotsByTaskManager.getOrDefault(resourceId, Collections.emptySet()); } + + Map> getSlotsByTaskManager() { + return Collections.unmodifiableMap(availableSlotsByTaskManager); + } } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSelectionStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSelectionStrategy.java index 7d6972057a0c..8a3bb1443d34 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSelectionStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSelectionStrategy.java @@ -56,9 +56,12 @@ final class SlotInfoAndResources { @Nonnull private final ResourceProfile remainingResources; - public SlotInfoAndResources(@Nonnull SlotInfo slotInfo, @Nonnull ResourceProfile remainingResources) { + private final double taskExecutorUtilization; + + public SlotInfoAndResources(@Nonnull SlotInfo slotInfo, @Nonnull ResourceProfile remainingResources, double taskExecutorUtilization) { this.slotInfo = slotInfo; this.remainingResources = remainingResources; + this.taskExecutorUtilization = taskExecutorUtilization; } @Nonnull @@ -71,8 +74,15 @@ public ResourceProfile getRemainingResources() { return remainingResources; } - public static SlotInfoAndResources fromSingleSlot(@Nonnull SlotInfo slotInfo) { - return new SlotInfoAndResources(slotInfo, slotInfo.getResourceProfile()); + public double getTaskExecutorUtilization() { + return taskExecutorUtilization; + } + + public static SlotInfoAndResources fromSingleSlot(@Nonnull SlotInfoWithUtilization slotInfoWithUtilization) { + return new SlotInfoAndResources( + slotInfoWithUtilization, + slotInfoWithUtilization.getResourceProfile(), + slotInfoWithUtilization.getTaskExecutorUtilization()); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSharingManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSharingManager.java index 08d823661931..ac2363b4c7d0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSharingManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSharingManager.java @@ -53,6 +53,7 @@ import java.util.function.Function; import java.util.function.Predicate; import java.util.stream.Collectors; +import java.util.stream.Stream; /** * Manager which is responsible for slot sharing. Slot sharing allows to run different @@ -185,18 +186,55 @@ public Collection listResolvedRootSl return resolvedRootSlots .values() .stream() - .flatMap((Map map) -> map.values().stream()) - .filter(validMultiTaskSlotAndDoesNotContain(groupId)) - .map((MultiTaskSlot multiTaskSlot) -> { - SlotInfo slotInfo = multiTaskSlot.getSlotContextFuture().join(); + .flatMap((Map map) -> createValidMultiTaskSlotInfos(map, groupId)) + .map((MultiTaskSlotInfo multiTaskSlotInfo) -> { + SlotInfo slotInfo = multiTaskSlotInfo.getSlotInfo(); return new SlotSelectionStrategy.SlotInfoAndResources( - slotInfo, - slotInfo.getResourceProfile().subtract(multiTaskSlot.getReservedResources())); + slotInfo, + slotInfo.getResourceProfile().subtract(multiTaskSlotInfo.getReservedResources()), + multiTaskSlotInfo.getTaskExecutorUtilization()); }).collect(Collectors.toList()); } + private Stream createValidMultiTaskSlotInfos(Map taskExecutorSlots, AbstractID groupId) { + final double taskExecutorUtilization = calculateTaskExecutorUtilization(taskExecutorSlots, groupId); + + return taskExecutorSlots.values().stream() + .filter(validMultiTaskSlotAndDoesNotContain(groupId)) + .map(multiTaskSlot -> + new MultiTaskSlotInfo( + multiTaskSlot.getSlotContextFuture().join(), + multiTaskSlot.getReservedResources(), + taskExecutorUtilization)); + } + + private double calculateTaskExecutorUtilization(Map map, AbstractID groupId) { + int numberValidSlots = 0; + int numberFreeSlots = 0; + + for (MultiTaskSlot multiTaskSlot : map.values()) { + if (isNotReleasing(multiTaskSlot)) { + numberValidSlots++; + + if (doesNotContain(groupId, multiTaskSlot)) { + numberFreeSlots++; + } + } + } + + return (double) (numberValidSlots - numberFreeSlots) / numberValidSlots; + } + + private boolean isNotReleasing(MultiTaskSlot multiTaskSlot) { + return !multiTaskSlot.isReleasing(); + } + + private boolean doesNotContain(@Nullable AbstractID groupId, MultiTaskSlot multiTaskSlot) { + return !multiTaskSlot.contains(groupId); + } + private Predicate validMultiTaskSlotAndDoesNotContain(@Nullable AbstractID groupId) { - return (MultiTaskSlot multiTaskSlot) -> !multiTaskSlot.contains(groupId) && !multiTaskSlot.isReleasing(); + return (MultiTaskSlot multiTaskSlot) -> doesNotContain(groupId, multiTaskSlot) && isNotReleasing(multiTaskSlot); } @Nullable @@ -816,4 +854,28 @@ private void progressToNextElement() { } } } + + private static class MultiTaskSlotInfo { + private final SlotInfo slotInfo; + private final ResourceProfile reservedResources; + private final double taskExecutorUtilization; + + private MultiTaskSlotInfo(SlotInfo slotInfo, ResourceProfile reservedResources, double taskExecutorUtilization) { + this.slotInfo = slotInfo; + this.reservedResources = reservedResources; + this.taskExecutorUtilization = taskExecutorUtilization; + } + + private ResourceProfile getReservedResources() { + return reservedResources; + } + + private double getTaskExecutorUtilization() { + return taskExecutorUtilization; + } + + private SlotInfo getSlotInfo() { + return slotInfo; + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/LocationPreferenceSlotSelectionStrategyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/LocationPreferenceSlotSelectionStrategyTest.java index 378d818c188f..4b9189c9edb0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/LocationPreferenceSlotSelectionStrategyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/LocationPreferenceSlotSelectionStrategyTest.java @@ -88,17 +88,17 @@ public void matchPreferredLocation() { SlotProfile slotProfile = new SlotProfile(resourceProfile, Collections.singletonList(tml2), Collections.emptySet()); Optional match = runMatching(slotProfile); - Assert.assertEquals(ssc2, match.get().getSlotInfo()); + Assert.assertEquals(slotInfo2, match.get().getSlotInfo()); slotProfile = new SlotProfile(resourceProfile, Arrays.asList(tmlX, tml4), Collections.emptySet()); match = runMatching(slotProfile); - Assert.assertEquals(ssc4, match.get().getSlotInfo()); + Assert.assertEquals(slotInfo4, match.get().getSlotInfo()); slotProfile = new SlotProfile(resourceProfile, Arrays.asList(tml3, tml1, tml3, tmlX), Collections.emptySet()); match = runMatching(slotProfile); - Assert.assertEquals(ssc3, match.get().getSlotInfo()); + Assert.assertEquals(slotInfo3, match.get().getSlotInfo()); } @Test @@ -112,6 +112,6 @@ public void matchPreviousLocationAvailableButAlsoBlacklisted() { Optional match = runMatching(slotProfile); // available previous allocation should override blacklisting - Assert.assertEquals(ssc3, match.get().getSlotInfo()); + Assert.assertEquals(slotInfo3, match.get().getSlotInfo()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/PreviousAllocationSlotSelectionStrategyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/PreviousAllocationSlotSelectionStrategyTest.java index 2a3a16f37f45..afbf8879eb2f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/PreviousAllocationSlotSelectionStrategyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/PreviousAllocationSlotSelectionStrategyTest.java @@ -41,12 +41,12 @@ public void matchPreviousAllocationOverridesPreferredLocation() { SlotProfile slotProfile = new SlotProfile(resourceProfile, Collections.singletonList(tml2), Collections.singleton(aid3)); Optional match = runMatching(slotProfile); - Assert.assertEquals(ssc3, match.get().getSlotInfo()); + Assert.assertEquals(slotInfo3, match.get().getSlotInfo()); slotProfile = new SlotProfile(resourceProfile, Arrays.asList(tmlX, tml1), new HashSet<>(Arrays.asList(aidX, aid2))); match = runMatching(slotProfile); - Assert.assertEquals(ssc2, match.get().getSlotInfo()); + Assert.assertEquals(slotInfo2, match.get().getSlotInfo()); } @Test @@ -55,7 +55,7 @@ public void matchPreviousLocationNotAvailableButByLocality() { SlotProfile slotProfile = new SlotProfile(resourceProfile, Collections.singletonList(tml4), Collections.singleton(aidX)); Optional match = runMatching(slotProfile); - Assert.assertEquals(ssc4, match.get().getSlotInfo()); + Assert.assertEquals(slotInfo4, match.get().getSlotInfo()); } @Test @@ -82,6 +82,6 @@ public void matchPreviousLocationNotAvailableAndSomeOthersBlacklisted() { Optional match = runMatching(slotProfile); // we expect that the candidate that is not blacklisted is returned - Assert.assertEquals(ssc2, match.get().getSlotInfo()); + Assert.assertEquals(slotInfo2, match.get().getSlotInfo()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/SlotSelectionStrategyTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/SlotSelectionStrategyTestBase.java index 9658decddc02..616d5ec730ad 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/SlotSelectionStrategyTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/SlotSelectionStrategyTestBase.java @@ -21,6 +21,7 @@ import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; import org.apache.flink.runtime.instance.SimpleSlotContext; import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; +import org.apache.flink.runtime.jobmaster.slotpool.SlotInfoWithUtilization; import org.apache.flink.runtime.jobmaster.slotpool.SlotSelectionStrategy; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.TestLogger; @@ -50,10 +51,18 @@ public abstract class SlotSelectionStrategyTestBase extends TestLogger { protected final TaskManagerGateway taskManagerGateway = new SimpleAckingTaskManagerGateway(); - protected final SimpleSlotContext ssc1 = new SimpleSlotContext(aid1, tml1, 1, taskManagerGateway, resourceProfile); - protected final SimpleSlotContext ssc2 = new SimpleSlotContext(aid2, tml2, 2, taskManagerGateway, biggerResourceProfile); - protected final SimpleSlotContext ssc3 = new SimpleSlotContext(aid3, tml3, 3, taskManagerGateway, resourceProfile); - protected final SimpleSlotContext ssc4 = new SimpleSlotContext(aid4, tml4, 4, taskManagerGateway, resourceProfile); + protected final SlotInfoWithUtilization slotInfo1 = SlotInfoWithUtilization.from( + new SimpleSlotContext(aid1, tml1, 1, taskManagerGateway, resourceProfile), + 0); + protected final SlotInfoWithUtilization slotInfo2 = SlotInfoWithUtilization.from( + new SimpleSlotContext(aid2, tml2, 2, taskManagerGateway, biggerResourceProfile), + 0); + protected final SlotInfoWithUtilization slotInfo3 = SlotInfoWithUtilization.from( + new SimpleSlotContext(aid3, tml3, 3, taskManagerGateway, resourceProfile), + 0); + protected final SlotInfoWithUtilization slotInfo4 = SlotInfoWithUtilization.from( + new SimpleSlotContext(aid4, tml4, 4, taskManagerGateway, resourceProfile), + 0); protected final Set candidates = Collections.unmodifiableSet(createCandidates()); @@ -65,10 +74,10 @@ public SlotSelectionStrategyTestBase(SlotSelectionStrategy slotSelectionStrategy private Set createCandidates() { Set candidates = new HashSet<>(4); - candidates.add(SlotSelectionStrategy.SlotInfoAndResources.fromSingleSlot(ssc1)); - candidates.add(SlotSelectionStrategy.SlotInfoAndResources.fromSingleSlot(ssc2)); - candidates.add(SlotSelectionStrategy.SlotInfoAndResources.fromSingleSlot(ssc3)); - candidates.add(SlotSelectionStrategy.SlotInfoAndResources.fromSingleSlot(ssc4)); + candidates.add(SlotSelectionStrategy.SlotInfoAndResources.fromSingleSlot(slotInfo1)); + candidates.add(SlotSelectionStrategy.SlotInfoAndResources.fromSingleSlot(slotInfo2)); + candidates.add(SlotSelectionStrategy.SlotInfoAndResources.fromSingleSlot(slotInfo3)); + candidates.add(SlotSelectionStrategy.SlotInfoAndResources.fromSingleSlot(slotInfo4)); return candidates; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java index 698b1928bb65..601e42e8e4c1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java @@ -88,6 +88,7 @@ import org.apache.flink.runtime.jobmaster.slotpool.DefaultSchedulerFactory; import org.apache.flink.runtime.jobmaster.slotpool.DefaultSlotPoolFactory; import org.apache.flink.runtime.jobmaster.slotpool.PhysicalSlot; +import org.apache.flink.runtime.jobmaster.slotpool.SlotInfoWithUtilization; import org.apache.flink.runtime.jobmaster.slotpool.SlotPool; import org.apache.flink.runtime.jobmaster.slotpool.SlotPoolFactory; import org.apache.flink.runtime.jobmaster.utils.JobMasterBuilder; @@ -570,8 +571,11 @@ public Optional failAllocation(AllocationID allocationID, Exception @Nonnull @Override - public Collection getAvailableSlotsInformation() { - final Collection allSlotInfos = registeredSlots.values().stream().flatMap(Collection::stream).collect(Collectors.toList()); + public Collection getAvailableSlotsInformation() { + final Collection allSlotInfos = registeredSlots.values().stream() + .flatMap(Collection::stream) + .map(slot -> SlotInfoWithUtilization.from(slot, 0)) + .collect(Collectors.toList()); return Collections.unmodifiableCollection(allSlotInfos); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolImplTest.java index d36be1ad5263..fc3b2787a4a5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolImplTest.java @@ -51,6 +51,8 @@ import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; +import org.apache.flink.shaded.guava18.com.google.common.collect.ImmutableMap; + import org.hamcrest.Description; import org.hamcrest.Matcher; import org.hamcrest.Matchers; @@ -73,9 +75,11 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; +import java.util.stream.IntStream; import static org.apache.flink.runtime.jobmaster.slotpool.AvailableSlotsTest.DEFAULT_TESTING_PROFILE; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.closeTo; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; @@ -706,6 +710,52 @@ public void testCreateAllocatedSlotReport() throws Exception { } } + @Test + public void testCalculationOfTaskExecutorUtilization() throws Exception { + try (final SlotPoolImpl slotPool = createSlotPoolImpl()) { + setupSlotPool(slotPool, resourceManagerGateway, mainThreadExecutor); + + final TaskManagerLocation firstTaskManagerLocation = new LocalTaskManagerLocation(); + final TaskManagerLocation secondTaskManagerLocation = new LocalTaskManagerLocation(); + + final List firstTaskManagersSlots = registerAndOfferSlots(firstTaskManagerLocation, slotPool, 4); + final List secondTaskManagersSlots = registerAndOfferSlots(secondTaskManagerLocation, slotPool, 4); + + slotPool.allocateAvailableSlot(new SlotRequestId(), firstTaskManagersSlots.get(0)); + slotPool.allocateAvailableSlot(new SlotRequestId(), firstTaskManagersSlots.get(1)); + slotPool.allocateAvailableSlot(new SlotRequestId(), secondTaskManagersSlots.get(3)); + + final Collection availableSlotsInformation = slotPool.getAvailableSlotsInformation(); + + final Map utilizationPerTaskExecutor = ImmutableMap.of( + firstTaskManagerLocation, 2.0 / 4, + secondTaskManagerLocation, 1.0 / 4); + + for (SlotInfoWithUtilization slotInfoWithUtilization : availableSlotsInformation) { + final double expectedTaskExecutorUtilization = utilizationPerTaskExecutor.get(slotInfoWithUtilization.getTaskManagerLocation()); + assertThat(slotInfoWithUtilization.getTaskExecutorUtilization(), is(closeTo(expectedTaskExecutorUtilization, 0.1))); + } + } + } + + private List registerAndOfferSlots(TaskManagerLocation taskManagerLocation, SlotPoolImpl slotPool, int numberOfSlotsToRegister) { + slotPool.registerTaskManager(taskManagerLocation.getResourceID()); + final List allocationIds = IntStream.range(0, numberOfSlotsToRegister) + .mapToObj(ignored -> new AllocationID()) + .collect(Collectors.toList()); + + Collection slotOffers = IntStream.range(0, numberOfSlotsToRegister) + .mapToObj(index -> new SlotOffer(allocationIds.get(index), index, ResourceProfile.ANY)) + .collect(Collectors.toList()); + + slotPool.offerSlots( + taskManagerLocation, + new SimpleAckingTaskManagerGateway(), + slotOffers); + + return allocationIds; + } + private static Collection> isEachEqual(Collection allocatedSlotInfos) { return allocatedSlotInfos .stream() diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSharingManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSharingManagerTest.java index 04ef8b2bacbb..c133e51a35a8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSharingManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSharingManagerTest.java @@ -48,13 +48,17 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.Matchers.closeTo; +import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.core.Is.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -427,15 +431,7 @@ public void testGetResolvedSlot() { allocatedSlotActions, SLOT_OWNER); - SlotSharingManager.MultiTaskSlot rootSlot = slotSharingManager.createRootSlot( - new SlotRequestId(), - CompletableFuture.completedFuture( - new SimpleSlotContext( - new AllocationID(), - new LocalTaskManagerLocation(), - 0, - new SimpleAckingTaskManagerGateway())), - new SlotRequestId()); + SlotSharingManager.MultiTaskSlot rootSlot = createRootSlot(new LocalTaskManagerLocation(), slotSharingManager); AbstractID groupId = new AbstractID(); @@ -476,26 +472,10 @@ public void testGetResolvedSlotWithLocationPreferences() { allocatedSlotActions, SLOT_OWNER); - SlotSharingManager.MultiTaskSlot rootSlot1 = slotSharingManager.createRootSlot( - new SlotRequestId(), - CompletableFuture.completedFuture( - new SimpleSlotContext( - new AllocationID(), - new LocalTaskManagerLocation(), - 0, - new SimpleAckingTaskManagerGateway())), - new SlotRequestId()); + SlotSharingManager.MultiTaskSlot rootSlot1 = createRootSlot(new LocalTaskManagerLocation(), slotSharingManager); LocalTaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); - SlotSharingManager.MultiTaskSlot rootSlot2 = slotSharingManager.createRootSlot( - new SlotRequestId(), - CompletableFuture.completedFuture( - new SimpleSlotContext( - new AllocationID(), - taskManagerLocation, - 0, - new SimpleAckingTaskManagerGateway())), - new SlotRequestId()); + SlotSharingManager.MultiTaskSlot rootSlot2 = createRootSlot(taskManagerLocation, slotSharingManager); AbstractID groupId = new AbstractID(); @@ -537,15 +517,7 @@ public void testResolvedSlotInReleasingIsNotAvailable() throws Exception { allocatedSlotActions, SLOT_OWNER); - final SlotSharingManager.MultiTaskSlot rootSlot = slotSharingManager.createRootSlot( - new SlotRequestId(), - CompletableFuture.completedFuture( - new SimpleSlotContext( - new AllocationID(), - new LocalTaskManagerLocation(), - 0, - new SimpleAckingTaskManagerGateway())), - new SlotRequestId()); + final SlotSharingManager.MultiTaskSlot rootSlot = createRootSlot(new LocalTaskManagerLocation(), slotSharingManager); final AbstractID groupId1 = new AbstractID(); final SlotSharingManager.SingleTaskSlot singleTaskSlot = rootSlot.allocateSingleTaskSlot( @@ -924,4 +896,49 @@ private class SlotSharingResourceTestContext { this.singleTaskSlotsInOrder = singleTaskSlotsInOrder; } } + + @Test + public void testTaskExecutorUtilizationCalculation() { + final TestingAllocatedSlotActions allocatedSlotActions = new TestingAllocatedSlotActions(); + final TaskManagerLocation firstTaskExecutorLocation = new LocalTaskManagerLocation(); + final TaskManagerLocation secondTaskExecutorLocation = new LocalTaskManagerLocation(); + + SlotSharingManager slotSharingManager = new SlotSharingManager( + SLOT_SHARING_GROUP_ID, + allocatedSlotActions, + SLOT_OWNER); + + final SlotSharingManager.MultiTaskSlot firstRootSlot = createRootSlot(firstTaskExecutorLocation, slotSharingManager); + createRootSlot(firstTaskExecutorLocation, slotSharingManager); + createRootSlot(secondTaskExecutorLocation, slotSharingManager); + + final AbstractID groupId = new AbstractID(); + + firstRootSlot.allocateSingleTaskSlot(new SlotRequestId(), ResourceProfile.UNKNOWN, groupId, Locality.UNCONSTRAINED); + + final Collection slotInfoAndResources = slotSharingManager.listResolvedRootSlotInfo(groupId); + + assertThat(slotInfoAndResources, hasSize(2)); + + final Map utilizationPerTaskExecutor = slotInfoAndResources.stream() + .collect( + Collectors.toMap( + slot -> slot.getSlotInfo().getTaskManagerLocation(), + SlotSelectionStrategy.SlotInfoAndResources::getTaskExecutorUtilization)); + + assertThat(utilizationPerTaskExecutor.get(firstTaskExecutorLocation), is(closeTo(1.0 / 2, 0.1))); + assertThat(utilizationPerTaskExecutor.get(secondTaskExecutorLocation), is(closeTo(0, 0.1))); + } + + private SlotSharingManager.MultiTaskSlot createRootSlot(TaskManagerLocation firstTaskExecutorLocation, SlotSharingManager slotSharingManager) { + return slotSharingManager.createRootSlot( + new SlotRequestId(), + CompletableFuture.completedFuture( + new SimpleSlotContext( + new AllocationID(), + firstTaskExecutorLocation, + 0, + new SimpleAckingTaskManagerGateway())), + new SlotRequestId()); + } } From 2ab8b61f2f22f1a1ce7f92cd6b8dd32d2c0c227d Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 16 Oct 2019 15:39:22 +0200 Subject: [PATCH 500/746] [FLINK-12122] Add EvenlySpreadOutLocationPreferenceSlotSelectionStrategy The EvenlySpreadOutLocationPreferenceSlotSelectionStrategy is a special implementation of the LocationPreferenceSlotSelectionStrategy which tries to evenly spread out the workload across all TaskExecutors by choosing the slot with the least utilization if there is a tie wrt the locality. --- ...cationPreferenceSlotSelectionStrategy.java | 46 +++++ .../slotpool/DefaultSchedulerFactory.java | 4 +- ...cationPreferenceSlotSelectionStrategy.java | 46 +++++ ...cationPreferenceSlotSelectionStrategy.java | 51 +++-- ...eviousAllocationSlotSelectionStrategy.java | 14 +- ...onPreferenceSlotSelectionStrategyTest.java | 2 +- ...usAllocationSlotSelectionStrategyTest.java | 2 +- .../ExecutionGraphNotEnoughResourceTest.java | 2 +- .../ExecutionGraphRestartTest.java | 2 +- .../scheduler/SchedulerTestBase.java | 2 +- .../slotpool/SlotPoolCoLocationTest.java | 2 +- .../jobmaster/slotpool/SlotPoolImplTest.java | 2 +- .../slotpool/SlotPoolInteractionsTest.java | 10 +- .../slotpool/SlotPoolSlotSharingTest.java | 2 +- .../slotpool/SlotPoolSlotSpreadOutTest.java | 176 ++++++++++++++++++ .../slotpool/SlotSharingManagerTest.java | 10 +- .../LegacySchedulerBatchSchedulingTest.java | 2 +- 17 files changed, 326 insertions(+), 49 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultLocationPreferenceSlotSelectionStrategy.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/EvenlySpreadOutLocationPreferenceSlotSelectionStrategy.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolSlotSpreadOutTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultLocationPreferenceSlotSelectionStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultLocationPreferenceSlotSelectionStrategy.java new file mode 100644 index 000000000000..b69e63323bac --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultLocationPreferenceSlotSelectionStrategy.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmaster.slotpool; + +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.jobmanager.scheduler.Locality; + +import javax.annotation.Nonnull; + +import java.util.Collection; +import java.util.Optional; + +class DefaultLocationPreferenceSlotSelectionStrategy extends LocationPreferenceSlotSelectionStrategy { + + @Nonnull + @Override + protected Optional selectWithoutLocationPreference(@Nonnull Collection availableSlots, @Nonnull ResourceProfile resourceProfile) { + for (SlotInfoAndResources candidate : availableSlots) { + if (candidate.getRemainingResources().isMatching(resourceProfile)) { + return Optional.of(SlotInfoAndLocality.of(candidate.getSlotInfo(), Locality.UNCONSTRAINED)); + } + } + return Optional.empty(); + } + + @Override + protected double calculateCandidateScore(int localWeigh, int hostLocalWeigh, double taskExecutorUtilization) { + return localWeigh * 10 + hostLocalWeigh; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultSchedulerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultSchedulerFactory.java index 72345df6b8b0..e0ae00e6606e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultSchedulerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultSchedulerFactory.java @@ -44,9 +44,9 @@ public Scheduler createScheduler(@Nonnull SlotPool slotPool) { @Nonnull private static SlotSelectionStrategy selectSlotSelectionStrategy(@Nonnull Configuration configuration) { if (configuration.getBoolean(CheckpointingOptions.LOCAL_RECOVERY)) { - return PreviousAllocationSlotSelectionStrategy.INSTANCE; + return PreviousAllocationSlotSelectionStrategy.create(); } else { - return LocationPreferenceSlotSelectionStrategy.INSTANCE; + return LocationPreferenceSlotSelectionStrategy.createDefault(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/EvenlySpreadOutLocationPreferenceSlotSelectionStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/EvenlySpreadOutLocationPreferenceSlotSelectionStrategy.java new file mode 100644 index 000000000000..db492a13e3ce --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/EvenlySpreadOutLocationPreferenceSlotSelectionStrategy.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmaster.slotpool; + +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.jobmanager.scheduler.Locality; + +import javax.annotation.Nonnull; + +import java.util.Collection; +import java.util.Comparator; +import java.util.Optional; + +class EvenlySpreadOutLocationPreferenceSlotSelectionStrategy extends LocationPreferenceSlotSelectionStrategy { + @Nonnull + @Override + protected Optional selectWithoutLocationPreference(@Nonnull Collection availableSlots, @Nonnull ResourceProfile resourceProfile) { + return availableSlots.stream() + .filter(slotInfoAndResources -> slotInfoAndResources.getRemainingResources().isMatching(resourceProfile)) + .min(Comparator.comparing(SlotInfoAndResources::getTaskExecutorUtilization)) + .map(slotInfoAndResources -> SlotInfoAndLocality.of(slotInfoAndResources.getSlotInfo(), Locality.UNCONSTRAINED)); + } + + @Override + protected double calculateCandidateScore(int localWeigh, int hostLocalWeigh, double taskExecutorUtilization) { + // taskExecutorUtilization in [0, 1] --> only affects choice if localWeigh and hostLocalWeigh + // between two candidates are equal + return localWeigh * 20 + hostLocalWeigh * 2 - taskExecutorUtilization; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/LocationPreferenceSlotSelectionStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/LocationPreferenceSlotSelectionStrategy.java index 1a80c2d242be..266da836e3c4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/LocationPreferenceSlotSelectionStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/LocationPreferenceSlotSelectionStrategy.java @@ -30,20 +30,13 @@ import java.util.HashMap; import java.util.Map; import java.util.Optional; -import java.util.function.BiFunction; /** * This class implements a {@link SlotSelectionStrategy} that is based on location preference hints. */ -public enum LocationPreferenceSlotSelectionStrategy implements SlotSelectionStrategy { +public abstract class LocationPreferenceSlotSelectionStrategy implements SlotSelectionStrategy { - INSTANCE; - - /** - * Calculates the candidate's locality score. - */ - private static final BiFunction LOCALITY_EVALUATION_FUNCTION = - (localWeigh, hostLocalWeigh) -> localWeigh * 10 + hostLocalWeigh; + LocationPreferenceSlotSelectionStrategy() {} @Override public Optional selectBestSlotForProfile( @@ -64,19 +57,6 @@ public Optional selectBestSlotForProfile( selectWitLocationPreference(availableSlots, locationPreferences, resourceProfile); } - @Nonnull - private Optional selectWithoutLocationPreference( - @Nonnull Collection availableSlots, - @Nonnull ResourceProfile resourceProfile) { - - for (SlotInfoAndResources candidate : availableSlots) { - if (candidate.getRemainingResources().isMatching(resourceProfile)) { - return Optional.of(SlotInfoAndLocality.of(candidate.getSlotInfo(), Locality.UNCONSTRAINED)); - } - } - return Optional.empty(); - } - @Nonnull private Optional selectWitLocationPreference( @Nonnull Collection availableSlots, @@ -94,21 +74,21 @@ private Optional selectWitLocationPreference( SlotInfoAndResources bestCandidate = null; Locality bestCandidateLocality = Locality.UNKNOWN; - int bestCandidateScore = Integer.MIN_VALUE; + double bestCandidateScore = Double.MIN_VALUE; for (SlotInfoAndResources candidate : availableSlots) { if (candidate.getRemainingResources().isMatching(resourceProfile)) { // this gets candidate is local-weigh - Integer localWeigh = preferredResourceIDs.getOrDefault( + int localWeigh = preferredResourceIDs.getOrDefault( candidate.getSlotInfo().getTaskManagerLocation().getResourceID(), 0); // this gets candidate is host-local-weigh - Integer hostLocalWeigh = preferredFQHostNames.getOrDefault( + int hostLocalWeigh = preferredFQHostNames.getOrDefault( candidate.getSlotInfo().getTaskManagerLocation().getFQDNHostname(), 0); - int candidateScore = LOCALITY_EVALUATION_FUNCTION.apply(localWeigh, hostLocalWeigh); + double candidateScore = calculateCandidateScore(localWeigh, hostLocalWeigh, candidate.getTaskExecutorUtilization()); if (candidateScore > bestCandidateScore) { bestCandidateScore = candidateScore; bestCandidate = candidate; @@ -124,4 +104,23 @@ private Optional selectWitLocationPreference( Optional.of(SlotInfoAndLocality.of(bestCandidate.getSlotInfo(), bestCandidateLocality)) : Optional.empty(); } + + @Nonnull + protected abstract Optional selectWithoutLocationPreference( + @Nonnull Collection availableSlots, + @Nonnull ResourceProfile resourceProfile); + + protected abstract double calculateCandidateScore(int localWeigh, int hostLocalWeigh, double taskExecutorUtilization); + + // ------------------------------------------------------------------------------------------- + // Factory methods + // ------------------------------------------------------------------------------------------- + + public static LocationPreferenceSlotSelectionStrategy createDefault() { + return new DefaultLocationPreferenceSlotSelectionStrategy(); + } + + public static LocationPreferenceSlotSelectionStrategy createEvenlySpreadOut() { + return new EvenlySpreadOutLocationPreferenceSlotSelectionStrategy(); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PreviousAllocationSlotSelectionStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PreviousAllocationSlotSelectionStrategy.java index f8feda8ac4d9..85ed43eb3ec2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PreviousAllocationSlotSelectionStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PreviousAllocationSlotSelectionStrategy.java @@ -34,9 +34,13 @@ * This class implements a {@link SlotSelectionStrategy} that is based on previous allocations and * falls back to using location preference hints if there is no previous allocation. */ -public enum PreviousAllocationSlotSelectionStrategy implements SlotSelectionStrategy { +public class PreviousAllocationSlotSelectionStrategy implements SlotSelectionStrategy { - INSTANCE; + private final SlotSelectionStrategy fallbackSlotSelectionStrategy; + + private PreviousAllocationSlotSelectionStrategy(SlotSelectionStrategy fallbackSlotSelectionStrategy) { + this.fallbackSlotSelectionStrategy = fallbackSlotSelectionStrategy; + } @Override public Optional selectBestSlotForProfile( @@ -58,7 +62,7 @@ public Optional selectBestSlotForProfile( // Second, select based on location preference, excluding blacklisted allocations Set blackListedAllocations = slotProfile.getPreviousExecutionGraphAllocations(); Collection availableAndAllowedSlots = computeWithoutBlacklistedSlots(availableSlots, blackListedAllocations); - return LocationPreferenceSlotSelectionStrategy.INSTANCE.selectBestSlotForProfile(availableAndAllowedSlots, slotProfile); + return fallbackSlotSelectionStrategy.selectBestSlotForProfile(availableAndAllowedSlots, slotProfile); } @Nonnull @@ -79,4 +83,8 @@ private Collection computeWithoutBlacklistedSlots( return availableAndAllowedSlots; } + + public static PreviousAllocationSlotSelectionStrategy create() { + return new PreviousAllocationSlotSelectionStrategy(LocationPreferenceSlotSelectionStrategy.createDefault()); + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/LocationPreferenceSlotSelectionStrategyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/LocationPreferenceSlotSelectionStrategyTest.java index 4b9189c9edb0..148d81842678 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/LocationPreferenceSlotSelectionStrategyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/LocationPreferenceSlotSelectionStrategyTest.java @@ -33,7 +33,7 @@ public class LocationPreferenceSlotSelectionStrategyTest extends SlotSelectionStrategyTestBase { public LocationPreferenceSlotSelectionStrategyTest() { - super(LocationPreferenceSlotSelectionStrategy.INSTANCE); + super(LocationPreferenceSlotSelectionStrategy.createDefault()); } protected LocationPreferenceSlotSelectionStrategyTest(SlotSelectionStrategy slotSelectionStrategy) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/PreviousAllocationSlotSelectionStrategyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/PreviousAllocationSlotSelectionStrategyTest.java index afbf8879eb2f..8575b07910cb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/PreviousAllocationSlotSelectionStrategyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/PreviousAllocationSlotSelectionStrategyTest.java @@ -32,7 +32,7 @@ public class PreviousAllocationSlotSelectionStrategyTest extends LocationPreferenceSlotSelectionStrategyTest { public PreviousAllocationSlotSelectionStrategyTest() { - super(PreviousAllocationSlotSelectionStrategy.INSTANCE); + super(PreviousAllocationSlotSelectionStrategy.create()); } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphNotEnoughResourceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphNotEnoughResourceTest.java index e57c0b25b3c1..4d4a4a43b3f1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphNotEnoughResourceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphNotEnoughResourceTest.java @@ -150,7 +150,7 @@ private static Scheduler createSchedulerWithSlots( final ResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway(); slotPool.start(JobMasterId.generate(), jobManagerAddress, mainThreadExecutor); slotPool.connectToResourceManager(resourceManagerGateway); - Scheduler scheduler = new SchedulerImpl(LocationPreferenceSlotSelectionStrategy.INSTANCE, slotPool); + Scheduler scheduler = new SchedulerImpl(LocationPreferenceSlotSelectionStrategy.createDefault(), slotPool); scheduler.start(mainThreadExecutor); CompletableFuture.runAsync(() -> slotPool.registerTaskManager(taskManagerLocation.getResourceID()), mainThreadExecutor).join(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java index 44a7758f633a..33faf40a2a13 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java @@ -761,7 +761,7 @@ private static Scheduler createSchedulerWithSlots( final TaskManagerGateway taskManagerGateway = new SimpleAckingTaskManagerGateway(); setupSlotPool(slotPool); - Scheduler scheduler = new SchedulerImpl(LocationPreferenceSlotSelectionStrategy.INSTANCE, slotPool); + Scheduler scheduler = new SchedulerImpl(LocationPreferenceSlotSelectionStrategy.createDefault(), slotPool); scheduler.start(mainThreadExecutor); slotPool.registerTaskManager(taskManagerLocation.getResourceID()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestBase.java index 2ad4861a046d..ffaf9339bf2e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestBase.java @@ -72,7 +72,7 @@ public abstract class SchedulerTestBase extends TestLogger { public void setup() throws Exception { final JobID jobId = new JobID(); slotPool = new TestingSlotPoolImpl(jobId); - scheduler = new SchedulerImpl(LocationPreferenceSlotSelectionStrategy.INSTANCE, slotPool); + scheduler = new SchedulerImpl(LocationPreferenceSlotSelectionStrategy.createDefault(), slotPool); testingSlotProvider = new TestingSlotPoolSlotProvider(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolCoLocationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolCoLocationTest.java index b9601ccb1d2d..9e821020dfe6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolCoLocationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolCoLocationTest.java @@ -64,7 +64,7 @@ public class SlotPoolCoLocationTest extends TestLogger { @Rule public final SlotPoolResource slotPoolResource = - new SlotPoolResource(PreviousAllocationSlotSelectionStrategy.INSTANCE); + new SlotPoolResource(PreviousAllocationSlotSelectionStrategy.create()); /** * Tests the scheduling of two tasks with a parallelism of 2 and a co-location constraint. diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolImplTest.java index fc3b2787a4a5..d41b76b2c56a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolImplTest.java @@ -814,7 +814,7 @@ private static void setupSlotPool( private static Scheduler setupScheduler( SlotPool slotPool, ComponentMainThreadExecutor mainThreadExecutable) { - Scheduler scheduler = new SchedulerImpl(LocationPreferenceSlotSelectionStrategy.INSTANCE, slotPool); + Scheduler scheduler = new SchedulerImpl(LocationPreferenceSlotSelectionStrategy.createDefault(), slotPool); scheduler.start(mainThreadExecutable); return scheduler; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolInteractionsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolInteractionsTest.java index cda9370620bb..fe0d279ef273 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolInteractionsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolInteractionsTest.java @@ -91,7 +91,7 @@ public void testSlotAllocationNoResourceManager() throws Exception { )) { pool.start(JobMasterId.generate(), "foobar", testMainThreadExecutor.getMainThreadExecutor()); - Scheduler scheduler = new SchedulerImpl(LocationPreferenceSlotSelectionStrategy.INSTANCE, pool); + Scheduler scheduler = new SchedulerImpl(LocationPreferenceSlotSelectionStrategy.createDefault(), pool); scheduler.start(testMainThreadExecutor.getMainThreadExecutor()); CompletableFuture future = testMainThreadExecutor.execute(() -> scheduler.allocateSlot( @@ -118,7 +118,7 @@ public void testCancelSlotAllocationWithoutResourceManager() throws Exception { final CompletableFuture timeoutFuture = new CompletableFuture<>(); pool.setTimeoutPendingSlotRequestConsumer(timeoutFuture::complete); pool.start(JobMasterId.generate(), "foobar", testMainThreadExecutor.getMainThreadExecutor()); - Scheduler scheduler = new SchedulerImpl(LocationPreferenceSlotSelectionStrategy.INSTANCE, pool); + Scheduler scheduler = new SchedulerImpl(LocationPreferenceSlotSelectionStrategy.createDefault(), pool); scheduler.start(testMainThreadExecutor.getMainThreadExecutor()); SlotRequestId requestId = new SlotRequestId(); @@ -169,7 +169,7 @@ public void testSlotAllocationTimeout() throws Exception { ResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway(); pool.connectToResourceManager(resourceManagerGateway); - Scheduler scheduler = new SchedulerImpl(LocationPreferenceSlotSelectionStrategy.INSTANCE, pool); + Scheduler scheduler = new SchedulerImpl(LocationPreferenceSlotSelectionStrategy.createDefault(), pool); scheduler.start(testMainThreadExecutor.getMainThreadExecutor()); SlotRequestId requestId = new SlotRequestId(); @@ -204,7 +204,7 @@ public void testExtraSlotsAreKept() throws Exception { pool.start(JobMasterId.generate(), "foobar", testMainThreadExecutor.getMainThreadExecutor()); - Scheduler scheduler = new SchedulerImpl(LocationPreferenceSlotSelectionStrategy.INSTANCE, pool); + Scheduler scheduler = new SchedulerImpl(LocationPreferenceSlotSelectionStrategy.createDefault(), pool); scheduler.start(testMainThreadExecutor.getMainThreadExecutor()); final CompletableFuture allocationIdFuture = new CompletableFuture<>(); @@ -271,7 +271,7 @@ public void testProviderAndOwnerSlotAllocationTimeout() throws Exception { ResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway(); pool.connectToResourceManager(resourceManagerGateway); - Scheduler scheduler = new SchedulerImpl(LocationPreferenceSlotSelectionStrategy.INSTANCE, pool); + Scheduler scheduler = new SchedulerImpl(LocationPreferenceSlotSelectionStrategy.createDefault(), pool); scheduler.start(testMainThreadExecutor.getMainThreadExecutor()); // test the pending request is clear when timed out diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolSlotSharingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolSlotSharingTest.java index 08aafd02411b..ef1fe31cbd63 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolSlotSharingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolSlotSharingTest.java @@ -56,7 +56,7 @@ public class SlotPoolSlotSharingTest extends TestLogger { @Rule public final SlotPoolResource slotPoolResource = - new SlotPoolResource(PreviousAllocationSlotSelectionStrategy.INSTANCE); + new SlotPoolResource(PreviousAllocationSlotSelectionStrategy.create()); @Test public void testSingleQueuedSharedSlotScheduling() throws Exception { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolSlotSpreadOutTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolSlotSpreadOutTest.java new file mode 100644 index 000000000000..2edd504d8a11 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolSlotSpreadOutTest.java @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.slotpool; + +import org.apache.flink.api.common.time.Time; +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.utils.SimpleAckingTaskManagerGateway; +import org.apache.flink.runtime.instance.SlotSharingGroupId; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit; +import org.apache.flink.runtime.jobmaster.LogicalSlot; +import org.apache.flink.runtime.jobmaster.SlotRequestId; +import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; +import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; +import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.util.TestLogger; + +import org.junit.Rule; +import org.junit.Test; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.not; + +/** + * Tests for the {@link SchedulerImpl} and {@link SlotPoolImpl} which verify + * the spread out of slots. + */ +public class SlotPoolSlotSpreadOutTest extends TestLogger { + + public static final Time TIMEOUT = Time.seconds(10L); + + @Rule + public final SlotPoolResource slotPoolResource = new SlotPoolResource(LocationPreferenceSlotSelectionStrategy.createEvenlySpreadOut()); + + @Test + public void allocateSingleSlot_withNoRequirements_selectsSlotSoThatWorkloadIsSpreadOut() { + registerTaskExecutors(2, 4); + + final ScheduledUnit firstSlotRequest = createSimpleSlotRequest(); + final ScheduledUnit secondSlotRequest = createSimpleSlotRequest(); + + final CompletableFuture firstSlotFuture = allocateSlot(firstSlotRequest); + final CompletableFuture secondSlotFuture = allocateSlot(secondSlotRequest); + + final TaskManagerLocation firstTaskManagerLocation = getTaskManagerLocation(firstSlotFuture); + final TaskManagerLocation secondTaskManagerLocation = getTaskManagerLocation(secondSlotFuture); + + assertThat(firstTaskManagerLocation, is(not(equalTo(secondTaskManagerLocation)))); + } + + @Test + public void allocateSingleSlot_withInputPreference_inputPreferenceHasPrecedenceOverSpreadOut() { + registerTaskExecutors(2, 2); + + final ScheduledUnit sourceSlotRequest = createSimpleSlotRequest(); + final ScheduledUnit sinkSlotRequest = createSimpleSlotRequest(); + + final CompletableFuture sourceSlotFuture = allocateSlot(sourceSlotRequest); + final TaskManagerLocation sourceTaskManagerLocation = getTaskManagerLocation(sourceSlotFuture); + + Collection preferredLocations = Collections.singleton(sourceTaskManagerLocation); + final CompletableFuture sinkSlotFuture = allocateSlotWithInputPreference(sinkSlotRequest, preferredLocations); + final TaskManagerLocation sinkTaskManagerLocation = getTaskManagerLocation(sinkSlotFuture); + + // input preference should have precedence over task spread out + assertThat(sinkTaskManagerLocation, is(equalTo(sourceTaskManagerLocation))); + } + + @Test + public void allocateSharedSlot_withNoRequirements_selectsSlotsSoThatWorkloadIsSpreadOut() { + final int numberSlotsPerTaskExecutor = 2; + final int numberTaskExecutors = 2; + final int numberSlots = numberTaskExecutors * numberSlotsPerTaskExecutor; + + registerTaskExecutors(numberTaskExecutors, numberSlotsPerTaskExecutor); + + final JobVertexID sourceJobVertexId = new JobVertexID(); + final JobVertexID sinkJobVertexId = new JobVertexID(); + final SlotSharingGroupId slotSharingGroupId = new SlotSharingGroupId(); + + final List sourceScheduledUnits = IntStream.range(0, numberSlots) + .mapToObj(ignored -> createSharedSlotRequest(sourceJobVertexId, slotSharingGroupId)) + .collect(Collectors.toList()); + + final List sinkScheduledUnits = IntStream.range(0, numberTaskExecutors) + .mapToObj(ignored -> createSharedSlotRequest(sinkJobVertexId, slotSharingGroupId)) + .collect(Collectors.toList()); + + sourceScheduledUnits.forEach(this::allocateSlot); + final Set sinkLocations = sinkScheduledUnits.stream() + .map(this::allocateSlot) + .map(this::getTaskManagerLocation) + .collect(Collectors.toSet()); + + // verify that the sinks have been evenly spread across the available TaskExecutors + assertThat(sinkLocations, hasSize(numberTaskExecutors)); + } + + private ScheduledUnit createSharedSlotRequest(JobVertexID jobVertexId, SlotSharingGroupId slotSharingGroupId) { + return new ScheduledUnit(jobVertexId, slotSharingGroupId, null); + } + + private ScheduledUnit createSimpleSlotRequest() { + return new ScheduledUnit(new JobVertexID(), null, null); + } + + private CompletableFuture allocateSlot(ScheduledUnit scheduledUnit) { + return internalAllocateSlot(scheduledUnit, SlotProfile.noRequirements()); + } + + private CompletableFuture internalAllocateSlot(ScheduledUnit scheduledUnit, SlotProfile slotProfile) { + SlotProvider slotProvider = slotPoolResource.getSlotProvider(); + return slotProvider.allocateSlot( + new SlotRequestId(), + scheduledUnit, + slotProfile, + TIMEOUT); + } + + private CompletableFuture allocateSlotWithInputPreference(ScheduledUnit scheduledUnit, Collection preferredLocations) { + return internalAllocateSlot(scheduledUnit, SlotProfile.preferredLocality(ResourceProfile.UNKNOWN, preferredLocations)); + } + + private TaskManagerLocation getTaskManagerLocation(CompletableFuture slotFuture) { + return slotFuture.join().getTaskManagerLocation(); + } + + private void registerTaskExecutors(int numberTaskExecutors, int numberSlotsPerTaskExecutor) { + for (int i = 0; i < numberTaskExecutors; i++) { + registerTaskExecutor(numberSlotsPerTaskExecutor); + } + } + + private void registerTaskExecutor(int numberSlotsPerTaskExecutor) { + final SlotPool slotPool = slotPoolResource.getSlotPool(); + final LocalTaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); + + slotPool.registerTaskManager(taskManagerLocation.getResourceID()); + + final Collection slotOffers = IntStream + .range(0, numberSlotsPerTaskExecutor) + .mapToObj(index -> new SlotOffer(new AllocationID(), index, ResourceProfile.ANY)) + .collect(Collectors.toList()); + + slotPool.offerSlots(taskManagerLocation, new SimpleAckingTaskManagerGateway(), slotOffers); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSharingManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSharingManagerTest.java index c133e51a35a8..904b993426ee 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSharingManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSharingManagerTest.java @@ -442,8 +442,9 @@ public void testGetResolvedSlot() { SlotSharingManager.MultiTaskSlot resolvedMultiTaskSlot = slotSharingManager.getResolvedRootSlot(slotInfoAndRemainingResource.getSlotInfo()); - SlotSelectionStrategy.SlotInfoAndLocality slotInfoAndLocality = - LocationPreferenceSlotSelectionStrategy.INSTANCE.selectBestSlotForProfile(slotInfos, SlotProfile.noRequirements()).get(); + final LocationPreferenceSlotSelectionStrategy locationPreferenceSlotSelectionStrategy = LocationPreferenceSlotSelectionStrategy.createDefault(); + SlotSelectionStrategy.SlotInfoAndLocality slotInfoAndLocality = locationPreferenceSlotSelectionStrategy + .selectBestSlotForProfile(slotInfos, SlotProfile.noRequirements()).get(); assertNotNull(resolvedMultiTaskSlot); assertEquals(Locality.UNCONSTRAINED, slotInfoAndLocality.getLocality()); @@ -482,8 +483,9 @@ public void testGetResolvedSlotWithLocationPreferences() { SlotProfile slotProfile = SlotProfile.preferredLocality(ResourceProfile.UNKNOWN, Collections.singleton(taskManagerLocation)); Collection slotInfos = slotSharingManager.listResolvedRootSlotInfo(groupId); + final LocationPreferenceSlotSelectionStrategy locationPreferenceSlotSelectionStrategy = LocationPreferenceSlotSelectionStrategy.createDefault(); SlotSelectionStrategy.SlotInfoAndLocality slotInfoAndLocality = - LocationPreferenceSlotSelectionStrategy.INSTANCE.selectBestSlotForProfile(slotInfos, slotProfile).get(); + locationPreferenceSlotSelectionStrategy.selectBestSlotForProfile(slotInfos, slotProfile).get(); SlotSharingManager.MultiTaskSlot resolvedRootSlot = slotSharingManager.getResolvedRootSlot(slotInfoAndLocality.getSlotInfo()); assertNotNull(resolvedRootSlot); @@ -498,7 +500,7 @@ public void testGetResolvedSlotWithLocationPreferences() { slotInfoAndLocality.getLocality()); slotInfos = slotSharingManager.listResolvedRootSlotInfo(groupId); - slotInfoAndLocality = LocationPreferenceSlotSelectionStrategy.INSTANCE.selectBestSlotForProfile(slotInfos, slotProfile).get(); + slotInfoAndLocality = locationPreferenceSlotSelectionStrategy.selectBestSlotForProfile(slotInfos, slotProfile).get(); resolvedRootSlot = slotSharingManager.getResolvedRootSlot(slotInfoAndLocality.getSlotInfo()); assertNotNull(resolvedRootSlot); assertNotSame(Locality.LOCAL, (slotInfoAndLocality.getLocality())); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/LegacySchedulerBatchSchedulingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/LegacySchedulerBatchSchedulingTest.java index e2155d99e057..73fa8f03ba4e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/LegacySchedulerBatchSchedulingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/LegacySchedulerBatchSchedulingTest.java @@ -170,7 +170,7 @@ private void finishExecution( @Nonnull private SchedulerImpl createScheduler(SlotPool slotPool, ComponentMainThreadExecutor mainThreadExecutor) { - final SchedulerImpl scheduler = new SchedulerImpl(LocationPreferenceSlotSelectionStrategy.INSTANCE, slotPool); + final SchedulerImpl scheduler = new SchedulerImpl(LocationPreferenceSlotSelectionStrategy.createDefault(), slotPool); scheduler.start(mainThreadExecutor); return scheduler; From 269e7f007e855c2bdedf8bad64ef13f516a608a6 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 16 Oct 2019 15:44:32 +0200 Subject: [PATCH 501/746] [FLINK-12122] Choose SlotSelectionStrategy based on ClusterOptions#EVENLY_SPREAD_OUT_SLOTS_STRATEGY If ClusterOptions#EVENLY_SPREAD_OUT_SLOTS_STRATEGY is enabled, then Flink will use the evenly spread out location preference strategy to spread out the workload as much as possible. This closes #9928. --- .../slotpool/DefaultSchedulerFactory.java | 15 +++++++++++++-- .../PreviousAllocationSlotSelectionStrategy.java | 6 +++++- 2 files changed, 18 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultSchedulerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultSchedulerFactory.java index e0ae00e6606e..0ca5c19f54f5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultSchedulerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/DefaultSchedulerFactory.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.jobmaster.slotpool; import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.ClusterOptions; import org.apache.flink.configuration.Configuration; import javax.annotation.Nonnull; @@ -43,10 +44,20 @@ public Scheduler createScheduler(@Nonnull SlotPool slotPool) { @Nonnull private static SlotSelectionStrategy selectSlotSelectionStrategy(@Nonnull Configuration configuration) { + final boolean evenlySpreadOutSlots = configuration.getBoolean(ClusterOptions.EVENLY_SPREAD_OUT_SLOTS_STRATEGY); + + final SlotSelectionStrategy locationPreferenceSlotSelectionStrategy; + + if (evenlySpreadOutSlots) { + locationPreferenceSlotSelectionStrategy = LocationPreferenceSlotSelectionStrategy.createEvenlySpreadOut(); + } else { + locationPreferenceSlotSelectionStrategy = LocationPreferenceSlotSelectionStrategy.createDefault(); + } + if (configuration.getBoolean(CheckpointingOptions.LOCAL_RECOVERY)) { - return PreviousAllocationSlotSelectionStrategy.create(); + return PreviousAllocationSlotSelectionStrategy.create(locationPreferenceSlotSelectionStrategy); } else { - return LocationPreferenceSlotSelectionStrategy.createDefault(); + return locationPreferenceSlotSelectionStrategy; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PreviousAllocationSlotSelectionStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PreviousAllocationSlotSelectionStrategy.java index 85ed43eb3ec2..55499887ad2c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PreviousAllocationSlotSelectionStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/PreviousAllocationSlotSelectionStrategy.java @@ -85,6 +85,10 @@ private Collection computeWithoutBlacklistedSlots( } public static PreviousAllocationSlotSelectionStrategy create() { - return new PreviousAllocationSlotSelectionStrategy(LocationPreferenceSlotSelectionStrategy.createDefault()); + return create(LocationPreferenceSlotSelectionStrategy.createDefault()); + } + + public static PreviousAllocationSlotSelectionStrategy create(SlotSelectionStrategy fallbackSlotSelectionStrategy) { + return new PreviousAllocationSlotSelectionStrategy(fallbackSlotSelectionStrategy); } } From fd1deff5f1510f840c290f89f0224ae2dadf7177 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 16 Oct 2019 16:08:06 +0200 Subject: [PATCH 502/746] [hotfix] Fix checkstyle violations in SlotSharingManagerTest --- .../runtime/jobmaster/slotpool/SlotSharingManagerTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSharingManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSharingManagerTest.java index 904b993426ee..e16910d9ed81 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSharingManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSharingManagerTest.java @@ -174,7 +174,7 @@ public void testNestedSlotCreation() { } /** - * Tests that we can release nested slots from the leaves onwards + * Tests that we can release nested slots from the leaves onwards. */ @Test public void testNestedSlotRelease() throws Exception { @@ -351,7 +351,7 @@ public void testSlotContextFutureCompletion() throws Exception { } /** - * Tests that slot context future failures will release the root slot + * Tests that slot context future failures will release the root slot. */ @Test public void testSlotContextFutureFailure() { @@ -387,7 +387,7 @@ public void testSlotContextFutureFailure() { /** * Tests that the root slot are moved from unresolved to resolved once the - * slot context future is successfully completed + * slot context future is successfully completed. */ @Test public void testRootSlotTransition() { From 7e8218515baf630e668348a68ff051dfa49c90c3 Mon Sep 17 00:00:00 2001 From: klion26 Date: Mon, 4 Nov 2019 21:13:14 +0800 Subject: [PATCH 503/746] [FLINK-13969][Checkpointing] Do not allow trigger new checkpoitn after stop the coordinator Currently, we just check whether coordinator has been stopped in eager pre-check, if the coordinator stopped after eager pre-check has been done, we'll trigger a new checkpoint even if the coordinator has been stopped. In this commit we'll prevent triggering new checkpoint in such case. [hotfix] extract common logic for checkpoint trigger check [hotfix] Get rid of Mockito in test This closes #10111. --- .../checkpoint/CheckpointCoordinator.java | 68 ++++++++----------- .../checkpoint/CheckpointCoordinatorTest.java | 63 +++++++++++++++++ 2 files changed, 92 insertions(+), 39 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index 8cabf675ea9b..052534d66d8d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -504,30 +504,7 @@ public CompletableFuture triggerCheckpoint( // make some eager pre-checks synchronized (lock) { - // abort if the coordinator has been shutdown in the meantime - if (shutdown) { - throw new CheckpointException(CheckpointFailureReason.CHECKPOINT_COORDINATOR_SHUTDOWN); - } - - // Don't allow periodic checkpoint if scheduling has been disabled - if (isPeriodic && !periodicScheduling) { - throw new CheckpointException(CheckpointFailureReason.PERIODIC_SCHEDULER_SHUTDOWN); - } - - // validate whether the checkpoint can be triggered, with respect to the limit of - // concurrent checkpoints, and the minimum time between checkpoints. - // these checks are not relevant for savepoints - if (!props.forceCheckpoint()) { - // sanity check: there should never be more than one trigger request queued - if (triggerRequestQueued) { - LOG.warn("Trying to trigger another checkpoint for job {} while one was queued already.", job); - throw new CheckpointException(CheckpointFailureReason.ALREADY_QUEUED); - } - - checkConcurrentCheckpoints(); - - checkMinPauseBetweenCheckpoints(); - } + preCheckBeforeTriggeringCheckpoint(isPeriodic, props.forceCheckpoint()); } // check if all tasks that we need to trigger are running. @@ -629,21 +606,7 @@ public CompletableFuture triggerCheckpoint( try { // re-acquire the coordinator-wide lock synchronized (lock) { - // since we released the lock in the meantime, we need to re-check - // that the conditions still hold. - if (shutdown) { - throw new CheckpointException(CheckpointFailureReason.CHECKPOINT_COORDINATOR_SHUTDOWN); - } - else if (!props.forceCheckpoint()) { - if (triggerRequestQueued) { - LOG.warn("Trying to trigger another checkpoint for job {} while one was queued already.", job); - throw new CheckpointException(CheckpointFailureReason.ALREADY_QUEUED); - } - - checkConcurrentCheckpoints(); - - checkMinPauseBetweenCheckpoints(); - } + preCheckBeforeTriggeringCheckpoint(isPeriodic, props.forceCheckpoint()); LOG.info("Triggering checkpoint {} @ {} for job {}.", checkpointID, timestamp, job); @@ -704,6 +667,10 @@ else if (!props.forceCheckpoint()) { LOG.warn("Cannot dispose failed checkpoint storage location {}", checkpointStorageLocation, t2); } + // rethrow the CheckpointException directly. + if (t instanceof CheckpointException) { + throw (CheckpointException) t; + } throw new CheckpointException(CheckpointFailureReason.EXCEPTION, t); } } @@ -1515,4 +1482,27 @@ private void checkAndResetCheckpointScheduler() { private boolean allPendingCheckpointsDiscarded() { return pendingCheckpoints.values().stream().allMatch(PendingCheckpoint::isDiscarded); } + + private void preCheckBeforeTriggeringCheckpoint(boolean isPeriodic, boolean forceCheckpoint) throws CheckpointException { + // abort if the coordinator has been shutdown in the meantime + if (shutdown) { + throw new CheckpointException(CheckpointFailureReason.CHECKPOINT_COORDINATOR_SHUTDOWN); + } + + // Don't allow periodic checkpoint if scheduling has been disabled + if (isPeriodic && !periodicScheduling) { + throw new CheckpointException(CheckpointFailureReason.PERIODIC_SCHEDULER_SHUTDOWN); + } + + if (!forceCheckpoint) { + if (triggerRequestQueued) { + LOG.warn("Trying to trigger another checkpoint for job {} while one was queued already.", job); + throw new CheckpointException(CheckpointFailureReason.ALREADY_QUEUED); + } + + checkConcurrentCheckpoints(); + + checkMinPauseBetweenCheckpoints(); + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java index b3894aa33852..e458437a90d0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java @@ -83,6 +83,7 @@ import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.mockExecutionJobVertex; import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.mockExecutionVertex; +import static org.apache.flink.util.Preconditions.checkNotNull; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; @@ -2491,6 +2492,53 @@ public void failJobDueToTaskFailure(Throwable cause, ExecutionAttemptID failingT coordinator.shutdown(JobStatus.FAILING); } + /** + * Tests that do not trigger checkpoint when stop the coordinator after the eager pre-check. + */ + @Test + public void testTriggerCheckpointAfterCancel() throws Exception { + ExecutionVertex vertex1 = mockExecutionVertex(new ExecutionAttemptID()); + + // set up the coordinator + CheckpointCoordinatorConfiguration chkConfig = new CheckpointCoordinatorConfiguration( + 600000, + 600000, + 0, + Integer.MAX_VALUE, + CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION, + true, + false, + 0); + TestingCheckpointIDCounter idCounter = new TestingCheckpointIDCounter(); + CheckpointCoordinator coord = new CheckpointCoordinator( + new JobID(), + chkConfig, + new ExecutionVertex[]{vertex1}, + new ExecutionVertex[]{vertex1}, + new ExecutionVertex[]{vertex1}, + idCounter, + new StandaloneCompletedCheckpointStore(1), + new MemoryStateBackend(), + Executors.directExecutor(), + manuallyTriggeredScheduledExecutor, + SharedStateRegistry.DEFAULT_FACTORY, + failureManager); + idCounter.setOwner(coord); + + try { + // start the coordinator + coord.startCheckpointScheduler(); + try { + coord.triggerCheckpoint(System.currentTimeMillis(), CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION), null, true, false); + fail("should not trigger periodic checkpoint after stop the coordinator."); + } catch (CheckpointException e) { + assertEquals(CheckpointFailureReason.PERIODIC_SCHEDULER_SHUTDOWN, e.getCheckpointFailureReason()); + } + } finally { + coord.shutdown(JobStatus.FINISHED); + } + } + private CheckpointCoordinator getCheckpointCoordinator( final JobID jobId, final ExecutionVertex vertex1, @@ -2604,4 +2652,19 @@ private void performIncrementalCheckpoint( coord.receiveAcknowledgeMessage(acknowledgeCheckpoint, TASK_MANAGER_LOCATION_INFO); } } + + private static class TestingCheckpointIDCounter extends StandaloneCheckpointIDCounter { + private CheckpointCoordinator owner; + + @Override + public long getAndIncrement() throws Exception { + checkNotNull(owner); + owner.stopCheckpointScheduler(); + return super.getAndIncrement(); + } + + void setOwner(CheckpointCoordinator coordinator) { + this.owner = checkNotNull(coordinator); + } + } } From 96df4fae121fbf75d90a884afa2c3541d774ec97 Mon Sep 17 00:00:00 2001 From: "yuzhao.cyz" Date: Wed, 6 Nov 2019 13:29:12 +0800 Subject: [PATCH 504/746] [FLINK-14623][table-api] Add computed column information into TableSchema This closes #10096 --- .../src/main/codegen/includes/parserImpls.ftl | 1 - .../sql/parser/FlinkSqlParserImplTest.java | 51 +-- .../apache/flink/table/api/TableColumn.java | 132 +++++++ .../apache/flink/table/api/TableSchema.java | 330 ++++++++++++------ .../descriptors/DescriptorProperties.java | 71 +++- .../flink/table/api/TableSchemaTest.java | 12 +- .../descriptors/DescriptorPropertiesTest.java | 6 +- 7 files changed, 452 insertions(+), 151 deletions(-) create mode 100644 flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/TableColumn.java 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 f752a55248e3..e3ff107b4a8a 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 @@ -56,7 +56,6 @@ void ComputedColumn(TableCreationContext context) : { SqlNode identifier; SqlNode expr; - boolean hidden = false; SqlParserPos pos; } { 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 7cb9014849bf..f4371ee7d7dd 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 @@ -19,6 +19,7 @@ package org.apache.flink.sql.parser; import org.apache.flink.sql.parser.ddl.SqlCreateTable; +import org.apache.flink.sql.parser.error.SqlValidateException; import org.apache.flink.sql.parser.impl.FlinkSqlParserImpl; import org.apache.flink.sql.parser.validate.FlinkSqlConformance; @@ -38,6 +39,7 @@ import java.io.Reader; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; /** FlinkSqlParserImpl tests. **/ @@ -351,7 +353,7 @@ public void testCreateTableWithUserDefinedType() { @Test public void testInvalidComputedColumn() { - checkFails("CREATE TABLE sls_stream (\n" + + final String sql0 = "CREATE TABLE t1 (\n" + " a bigint, \n" + " b varchar,\n" + " toTimestamp^(^b, 'yyyy-MM-dd HH:mm:ss'), \n" + @@ -359,11 +361,25 @@ public void testInvalidComputedColumn() { ") with (\n" + " 'x' = 'y', \n" + " 'asd' = 'data'\n" + - ")\n", "(?s).*Encountered \"\\(\" at line 4, column 14.\n" + + ")\n"; + final String expect0 = "(?s).*Encountered \"\\(\" at line 4, column 14.\n" + "Was expecting one of:\n" + " \"AS\" ...\n" + " \"STRING\" ...\n" + - ".*"); + ".*"; + sql(sql0).fails(expect0); + // Sub-query computed column expression is forbidden. + final String sql1 = "CREATE TABLE t1 (\n" + + " a bigint, \n" + + " b varchar,\n" + + " c as ^(^select max(d) from t2), \n" + + " PRIMARY KEY (a, b) \n" + + ") with (\n" + + " 'x' = 'y', \n" + + " 'asd' = 'data'\n" + + ")\n"; + final String expect1 = "(?s).*Query expression encountered in illegal context.*"; + sql(sql1).fails(expect1); } @Test @@ -505,24 +521,6 @@ public void testInsertExtendedColumnAsStaticPartition2() { .fails("Extended columns not allowed under the current SQL conformance level")); } - @Test - public void testInsertWithInvalidPartitionColumns() { - conformance0 = FlinkSqlConformance.HIVE; - final String sql2 = "insert into emp (empno, ename, job, mgr, hiredate,\n" - + " sal, comm, deptno, slacker)\n" - + "partition(^xxx^='1', job='job')\n" - + "select 'nom', 0, timestamp '1970-01-01 00:00:00',\n" - + " 1, 1, 1, false\n" - + "from (values 'a')"; - sql(sql2).node(new ValidationMatcher().fails("Unknown target column 'XXX'")); - final String sql3 = "insert into ^empnullables^ (ename, empno, deptno)\n" - + "partition(empno='1')\n" - + "values ('Pat', null)"; - sql(sql3).node(new ValidationMatcher().fails( - "\"Number of INSERT target columns \\\\(3\\\\) does not \"\n" - + "\t\t\t\t+ \"equal number of source items \\\\(2\\\\)\"")); - } - @Test public void testInsertOverwrite() { conformance0 = FlinkSqlConformance.HIVE; @@ -646,10 +644,13 @@ public void describeTo(Description description) { public boolean matches(Object item) { if (item instanceof ExtendedSqlNode) { ExtendedSqlNode createTable = (ExtendedSqlNode) item; - try { - createTable.validate(); - } catch (Exception e) { - assertEquals(failMsg, e.getMessage()); + if (failMsg != null) { + try { + createTable.validate(); + fail("expected exception"); + } catch (SqlValidateException e) { + assertEquals(failMsg, e.getMessage()); + } } if (expectedColumnSql != null && item instanceof SqlCreateTable) { assertEquals(expectedColumnSql, diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/TableColumn.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/TableColumn.java new file mode 100644 index 000000000000..bc5c1885e4c1 --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/TableColumn.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.api; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.table.types.DataType; +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nullable; + +import java.util.Objects; +import java.util.Optional; + +/** + * A table column represents a table column's structure with + * column name, column data type and computation expression(if it is a computed column). + */ +@PublicEvolving +public class TableColumn { + + //~ Instance fields -------------------------------------------------------- + + private final String name; + private final DataType type; + @Nullable + private final String expr; + + //~ Constructors ----------------------------------------------------------- + + /** + * Creates a {@link TableColumn} instance. + * + * @param name Column name + * @param type Column data type + * @param expr Column computation expression if it is a computed column + */ + private TableColumn( + String name, + DataType type, + @Nullable String expr) { + this.name = name; + this.type = type; + this.expr = expr; + } + + //~ Methods ---------------------------------------------------------------- + + /** + * Creates a table column from given name and data type. + */ + public static TableColumn of(String name, DataType type) { + Preconditions.checkNotNull(name, "Column name can not be null!"); + Preconditions.checkNotNull(type, "Column type can not be null!"); + return new TableColumn(name, type, null); + } + + /** + * Creates a table column from given name and computation expression. + * + * @param name Name of the column + * @param expression SQL-style expression + */ + public static TableColumn of(String name, DataType type, String expression) { + Preconditions.checkNotNull(name, "Column name can not be null!"); + Preconditions.checkNotNull(type, "Column type can not be null!"); + Preconditions.checkNotNull(expression, "Column expression can not be null!"); + return new TableColumn(name, type, expression); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TableColumn that = (TableColumn) o; + return Objects.equals(this.name, that.name) + && Objects.equals(this.type, that.type) + && Objects.equals(this.expr, that.expr); + } + + @Override + public int hashCode() { + return Objects.hash(this.name, this.type, this.expr); + } + + //~ Getter/Setter ---------------------------------------------------------- + + /** Returns data type of this column. */ + public DataType getType() { + return this.type; + } + + /** Returns name of this column. */ + public String getName() { + return name; + } + + /** Returns computation expression of this column. Or empty if this column + * is not a computed column. */ + public Optional getExpr() { + return Optional.ofNullable(this.expr); + } + + /** + * Returns if this column is a computed column that is generated from an expression. + * + * @return true if this column is generated + */ + public boolean isGenerated() { + return this.expr != null; + } + +} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/TableSchema.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/TableSchema.java index 7cfac1d776ab..454baf191de7 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/TableSchema.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/TableSchema.java @@ -34,7 +34,9 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; +import java.util.stream.Collectors; import java.util.stream.IntStream; import static org.apache.flink.table.api.DataTypes.FIELD; @@ -52,56 +54,13 @@ public class TableSchema { private static final String ATOMIC_TYPE_FIELD_NAME = "f0"; - private final String[] fieldNames; - - private final DataType[] fieldDataTypes; - - /** Mapping from qualified field name to (nested) field type. */ - private final Map fieldNameToType; + private final List columns; private final List watermarkSpecs; - private TableSchema(String[] fieldNames, DataType[] fieldDataTypes, List watermarkSpecs) { - this.fieldNames = Preconditions.checkNotNull(fieldNames); - this.fieldDataTypes = Preconditions.checkNotNull(fieldDataTypes); + private TableSchema(List columns, List watermarkSpecs) { + this.columns = Preconditions.checkNotNull(columns); this.watermarkSpecs = Preconditions.checkNotNull(watermarkSpecs); - - if (fieldNames.length != fieldDataTypes.length) { - throw new ValidationException( - "Number of field names and field data types must be equal.\n" + - "Number of names is " + fieldNames.length + ", number of data types is " + fieldDataTypes.length + ".\n" + - "List of field names: " + Arrays.toString(fieldNames) + "\n" + - "List of field data types: " + Arrays.toString(fieldDataTypes)); - } - - // validate and create name to type mapping - fieldNameToType = new HashMap<>(); - for (int i = 0; i < fieldNames.length; i++) { - // check for null - DataType fieldType = Preconditions.checkNotNull(fieldDataTypes[i]); - String fieldName = Preconditions.checkNotNull(fieldNames[i]); - validateAndCreateNameToTypeMapping(fieldName, fieldType, ""); - } - - // validate watermark and rowtime attribute - for (WatermarkSpec watermark : watermarkSpecs) { - String rowtimeAttribute = watermark.getRowtimeAttribute(); - DataType rowtimeType = getFieldDataType(rowtimeAttribute) - .orElseThrow(() -> new ValidationException(String.format( - "Rowtime attribute '%s' is not defined in schema.", rowtimeAttribute))); - if (rowtimeType.getLogicalType().getTypeRoot() != TIMESTAMP_WITHOUT_TIME_ZONE) { - throw new ValidationException(String.format( - "Rowtime attribute '%s' must be of type TIMESTAMP but is of type '%s'.", - rowtimeAttribute, rowtimeType)); - } - LogicalType watermarkOutputType = watermark.getWatermarkExprOutputType().getLogicalType(); - if (watermarkOutputType.getTypeRoot() != TIMESTAMP_WITHOUT_TIME_ZONE) { - throw new ValidationException(String.format( - "Watermark strategy '%s' must be of type TIMESTAMP but is of type '%s'.", - watermark.getWatermarkExpressionString(), - watermarkOutputType.asSerializableString())); - } - } } /** @@ -109,21 +68,31 @@ private TableSchema(String[] fieldNames, DataType[] fieldDataTypes, List[] fieldTypes) { - this(fieldNames, fromLegacyInfoToDataType(fieldTypes), Collections.emptyList()); + DataType[] fieldDataTypes = fromLegacyInfoToDataType(fieldTypes); + validateNameTypeNumberEqual(fieldNames, fieldDataTypes); + List columns = new ArrayList<>(); + for (int i = 0; i < fieldNames.length; i++) { + columns.add(TableColumn.of(fieldNames[i], fieldDataTypes[i])); + } + validateColumnsAndWatermarkSpecs(columns, Collections.emptyList()); + this.columns = columns; + this.watermarkSpecs = Collections.emptyList(); } /** * Returns a deep copy of the table schema. */ public TableSchema copy() { - return new TableSchema(fieldNames.clone(), fieldDataTypes.clone(), new ArrayList<>(watermarkSpecs)); + return new TableSchema(new ArrayList<>(columns), new ArrayList<>(watermarkSpecs)); } /** * Returns all field data types as an array. */ public DataType[] getFieldDataTypes() { - return fieldDataTypes; + return columns.stream() + .map(TableColumn::getType) + .toArray(DataType[]::new); } /** @@ -135,7 +104,7 @@ public DataType[] getFieldDataTypes() { */ @Deprecated public TypeInformation[] getFieldTypes() { - return fromDataTypeToLegacyInfo(fieldDataTypes); + return fromDataTypeToLegacyInfo(getFieldDataTypes()); } /** @@ -144,10 +113,10 @@ public TypeInformation[] getFieldTypes() { * @param fieldIndex the index of the field */ public Optional getFieldDataType(int fieldIndex) { - if (fieldIndex < 0 || fieldIndex >= fieldDataTypes.length) { + if (fieldIndex < 0 || fieldIndex >= columns.size()) { return Optional.empty(); } - return Optional.of(fieldDataTypes[fieldIndex]); + return Optional.of(columns.get(fieldIndex).getType()); } /** @@ -166,14 +135,12 @@ public Optional> getFieldType(int fieldIndex) { /** * Returns the specified data type for the given field name. * - * @param fieldName the name of the field. the field name can be a nested field using a dot separator, - * e.g. "field1.innerField2" + * @param fieldName the name of the field */ public Optional getFieldDataType(String fieldName) { - if (fieldNameToType.containsKey(fieldName)) { - return Optional.of(fieldNameToType.get(fieldName)); - } - return Optional.empty(); + return this.columns.stream() + .filter(column -> column.getName().equals(fieldName)) + .findFirst().map(TableColumn::getType); } /** @@ -193,14 +160,16 @@ public Optional> getFieldType(String fieldName) { * Returns the number of fields. */ public int getFieldCount() { - return fieldNames.length; + return columns.size(); } /** * Returns all field names as an array. */ public String[] getFieldNames() { - return fieldNames; + return this.columns.stream() + .map(TableColumn::getName) + .toArray(String[]::new); } /** @@ -209,18 +178,53 @@ public String[] getFieldNames() { * @param fieldIndex the index of the field */ public Optional getFieldName(int fieldIndex) { - if (fieldIndex < 0 || fieldIndex >= fieldNames.length) { + if (fieldIndex < 0 || fieldIndex >= columns.size()) { return Optional.empty(); } - return Optional.of(fieldNames[fieldIndex]); + return Optional.of(this.columns.get(fieldIndex).getName()); } /** - * Converts a table schema into a (nested) data type describing a {@link DataTypes#ROW(Field...)}. + * Returns the {@link TableColumn} instance for the given field index. + * + * @param fieldIndex the index of the field + */ + public Optional getTableColumn(int fieldIndex) { + if (fieldIndex < 0 || fieldIndex >= columns.size()) { + return Optional.empty(); + } + return Optional.of(this.columns.get(fieldIndex)); + } + + /** + * Returns the {@link TableColumn} instance for the given field name. + * + * @param fieldName the name of the field + */ + public Optional getTableColumn(String fieldName) { + return this.columns.stream() + .filter(column -> column.getName().equals(fieldName)) + .findFirst(); + } + + /** + * Returns all the {@link TableColumn}s for this table schema. + */ + public List getTableColumns() { + return new ArrayList<>(this.columns); + } + + /** + * Converts a table schema into a (nested) data type describing a + * {@link DataTypes#ROW(Field...)}. + * + *

    Note that the returned row type contains field types for all the columns, including + * normal columns and computed columns. Be caution with the computed column data types, because + * they are not expected to be included in the row type of TableSource or TableSink. */ public DataType toRowDataType() { - final Field[] fields = IntStream.range(0, fieldDataTypes.length) - .mapToObj(i -> FIELD(fieldNames[i], fieldDataTypes[i])) + final Field[] fields = columns.stream() + .map(column -> FIELD(column.getName(), column.getType())) .toArray(Field[]::new); return ROW(fields); } @@ -249,8 +253,15 @@ public List getWatermarkSpecs() { public String toString() { final StringBuilder sb = new StringBuilder(); sb.append("root\n"); - for (int i = 0; i < fieldNames.length; i++) { - sb.append(" |-- ").append(fieldNames[i]).append(": ").append(fieldDataTypes[i]).append('\n'); + for (TableColumn column : columns) { + sb.append(" |-- ") + .append(column.getName()) + .append(": "); + sb.append(column.getType()); + if (column.getExpr().isPresent()) { + sb.append(" AS ").append(column.getExpr().get()); + } + sb.append('\n'); } if (!watermarkSpecs.isEmpty()) { for (WatermarkSpec watermark : watermarkSpecs) { @@ -271,47 +282,17 @@ public boolean equals(Object o) { return false; } TableSchema schema = (TableSchema) o; - return Arrays.equals(fieldNames, schema.fieldNames) && - Arrays.equals(fieldDataTypes, schema.fieldDataTypes) && - watermarkSpecs.equals(schema.getWatermarkSpecs()); + return Objects.equals(columns, schema.columns) + && Objects.equals(watermarkSpecs, schema.getWatermarkSpecs()); } @Override public int hashCode() { - int result = Arrays.hashCode(fieldNames); - result = 31 * result + Arrays.hashCode(fieldDataTypes); + int result = Objects.hash(columns); result = 31 * result + watermarkSpecs.hashCode(); return result; } - /** - * Creates a mapping from field name to data type, the field name can be a nested field. - * This is mainly used for validating whether the rowtime attribute (might be nested) exists - * in the schema. During creating, it also validates whether there is duplicate field names. - * - *

    For example, a "f0" field of ROW type has two nested fields "q1" and "q2". Then the - * mapping will be ["f0" -> ROW, "f0.q1" -> INT, "f0.q2" -> STRING]. - *

    -	 * {@code
    -	 *     f0 ROW
    -	 * }
    -	 * 
    - * @param fieldName name of this field, e.g. "q1" or "q2" in the above example. - * @param fieldType data type of this field - * @param parentFieldName the field name of parent type, e.g. "f0" in the above example. - */ - private void validateAndCreateNameToTypeMapping(String fieldName, DataType fieldType, String parentFieldName) { - String fullFieldName = parentFieldName.isEmpty() ? fieldName : parentFieldName + "." + fieldName; - DataType oldType = fieldNameToType.put(fullFieldName, fieldType); - if (oldType != null) { - throw new ValidationException("Field names must be unique. Duplicate field: '" + fullFieldName + "'"); - } - if (fieldType instanceof FieldsDataType) { - Map fieldDataTypes = ((FieldsDataType) fieldType).getFieldDataTypes(); - fieldDataTypes.forEach((key, value) -> validateAndCreateNameToTypeMapping(key, value, fullFieldName)); - } - } - /** * Creates a table schema from a {@link TypeInformation} instance. If the type information is * a {@link CompositeType}, the field names and types for the composite type are used to @@ -346,6 +327,99 @@ public static Builder builder() { return new Builder(); } + //~ Tools ------------------------------------------------------------------ + + /** + * Validate the field names {@code fieldNames} and field types {@code fieldTypes} + * have equal number. + * + * @param fieldNames Field names + * @param fieldTypes Field data types + */ + private static void validateNameTypeNumberEqual(String[] fieldNames, DataType[] fieldTypes) { + if (fieldNames.length != fieldTypes.length) { + throw new ValidationException( + "Number of field names and field data types must be equal.\n" + + "Number of names is " + fieldNames.length + + ", number of data types is " + fieldTypes.length + ".\n" + + "List of field names: " + Arrays.toString(fieldNames) + "\n" + + "List of field data types: " + Arrays.toString(fieldTypes)); + } + } + + /** Table column and watermark specification sanity check. */ + private static void validateColumnsAndWatermarkSpecs(List columns, + List watermarkSpecs) { + // Validate and create name to type mapping. + // Field name to data type mapping, we need this because the row time attribute + // field can be nested. + + // This also check duplicate fields. + final Map fieldNameToType = new HashMap<>(); + for (TableColumn column : columns) { + validateAndCreateNameToTypeMapping(fieldNameToType, + column.getName(), + column.getType(), + ""); + } + + // Validate watermark and rowtime attribute. + for (WatermarkSpec watermark : watermarkSpecs) { + String rowtimeAttribute = watermark.getRowtimeAttribute(); + DataType rowtimeType = Optional.ofNullable(fieldNameToType.get(rowtimeAttribute)) + .orElseThrow(() -> new ValidationException(String.format( + "Rowtime attribute '%s' is not defined in schema.", rowtimeAttribute))); + if (rowtimeType.getLogicalType().getTypeRoot() != TIMESTAMP_WITHOUT_TIME_ZONE) { + throw new ValidationException(String.format( + "Rowtime attribute '%s' must be of type TIMESTAMP but is of type '%s'.", + rowtimeAttribute, rowtimeType)); + } + LogicalType watermarkOutputType = watermark.getWatermarkExprOutputType().getLogicalType(); + if (watermarkOutputType.getTypeRoot() != TIMESTAMP_WITHOUT_TIME_ZONE) { + throw new ValidationException(String.format( + "Watermark strategy '%s' must be of type TIMESTAMP but is of type '%s'.", + watermark.getWatermarkExpressionString(), + watermarkOutputType.asSerializableString())); + } + } + } + + /** + * Creates a mapping from field name to data type, the field name can be a nested field. + * This is mainly used for validating whether the rowtime attribute (might be nested) exists + * in the schema. During creating, it also validates whether there is duplicate field names. + * + *

    For example, a "f0" field of ROW type has two nested fields "q1" and "q2". Then the + * mapping will be ["f0" -> ROW, "f0.q1" -> INT, "f0.q2" -> STRING]. + * + *

    +	 * {@code
    +	 *     f0 ROW
    +	 * }
    +	 * 
    + * + * @param fieldNameToType Field name to type mapping that to update + * @param fieldName Name of this field, e.g. "q1" or "q2" in the above example + * @param fieldType Data type of this field + * @param parentFieldName Field name of parent type, e.g. "f0" in the above example + */ + private static void validateAndCreateNameToTypeMapping( + Map fieldNameToType, + String fieldName, + DataType fieldType, + String parentFieldName) { + String fullFieldName = parentFieldName.isEmpty() ? fieldName : parentFieldName + "." + fieldName; + DataType oldType = fieldNameToType.put(fullFieldName, fieldType); + if (oldType != null) { + throw new ValidationException("Field names must be unique. Duplicate field: '" + fullFieldName + "'"); + } + if (fieldType instanceof FieldsDataType) { + Map fieldDataTypes = ((FieldsDataType) fieldType).getFieldDataTypes(); + fieldDataTypes.forEach((key, value) -> + validateAndCreateNameToTypeMapping(fieldNameToType, key, value, fullFieldName)); + } + } + // -------------------------------------------------------------------------------------------- /** @@ -353,15 +427,12 @@ public static Builder builder() { */ public static class Builder { - private final List fieldNames; - - private final List fieldDataTypes; + private List columns; private final List watermarkSpecs; public Builder() { - fieldNames = new ArrayList<>(); - fieldDataTypes = new ArrayList<>(); + columns = new ArrayList<>(); watermarkSpecs = new ArrayList<>(); } @@ -373,8 +444,39 @@ public Builder() { public Builder field(String name, DataType dataType) { Preconditions.checkNotNull(name); Preconditions.checkNotNull(dataType); - fieldNames.add(name); - fieldDataTypes.add(dataType); + columns.add(TableColumn.of(name, dataType)); + return this; + } + + /** + * Add a computed field which is generated by the given expression. + * This also defines the field name and the data type. + * + *

    The call order of this method determines the order of fields in the schema. + * + * @param name Field name + * @param dataType Field data type + * @param expression Computed column expression, it should be a SQL-style expression whose + * identifiers should be all quoted and expanded. + * + * It should be expanded because this expression may be persisted + * then deserialized from the catalog, an expanded identifier would + * avoid the ambiguity if there are same name UDF referenced from + * different paths. For example, if there is a UDF named "my_udf" from + * path "my_catalog.my_database", you could pass in an expression like + * "`my_catalog`.`my_database`.`my_udf`(`f0`) + 1"; + * + * It should be quoted because user could use a reserved keyword as the + * identifier, and we have no idea if it is quoted when deserialize from + * the catalog, so we force to use quoted identifier here. But framework + * will not check whether it is qualified and quoted or not. + * + */ + public Builder field(String name, DataType dataType, String expression) { + Preconditions.checkNotNull(name); + Preconditions.checkNotNull(dataType); + Preconditions.checkNotNull(expression); + columns.add(TableColumn.of(name, dataType, expression)); return this; } @@ -386,9 +488,11 @@ public Builder field(String name, DataType dataType) { public Builder fields(String[] names, DataType[] dataTypes) { Preconditions.checkNotNull(names); Preconditions.checkNotNull(dataTypes); - - fieldNames.addAll(Arrays.asList(names)); - fieldDataTypes.addAll(Arrays.asList(dataTypes)); + validateNameTypeNumberEqual(names, dataTypes); + List columns = IntStream.range(0, names.length) + .mapToObj(idx -> TableColumn.of(names[idx], dataTypes[idx])) + .collect(Collectors.toList()); + this.columns.addAll(columns); return this; } @@ -430,10 +534,8 @@ public Builder watermark(String rowtimeAttribute, String watermarkExpressionStri * Returns a {@link TableSchema} instance. */ public TableSchema build() { - return new TableSchema( - fieldNames.toArray(new String[0]), - fieldDataTypes.toArray(new DataType[0]), - watermarkSpecs); + validateColumnsAndWatermarkSpecs(this.columns, this.watermarkSpecs); + return new TableSchema(columns, watermarkSpecs); } } } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/DescriptorProperties.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/DescriptorProperties.java index 526907979972..20ade9776864 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/DescriptorProperties.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/DescriptorProperties.java @@ -28,6 +28,7 @@ import org.apache.flink.table.api.WatermarkSpec; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.utils.LogicalTypeParser; +import org.apache.flink.table.types.utils.LegacyTypeInfoDataTypeConverter; import org.apache.flink.table.types.utils.TypeConversions; import org.apache.flink.table.utils.EncodingUtils; import org.apache.flink.table.utils.TypeStringUtils; @@ -76,11 +77,13 @@ public class DescriptorProperties { public static final String TABLE_SCHEMA_TYPE = "type"; + public static final String TABLE_SCHEMA_EXPR = "expr"; + public static final String WATERMARK = "watermark"; public static final String WATERMARK_ROWTIME = "rowtime"; - public static final String WATERMARK_STRATEGY_EXPRESSION = "strategy.expression"; + public static final String WATERMARK_STRATEGY_EXPR = "strategy.expr"; public static final String WATERMARK_STRATEGY_DATATYPE = "strategy.datatype"; @@ -195,15 +198,22 @@ public void putTableSchema(String key, TableSchema schema) { final String[] fieldNames = schema.getFieldNames(); final TypeInformation[] fieldTypes = schema.getFieldTypes(); + final String[] fieldExpressions = schema.getTableColumns().stream() + .map(column -> column.getExpr().orElse(null)) + .toArray(String[]::new); final List> values = new ArrayList<>(); for (int i = 0; i < schema.getFieldCount(); i++) { - values.add(Arrays.asList(fieldNames[i], TypeStringUtils.writeTypeInfo(fieldTypes[i]))); + values.add( + Arrays.asList( + fieldNames[i], + TypeStringUtils.writeTypeInfo(fieldTypes[i]), + fieldExpressions[i])); } - putIndexedFixedProperties( + putIndexedOptionalProperties( key, - Arrays.asList(TABLE_SCHEMA_NAME, TABLE_SCHEMA_TYPE), + Arrays.asList(TABLE_SCHEMA_NAME, TABLE_SCHEMA_TYPE, TABLE_SCHEMA_EXPR), values); if (!schema.getWatermarkSpecs().isEmpty()) { @@ -216,7 +226,7 @@ public void putTableSchema(String key, TableSchema schema) { } putIndexedFixedProperties( key + '.' + WATERMARK, - Arrays.asList(WATERMARK_ROWTIME, WATERMARK_STRATEGY_EXPRESSION, WATERMARK_STRATEGY_DATATYPE), + Arrays.asList(WATERMARK_ROWTIME, WATERMARK_STRATEGY_EXPR, WATERMARK_STRATEGY_DATATYPE), watermarkValues); } } @@ -257,6 +267,43 @@ public void putIndexedFixedProperties(String key, List subKeys, ListFor example: + * + *

    +	 *     schema.fields.0.type = INT, schema.fields.0.name = test
    +	 *     schema.fields.1.type = LONG, schema.fields.1.name = test2
    +	 *     schema.fields.2.type = LONG, schema.fields.2.name = test3, schema.fields.2.expr = test2 + 1
    +	 * 
    + * + *

    The arity of each subKeyValues must match the arity of propertyKeys. + */ + public void putIndexedOptionalProperties(String key, List subKeys, List> subKeyValues) { + checkNotNull(key); + checkNotNull(subKeys); + checkNotNull(subKeyValues); + for (int idx = 0; idx < subKeyValues.size(); idx++) { + final List values = subKeyValues.get(idx); + if (values == null || values.size() != subKeys.size()) { + throw new ValidationException("Values must have same arity as keys."); + } + if (values.stream().allMatch(Objects::isNull)) { + throw new ValidationException("Values must have at least one non-null value."); + } + for (int keyIdx = 0; keyIdx < values.size(); keyIdx++) { + String value = values.get(keyIdx); + if (value != null) { + put(key + '.' + idx + '.' + subKeys.get(keyIdx), values.get(keyIdx)); + } + } + } + } + /** * Adds an indexed mapping of properties under a common key. * @@ -536,14 +583,22 @@ public Optional getOptionalTableSchema(String key) { for (int i = 0; i < fieldCount; i++) { final String nameKey = key + '.' + i + '.' + TABLE_SCHEMA_NAME; final String typeKey = key + '.' + i + '.' + TABLE_SCHEMA_TYPE; + final String exprKey = key + '.' + i + '.' + TABLE_SCHEMA_EXPR; final String name = optionalGet(nameKey).orElseThrow(exceptionSupplier(nameKey)); final TypeInformation type = optionalGet(typeKey) .map(TypeStringUtils::readTypeInfo) .orElseThrow(exceptionSupplier(typeKey)); - - schemaBuilder.field(name, type); + final Optional expr = optionalGet(exprKey); + if (expr.isPresent()) { + schemaBuilder.field( + name, + LegacyTypeInfoDataTypeConverter.toDataType(type), + expr.get()); + } else { + schemaBuilder.field(name, type); + } } // extract watermark information @@ -557,7 +612,7 @@ public Optional getOptionalTableSchema(String key) { if (watermarkCount > 0) { for (int i = 0; i < watermarkCount; i++) { final String rowtimeKey = watermarkPrefixKey + '.' + i + '.' + WATERMARK_ROWTIME; - final String exprKey = watermarkPrefixKey + '.' + i + '.' + WATERMARK_STRATEGY_EXPRESSION; + final String exprKey = watermarkPrefixKey + '.' + i + '.' + WATERMARK_STRATEGY_EXPR; final String typeKey = watermarkPrefixKey + '.' + i + '.' + WATERMARK_STRATEGY_DATATYPE; final String rowtime = optionalGet(rowtimeKey).orElseThrow(exceptionSupplier(rowtimeKey)); final String exprString = optionalGet(exprKey).orElseThrow(exceptionSupplier(exprKey)); diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/api/TableSchemaTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/api/TableSchemaTest.java index 15d9132ef17d..9b0a312816cd 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/api/TableSchemaTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/api/TableSchemaTest.java @@ -21,6 +21,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.FieldsDataType; import org.junit.Rule; import org.junit.Test; @@ -53,6 +54,7 @@ public void testTableSchema() { DataTypes.FIELD("q1", DataTypes.STRING()), DataTypes.FIELD("q2", DataTypes.TIMESTAMP(3)))) .field("f2", DataTypes.STRING()) + .field("f3", DataTypes.BIGINT(), "f0 + 1") .watermark("f1.q2", WATERMARK_EXPRESSION, WATERMARK_DATATYPE) .build(); @@ -61,16 +63,22 @@ public void testTableSchema() { " |-- f0: BIGINT\n" + " |-- f1: ROW<`q1` STRING, `q2` TIMESTAMP(3)>\n" + " |-- f2: STRING\n" + + " |-- f3: BIGINT AS f0 + 1\n" + " |-- WATERMARK FOR f1.q2 AS now()"; assertEquals(expected, schema.toString()); // test getFieldNames and getFieldDataType assertEquals(Optional.of("f2"), schema.getFieldName(2)); + assertEquals(Optional.of(DataTypes.BIGINT()), schema.getFieldDataType(3)); + assertEquals(Optional.of(TableColumn.of("f3", DataTypes.BIGINT(), "f0 + 1")), + schema.getTableColumn(3)); assertEquals(Optional.of(DataTypes.STRING()), schema.getFieldDataType("f2")); - assertEquals(Optional.of(DataTypes.STRING()), schema.getFieldDataType("f1.q1")); - assertFalse(schema.getFieldName(3).isPresent()); + assertEquals(Optional.of(DataTypes.STRING()), schema.getFieldDataType("f1") + .map(r -> ((FieldsDataType) r).getFieldDataTypes().get("q1"))); + assertFalse(schema.getFieldName(4).isPresent()); assertFalse(schema.getFieldType(-1).isPresent()); assertFalse(schema.getFieldType("c").isPresent()); + assertFalse(schema.getFieldDataType("f1.q1").isPresent()); assertFalse(schema.getFieldDataType("f1.q3").isPresent()); // test copy() and equals() diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/descriptors/DescriptorPropertiesTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/descriptors/DescriptorPropertiesTest.java index 38e39960f167..578e3709e40d 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/descriptors/DescriptorPropertiesTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/descriptors/DescriptorPropertiesTest.java @@ -180,6 +180,7 @@ public void testTableSchema() { // the "bridgedTo" is a temporary solution because the type string format is based on TypeInformation. DataTypes.FIELD("q2", DataTypes.TIMESTAMP(3).bridgedTo(Timestamp.class)))) .field("f2", DataTypes.STRING()) + .field("f3", DataTypes.BIGINT(), "f0 + 1") .watermark( "f1.q2", "`f1`.`q2` - INTERVAL '5' SECOND", @@ -196,8 +197,11 @@ public void testTableSchema() { expected.put("schema.1.type", "ROW"); expected.put("schema.2.name", "f2"); expected.put("schema.2.type", "VARCHAR"); + expected.put("schema.3.name", "f3"); + expected.put("schema.3.type", "BIGINT"); + expected.put("schema.3.expr", "f0 + 1"); expected.put("schema.watermark.0.rowtime", "f1.q2"); - expected.put("schema.watermark.0.strategy.expression", "`f1`.`q2` - INTERVAL '5' SECOND"); + expected.put("schema.watermark.0.strategy.expr", "`f1`.`q2` - INTERVAL '5' SECOND"); expected.put("schema.watermark.0.strategy.datatype", "TIMESTAMP(3)"); assertEquals(expected, actual); From d938c19480c220344827271ff8da729cd91735b3 Mon Sep 17 00:00:00 2001 From: tison Date: Fri, 8 Nov 2019 10:23:59 +0800 Subject: [PATCH 505/746] [FLINK-14593][client] Port ClusterClient to asynchronous interface version This closes #10069 . --- .../apache/flink/client/cli/CliFrontend.java | 24 ++- .../flink/client/program/ClusterClient.java | 145 +++++++++--------- .../client/program/MiniClusterClient.java | 54 +++++-- .../program/rest/RestClusterClient.java | 62 +++----- .../client/cli/CliFrontendCancelTest.java | 60 +++++--- .../cli/CliFrontendStopWithSavepointTest.java | 118 ++++++++------ .../client/cli/util/MockedCliFrontend.java | 2 +- .../client/program/TestingClusterClient.java | 136 ++++++++++++++++ ...RestClusterClientSavepointTriggerTest.java | 2 +- .../program/rest/RestClusterClientTest.java | 9 +- .../kafka/KafkaConsumerTestBase.java | 14 +- .../state/api/SavepointReaderITTestBase.java | 2 +- .../api/SavepointReaderKeyedStateITCase.java | 2 +- .../AbstractQueryableStateTestBase.java | 2 +- .../client/gateway/local/LocalExecutor.java | 2 +- .../JobMasterStopWithSavepointIT.java | 2 +- .../JobMasterTriggerSavepointITCase.java | 4 +- .../accumulators/AccumulatorLiveITCase.java | 2 +- .../test/cancelling/CancelingTestBase.java | 2 +- .../test/checkpointing/RescalingITCase.java | 8 +- .../ResumeCheckpointManuallyITCase.java | 2 +- .../test/checkpointing/SavepointITCase.java | 8 +- .../utils/SavepointMigrationTestBase.java | 4 +- .../test/classloading/ClassLoaderITCase.java | 2 +- .../ProcessFailureCancelingITCase.java | 2 +- .../AbstractOperatorRestoreTestBase.java | 2 +- .../streaming/runtime/TimestampITCase.java | 2 +- .../yarn/CliFrontendRunWithYarnTest.java | 4 +- .../flink/yarn/util/FakeClusterClient.java | 111 -------------- 29 files changed, 418 insertions(+), 371 deletions(-) create mode 100644 flink-clients/src/test/java/org/apache/flink/client/program/TestingClusterClient.java delete mode 100644 flink-yarn-tests/src/test/java/org/apache/flink/yarn/util/FakeClusterClient.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 cb38ed631357..6f6e93326538 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 @@ -536,7 +536,7 @@ protected void stop(String[] args) throws Exception { clusterClient -> { final String savepointPath; try { - savepointPath = clusterClient.stopWithSavepoint(jobId, advanceToEndOfEventTime, targetDirectory); + savepointPath = clusterClient.stopWithSavepoint(jobId, advanceToEndOfEventTime, targetDirectory).get(clientTimeout.toMillis(), TimeUnit.MILLISECONDS); } catch (Exception e) { throw new FlinkException("Could not stop with a savepoint job \"" + jobId + "\".", e); } @@ -597,7 +597,7 @@ protected void cancel(String[] args) throws Exception { clusterClient -> { final String savepointPath; try { - savepointPath = clusterClient.cancelWithSavepoint(jobId, targetDirectory); + savepointPath = clusterClient.cancelWithSavepoint(jobId, targetDirectory).get(clientTimeout.toMillis(), TimeUnit.MILLISECONDS); } catch (Exception e) { throw new FlinkException("Could not cancel job " + jobId + '.', e); } @@ -619,7 +619,7 @@ protected void cancel(String[] args) throws Exception { commandLine, clusterClient -> { try { - clusterClient.cancel(jobId); + clusterClient.cancel(jobId).get(clientTimeout.toMillis(), TimeUnit.MILLISECONDS); } catch (Exception e) { throw new FlinkException("Could not cancel job " + jobId + '.', e); } @@ -695,26 +695,22 @@ protected void savepoint(String[] args) throws Exception { /** * Sends a SavepointTriggerMessage to the job manager. */ - private String triggerSavepoint(ClusterClient clusterClient, JobID jobId, String savepointDirectory) throws FlinkException { + private void triggerSavepoint(ClusterClient clusterClient, JobID jobId, String savepointDirectory) throws FlinkException { logAndSysout("Triggering savepoint for job " + jobId + '.'); + CompletableFuture savepointPathFuture = clusterClient.triggerSavepoint(jobId, savepointDirectory); logAndSysout("Waiting for response..."); - final String savepointPath; - try { - savepointPath = savepointPathFuture.get(); - } - catch (Exception e) { + final String savepointPath = savepointPathFuture.get(clientTimeout.toMillis(), TimeUnit.MILLISECONDS); + + logAndSysout("Savepoint completed. Path: " + savepointPath); + logAndSysout("You can resume your program from this savepoint with the run command."); + } catch (Exception e) { Throwable cause = ExceptionUtils.stripExecutionException(e); throw new FlinkException("Triggering a savepoint for the job " + jobId + " failed.", cause); } - - logAndSysout("Savepoint completed. Path: " + savepointPath); - logAndSysout("You can resume your program from this savepoint with the run command."); - - return savepointPath; } /** 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 eb3893539bbe..3c1d1a5d6825 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 @@ -41,78 +41,75 @@ * * @param type of the cluster id */ -public abstract class ClusterClient implements AutoCloseable { - - /** - * User overridable hook to close the client, possibly closes internal services. - * @deprecated use the {@link #close()} instead. This method stays for backwards compatibility. - */ - public void shutdown() throws Exception { - close(); - } +public interface ClusterClient extends AutoCloseable { @Override - public void close() throws Exception { + default void close() throws Exception { } /** - * Requests the {@link JobStatus} of the job with the given {@link JobID}. + * Returns the cluster id identifying the cluster to which the client is connected. + * + * @return cluster id of the connected cluster */ - public abstract CompletableFuture getJobStatus(JobID jobId); + T getClusterId(); /** - * Cancels a job identified by the job id. - * @param jobId the job id - * @throws Exception In case an error occurred. + * Return the Flink configuration object. + * + * @return The Flink configuration object */ - public abstract void cancel(JobID jobId) throws Exception; + Configuration getFlinkConfiguration(); /** - * Cancels a job identified by the job id and triggers a savepoint. - * @param jobId the job id - * @param savepointDirectory directory the savepoint should be written to - * @return path where the savepoint is located - * @throws Exception In case an error occurred. + * Shut down the cluster that this client communicate with. */ - public abstract String cancelWithSavepoint(JobID jobId, @Nullable String savepointDirectory) throws Exception; + default void shutDownCluster() { + throw new UnsupportedOperationException(); + } /** - * Stops a program on Flink cluster whose job-manager is configured in this client's configuration. - * Stopping works only for streaming programs. Be aware, that the program might continue to run for - * a while after sending the stop command, because after sources stopped to emit data all operators - * need to finish processing. + * Returns an URL (as a string) to the cluster web interface. + */ + String getWebInterfaceURL(); + + /** + * Lists the currently running and finished jobs on the cluster. * - * @param jobId the job ID of the streaming program to stop - * @param advanceToEndOfEventTime flag indicating if the source should inject a {@code MAX_WATERMARK} in the pipeline - * @param savepointDirectory directory the savepoint should be written to - * @return a {@link CompletableFuture} containing the path where the savepoint is located - * @throws Exception - * If the job ID is invalid (ie, is unknown or refers to a batch job) or if sending the stop signal - * failed. That might be due to an I/O problem, ie, the job-manager is unreachable. + * @return future collection of running and finished jobs + * @throws Exception if no connection to the cluster could be established */ - public abstract String stopWithSavepoint(final JobID jobId, final boolean advanceToEndOfEventTime, @Nullable final String savepointDirectory) throws Exception; + CompletableFuture> listJobs() throws Exception; /** - * Triggers a savepoint for the job identified by the job id. The savepoint will be written to the given savepoint - * directory, or {@link org.apache.flink.configuration.CheckpointingOptions#SAVEPOINT_DIRECTORY} if it is null. + * Dispose the savepoint under the given path. * - * @param jobId job id - * @param savepointDirectory directory the savepoint should be written to - * @return path future where the savepoint is located - * @throws FlinkException if no connection to the cluster could be established + * @param savepointPath path to the savepoint to be disposed + * @return acknowledge future of the dispose action */ - public abstract CompletableFuture triggerSavepoint(JobID jobId, @Nullable String savepointDirectory) throws FlinkException; + CompletableFuture disposeSavepoint(String savepointPath) throws FlinkException; - public abstract CompletableFuture disposeSavepoint(String savepointPath) throws FlinkException; + /** + * 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); /** - * Lists the currently running and finished jobs on the cluster. + * Requests the {@link JobStatus} of the job with the given {@link JobID}. + */ + CompletableFuture getJobStatus(JobID jobId); + + /** + * Request the {@link JobResult} for the given {@link JobID}. * - * @return future collection of running and finished jobs - * @throws Exception if no connection to the cluster could be established + * @param jobId for which to request the {@link JobResult} + * @return Future which is completed with the {@link JobResult} */ - public abstract CompletableFuture> listJobs() throws Exception; + CompletableFuture requestJobResult(@Nonnull JobID jobId); /** * Requests and returns the accumulators for the given job identifier. Accumulators can be @@ -121,7 +118,7 @@ public void close() throws Exception { * @param jobID The job identifier of a job. * @return A Map containing the accumulator's name and its value. */ - public Map> getAccumulators(JobID jobID) throws Exception { + default CompletableFuture>> getAccumulators(JobID jobID) { return getAccumulators(jobID, ClassLoader.getSystemClassLoader()); } @@ -132,47 +129,45 @@ public Map> getAccumulators(JobID jobID) throws * @param loader The class loader for deserializing the accumulator results. * @return A Map containing the accumulator's name and its value. */ - public abstract Map> getAccumulators(JobID jobID, ClassLoader loader) throws Exception; - - // ------------------------------------------------------------------------ - // Abstract methods to be implemented by the cluster specific Client - // ------------------------------------------------------------------------ - - /** - * Returns an URL (as a string) to the JobManager web interface. - */ - public abstract String getWebInterfaceURL(); + CompletableFuture>> getAccumulators(JobID jobID, ClassLoader loader); /** - * Returns the cluster id identifying the cluster to which the client is connected. + * Cancels a job identified by the job id. * - * @return cluster id of the connected cluster + * @param jobId the job id */ - public abstract T getClusterId(); + CompletableFuture cancel(JobID jobId); /** - * Return the Flink configuration object. - * @return The Flink configuration object + * Cancels a job identified by the job id and triggers a savepoint. + * + * @param jobId the job id + * @param savepointDirectory directory the savepoint should be written to + * @return future of path where the savepoint is located */ - public abstract Configuration getFlinkConfiguration(); + CompletableFuture cancelWithSavepoint(JobID jobId, @Nullable String savepointDirectory); /** - * Submit the given {@link JobGraph} to the cluster. + * Stops a program on Flink cluster whose job-manager is configured in this client's configuration. + * Stopping works only for streaming programs. Be aware, that the program might continue to run for + * a while after sending the stop command, because after sources stopped to emit data all operators + * need to finish processing. * - * @param jobGraph to submit - * @return Future which is completed with the {@link JobSubmissionResult} + * @param jobId the job ID of the streaming program to stop + * @param advanceToEndOfEventTime flag indicating if the source should inject a {@code MAX_WATERMARK} in the pipeline + * @param savepointDirectory directory the savepoint should be written to + * @return a {@link CompletableFuture} containing the path where the savepoint is located */ - public abstract CompletableFuture submitJob(@Nonnull JobGraph jobGraph); + CompletableFuture stopWithSavepoint(final JobID jobId, final boolean advanceToEndOfEventTime, @Nullable final String savepointDirectory); /** - * Request the {@link JobResult} for the given {@link JobID}. + * Triggers a savepoint for the job identified by the job id. The savepoint will be written to the given savepoint + * directory, or {@link org.apache.flink.configuration.CheckpointingOptions#SAVEPOINT_DIRECTORY} if it is null. * - * @param jobId for which to request the {@link JobResult} - * @return Future which is completed with the {@link JobResult} + * @param jobId job id + * @param savepointDirectory directory the savepoint should be written to + * @return path future where the savepoint is located + * @throws FlinkException if no connection to the cluster could be established */ - public abstract CompletableFuture requestJobResult(@Nonnull JobID jobId); - - public void shutDownCluster() { - throw new UnsupportedOperationException("The " + getClass().getSimpleName() + " does not support shutDownCluster."); - } + CompletableFuture triggerSavepoint(JobID jobId, @Nullable String savepointDirectory) throws FlinkException; } 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 8d9460407bae..bc24b781f809 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 @@ -28,9 +28,13 @@ import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.SerializedValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -38,11 +42,15 @@ import java.util.HashMap; import java.util.Map; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ExecutionException; /** * Client to interact with a {@link MiniCluster}. */ -public class MiniClusterClient extends ClusterClient { +public class MiniClusterClient implements ClusterClient { + + private static final Logger LOG = LoggerFactory.getLogger(MiniClusterClient.class); private final MiniCluster miniCluster; private final Configuration configuration; @@ -68,18 +76,18 @@ public CompletableFuture requestJobResult(@Nonnull JobID jobId) { } @Override - public void cancel(JobID jobId) throws Exception { - miniCluster.cancelJob(jobId).get(); + public CompletableFuture cancel(JobID jobId) { + return miniCluster.cancelJob(jobId); } @Override - public String cancelWithSavepoint(JobID jobId, @Nullable String savepointDirectory) throws Exception { - return miniCluster.triggerSavepoint(jobId, savepointDirectory, true).get(); + public CompletableFuture cancelWithSavepoint(JobID jobId, @Nullable String savepointDirectory) { + return miniCluster.triggerSavepoint(jobId, savepointDirectory, true); } @Override - public String stopWithSavepoint(JobID jobId, boolean advanceToEndOfEventTime, @Nullable String savepointDirector) throws Exception { - return miniCluster.stopWithSavepoint(jobId, savepointDirector, advanceToEndOfEventTime).get(); + public CompletableFuture stopWithSavepoint(JobID jobId, boolean advanceToEndOfEventTime, @Nullable String savepointDirector) { + return miniCluster.stopWithSavepoint(jobId, savepointDirector, advanceToEndOfEventTime); } @Override @@ -98,14 +106,21 @@ public CompletableFuture> listJobs() { } @Override - public Map> getAccumulators(JobID jobID, ClassLoader loader) throws Exception { - AccessExecutionGraph executionGraph = miniCluster.getExecutionGraph(jobID).get(); - Map>> accumulatorsSerialized = executionGraph.getAccumulatorsSerialized(); - Map> result = new HashMap<>(accumulatorsSerialized.size()); - for (Map.Entry>> acc : accumulatorsSerialized.entrySet()) { - result.put(acc.getKey(), acc.getValue().deserializeValue(loader)); - } - return result; + public CompletableFuture>> getAccumulators(JobID jobID, ClassLoader loader) { + return miniCluster + .getExecutionGraph(jobID) + .thenApply(AccessExecutionGraph::getAccumulatorsSerialized) + .thenApply(accumulators -> { + Map> result = new HashMap<>(accumulators.size()); + for (Map.Entry>> acc : accumulators.entrySet()) { + try { + result.put(acc.getKey(), acc.getValue().deserializeValue(loader)); + } catch (Exception e) { + throw new CompletionException("Cannot deserialize accumulators.", e); + } + } + return result; + }); } @Override @@ -120,7 +135,14 @@ public MiniClusterClient.MiniClusterId getClusterId() { @Override public String getWebInterfaceURL() { - return miniCluster.getRestAddress().toString(); + try { + return miniCluster.getRestAddress().get().toString(); + } catch (InterruptedException | ExecutionException e) { + ExceptionUtils.checkInterrupted(e); + + LOG.warn("Could not retrieve the web interface URL for the cluster.", e); + return "Unknown address."; + } } enum MiniClusterId { 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 27676556b991..c6e195415df4 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 @@ -30,7 +30,6 @@ 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.JobStatusMessage; import org.apache.flink.runtime.client.JobSubmissionException; import org.apache.flink.runtime.concurrent.FutureUtils; @@ -108,7 +107,6 @@ 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; @@ -130,7 +128,7 @@ /** * A {@link ClusterClient} implementation that communicates via HTTP REST requests. */ -public class RestClusterClient extends ClusterClient { +public class RestClusterClient implements ClusterClient { private static final Logger LOG = LoggerFactory.getLogger(RestClusterClient.class); @@ -138,9 +136,6 @@ public class RestClusterClient extends ClusterClient { private final Configuration configuration; - /** Timeout for futures. */ - private final Duration timeout; - private final RestClient restClient; private final ExecutorService executorService = Executors.newFixedThreadPool(4, new ExecutorThreadFactory("Flink-RestClusterClient-IO")); @@ -174,7 +169,6 @@ public RestClusterClient(Configuration config, T clusterId) throws Exception { WaitStrategy waitStrategy) throws Exception { this.configuration = checkNotNull(configuration); - this.timeout = AkkaUtils.getClientTimeout(configuration); this.restClusterClientConfiguration = RestClusterClientConfiguration.fromConfiguration(configuration); @@ -221,12 +215,6 @@ public void close() { } catch (Exception e) { LOG.error("An error occurred during stopping the ClientHighAvailabilityServices", e); } - - try { - super.close(); - } catch (Exception e) { - LOG.error("Error while closing the Cluster Client", e); - } } /** @@ -344,21 +332,21 @@ public CompletableFuture submitJob(@Nonnull JobGraph jobGra } @Override - public void cancel(JobID jobID) throws Exception { + public CompletableFuture cancel(JobID jobID) { JobCancellationMessageParameters params = new JobCancellationMessageParameters(); params.jobPathParameter.resolve(jobID); params.terminationModeQueryParameter.resolve(Collections.singletonList(TerminationModeQueryParameter.TerminationMode.CANCEL)); CompletableFuture responseFuture = sendRequest( JobCancellationHeaders.getInstance(), params); - responseFuture.get(timeout.toMillis(), TimeUnit.MILLISECONDS); + return responseFuture.thenApply(ignore -> Acknowledge.get()); } @Override - public String stopWithSavepoint( + public CompletableFuture stopWithSavepoint( final JobID jobId, final boolean advanceToEndOfTime, - @Nullable final String savepointDirectory) throws Exception { + @Nullable final String savepointDirectory) { final StopWithSavepointTriggerHeaders stopWithSavepointTriggerHeaders = StopWithSavepointTriggerHeaders.getInstance(); @@ -379,12 +367,12 @@ public String stopWithSavepoint( throw new CompletionException(savepointInfo.getFailureCause()); } return savepointInfo.getLocation(); - }).get(); + }); } @Override - public String cancelWithSavepoint(JobID jobId, @Nullable String savepointDirectory) throws Exception { - return triggerSavepoint(jobId, savepointDirectory, true).get(); + public CompletableFuture cancelWithSavepoint(JobID jobId, @Nullable String savepointDirectory) { + return triggerSavepoint(jobId, savepointDirectory, true); } @Override @@ -420,7 +408,7 @@ private CompletableFuture triggerSavepoint( } @Override - public Map> getAccumulators(final JobID jobID, ClassLoader loader) throws Exception { + public CompletableFuture>> getAccumulators(JobID jobID, ClassLoader loader) { final JobAccumulatorsHeaders accumulatorsHeaders = JobAccumulatorsHeaders.getInstance(); final JobAccumulatorsMessageParameters accMsgParams = accumulatorsHeaders.getUnresolvedMessageParameters(); accMsgParams.jobPathParameter.resolve(jobID); @@ -430,26 +418,18 @@ public Map> getAccumulators(final JobID jobID, C accumulatorsHeaders, accMsgParams); - Map> result = Collections.emptyMap(); - - try { - result = responseFuture.thenApply((JobAccumulatorsInfo accumulatorsInfo) -> { - try { - return AccumulatorHelper.deserializeAccumulators( - accumulatorsInfo.getSerializedUserAccumulators(), - loader); - } catch (Exception e) { - throw new CompletionException( - new FlinkException( - String.format("Deserialization of accumulators for job %s failed.", jobID), - e)); - } - }).get(timeout.toMillis(), TimeUnit.MILLISECONDS); - } catch (ExecutionException ee) { - ExceptionUtils.rethrowException(ExceptionUtils.stripExecutionException(ee)); - } - - return result; + return responseFuture.thenApply((JobAccumulatorsInfo accumulatorsInfo) -> { + try { + return AccumulatorHelper.deserializeAccumulators( + accumulatorsInfo.getSerializedUserAccumulators(), + loader); + } catch (Exception e) { + throw new CompletionException( + new FlinkException( + String.format("Deserialization of accumulators for job %s failed.", jobID), + e)); + } + }); } private CompletableFuture pollSavepointAsync( diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java index 638150a2e72a..ffc3689810b7 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java @@ -20,21 +20,20 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.client.cli.util.MockedCliFrontend; -import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.client.program.TestingClusterClient; import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.messages.Acknowledge; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; -import org.mockito.Mockito; import java.util.Collections; +import java.util.concurrent.CompletableFuture; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.isNull; -import static org.mockito.Matchers.notNull; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; /** * Tests for the CANCEL command. @@ -56,13 +55,20 @@ public void testCancel() throws Exception { // test cancel properly JobID jid = new JobID(); + OneShotLatch cancelLatch = new OneShotLatch(); + String[] parameters = { jid.toString() }; - final ClusterClient clusterClient = createClusterClient(); - MockedCliFrontend testFrontend = new MockedCliFrontend(clusterClient); - testFrontend.cancel(parameters); + TestingClusterClient clusterClient = new TestingClusterClient<>(); + + clusterClient.setCancelFunction(jobID -> { + cancelLatch.trigger(); + return CompletableFuture.completedFuture(Acknowledge.get()); + }); - Mockito.verify(clusterClient, times(1)).cancel(any(JobID.class)); + MockedCliFrontend testFrontend = new MockedCliFrontend(clusterClient); + testFrontend.cancel(parameters); + cancelLatch.await(); } @Test(expected = CliArgsException.class) @@ -94,26 +100,36 @@ public void testCancelWithSavepoint() throws Exception { // Cancel with savepoint (no target directory) JobID jid = new JobID(); + OneShotLatch cancelWithSavepointLatch = new OneShotLatch(); + String[] parameters = { "-s", jid.toString() }; - final ClusterClient clusterClient = createClusterClient(); + TestingClusterClient clusterClient = new TestingClusterClient<>(); + clusterClient.setCancelWithSavepointFunction((jobID, savepointDirectory) -> { + assertNull(savepointDirectory); + cancelWithSavepointLatch.trigger(); + return CompletableFuture.completedFuture(savepointDirectory); + }); MockedCliFrontend testFrontend = new MockedCliFrontend(clusterClient); testFrontend.cancel(parameters); - - Mockito.verify(clusterClient, times(1)) - .cancelWithSavepoint(any(JobID.class), isNull(String.class)); + cancelWithSavepointLatch.await(); } { // Cancel with savepoint (with target directory) JobID jid = new JobID(); + OneShotLatch cancelWithSavepointLatch = new OneShotLatch(); + String[] parameters = { "-s", "targetDirectory", jid.toString() }; - final ClusterClient clusterClient = createClusterClient(); + TestingClusterClient clusterClient = new TestingClusterClient<>(); + clusterClient.setCancelWithSavepointFunction((jobID, savepointDirectory) -> { + assertNotNull(savepointDirectory); + cancelWithSavepointLatch.trigger(); + return CompletableFuture.completedFuture(savepointDirectory); + }); MockedCliFrontend testFrontend = new MockedCliFrontend(clusterClient); testFrontend.cancel(parameters); - - Mockito.verify(clusterClient, times(1)) - .cancelWithSavepoint(any(JobID.class), notNull(String.class)); + cancelWithSavepointLatch.await(); } } @@ -138,10 +154,4 @@ public void testCancelWithSavepointWithoutParameters() throws Exception { Collections.singletonList(getCli(configuration))); testFrontend.cancel(parameters); } - - private static ClusterClient createClusterClient() throws Exception { - final ClusterClient clusterClient = mock(ClusterClient.class); - - return clusterClient; - } } diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopWithSavepointTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopWithSavepointTest.java index 19872aaf88ed..4ab07d033a08 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopWithSavepointTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopWithSavepointTest.java @@ -20,30 +20,25 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.client.cli.util.MockedCliFrontend; -import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.client.program.TestingClusterClient; import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; -import org.mockito.Mockito; - -import javax.annotation.Nullable; import java.util.Collections; +import java.util.concurrent.CompletableFuture; +import static org.hamcrest.CoreMatchers.is; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.ArgumentMatchers.anyBoolean; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.ArgumentMatchers.isNull; -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.powermock.api.mockito.PowerMockito.doThrow; /** * Tests for the STOP command. @@ -67,13 +62,20 @@ public void testStopWithOnlyJobId() throws Exception { String jidString = jid.toString(); String[] parameters = { jidString }; - final ClusterClient clusterClient = createClusterClient(null); + OneShotLatch stopWithSavepointLatch = new OneShotLatch(); + TestingClusterClient clusterClient = new TestingClusterClient<>(); + clusterClient.setStopWithSavepointFunction((jobID, advanceToEndOfEventTime, savepointDirectory) -> { + assertThat(jobID, is(jid)); + assertThat(advanceToEndOfEventTime, is(false)); + assertNull(savepointDirectory); + stopWithSavepointLatch.trigger(); + return CompletableFuture.completedFuture(savepointDirectory); + }); MockedCliFrontend testFrontend = new MockedCliFrontend(clusterClient); testFrontend.stop(parameters); - Mockito.verify(clusterClient, times(1)) - .stopWithSavepoint(eq(jid), eq(false), isNull()); + stopWithSavepointLatch.await(); } @Test @@ -81,12 +83,19 @@ public void testStopWithDefaultSavepointDir() throws Exception { JobID jid = new JobID(); String[] parameters = {jid.toString() }; - final ClusterClient clusterClient = createClusterClient(null); + OneShotLatch stopWithSavepointLatch = new OneShotLatch(); + TestingClusterClient clusterClient = new TestingClusterClient<>(); + clusterClient.setStopWithSavepointFunction((jobID, advanceToEndOfEventTime, savepointDirectory) -> { + assertThat(jobID, is(jid)); + assertThat(advanceToEndOfEventTime, is(false)); + assertNull(savepointDirectory); + stopWithSavepointLatch.trigger(); + return CompletableFuture.completedFuture(savepointDirectory); + }); MockedCliFrontend testFrontend = new MockedCliFrontend(clusterClient); testFrontend.stop(parameters); - Mockito.verify(clusterClient, times(1)) - .stopWithSavepoint(eq(jid), eq(false), isNull()); + stopWithSavepointLatch.await(); } @Test @@ -94,12 +103,18 @@ public void testStopWithExplicitSavepointDir() throws Exception { JobID jid = new JobID(); String[] parameters = { "-p", "test-target-dir", jid.toString() }; - final ClusterClient clusterClient = createClusterClient(null); + OneShotLatch stopWithSavepointLatch = new OneShotLatch(); + TestingClusterClient clusterClient = new TestingClusterClient<>(); + clusterClient.setStopWithSavepointFunction((jobID, advanceToEndOfEventTime, savepointDirectory) -> { + assertThat(jobID, is(jid)); + assertThat(advanceToEndOfEventTime, is(false)); + assertThat(savepointDirectory, is("test-target-dir")); + stopWithSavepointLatch.trigger(); + return CompletableFuture.completedFuture(savepointDirectory); + }); MockedCliFrontend testFrontend = new MockedCliFrontend(clusterClient); testFrontend.stop(parameters); - - Mockito.verify(clusterClient, times(1)) - .stopWithSavepoint(eq(jid), eq(false), eq("test-target-dir")); + stopWithSavepointLatch.await(); } @Test @@ -107,12 +122,19 @@ public void testStopOnlyWithMaxWM() throws Exception { JobID jid = new JobID(); String[] parameters = { "-d", jid.toString() }; - final ClusterClient clusterClient = createClusterClient(null); + OneShotLatch stopWithSavepointLatch = new OneShotLatch(); + TestingClusterClient clusterClient = new TestingClusterClient<>(); + clusterClient.setStopWithSavepointFunction((jobID, advanceToEndOfEventTime, savepointDirectory) -> { + assertThat(jobID, is(jid)); + assertThat(advanceToEndOfEventTime, is(true)); + assertNull(savepointDirectory); + stopWithSavepointLatch.trigger(); + return CompletableFuture.completedFuture(savepointDirectory); + }); MockedCliFrontend testFrontend = new MockedCliFrontend(clusterClient); testFrontend.stop(parameters); - Mockito.verify(clusterClient, times(1)) - .stopWithSavepoint(eq(jid), eq(true), isNull()); + stopWithSavepointLatch.await(); } @Test @@ -120,12 +142,19 @@ public void testStopWithMaxWMAndDefaultSavepointDir() throws Exception { JobID jid = new JobID(); String[] parameters = { "-p", "-d", jid.toString() }; - final ClusterClient clusterClient = createClusterClient(null); + OneShotLatch stopWithSavepointLatch = new OneShotLatch(); + TestingClusterClient clusterClient = new TestingClusterClient<>(); + clusterClient.setStopWithSavepointFunction((jobID, advanceToEndOfEventTime, savepointDirectory) -> { + assertThat(jobID, is(jid)); + assertThat(advanceToEndOfEventTime, is(true)); + assertNull(savepointDirectory); + stopWithSavepointLatch.trigger(); + return CompletableFuture.completedFuture(savepointDirectory); + }); MockedCliFrontend testFrontend = new MockedCliFrontend(clusterClient); testFrontend.stop(parameters); - Mockito.verify(clusterClient, times(1)) - .stopWithSavepoint(eq(jid), eq(true), isNull()); + stopWithSavepointLatch.await(); } @Test @@ -133,12 +162,19 @@ public void testStopWithMaxWMAndExplicitSavepointDir() throws Exception { JobID jid = new JobID(); String[] parameters = { "-d", "-p", "test-target-dir", jid.toString() }; - final ClusterClient clusterClient = createClusterClient(null); + OneShotLatch stopWithSavepointLatch = new OneShotLatch(); + TestingClusterClient clusterClient = new TestingClusterClient<>(); + clusterClient.setStopWithSavepointFunction((jobID, advanceToEndOfEventTime, savepointDirectory) -> { + assertThat(jobID, is(jid)); + assertThat(advanceToEndOfEventTime, is(true)); + assertThat(savepointDirectory, is("test-target-dir")); + stopWithSavepointLatch.trigger(); + return CompletableFuture.completedFuture(savepointDirectory); + }); MockedCliFrontend testFrontend = new MockedCliFrontend(clusterClient); testFrontend.stop(parameters); - Mockito.verify(clusterClient, times(1)) - .stopWithSavepoint(eq(jid), eq(true), eq("test-target-dir")); + stopWithSavepointLatch.await(); } @Test(expected = CliArgsException.class) @@ -166,14 +202,9 @@ public void testMissingJobId() throws Exception { @Test(expected = CliArgsException.class) public void testWrongSavepointDirOrder() throws Exception { JobID jid = new JobID(); - String[] parameters = { "-s", "-d", "test-target-dir", jid.toString() }; - final ClusterClient clusterClient = createClusterClient(null); - MockedCliFrontend testFrontend = new MockedCliFrontend(clusterClient); + MockedCliFrontend testFrontend = new MockedCliFrontend(new TestingClusterClient()); testFrontend.stop(parameters); - - Mockito.verify(clusterClient, times(1)) - .stopWithSavepoint(eq(jid), eq(false), eq("test-target-dir")); } @Test @@ -184,7 +215,8 @@ public void testUnknownJobId() throws Exception { String[] parameters = { "-p", "test-target-dir", jid.toString() }; String expectedMessage = "Test exception"; FlinkException testException = new FlinkException(expectedMessage); - final ClusterClient clusterClient = createClusterClient(testException); + TestingClusterClient clusterClient = new TestingClusterClient<>(); + clusterClient.setStopWithSavepointFunction((jobID, advanceToEndOfEventTime, savepointDirectory) -> FutureUtils.completedExceptionally(testException)); MockedCliFrontend testFrontend = new MockedCliFrontend(clusterClient); try { @@ -194,14 +226,4 @@ public void testUnknownJobId() throws Exception { assertTrue(ExceptionUtils.findThrowableWithMessage(e, expectedMessage).isPresent()); } } - - private static ClusterClient createClusterClient(@Nullable Exception exception) throws Exception { - final ClusterClient clusterClient = mock(ClusterClient.class); - - if (exception != null) { - doThrow(exception).when(clusterClient).stopWithSavepoint(any(JobID.class), anyBoolean(), anyString()); - } - - return clusterClient; - } } diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java b/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java index e76175e5c75b..91bbec57c066 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/util/MockedCliFrontend.java @@ -31,7 +31,7 @@ */ public class MockedCliFrontend extends CliFrontend { - public MockedCliFrontend(ClusterClient clusterClient) { + public MockedCliFrontend(ClusterClient clusterClient) { super( new Configuration(), new DummyClusterClientServiceLoader<>(clusterClient), diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/TestingClusterClient.java b/flink-clients/src/test/java/org/apache/flink/client/program/TestingClusterClient.java new file mode 100644 index 000000000000..b13ab94eb10c --- /dev/null +++ b/flink-clients/src/test/java/org/apache/flink/client/program/TestingClusterClient.java @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.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.flink.util.function.TriFunction; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.function.BiFunction; +import java.util.function.Function; + +/** + * Settable implementation of ClusterClient used for testing. + */ +public class TestingClusterClient implements ClusterClient { + + private Function> cancelFunction = ignore -> CompletableFuture.completedFuture(Acknowledge.get()); + private BiFunction> cancelWithSavepointFunction = (ignore, savepointPath) -> CompletableFuture.completedFuture(savepointPath); + private TriFunction> stopWithSavepointFunction = (ignore1, ignore2, savepointPath) -> CompletableFuture.completedFuture(savepointPath); + private BiFunction> triggerSavepointFunction = (ignore, savepointPath) -> CompletableFuture.completedFuture(savepointPath); + + public void setCancelFunction(Function> cancelFunction) { + this.cancelFunction = cancelFunction; + } + + public void setCancelWithSavepointFunction(BiFunction> cancelWithSavepointFunction) { + this.cancelWithSavepointFunction = cancelWithSavepointFunction; + } + + public void setStopWithSavepointFunction(TriFunction> stopWithSavepointFunction) { + this.stopWithSavepointFunction = stopWithSavepointFunction; + } + + public void setTriggerSavepointFunction(BiFunction> triggerSavepointFunction) { + this.triggerSavepointFunction = triggerSavepointFunction; + } + + @Override + public T getClusterId() { + throw new UnsupportedOperationException(); + } + + @Override + public Configuration getFlinkConfiguration() { + throw new UnsupportedOperationException(); + } + + @Override + public void shutDownCluster() { + throw new UnsupportedOperationException(); + } + + @Override + public String getWebInterfaceURL() { + throw new UnsupportedOperationException(); + } + + @Override + public CompletableFuture> listJobs() { + throw new UnsupportedOperationException(); + } + + @Override + public CompletableFuture disposeSavepoint(String savepointPath) { + throw new UnsupportedOperationException(); + } + + @Override + public CompletableFuture submitJob(@Nonnull JobGraph jobGraph) { + throw new UnsupportedOperationException(); + } + + @Override + public CompletableFuture getJobStatus(JobID jobId) { + throw new UnsupportedOperationException(); + } + + @Override + public CompletableFuture requestJobResult(@Nonnull JobID jobId) { + throw new UnsupportedOperationException(); + } + + @Override + public CompletableFuture>> getAccumulators(JobID jobID, ClassLoader loader) { + throw new UnsupportedOperationException(); + } + + @Override + public CompletableFuture cancel(JobID jobId) { + return cancelFunction.apply(jobId); + } + + @Override + public CompletableFuture cancelWithSavepoint(JobID jobId, @Nullable String savepointDirectory) { + return cancelWithSavepointFunction.apply(jobId, savepointDirectory); + } + + @Override + public CompletableFuture stopWithSavepoint(JobID jobId, boolean advanceToEndOfEventTime, @Nullable String savepointDirectory) { + return stopWithSavepointFunction.apply(jobId, advanceToEndOfEventTime, savepointDirectory); + } + + @Override + public CompletableFuture triggerSavepoint(JobID jobId, @Nullable String savepointDirectory) { + return triggerSavepointFunction.apply(jobId, savepointDirectory); + } +} diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientSavepointTriggerTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientSavepointTriggerTest.java index 31f6f9ed7e72..5c6e21dfd7b2 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientSavepointTriggerTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientSavepointTriggerTest.java @@ -184,7 +184,7 @@ public void testTriggerSavepointCancelJob() throws Exception { final RestClusterClient restClusterClient = createRestClusterClient(restServerEndpoint.getServerAddress().getPort()); - final String savepointPath = restClusterClient.cancelWithSavepoint(new JobID(), null); + final String savepointPath = restClusterClient.cancelWithSavepoint(new JobID(), null).get(); assertEquals(expectedSavepointDir, savepointPath); } } diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java index 691a3edb5496..3af83cef7e13 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java @@ -231,18 +231,15 @@ public void testJobSubmitCancel() throws Exception { submitHandler, terminationHandler, testJobExecutionResultHandler)) { - RestClusterClient restClusterClient = createRestClusterClient(restServerEndpoint.getServerAddress().getPort()); - try { + try (RestClusterClient restClusterClient = createRestClusterClient(restServerEndpoint.getServerAddress().getPort())) { Assert.assertFalse(submitHandler.jobSubmitted); ClientUtils.submitJobAndWaitForResult(restClusterClient, jobGraph, ClassLoader.getSystemClassLoader()); Assert.assertTrue(submitHandler.jobSubmitted); Assert.assertFalse(terminationHandler.jobCanceled); - restClusterClient.cancel(jobId); + restClusterClient.cancel(jobId).get(); Assert.assertTrue(terminationHandler.jobCanceled); - } finally { - restClusterClient.close(); } } } @@ -521,7 +518,7 @@ public void testGetAccumulators() throws Exception { JobID id = new JobID(); { - Map> accumulators = restClusterClient.getAccumulators(id); + Map> accumulators = restClusterClient.getAccumulators(id).get(); assertNotNull(accumulators); assertEquals(1, accumulators.size()); diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java index 6c69eb90ff95..5e99be73072e 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java @@ -253,7 +253,7 @@ public void run() { while (System.nanoTime() < deadline); // cancel the job & wait for the job to finish - client.cancel(Iterables.getOnlyElement(getRunningJobs(client))); + client.cancel(Iterables.getOnlyElement(getRunningJobs(client))).get(); runner.join(); final Throwable t = errorRef.get(); @@ -338,7 +338,7 @@ public void run() { while (System.nanoTime() < deadline); // cancel the job & wait for the job to finish - client.cancel(Iterables.getOnlyElement(getRunningJobs(client))); + client.cancel(Iterables.getOnlyElement(getRunningJobs(client))).get(); runner.join(); final Throwable t = errorRef.get(); @@ -504,7 +504,7 @@ public void cancel() { } // cancel the consume job after all extra records are written - client.cancel(consumeJobId); + client.cancel(consumeJobId).get(); consumeThread.join(); kafkaOffsetHandler.close(); @@ -1020,7 +1020,7 @@ public void runCancelingOnFullInputTest() throws Exception { } // cancel - client.cancel(jobId); + client.cancel(jobId).get(); // wait for the program to be done and validate that we failed with the right exception runnerThread.join(); @@ -1090,7 +1090,7 @@ public void runCancelingOnEmptyInputTest() throws Exception { Assert.fail("Test failed prematurely with: " + failueCause.getMessage()); } // cancel - client.cancel(jobId); + client.cancel(jobId).get(); // wait for the program to be done and validate that we failed with the right exception runnerThread.join(); @@ -1643,7 +1643,7 @@ public void cancel() { LOG.info("Found all JMX metrics. Cancelling job."); } finally { // cancel - client.cancel(jobId); + client.cancel(jobId).get(); // wait for the job to finish (it should due to the cancel command above) jobThread.join(); } @@ -2055,7 +2055,7 @@ public Tuple2 map(Tuple2 value) throws Excep // did not finish in time, maybe the producer dropped one or more records and // the validation did not reach the exit point success = false; - client.cancel(jobId); + client.cancel(jobId).get(); } else { Throwable error = errorRef.get(); diff --git a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointReaderITTestBase.java b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointReaderITTestBase.java index 06148435e291..f7e3a871b665 100644 --- a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointReaderITTestBase.java +++ b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointReaderITTestBase.java @@ -196,7 +196,7 @@ private String takeSavepoint(JobGraph jobGraph) throws Exception { CompletableFuture path = client.triggerSavepoint(result.getJobID(), dirPath); return path.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); } finally { - client.cancel(jobId); + client.cancel(jobId).get(); } } diff --git a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointReaderKeyedStateITCase.java b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointReaderKeyedStateITCase.java index 39825bd34207..1ce2376be9a4 100644 --- a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointReaderKeyedStateITCase.java +++ b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointReaderKeyedStateITCase.java @@ -132,7 +132,7 @@ private String takeSavepoint(JobGraph jobGraph) throws Exception { CompletableFuture path = client.triggerSavepoint(result.getJobID(), dirPath); return path.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); } finally { - client.cancel(jobId); + client.cancel(jobId).get(); } } diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java index 6791ab6fafdf..292642c93668 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java @@ -1255,7 +1255,7 @@ JobID getJobId() { @Override public void close() throws Exception { // Free cluster resources - clusterClient.cancel(jobId); + clusterClient.cancel(jobId).get(); // cancel() is non-blocking so do this to make sure the job finished CompletableFuture jobStatusFuture = FutureUtils.retrySuccessfulWithDelay( () -> clusterClient.getJobStatus(jobId), 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 9796b4cb80ca..f6166364e46e 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 @@ -429,7 +429,7 @@ private void cancelQueryInternal(ExecutionContext context, String resultI // retrieve existing cluster clusterClient = clusterDescriptor.retrieve(context.getClusterId()); try { - clusterClient.cancel(new JobID(StringUtils.hexStringToByte(resultId))); + clusterClient.cancel(new JobID(StringUtils.hexStringToByte(resultId))).get(); } catch (Throwable t) { // the job might has finished earlier } diff --git a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointIT.java b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointIT.java index d585bf1aa586..d7578c6535a4 100644 --- a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointIT.java +++ b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointIT.java @@ -190,7 +190,7 @@ private void throwingExceptionOnCallbackWithRestartsHelper(final boolean termina final long syncSavepoint = syncSavepointId.get(); assertTrue(syncSavepoint > 0 && syncSavepoint < numberOfCheckpointsToExpect); - clusterClient.cancel(jobGraph.getJobID()); + clusterClient.cancel(jobGraph.getJobID()).get(); assertThat(getJobStatus(), either(equalTo(JobStatus.CANCELLING)).or(equalTo(JobStatus.CANCELED))); } diff --git a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointITCase.java index 1c5f270d00de..bde424643aa6 100644 --- a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointITCase.java @@ -184,7 +184,7 @@ public void testCancelWithSavepointWithoutConfiguredSavepointDirectory() throws setUpWithCheckpointInterval(10L); try { - clusterClient.cancelWithSavepoint(jobGraph.getJobID(), null); + clusterClient.cancelWithSavepoint(jobGraph.getJobID(), null).get(); } catch (Exception e) { if (!ExceptionUtils.findThrowableWithMessage(e, "savepoint directory").isPresent()) { throw e; @@ -254,7 +254,7 @@ public Future notifyCheckpointCompleteAsync(final long checkpointId) { private String cancelWithSavepoint() throws Exception { return clusterClient.cancelWithSavepoint( jobGraph.getJobID(), - savepointDirectory.toAbsolutePath().toString()); + savepointDirectory.toAbsolutePath().toString()).get(); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java index 98062555a731..20195d5c6885 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java @@ -160,7 +160,7 @@ public void go() throws Exception { FutureUtils.retrySuccessfulWithDelay( () -> { try { - return CompletableFuture.completedFuture(client.getAccumulators(jobGraph.getJobID())); + return CompletableFuture.completedFuture(client.getAccumulators(jobGraph.getJobID()).get()); } catch (Exception e) { return FutureUtils.completedExceptionally(e); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java index d6927221da22..579069c7121b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java @@ -111,7 +111,7 @@ protected void runAndCancelJob(Plan plan, final int msecsTillCanceling, int maxT Thread.sleep(msecsTillCanceling); - client.cancel(jobSubmissionResult.getJobID()); + client.cancel(jobSubmissionResult.getJobID()).get(); Deadline cancelDeadline = new FiniteDuration(maxTimeTillCanceled, TimeUnit.MILLISECONDS).fromNow(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java index cf1b00a5d2fe..923933fbfc07 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java @@ -217,7 +217,7 @@ public void testSavepointRescalingKeyedState(boolean scaleOut, boolean deriveMax final String savepointPath = savepointPathFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); - client.cancel(jobID); + client.cancel(jobID).get(); while (!getRunningJobs(client).isEmpty()) { Thread.sleep(50); @@ -279,7 +279,7 @@ public void testSavepointRescalingNonPartitionedStateCausesException() throws Ex final String savepointPath = savepointPathFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); - client.cancel(jobID); + client.cancel(jobID).get(); while (!getRunningJobs(client).isEmpty()) { Thread.sleep(50); @@ -361,7 +361,7 @@ public void testSavepointRescalingWithKeyedAndNonPartitionedState() throws Excep final String savepointPath = savepointPathFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); - client.cancel(jobID); + client.cancel(jobID).get(); while (!getRunningJobs(client).isEmpty()) { Thread.sleep(50); @@ -478,7 +478,7 @@ public void testSavepointRescalingPartitionedOperatorState(boolean scaleOut, Ope final String savepointPath = savepointPathFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); - client.cancel(jobID); + client.cancel(jobID).get(); while (!getRunningJobs(client).isEmpty()) { Thread.sleep(50); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java index e7df79792d2e..f356d6a56adb 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java @@ -304,7 +304,7 @@ private static String runJobAndGetExternalizedCheckpoint(StateBackend backend, F NotifyingInfiniteTupleSource.countDownLatch.await(); waitUntilExternalizedCheckpointCreated(checkpointDir, initialJobGraph.getJobID()); - client.cancel(initialJobGraph.getJobID()); + client.cancel(initialJobGraph.getJobID()).get(); waitUntilCanceled(initialJobGraph.getJobID(), client); return getExternalizedCheckpointCheckpointPath(checkpointDir, initialJobGraph.getJobID()); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java index 032db60bbf8a..909b640539fd 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 @@ -195,7 +195,7 @@ private String submitJobAndTakeSavepoint(MiniClusterResourceFactory clusterFacto StatefulCounter.getProgressLatch().await(); - return client.cancelWithSavepoint(jobId, null); + return client.cancelWithSavepoint(jobId, null).get(); } finally { cluster.after(); StatefulCounter.resetForTest(parallelism); @@ -240,7 +240,7 @@ private void restoreJobAndVerifyState(String savepointPath, MiniClusterResourceF // Await some progress after restore StatefulCounter.getProgressLatch().await(); - client.cancel(jobId); + client.cancel(jobId).get(); FutureUtils.retrySuccessfulWithDelay( () -> client.getJobStatus(jobId), @@ -681,7 +681,7 @@ public Integer map(Integer value) throws Exception { } savepointPath = client.triggerSavepoint(jobGraph.getJobID(), null).get(); - client.cancel(jobGraph.getJobID()); + client.cancel(jobGraph.getJobID()).get(); while (!client.getJobStatus(jobGraph.getJobID()).get().isGloballyTerminalState()) { Thread.sleep(100); } @@ -694,7 +694,7 @@ public Integer map(Integer value) throws Exception { latch.await(); } - client.cancel(jobGraph.getJobID()); + client.cancel(jobGraph.getJobID()).get(); while (!client.getJobStatus(jobGraph.getJobID()).get().isGloballyTerminalState()) { Thread.sleep(100); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java index 7a43704e5e9d..3c2446f3ff16 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java @@ -146,7 +146,7 @@ protected final void executeAndSavepoint( boolean done = false; while (deadLine.hasTimeLeft()) { Thread.sleep(100); - Map> accumulators = client.getAccumulators(jobSubmissionResult.getJobID()); + Map> accumulators = client.getAccumulators(jobSubmissionResult.getJobID()).get(); boolean allDone = true; for (Tuple2 acc : expectedAccumulators) { @@ -226,7 +226,7 @@ protected final void restoreAndExecute( } Thread.sleep(100); - Map> accumulators = client.getAccumulators(jobId); + Map> accumulators = client.getAccumulators(jobId).get(); boolean allDone = true; for (Tuple2 acc : expectedAccumulators) { 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 2d8afc0f7822..2f755193844d 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 @@ -370,7 +370,7 @@ public void testDisposeSavepointWithCustomKvState() throws Exception { clusterClient.disposeSavepoint(savepointPath).get(); - clusterClient.cancel(jobId); + clusterClient.cancel(jobId).get(); // make sure, the execution is finished to not influence other test methods invokeThread.join(deadline.timeLeft().toMillis()); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java index db0b1be177d5..a3e153acbc1a 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 @@ -217,7 +217,7 @@ public Long map(Long value) throws Exception { taskManagerProcess = null; // try to cancel the job - clusterClient.cancel(jobId); + clusterClient.cancel(jobId).get(); // we should see a failure within reasonable time (10s is the ask timeout). // since the CI environment is often slow, we conservatively give it up to 2 minutes, diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java index ae1484c27ae0..f04280573d17 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java @@ -152,7 +152,7 @@ private String migrateJob(ClusterClient clusterClient, Deadline deadline) thr try { savepointPath = clusterClient.cancelWithSavepoint( jobToMigrate.getJobID(), - targetDirectory.getAbsolutePath()); + targetDirectory.getAbsolutePath()).get(); } catch (Exception e) { String exceptionString = ExceptionUtils.stringifyException(e); if (!PATTERN_CANCEL_WITH_SAVEPOINT_TOLERATED_EXCEPTIONS.matcher(exceptionString).find()) { 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 83a57459db18..14c41e057aa0 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 @@ -197,7 +197,7 @@ public void run() { // send stop until the job is stopped do { try { - clusterClient.stopWithSavepoint(id, false, "test"); + clusterClient.stopWithSavepoint(id, false, "test").get(); } catch (Exception e) { boolean ignoreException = ExceptionUtils.findThrowable(e, CheckpointException.class) 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 4e7ece316a54..1068d02a0a62 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 @@ -23,10 +23,10 @@ import org.apache.flink.client.deployment.ClusterClientFactory; import org.apache.flink.client.deployment.ClusterClientServiceLoader; import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.client.program.TestingClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.yarn.cli.FlinkYarnSessionCli; -import org.apache.flink.yarn.util.FakeClusterClient; import org.apache.flink.yarn.util.NonDeployingYarnClusterDescriptor; import org.apache.hadoop.yarn.api.records.ApplicationId; @@ -71,7 +71,7 @@ public void testRun() throws Exception { configuration.setInteger(JobManagerOptions.PORT, 8081); final ClusterClientServiceLoader testServiceLoader = - new TestingYarnClusterClientServiceLoader(new FakeClusterClient()); + new TestingYarnClusterClientServiceLoader(new TestingClusterClient<>()); final FlinkYarnSessionCli yarnCLI = new FlinkYarnSessionCli( configuration, 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 deleted file mode 100644 index 97db45df4f1e..000000000000 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/util/FakeClusterClient.java +++ /dev/null @@ -1,111 +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.util; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.JobSubmissionResult; -import org.apache.flink.client.program.ClusterClient; -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; -import java.util.Collections; -import java.util.Map; -import java.util.concurrent.CompletableFuture; - -/** - * Dummy {@link ClusterClient} for testing purposes (extend as needed). - */ -public class FakeClusterClient extends ClusterClient { - - @Override - public Configuration getFlinkConfiguration() { - throw new UnsupportedOperationException("Not needed in test."); - } - - @Override - public String getWebInterfaceURL() { - return ""; - } - - @Override - public ApplicationId getClusterId() { - 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."); - } - - @Override - public String stopWithSavepoint(JobID jobId, boolean advanceToEndOfEventTime, @Nullable String savepointDirectory) throws Exception { - 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."); - } -} From fe77fdcfecd558d09cb035dc6c0735f32868806e Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Wed, 30 Oct 2019 15:18:07 +0800 Subject: [PATCH 506/746] [FLINK-14262][table] FunctionCatalog should return empty when can not get the catalog from catalog name This closes #10039 --- .../java/org/apache/flink/table/catalog/FunctionCatalog.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) 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 d56645321314..90c0bcb3de8d 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 @@ -307,9 +307,10 @@ private Optional resolvePreciseFunctionReference(ObjectId ); } - Catalog catalog = catalogManager.getCatalog(oi.getCatalogName()).get(); + Optional catalogOptional = catalogManager.getCatalog(oi.getCatalogName()); - if (catalog != null) { + if (catalogOptional.isPresent()) { + Catalog catalog = catalogOptional.get(); try { CatalogFunction catalogFunction = catalog.getFunction( new ObjectPath(oi.getDatabaseName(), oi.getObjectName())); From 0a5ae34318ed2ceaa4d2807054a60384d2b001bf Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Fri, 1 Nov 2019 13:05:33 +0800 Subject: [PATCH 507/746] [FLINK-14262][table-planner-blink] Add getCatalogManager to FlinkContext This closes #10039 --- .../flink/table/planner/delegation/PlannerContext.java | 4 +++- .../apache/flink/table/planner/calcite/FlinkContext.scala | 7 ++++++- .../flink/table/planner/calcite/FlinkContextImpl.scala | 7 +++++-- .../flink/table/planner/calcite/FlinkRelBuilder.scala | 4 +++- .../flink/table/planner/delegation/PlannerBase.scala | 3 ++- .../plan/optimize/BatchCommonSubGraphBasedOptimizer.scala | 4 +++- .../plan/optimize/StreamCommonSubGraphBasedOptimizer.scala | 4 +++- .../flink/table/sqlexec/SqlToOperationConverterTest.java | 1 + .../plan/metadata/AggCallSelectivityEstimatorTest.scala | 2 +- .../planner/plan/metadata/FlinkRelMdHandlerTestBase.scala | 1 + .../planner/plan/metadata/SelectivityEstimatorTest.scala | 2 +- 11 files changed, 29 insertions(+), 10 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java index 43cde6d12a97..b989dce380af 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java @@ -24,6 +24,7 @@ import org.apache.flink.table.api.SqlDialect; import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableException; +import org.apache.flink.table.catalog.CatalogManager; import org.apache.flink.table.catalog.FunctionCatalog; import org.apache.flink.table.planner.calcite.CalciteConfig; import org.apache.flink.table.planner.calcite.CalciteConfig$; @@ -84,11 +85,12 @@ public class PlannerContext { public PlannerContext( TableConfig tableConfig, FunctionCatalog functionCatalog, + CatalogManager catalogManager, CalciteSchema rootSchema, List traitDefs) { this.tableConfig = tableConfig; this.functionCatalog = functionCatalog; - this.context = new FlinkContextImpl(tableConfig, functionCatalog); + this.context = new FlinkContextImpl(tableConfig, functionCatalog, catalogManager); this.rootSchema = rootSchema; this.traitDefs = traitDefs; // Make a framework config to initialize the RelOptCluster instance, diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkContext.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkContext.scala index f974d88c2fbc..06f6998ab6ca 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkContext.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkContext.scala @@ -19,7 +19,7 @@ package org.apache.flink.table.planner.calcite import org.apache.flink.table.api.TableConfig -import org.apache.flink.table.catalog.FunctionCatalog +import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog} import org.apache.calcite.plan.Context @@ -38,6 +38,11 @@ trait FlinkContext extends Context { */ def getFunctionCatalog: FunctionCatalog + /** + * Gets [[CatalogManager]] instance defined in [[org.apache.flink.table.api.TableEnvironment]]. + */ + def getCatalogManager: CatalogManager + override def unwrap[C](clazz: Class[C]): C = { if (clazz.isInstance(this)) clazz.cast(this) else null.asInstanceOf[C] } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkContextImpl.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkContextImpl.scala index 263547d5c78b..99db1eb34e88 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkContextImpl.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkContextImpl.scala @@ -19,14 +19,17 @@ package org.apache.flink.table.planner.calcite import org.apache.flink.table.api.TableConfig -import org.apache.flink.table.catalog.FunctionCatalog +import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog} class FlinkContextImpl( tableConfig: TableConfig, - functionCatalog: FunctionCatalog) + functionCatalog: FunctionCatalog, + catalogManager: CatalogManager) extends FlinkContext { override def getTableConfig: TableConfig = tableConfig override def getFunctionCatalog: FunctionCatalog = functionCatalog + + override def getCatalogManager: CatalogManager = catalogManager } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelBuilder.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelBuilder.scala index c493907ffd4f..b61696893b4c 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelBuilder.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelBuilder.scala @@ -19,7 +19,7 @@ package org.apache.flink.table.planner.calcite import org.apache.flink.table.api.TableConfig -import org.apache.flink.table.catalog.FunctionCatalog +import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog} import org.apache.flink.table.operations.QueryOperation import org.apache.flink.table.planner.calcite.FlinkRelBuilder.PlannerNamedWindowProperty import org.apache.flink.table.planner.calcite.FlinkRelFactories.{ExpandFactory, RankFactory, SinkFactory} @@ -177,6 +177,8 @@ object FlinkRelBuilder { override def getFunctionCatalog: FunctionCatalog = clusterContext.getFunctionCatalog override def unwrap[C](clazz: Class[C]): C = context.unwrap(clazz) + + override def getCatalogManager: CatalogManager = clusterContext.getCatalogManager } new FlinkRelBuilder(mergedContext, cluster, schema) } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala index 0e9436dc6b17..9b6442a91a8c 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala @@ -71,7 +71,7 @@ abstract class PlannerBase( executor: Executor, config: TableConfig, val functionCatalog: FunctionCatalog, - catalogManager: CatalogManager, + val catalogManager: CatalogManager, isStreamingMode: Boolean) extends Planner { @@ -85,6 +85,7 @@ abstract class PlannerBase( new PlannerContext( config, functionCatalog, + catalogManager, asRootSchema(new CatalogManagerCalciteSchema(catalogManager, isStreamingMode)), getTraitDefs.toList ) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala index 549a519b3e5b..da0871628190 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala @@ -19,7 +19,7 @@ package org.apache.flink.table.planner.plan.optimize import org.apache.flink.table.api.TableConfig -import org.apache.flink.table.catalog.FunctionCatalog +import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog} import org.apache.flink.table.planner.delegation.BatchPlanner import org.apache.flink.table.planner.plan.nodes.physical.batch.BatchExecSink import org.apache.flink.table.planner.plan.optimize.program.{BatchOptimizeContext, FlinkBatchProgram} @@ -81,6 +81,8 @@ class BatchCommonSubGraphBasedOptimizer(planner: BatchPlanner) override def getTableConfig: TableConfig = config override def getFunctionCatalog: FunctionCatalog = planner.functionCatalog + + override def getCatalogManager: CatalogManager = planner.catalogManager }) } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala index 5f3ee5229197..fc47bdbcc0bc 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala @@ -20,7 +20,7 @@ package org.apache.flink.table.planner.plan.optimize import org.apache.flink.table.api.TableConfig import org.apache.flink.table.api.config.ExecutionConfigOptions -import org.apache.flink.table.catalog.FunctionCatalog +import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog} import org.apache.flink.table.planner.delegation.StreamPlanner import org.apache.flink.table.planner.plan.`trait`.{AccMode, AccModeTraitDef, MiniBatchInterval, MiniBatchIntervalTrait, MiniBatchIntervalTraitDef, MiniBatchMode, UpdateAsRetractionTraitDef} import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery @@ -169,6 +169,8 @@ class StreamCommonSubGraphBasedOptimizer(planner: StreamPlanner) override def getFunctionCatalog: FunctionCatalog = planner.functionCatalog + override def getCatalogManager: CatalogManager = planner.catalogManager + override def getRexBuilder: RexBuilder = planner.getRelBuilder.getRexBuilder override def updateAsRetraction: Boolean = updatesAsRetraction diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java index 5c11d89725d9..a94d5412b990 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java @@ -78,6 +78,7 @@ public class SqlToOperationConverterTest { private final PlannerContext plannerContext = new PlannerContext(tableConfig, functionCatalog, + catalogManager, asRootSchema(new CatalogManagerCalciteSchema(catalogManager, false)), new ArrayList<>()); diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/AggCallSelectivityEstimatorTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/AggCallSelectivityEstimatorTest.scala index 554d51b4ceb0..dd280ba437f3 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/AggCallSelectivityEstimatorTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/AggCallSelectivityEstimatorTest.scala @@ -85,7 +85,7 @@ class AggCallSelectivityEstimatorTest { val catalogManager = mock(classOf[CatalogManager]) val moduleManager = mock(classOf[ModuleManager]) val functionCatalog = new FunctionCatalog(catalogManager, moduleManager) - val context = new FlinkContextImpl(new TableConfig, functionCatalog) + val context = new FlinkContextImpl(new TableConfig, functionCatalog, catalogManager) when(tableScan, "getCluster").thenReturn(cluster) when(cluster, "getRexBuilder").thenReturn(rexBuilder) when(cluster, "getTypeFactory").thenReturn(typeFactory) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala index 3802d44f3e99..9fd90a3475a2 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala @@ -91,6 +91,7 @@ class FlinkRelMdHandlerTestBase { new PlannerContext( tableConfig, new FunctionCatalog(catalogManager, moduleManager), + catalogManager, CalciteSchema.from(rootSchema), util.Arrays.asList( ConventionTraitDef.INSTANCE, diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimatorTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimatorTest.scala index f16ae27adecb..0386dbb0573a 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimatorTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimatorTest.scala @@ -87,7 +87,7 @@ class SelectivityEstimatorTest { val catalogManager = mock(classOf[CatalogManager]) val moduleManager = mock(classOf[ModuleManager]) val functionCatalog = new FunctionCatalog(catalogManager, moduleManager) - val context: FlinkContext = new FlinkContextImpl(tableConfig, functionCatalog) + val context: FlinkContext = new FlinkContextImpl(tableConfig, functionCatalog, catalogManager) when(tableScan, "getCluster").thenReturn(cluster) when(cluster, "getRexBuilder").thenReturn(rexBuilder) when(cluster, "getPlanner").thenReturn(planner) From 1b4ebce86b71d56f44185f1cb83d9a3b51de13df Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Wed, 6 Nov 2019 19:29:43 +0800 Subject: [PATCH 508/746] [FLINK-14262][table-planner-blink] support referencing function with fully/partially qualified names in blink This closes #10039 --- .../catalog/FunctionCatalogOperatorTable.java | 61 +++++++++++-------- .../planner/delegation/PlannerContext.java | 21 ++++--- .../expressions/SqlAggFunctionVisitor.java | 5 +- .../converter/ScalarFunctionConvertRule.java | 3 +- .../functions/utils/FunctionUtils.java | 58 ++++++++++++++++++ .../functions/utils/HiveAggSqlFunction.java | 8 ++- .../utils/HiveScalarSqlFunction.java | 7 ++- .../functions/utils/HiveTableSqlFunction.java | 18 ++++-- .../planner/plan/QueryOperationConverter.java | 3 +- .../functions/utils/AggSqlFunction.scala | 26 ++++---- .../functions/utils/ScalarSqlFunction.scala | 30 ++++----- .../functions/utils/TableSqlFunction.scala | 32 +++++----- .../utils/UserDefinedFunctionUtils.scala | 18 +++--- .../rules/logical/LogicalUnnestRule.scala | 5 +- .../PushFilterIntoTableSourceScanRule.scala | 5 +- .../planner/plan/utils/RexNodeExtractor.scala | 32 ++++++---- .../planner/plan/utils/SetOpRewriteUtil.scala | 3 +- .../planner/catalog/CatalogTableITCase.scala | 52 +++++++++++++++- .../metadata/FlinkRelMdHandlerTestBase.scala | 16 +++-- .../plan/utils/PartitionPrunerTest.scala | 8 ++- .../plan/utils/RexNodeExtractorTest.scala | 14 +++-- 21 files changed, 294 insertions(+), 131 deletions(-) create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/FunctionUtils.java diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/FunctionCatalogOperatorTable.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/FunctionCatalogOperatorTable.java index b9e0a165ad59..cf17ff892733 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/FunctionCatalogOperatorTable.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/FunctionCatalogOperatorTable.java @@ -20,14 +20,20 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.java.typeutils.GenericTypeInfo; +import org.apache.flink.table.catalog.CatalogManager; import org.apache.flink.table.catalog.FunctionCatalog; import org.apache.flink.table.catalog.FunctionLookup; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.UnresolvedIdentifier; +import org.apache.flink.table.expressions.CallExpression; import org.apache.flink.table.functions.AggregateFunctionDefinition; import org.apache.flink.table.functions.FunctionDefinition; import org.apache.flink.table.functions.FunctionIdentifier; import org.apache.flink.table.functions.ScalarFunctionDefinition; import org.apache.flink.table.functions.TableFunctionDefinition; +import org.apache.flink.table.functions.UserDefinedFunction; import org.apache.flink.table.planner.calcite.FlinkTypeFactory; +import org.apache.flink.table.planner.functions.utils.FunctionUtils; import org.apache.flink.table.planner.functions.utils.HiveAggSqlFunction; import org.apache.flink.table.planner.functions.utils.HiveScalarSqlFunction; import org.apache.flink.table.planner.functions.utils.HiveTableSqlFunction; @@ -43,11 +49,14 @@ import org.apache.calcite.sql.SqlOperator; import org.apache.calcite.sql.SqlOperatorTable; import org.apache.calcite.sql.SqlSyntax; +import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.calcite.sql.validate.SqlNameMatcher; +import java.util.Arrays; import java.util.List; import java.util.Optional; +import static org.apache.flink.table.planner.functions.utils.FunctionUtils.toFunctionIdentifier; import static org.apache.flink.table.planner.functions.utils.HiveFunctionUtils.isHiveFunc; import static org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType; @@ -58,12 +67,15 @@ public class FunctionCatalogOperatorTable implements SqlOperatorTable { private final FunctionCatalog functionCatalog; + private final CatalogManager catalogManager; private final FlinkTypeFactory typeFactory; public FunctionCatalogOperatorTable( FunctionCatalog functionCatalog, + CatalogManager catalogManager, FlinkTypeFactory typeFactory) { this.functionCatalog = functionCatalog; + this.catalogManager = catalogManager; this.typeFactory = typeFactory; } @@ -74,7 +86,7 @@ public void lookupOperatorOverloads( SqlSyntax syntax, List operatorList, SqlNameMatcher nameMatcher) { - if (!opName.isSimple()) { + if (opName.isStar()) { return; } @@ -84,12 +96,12 @@ public void lookupOperatorOverloads( return; } - String name = opName.getSimple(); - Optional candidateFunction = functionCatalog.lookupFunction( - FunctionIdentifier.of(name)); + FunctionIdentifier identifier = toFunctionIdentifier(opName.names.toArray(new String[0]), catalogManager); + + Optional candidateFunction = functionCatalog.lookupFunction(identifier); candidateFunction.flatMap(lookupResult -> - convertToSqlFunction(category, name, lookupResult.getFunctionDefinition()) + convertToSqlFunction(category, identifier, lookupResult.getFunctionDefinition()) ).ifPresent(operatorList::add); } @@ -99,29 +111,25 @@ private boolean isNotUserFunction(SqlFunctionCategory category) { private Optional convertToSqlFunction( SqlFunctionCategory category, - String name, + FunctionIdentifier identifier, FunctionDefinition functionDefinition) { if (functionDefinition instanceof AggregateFunctionDefinition) { AggregateFunctionDefinition def = (AggregateFunctionDefinition) functionDefinition; if (isHiveFunc(def.getAggregateFunction())) { return Optional.of(new HiveAggSqlFunction( - name, - name, - def.getAggregateFunction(), - typeFactory)); + identifier, def.getAggregateFunction(), typeFactory)); } else { - return convertAggregateFunction(name, (AggregateFunctionDefinition) functionDefinition); + return convertAggregateFunction(identifier, (AggregateFunctionDefinition) functionDefinition); } } else if (functionDefinition instanceof ScalarFunctionDefinition) { ScalarFunctionDefinition def = (ScalarFunctionDefinition) functionDefinition; if (isHiveFunc(def.getScalarFunction())) { return Optional.of(new HiveScalarSqlFunction( - name, - name, + identifier, def.getScalarFunction(), typeFactory)); } else { - return convertScalarFunction(name, def); + return convertScalarFunction(identifier, def); } } else if (functionDefinition instanceof TableFunctionDefinition && category != null && @@ -130,15 +138,14 @@ private Optional convertToSqlFunction( if (isHiveFunc(def.getTableFunction())) { DataType returnType = fromLegacyInfoToDataType(new GenericTypeInfo<>(Row.class)); return Optional.of(new HiveTableSqlFunction( - name, - name, + identifier, def.getTableFunction(), returnType, typeFactory, new DeferredTypeFlinkTableFunction(def.getTableFunction(), returnType), - HiveTableSqlFunction.operandTypeChecker(name, def.getTableFunction()))); + HiveTableSqlFunction.operandTypeChecker(identifier, def.getTableFunction()))); } else { - return convertTableFunction(name, (TableFunctionDefinition) functionDefinition); + return convertTableFunction(identifier, (TableFunctionDefinition) functionDefinition); } } @@ -146,11 +153,11 @@ private Optional convertToSqlFunction( } private Optional convertAggregateFunction( - String name, + FunctionIdentifier identifier, AggregateFunctionDefinition functionDefinition) { SqlFunction aggregateFunction = UserDefinedFunctionUtils.createAggregateSqlFunction( - name, - name, + identifier, + identifier.toString(), functionDefinition.getAggregateFunction(), TypeConversions.fromLegacyInfoToDataType(functionDefinition.getResultTypeInfo()), TypeConversions.fromLegacyInfoToDataType(functionDefinition.getAccumulatorTypeInfo()), @@ -159,20 +166,20 @@ private Optional convertAggregateFunction( return Optional.of(aggregateFunction); } - private Optional convertScalarFunction(String name, ScalarFunctionDefinition functionDefinition) { + private Optional convertScalarFunction(FunctionIdentifier identifier, ScalarFunctionDefinition functionDefinition) { SqlFunction scalarFunction = UserDefinedFunctionUtils.createScalarSqlFunction( - name, - name, + identifier, + identifier.toString(), functionDefinition.getScalarFunction(), typeFactory ); return Optional.of(scalarFunction); } - private Optional convertTableFunction(String name, TableFunctionDefinition functionDefinition) { + private Optional convertTableFunction(FunctionIdentifier identifier, TableFunctionDefinition functionDefinition) { SqlFunction tableFunction = UserDefinedFunctionUtils.createTableSqlFunction( - name, - name, + identifier, + identifier.toString(), functionDefinition.getTableFunction(), TypeConversions.fromLegacyInfoToDataType(functionDefinition.getResultType()), typeFactory diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java index b989dce380af..997ce39f4cf3 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java @@ -29,6 +29,7 @@ import org.apache.flink.table.planner.calcite.CalciteConfig; import org.apache.flink.table.planner.calcite.CalciteConfig$; import org.apache.flink.table.planner.calcite.CalciteParser; +import org.apache.flink.table.planner.calcite.FlinkContext; import org.apache.flink.table.planner.calcite.FlinkContextImpl; import org.apache.flink.table.planner.calcite.FlinkPlannerImpl; import org.apache.flink.table.planner.calcite.FlinkRelBuilder; @@ -45,7 +46,6 @@ import org.apache.calcite.config.Lex; import org.apache.calcite.jdbc.CalciteSchema; -import org.apache.calcite.plan.Context; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelOptPlanner; import org.apache.calcite.plan.RelTraitDef; @@ -76,9 +76,8 @@ public class PlannerContext { private final RelDataTypeSystem typeSystem = new FlinkTypeSystem(); private final FlinkTypeFactory typeFactory = new FlinkTypeFactory(typeSystem); private final TableConfig tableConfig; - private final FunctionCatalog functionCatalog; private final RelOptCluster cluster; - private final Context context; + private final FlinkContext context; private final CalciteSchema rootSchema; private final List traitDefs; @@ -89,7 +88,6 @@ public PlannerContext( CalciteSchema rootSchema, List traitDefs) { this.tableConfig = tableConfig; - this.functionCatalog = functionCatalog; this.context = new FlinkContextImpl(tableConfig, functionCatalog, catalogManager); this.rootSchema = rootSchema; this.traitDefs = traitDefs; @@ -113,7 +111,7 @@ private FrameworkConfig createFrameworkConfig() { .costFactory(new FlinkCostFactory()) .typeSystem(typeSystem) .sqlToRelConverterConfig(getSqlToRelConverterConfig(getCalciteConfig(tableConfig))) - .operatorTable(getSqlOperatorTable(getCalciteConfig(tableConfig), functionCatalog)) + .operatorTable(getSqlOperatorTable(getCalciteConfig(tableConfig))) // set the executor to evaluate constant expressions .executor(new ExpressionReducer(tableConfig, false)) .context(context) @@ -248,23 +246,26 @@ private SqlToRelConverter.Config getSqlToRelConverterConfig(CalciteConfig calcit /** * Returns the operator table for this environment including a custom Calcite configuration. */ - private SqlOperatorTable getSqlOperatorTable(CalciteConfig calciteConfig, FunctionCatalog functionCatalog) { + private SqlOperatorTable getSqlOperatorTable(CalciteConfig calciteConfig) { return JavaScalaConversionUtil.toJava(calciteConfig.getSqlOperatorTable()).map(operatorTable -> { if (calciteConfig.replacesSqlOperatorTable()) { return operatorTable; } else { - return ChainedSqlOperatorTable.of(getBuiltinSqlOperatorTable(functionCatalog), operatorTable); + return ChainedSqlOperatorTable.of(getBuiltinSqlOperatorTable(), operatorTable); } } - ).orElseGet(() -> getBuiltinSqlOperatorTable(functionCatalog)); + ).orElseGet(this::getBuiltinSqlOperatorTable); } /** * Returns builtin the operator table and external the operator for this environment. */ - private SqlOperatorTable getBuiltinSqlOperatorTable(FunctionCatalog functionCatalog) { + private SqlOperatorTable getBuiltinSqlOperatorTable() { return ChainedSqlOperatorTable.of( - new FunctionCatalogOperatorTable(functionCatalog, typeFactory), + new FunctionCatalogOperatorTable( + context.getFunctionCatalog(), + context.getCatalogManager(), + typeFactory), FlinkSqlOperatorTable.instance()); } diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/SqlAggFunctionVisitor.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/SqlAggFunctionVisitor.java index bb6538151bd5..d1ec68478b70 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/SqlAggFunctionVisitor.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/SqlAggFunctionVisitor.java @@ -41,6 +41,7 @@ import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.isFunctionOfKind; import static org.apache.flink.table.functions.FunctionKind.AGGREGATE; import static org.apache.flink.table.functions.FunctionKind.TABLE_AGGREGATE; +import static org.apache.flink.table.planner.functions.utils.FunctionUtils.toFunctionIdentifier; import static org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType; /** @@ -89,7 +90,7 @@ public SqlAggFunction visit(CallExpression call) { AggregateFunctionDefinition aggDef = (AggregateFunctionDefinition) def; AggregateFunction aggFunc = aggDef.getAggregateFunction(); return new AggSqlFunction( - aggFunc.functionIdentifier(), + toFunctionIdentifier(call, aggFunc), aggFunc.toString(), aggFunc, fromLegacyInfoToDataType(aggDef.getResultTypeInfo()), @@ -101,7 +102,7 @@ public SqlAggFunction visit(CallExpression call) { TableAggregateFunctionDefinition aggDef = (TableAggregateFunctionDefinition) def; TableAggregateFunction aggFunc = aggDef.getTableAggregateFunction(); return new AggSqlFunction( - aggFunc.functionIdentifier(), + toFunctionIdentifier(call, aggFunc), aggFunc.toString(), aggFunc, fromLegacyInfoToDataType(aggDef.getResultTypeInfo()), diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ScalarFunctionConvertRule.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ScalarFunctionConvertRule.java index 8921092ad1ce..7b5a8a4dd59b 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ScalarFunctionConvertRule.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ScalarFunctionConvertRule.java @@ -30,6 +30,7 @@ import java.util.Optional; import static org.apache.flink.table.planner.expressions.converter.ExpressionConverter.toRexNodes; +import static org.apache.flink.table.planner.functions.utils.FunctionUtils.toFunctionIdentifier; /** * {@link CallExpressionConvertRule} to convert {@link ScalarFunctionDefinition}. @@ -42,7 +43,7 @@ public Optional convert(CallExpression call, ConvertContext context) { if (def instanceof ScalarFunctionDefinition) { ScalarFunction scalaFunc = ((ScalarFunctionDefinition) def).getScalarFunction(); SqlFunction sqlFunction = UserDefinedFunctionUtils.createScalarSqlFunction( - scalaFunc.functionIdentifier(), + toFunctionIdentifier(call, scalaFunc), scalaFunc.toString(), scalaFunc, context.getTypeFactory()); diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/FunctionUtils.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/FunctionUtils.java new file mode 100644 index 000000000000..01abf689d7a5 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/FunctionUtils.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.table.planner.functions.utils; + +import org.apache.flink.table.catalog.CatalogManager; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.UnresolvedIdentifier; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.functions.FunctionIdentifier; +import org.apache.flink.table.functions.UserDefinedFunction; + +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.parser.SqlParserPos; + +import java.util.Arrays; +import java.util.Optional; + +/** + * Utils for sql functions. + */ +public class FunctionUtils { + + public static FunctionIdentifier toFunctionIdentifier(String[] names, CatalogManager catalogManager) { + return names.length == 1 ? + FunctionIdentifier.of(names[0]) : + FunctionIdentifier.of( + catalogManager.qualifyIdentifier(UnresolvedIdentifier.of(names))); + } + + public static FunctionIdentifier toFunctionIdentifier(CallExpression call, UserDefinedFunction function) { + return call.getFunctionIdentifier() + .orElse(FunctionIdentifier.of(function.functionIdentifier())); + } + + public static SqlIdentifier toSqlIdentifier(FunctionIdentifier fi) { + Optional objectIdentifier = fi.getIdentifier(); + String[] names = objectIdentifier + .map(id -> new String[] {id.getCatalogName(), id.getDatabaseName(), id.getObjectName()}) + .orElseGet(() -> new String[]{fi.getSimpleName().get()}); + return new SqlIdentifier(Arrays.asList(names), SqlParserPos.ZERO); + } +} diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/HiveAggSqlFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/HiveAggSqlFunction.java index 54bde22d1f13..373e7ebe2dd7 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/HiveAggSqlFunction.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/HiveAggSqlFunction.java @@ -20,6 +20,7 @@ import org.apache.flink.api.java.typeutils.GenericTypeInfo; import org.apache.flink.table.functions.AggregateFunction; +import org.apache.flink.table.functions.FunctionIdentifier; import org.apache.flink.table.planner.calcite.FlinkTypeFactory; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.util.InstantiationUtil; @@ -49,9 +50,10 @@ public class HiveAggSqlFunction extends AggSqlFunction { private final AggregateFunction aggregateFunction; - public HiveAggSqlFunction(String name, String displayName, - AggregateFunction aggregateFunction, FlinkTypeFactory typeFactory) { - super(name, displayName, aggregateFunction, fromLegacyInfoToDataType(new GenericTypeInfo<>(Object.class)), + public HiveAggSqlFunction( + FunctionIdentifier identifier, AggregateFunction aggregateFunction, FlinkTypeFactory typeFactory) { + super(identifier, identifier.toString(), aggregateFunction, + fromLegacyInfoToDataType(new GenericTypeInfo<>(Object.class)), fromLegacyInfoToDataType(new GenericTypeInfo<>(Object.class)), typeFactory, false, new Some<>(createReturnTypeInference(aggregateFunction, typeFactory))); this.aggregateFunction = aggregateFunction; diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/HiveScalarSqlFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/HiveScalarSqlFunction.java index a44576ae737d..c35fea1642c5 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/HiveScalarSqlFunction.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/HiveScalarSqlFunction.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.functions.utils; +import org.apache.flink.table.functions.FunctionIdentifier; import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.planner.calcite.FlinkTypeFactory; import org.apache.flink.table.types.logical.LogicalType; @@ -49,9 +50,9 @@ public class HiveScalarSqlFunction extends ScalarSqlFunction { private final ScalarFunction function; public HiveScalarSqlFunction( - String name, String displayName, - ScalarFunction function, FlinkTypeFactory typeFactory) { - super(name, displayName, function, typeFactory, new Some<>(createReturnTypeInference(function, typeFactory))); + FunctionIdentifier identifier, ScalarFunction function, FlinkTypeFactory typeFactory) { + super(identifier, identifier.toString(), function, + typeFactory, new Some<>(createReturnTypeInference(function, typeFactory))); this.function = function; } diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/HiveTableSqlFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/HiveTableSqlFunction.java index 6800fb98f8f0..e2133c5c4f96 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/HiveTableSqlFunction.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/HiveTableSqlFunction.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.functions.utils; +import org.apache.flink.table.functions.FunctionIdentifier; import org.apache.flink.table.functions.TableFunction; import org.apache.flink.table.planner.calcite.FlinkTypeFactory; import org.apache.flink.table.planner.plan.schema.FlinkTableFunction; @@ -77,13 +78,15 @@ public class HiveTableSqlFunction extends TableSqlFunction { private final TableFunction hiveUdtf; private final HiveOperandTypeChecker operandTypeChecker; - public HiveTableSqlFunction(String name, String displayName, + public HiveTableSqlFunction( + FunctionIdentifier identifier, TableFunction hiveUdtf, DataType implicitResultType, FlinkTypeFactory typeFactory, FlinkTableFunction functionImpl, HiveOperandTypeChecker operandTypeChecker) { - super(name, displayName, hiveUdtf, implicitResultType, typeFactory, functionImpl, scala.Option.apply(operandTypeChecker)); + super(identifier, identifier.toString(), hiveUdtf, implicitResultType, typeFactory, + functionImpl, scala.Option.apply(operandTypeChecker)); this.hiveUdtf = hiveUdtf; this.operandTypeChecker = operandTypeChecker; } @@ -208,8 +211,10 @@ private static Object getValue(SqlNode right) throws NonLiteralException { private static class NonLiteralException extends Exception { } - public static HiveOperandTypeChecker operandTypeChecker(String name, TableFunction udtf) { - return new HiveOperandTypeChecker(name, udtf, UserDefinedFunctionUtils.checkAndExtractMethods(udtf, "eval")); + public static HiveOperandTypeChecker operandTypeChecker( + FunctionIdentifier identifier, TableFunction udtf) { + return new HiveOperandTypeChecker( + identifier, udtf, UserDefinedFunctionUtils.checkAndExtractMethods(udtf, "eval")); } /** @@ -222,8 +227,9 @@ public static class HiveOperandTypeChecker extends OperandTypeChecker { private LogicalType[] previousArgTypes; - private HiveOperandTypeChecker(String name, TableFunction udtf, Method[] methods) { - super(name, udtf, methods); + private HiveOperandTypeChecker( + FunctionIdentifier identifier, TableFunction udtf, Method[] methods) { + super(identifier, udtf, methods); } @Override diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java index 0e5b92b67f50..752a63605d32 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java @@ -33,6 +33,7 @@ import org.apache.flink.table.expressions.resolver.LookupCallResolver; import org.apache.flink.table.functions.BuiltInFunctionDefinitions; import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.functions.FunctionIdentifier; import org.apache.flink.table.functions.TableFunction; import org.apache.flink.table.operations.AggregateQueryOperation; import org.apache.flink.table.operations.CalculatedQueryOperation; @@ -276,7 +277,7 @@ public RelNode visit(CalculatedQueryOperation calculatedTable) { FlinkTypeFactory typeFactory = relBuilder.getTypeFactory(); TableSqlFunction sqlFunction = new TableSqlFunction( - tableFunction.functionIdentifier(), + FunctionIdentifier.of(tableFunction.functionIdentifier()), tableFunction.toString(), tableFunction, resultType, 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 5213929ecb55..94516884c933 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 @@ -19,9 +19,10 @@ package org.apache.flink.table.planner.functions.utils import org.apache.flink.table.api.ValidationException -import org.apache.flink.table.functions.{AggregateFunction, TableAggregateFunction, UserDefinedAggregateFunction} +import org.apache.flink.table.functions.{AggregateFunction, FunctionIdentifier, TableAggregateFunction, UserDefinedAggregateFunction} import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.functions.utils.AggSqlFunction.{createOperandTypeChecker, createOperandTypeInference, createReturnTypeInference} +import org.apache.flink.table.planner.functions.utils.FunctionUtils.toSqlIdentifier import org.apache.flink.table.planner.functions.utils.UserDefinedFunctionUtils._ import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType import org.apache.flink.table.types.DataType @@ -31,7 +32,6 @@ import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.sql._ import org.apache.calcite.sql.`type`.SqlOperandTypeChecker.Consistency import org.apache.calcite.sql.`type`._ -import org.apache.calcite.sql.parser.SqlParserPos import org.apache.calcite.sql.validate.SqlUserDefinedAggFunction import org.apache.calcite.util.Optionality @@ -41,7 +41,7 @@ import java.util * Calcite wrapper for user-defined aggregate functions. Currently, the aggregate function can be * an [[AggregateFunction]] or a [[TableAggregateFunction]] * - * @param name function name (used by SQL parser) + * @param identifier function identifier to uniquely identify this function * @param displayName name to be displayed in operator name * @param aggregateFunction aggregate function to be called * @param externalResultType the type information of returned value @@ -49,7 +49,7 @@ import java.util * @param typeFactory type factory for converting Flink's between Calcite's types */ class AggSqlFunction( - name: String, + identifier: FunctionIdentifier, displayName: String, val aggregateFunction: UserDefinedAggregateFunction[_, _], val externalResultType: DataType, @@ -58,11 +58,11 @@ class AggSqlFunction( requiresOver: Boolean, returnTypeInfer: Option[SqlReturnTypeInference] = None) extends SqlUserDefinedAggFunction( - new SqlIdentifier(name, SqlParserPos.ZERO), + toSqlIdentifier(identifier), returnTypeInfer.getOrElse(createReturnTypeInference( fromDataTypeToLogicalType(externalResultType), typeFactory)), - createOperandTypeInference(name, aggregateFunction, typeFactory, externalAccType), - createOperandTypeChecker(name, aggregateFunction, externalAccType), + createOperandTypeInference(identifier, aggregateFunction, typeFactory, externalAccType), + createOperandTypeChecker(identifier, 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, @@ -90,7 +90,7 @@ class AggSqlFunction( object AggSqlFunction { def apply( - name: String, + identifier: FunctionIdentifier, displayName: String, aggregateFunction: UserDefinedAggregateFunction[_, _], externalResultType: DataType, @@ -99,7 +99,7 @@ object AggSqlFunction { requiresOver: Boolean): AggSqlFunction = { new AggSqlFunction( - name, + identifier, displayName, aggregateFunction, externalResultType, @@ -109,7 +109,7 @@ object AggSqlFunction { } private[flink] def createOperandTypeInference( - name: String, + identifier: FunctionIdentifier, aggregateFunction: UserDefinedAggregateFunction[_, _], typeFactory: FlinkTypeFactory, externalAccType: DataType): SqlOperandTypeInference = { @@ -128,7 +128,7 @@ object AggSqlFunction { val foundSignature = getAccumulateMethodSignature(aggregateFunction, operandLogicalType) .getOrElse( throw new ValidationException( - s"Given parameters of function '$name' do not match any signature. \n" + + s"Given parameters of function '$identifier' do not match any signature. \n" + s"Actual: ${signatureInternalToString(actualSignature)} \n" + s"Expected: ${signaturesToString(aggregateFunction, "accumulate")}")) @@ -161,7 +161,7 @@ object AggSqlFunction { } private[flink] def createOperandTypeChecker( - name: String, + identifier: FunctionIdentifier, aggregateFunction: UserDefinedAggregateFunction[_, _], externalAccType: DataType): SqlOperandTypeChecker = { @@ -210,7 +210,7 @@ object AggSqlFunction { if (foundSignature.isEmpty) { if (throwOnFailure) { throw new ValidationException( - s"Given parameters of function '$name' do not match any signature. \n" + + s"Given parameters of function '$identifier' do not match any signature. \n" + s"Actual: ${signatureInternalToString(actualSignature)} \n" + s"Expected: ${signaturesToString(aggregateFunction, "accumulate")}") } else { 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 f036cbea952f..1840d4199542 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 @@ -19,8 +19,9 @@ package org.apache.flink.table.planner.functions.utils import org.apache.flink.table.api.ValidationException -import org.apache.flink.table.functions.ScalarFunction +import org.apache.flink.table.functions.{FunctionIdentifier, ScalarFunction} import org.apache.flink.table.planner.calcite.FlinkTypeFactory +import org.apache.flink.table.planner.functions.utils.FunctionUtils.toSqlIdentifier import org.apache.flink.table.planner.functions.utils.ScalarSqlFunction._ import org.apache.flink.table.planner.functions.utils.UserDefinedFunctionUtils.{getOperandType, _} import org.apache.flink.table.runtime.types.ClassLogicalTypeConverter.getDefaultExternalClassForType @@ -32,29 +33,28 @@ import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.sql._ import org.apache.calcite.sql.`type`.SqlOperandTypeChecker.Consistency import org.apache.calcite.sql.`type`._ -import org.apache.calcite.sql.parser.SqlParserPos import scala.collection.JavaConverters._ /** * Calcite wrapper for user-defined scalar functions. * - * @param name function name (used by SQL parser) + * @param identifier function identifier to uniquely identify this function * @param displayName name to be displayed in operator name * @param scalarFunction scalar function to be called * @param typeFactory type factory for converting Flink's between Calcite's types */ class ScalarSqlFunction( - name: String, + identifier: FunctionIdentifier, displayName: String, val scalarFunction: ScalarFunction, typeFactory: FlinkTypeFactory, returnTypeInfer: Option[SqlReturnTypeInference] = None) extends SqlFunction( - new SqlIdentifier(name, SqlParserPos.ZERO), - returnTypeInfer.getOrElse(createReturnTypeInference(name, scalarFunction, typeFactory)), - createOperandTypeInference(name, scalarFunction, typeFactory), - createOperandTypeChecker(name, scalarFunction), + toSqlIdentifier(identifier), + returnTypeInfer.getOrElse(createReturnTypeInference(identifier, scalarFunction, typeFactory)), + createOperandTypeInference(identifier, scalarFunction, typeFactory), + createOperandTypeChecker(identifier, scalarFunction), null, SqlFunctionCategory.USER_DEFINED_FUNCTION) { @@ -73,7 +73,7 @@ class ScalarSqlFunction( object ScalarSqlFunction { private[flink] def createReturnTypeInference( - name: String, + identifier: FunctionIdentifier, scalarFunction: ScalarFunction, typeFactory: FlinkTypeFactory): SqlReturnTypeInference = { /** @@ -102,7 +102,7 @@ object ScalarSqlFunction { } private[flink] def createOperandTypeInference( - name: String, + identifier: FunctionIdentifier, scalarFunction: ScalarFunction, typeFactory: FlinkTypeFactory): SqlOperandTypeInference = { /** @@ -114,13 +114,13 @@ object ScalarSqlFunction { returnType: RelDataType, operandTypes: Array[RelDataType]): Unit = { ScalarSqlFunction.inferOperandTypes( - name, scalarFunction, typeFactory, callBinding, returnType, operandTypes) + identifier, scalarFunction, typeFactory, callBinding, returnType, operandTypes) } } } def inferOperandTypes( - name: String, + identifier: FunctionIdentifier, func: ScalarFunction, typeFactory: FlinkTypeFactory, callBinding: SqlCallBinding, @@ -128,7 +128,7 @@ object ScalarSqlFunction { operandTypes: Array[RelDataType]): Unit = { val parameters = getOperandType(callBinding).toArray if (getEvalUserDefinedMethod(func, parameters).isEmpty) { - throwValidationException(name, func, parameters) + throwValidationException(identifier.toString, func, parameters) } func.getParameterTypes(getEvalMethodSignature(func, parameters)) .map(fromTypeInfoToLogicalType) @@ -140,7 +140,7 @@ object ScalarSqlFunction { } private[flink] def createOperandTypeChecker( - name: String, + identifier: FunctionIdentifier, scalarFunction: ScalarFunction): SqlOperandTypeChecker = { val methods = checkAndExtractMethods(scalarFunction, "eval") @@ -186,7 +186,7 @@ object ScalarSqlFunction { if (foundMethod.isEmpty) { if (throwOnFailure) { throw new ValidationException( - s"Given parameters of function '$name' do not match any signature. \n" + + s"Given parameters of function '$identifier' do not match any signature. \n" + s"Actual: ${signatureInternalToString(operandTypeInfo)} \n" + s"Expected: ${signaturesToString(scalarFunction, "eval")}") } else { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/TableSqlFunction.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/TableSqlFunction.scala index 4cb1d53c7660..1a6020f7b5e4 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/TableSqlFunction.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/TableSqlFunction.scala @@ -19,8 +19,9 @@ package org.apache.flink.table.planner.functions.utils import org.apache.flink.table.api.ValidationException -import org.apache.flink.table.functions.TableFunction +import org.apache.flink.table.functions.{FunctionIdentifier, TableFunction} import org.apache.flink.table.planner.calcite.FlinkTypeFactory +import org.apache.flink.table.planner.functions.utils.FunctionUtils.toSqlIdentifier import org.apache.flink.table.planner.functions.utils.TableSqlFunction._ import org.apache.flink.table.planner.functions.utils.UserDefinedFunctionUtils._ import org.apache.flink.table.planner.plan.schema.FlinkTableFunction @@ -32,7 +33,6 @@ import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory} import org.apache.calcite.sql._ import org.apache.calcite.sql.`type`.SqlOperandTypeChecker.Consistency import org.apache.calcite.sql.`type`._ -import org.apache.calcite.sql.parser.SqlParserPos import org.apache.calcite.sql.validate.{SqlUserDefinedTableFunction, SqlUserDefinedTableMacro} import java.lang.reflect.Method @@ -41,7 +41,7 @@ import java.util /** * Calcite wrapper for user-defined table functions. * - * @param name function name (used by SQL parser) + * @param identifier function identifier to uniquely identify this function * @param udtf user-defined table function to be called * @param implicitResultType Implicit result type information * @param typeFactory type factory for converting Flink's between Calcite's types @@ -49,7 +49,7 @@ import java.util * @return [[TableSqlFunction]] */ class TableSqlFunction( - name: String, + identifier: FunctionIdentifier, displayName: String, val udtf: TableFunction[_], implicitResultType: DataType, @@ -57,12 +57,12 @@ class TableSqlFunction( functionImpl: FlinkTableFunction, operandTypeInfer: Option[SqlOperandTypeChecker] = None) extends SqlUserDefinedTableFunction( - new SqlIdentifier(name, SqlParserPos.ZERO), + toSqlIdentifier(identifier), ReturnTypes.CURSOR, // type inference has the UNKNOWN operand types. - createOperandTypeInference(name, udtf, typeFactory), + createOperandTypeInference(identifier, udtf, typeFactory), // only checker has the real operand types. - operandTypeInfer.getOrElse(createOperandTypeChecker(name, udtf)), + operandTypeInfer.getOrElse(createOperandTypeChecker(identifier, udtf)), null, functionImpl) { @@ -98,7 +98,7 @@ class TableSqlFunction( object TableSqlFunction { private[flink] def createOperandTypeInference( - name: String, + identifier: FunctionIdentifier, udtf: TableFunction[_], typeFactory: FlinkTypeFactory): SqlOperandTypeInference = { /** @@ -110,13 +110,13 @@ object TableSqlFunction { returnType: RelDataType, operandTypes: Array[RelDataType]): Unit = { inferOperandTypesInternal( - name, udtf, typeFactory, callBinding, returnType, operandTypes) + identifier, udtf, typeFactory, callBinding, returnType, operandTypes) } } } def inferOperandTypesInternal( - name: String, + identifier: FunctionIdentifier, func: TableFunction[_], typeFactory: FlinkTypeFactory, callBinding: SqlCallBinding, @@ -124,7 +124,7 @@ object TableSqlFunction { operandTypes: Array[RelDataType]): Unit = { val parameters = getOperandType(callBinding).toArray if (getEvalUserDefinedMethod(func, parameters).isEmpty) { - throwValidationException(name, func, parameters) + throwValidationException(identifier.toString, func, parameters) } func.getParameterTypes(getEvalMethodSignature(func, parameters)) .map(fromTypeInfoToLogicalType) @@ -136,9 +136,9 @@ object TableSqlFunction { } private[flink] def createOperandTypeChecker( - name: String, + identifier: FunctionIdentifier, udtf: TableFunction[_]): SqlOperandTypeChecker = { - new OperandTypeChecker(name, udtf, checkAndExtractMethods(udtf, "eval")) + new OperandTypeChecker(identifier, udtf, checkAndExtractMethods(udtf, "eval")) } } @@ -146,7 +146,9 @@ object TableSqlFunction { * Operand type checker based on [[TableFunction]] given information. */ class OperandTypeChecker( - name: String, udtf: TableFunction[_], methods: Array[Method]) extends SqlOperandTypeChecker { + identifier: FunctionIdentifier, + udtf: TableFunction[_], + methods: Array[Method]) extends SqlOperandTypeChecker { override def getAllowedSignatures(op: SqlOperator, opName: String): String = { s"$opName[${signaturesToString(udtf, "eval")}]" @@ -181,7 +183,7 @@ class OperandTypeChecker( if (getEvalUserDefinedMethod(udtf, operandTypes).isEmpty) { if (throwOnFailure) { throw new ValidationException( - s"Given parameters of function '$name' do not match any signature. \n" + + s"Given parameters of function '$identifier' do not match any signature. \n" + s"Actual: ${signatureInternalToString(operandTypes)} \n" + s"Expected: ${signaturesToString(udtf, "eval")}") } else { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/UserDefinedFunctionUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/UserDefinedFunctionUtils.scala index 664dd8e3d267..f1f782dca1f7 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/UserDefinedFunctionUtils.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/UserDefinedFunctionUtils.scala @@ -449,17 +449,17 @@ object UserDefinedFunctionUtils { /** * Create [[SqlFunction]] for a [[ScalarFunction]] * - * @param name function name + * @param identifier function identifier * @param function scalar function * @param typeFactory type factory * @return the ScalarSqlFunction */ def createScalarSqlFunction( - name: String, + identifier: FunctionIdentifier, displayName: String, function: ScalarFunction, typeFactory: FlinkTypeFactory): SqlFunction = { - new ScalarSqlFunction(name, displayName, function, typeFactory) + new ScalarSqlFunction(identifier, displayName, function, typeFactory) } /** @@ -478,34 +478,34 @@ object UserDefinedFunctionUtils { * For all the other cases, please use * createTableSqlFunction (String, String, TableFunction, FlinkTypeFactory) instead. * - * @param name function name + * @param identifier function identifier * @param tableFunction table function * @param implicitResultType the implicit type information of returned table * @param typeFactory type factory * @return the TableSqlFunction */ def createTableSqlFunction( - name: String, + identifier: FunctionIdentifier, displayName: String, tableFunction: TableFunction[_], implicitResultType: DataType, typeFactory: FlinkTypeFactory): TableSqlFunction = { // we don't know the exact result type yet. val function = new DeferredTypeFlinkTableFunction(tableFunction, implicitResultType) - new TableSqlFunction(name, displayName, tableFunction, implicitResultType, + new TableSqlFunction(identifier, displayName, tableFunction, implicitResultType, typeFactory, function) } /** * Create [[SqlFunction]] for an [[AggregateFunction]] * - * @param name function name + * @param identifier function identifier * @param aggFunction aggregate function * @param typeFactory type factory * @return the TableSqlFunction */ def createAggregateSqlFunction( - name: String, + identifier: FunctionIdentifier, displayName: String, aggFunction: AggregateFunction[_, _], externalResultType: DataType, @@ -516,7 +516,7 @@ object UserDefinedFunctionUtils { checkAndExtractMethods(aggFunction, "accumulate") AggSqlFunction( - name, + identifier, displayName, aggFunction, externalResultType, diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.scala index 9692f40fc1cd..c99cdccf4d56 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.scala @@ -20,6 +20,7 @@ package org.apache.flink.table.planner.plan.rules.logical import org.apache.flink.api.java.typeutils.MapTypeInfo import org.apache.flink.table.api.TableException +import org.apache.flink.table.functions.FunctionIdentifier import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.calcite.FlinkTypeFactory.toLogicalType import org.apache.flink.table.planner.functions.utils.UserDefinedFunctionUtils @@ -36,7 +37,9 @@ import org.apache.calcite.rel.RelNode import org.apache.calcite.rel.`type`.{RelDataTypeFieldImpl, RelRecordType, StructKind} import org.apache.calcite.rel.core.Uncollect import org.apache.calcite.rel.logical._ +import org.apache.calcite.sql.SqlIdentifier import org.apache.calcite.sql.`type`.{AbstractSqlType, ArraySqlType, MapSqlType, MultisetSqlType} +import org.apache.calcite.sql.parser.SqlParserPos import java.util.Collections @@ -123,7 +126,7 @@ class LogicalUnnestRule( // create sql function val explodeSqlFunc = UserDefinedFunctionUtils.createTableSqlFunction( - "explode", + FunctionIdentifier.of("explode"), "explode", explodeTableFunc, fromLogicalTypeToDataType(toLogicalType(componentType)), 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 e63dc218b10c..902b6d9c9c2b 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 @@ -84,7 +84,7 @@ class PushFilterIntoTableSourceScanRule extends RelOptRule( relOptTable: FlinkRelOptTable): Unit = { val relBuilder = call.builder() - val functionCatalog = call.getPlanner.getContext.asInstanceOf[FlinkContext].getFunctionCatalog + val context = call.getPlanner.getContext.asInstanceOf[FlinkContext] val maxCnfNodeCount = FlinkRelOptUtil.getMaxCnfNodeCount(scan) val (predicates, unconvertedRexNodes) = RexNodeExtractor.extractConjunctiveConditions( @@ -92,7 +92,8 @@ class PushFilterIntoTableSourceScanRule extends RelOptRule( maxCnfNodeCount, filter.getInput.getRowType.getFieldNames, relBuilder.getRexBuilder, - functionCatalog, + context.getFunctionCatalog, + context.getCatalogManager, TimeZone.getTimeZone(scan.getCluster.getPlanner.getContext .asInstanceOf[FlinkContext].getTableConfig.getLocalTimeZone)) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala index 99fcda4c1ab8..c4fc15fe0c99 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala @@ -19,26 +19,29 @@ package org.apache.flink.table.planner.plan.utils import org.apache.flink.table.api.TableException -import org.apache.flink.table.catalog.{FunctionCatalog, FunctionLookup} +import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog, FunctionLookup} import org.apache.flink.table.dataformat.DataFormatConverters.{LocalDateConverter, LocalDateTimeConverter, LocalTimeConverter} import org.apache.flink.table.expressions._ import org.apache.flink.table.expressions.utils.ApiExpressionUtils._ import org.apache.flink.table.functions.BuiltInFunctionDefinitions.{AND, CAST, OR} +import org.apache.flink.table.functions.FunctionIdentifier import org.apache.flink.table.planner.calcite.FlinkTypeFactory +import org.apache.flink.table.planner.functions.utils.FunctionUtils.toFunctionIdentifier import org.apache.flink.table.planner.utils.Logging import org.apache.flink.table.runtime.functions.SqlDateTimeUtils.unixTimestampToLocalDateTime import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType import org.apache.flink.table.types.DataType import org.apache.flink.table.types.logical.LogicalTypeRoot._ import org.apache.flink.util.Preconditions + import org.apache.calcite.plan.RelOptUtil import org.apache.calcite.rex._ import org.apache.calcite.sql.fun.{SqlStdOperatorTable, SqlTrimFunction} import org.apache.calcite.sql.{SqlFunction, SqlPostfixOperator} import org.apache.calcite.util.Util -import java.util.{TimeZone, List => JList} -import org.apache.flink.table.functions.FunctionIdentifier +import java.util +import java.util.{TimeZone, List => JList} import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ @@ -91,6 +94,7 @@ object RexNodeExtractor extends Logging { inputFieldNames: JList[String], rexBuilder: RexBuilder, functionCatalog: FunctionCatalog, + catalogManager: CatalogManager, timeZone: TimeZone): (Array[Expression], Array[RexNode]) = { // converts the expanded expression to conjunctive normal form, // like "(a AND b) OR c" will be converted to "(a OR c) AND (b OR c)" @@ -101,7 +105,8 @@ object RexNodeExtractor extends Logging { val convertedExpressions = new mutable.ArrayBuffer[Expression] val unconvertedRexNodes = new mutable.ArrayBuffer[RexNode] val inputNames = inputFieldNames.asScala.toArray - val converter = new RexNodeToExpressionConverter(inputNames, functionCatalog, timeZone) + val converter = new RexNodeToExpressionConverter( + inputNames, functionCatalog, catalogManager, timeZone) conjunctions.asScala.foreach(rex => { rex.accept(converter) match { @@ -294,6 +299,7 @@ class RefFieldAccessorVisitor(usedFields: Array[Int]) extends RexVisitorImpl[Uni class RexNodeToExpressionConverter( inputNames: Array[String], functionCatalog: FunctionCatalog, + catalogManager: CatalogManager, timeZone: TimeZone) extends RexVisitor[Option[ResolvedExpression]] { @@ -403,12 +409,16 @@ class RexNodeToExpressionConverter( Option(operands.reduceLeft((l, r) => new CallExpression(AND, Seq(l, r), outputType))) case SqlStdOperatorTable.CAST => Option(new CallExpression(CAST, Seq(operands.head, typeLiteral(outputType)), outputType)) - case function: SqlFunction => - lookupFunction(replace(function.getName), operands, outputType) - case postfix: SqlPostfixOperator => - lookupFunction(replace(postfix.getName), operands, outputType) + case _: SqlFunction | _: SqlPostfixOperator => + val names = new util.ArrayList[String](rexCall.getOperator.getNameAsId.names) + names.set(names.size() - 1, replace(names.get(names.size() - 1))) + val id = toFunctionIdentifier(names.asScala.toArray, catalogManager) + lookupFunction(id, operands, outputType) case operator@_ => - lookupFunction(replace(s"${operator.getKind}"), operands, outputType) + lookupFunction( + FunctionIdentifier.of(replace(s"${operator.getKind}")), + operands, + outputType) } } } @@ -430,10 +440,10 @@ class RexNodeToExpressionConverter( fieldRef: RexPatternFieldRef): Option[ResolvedExpression] = None private def lookupFunction( - name: String, + identifier: FunctionIdentifier, operands: Seq[ResolvedExpression], outputType: DataType): Option[ResolvedExpression] = { - Try(functionCatalog.lookupFunction(FunctionIdentifier.of(name))) match { + Try(functionCatalog.lookupFunction(identifier)) match { case Success(f: java.util.Optional[FunctionLookup.Result]) => if (f.isPresent) { Some(new CallExpression(f.get().getFunctionDefinition, operands, outputType)) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/SetOpRewriteUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/SetOpRewriteUtil.scala index d0cd08de43ed..6ecdefd37a0c 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/SetOpRewriteUtil.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/SetOpRewriteUtil.scala @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.utils import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.table.functions.FunctionIdentifier import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.calcite.FlinkTypeFactory.toLogicalRowType import org.apache.flink.table.planner.functions.tablefunctions.ReplicateRows @@ -84,7 +85,7 @@ object SetOpRewriteUtil { val function = new TypedFlinkTableFunction(tf, fieldNames, resultType) val typeFactory = builder.getTypeFactory.asInstanceOf[FlinkTypeFactory] val sqlFunction = new TableSqlFunction( - tf.functionIdentifier, + FunctionIdentifier.of(tf.functionIdentifier), tf.toString, tf, resultType, diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala index 6a1aabb1bd50..bb3dadb79dc2 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala @@ -20,8 +20,10 @@ package org.apache.flink.table.planner.catalog import org.apache.flink.table.api.config.ExecutionConfigOptions import org.apache.flink.table.api.internal.TableEnvironmentImpl -import org.apache.flink.table.api.{EnvironmentSettings, TableEnvironment, TableException, ValidationException} +import org.apache.flink.table.api.{EnvironmentSettings, TableEnvironment, ValidationException} +import org.apache.flink.table.catalog.{CatalogFunctionImpl, ObjectPath} import org.apache.flink.table.planner.factories.utils.TestCollectionTableFactory +import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.JavaFunc0 import org.apache.flink.types.Row import org.junit.Assert.assertEquals @@ -70,6 +72,14 @@ class CatalogTableITCase(isStreamingMode: Boolean) { .setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1) TestCollectionTableFactory.reset() TestCollectionTableFactory.isStreaming = isStreamingMode + + val func = new CatalogFunctionImpl( + classOf[JavaFunc0].getName, + new util.HashMap[String, String]()) + tableEnv.getCatalog(tableEnv.getCurrentCatalog).get().createFunction( + new ObjectPath(tableEnv.getCurrentDatabase, "myfunc"), + func, + true) } def toRow(args: Any*):Row = { @@ -84,6 +94,46 @@ class CatalogTableITCase(isStreamingMode: Boolean) { tableEnv.execute(name) } + private def testUdf(funcPrefix: String): Unit = { + val sinkDDL = + """ + |create table sinkT( + | a bigint + |) with ( + | 'connector' = 'COLLECTION' + |) + """.stripMargin + tableEnv.sqlUpdate(sinkDDL) + tableEnv.sqlUpdate(s"insert into sinkT select ${funcPrefix}myfunc(cast(1 as bigint))") + tableEnv.execute("") + assertEquals(Seq(toRow(2L)), TestCollectionTableFactory.RESULT.sorted) + } + + @Test + def testUdfWithFullIdentifier(): Unit = { + testUdf("default_catalog.default_database.") + } + + @Test + def testUdfWithDatabase(): Unit = { + testUdf("default_database.") + } + + @Test + def testUdfWithNon(): Unit = { + testUdf("") + } + + @Test(expected = classOf[ValidationException]) + def testUdfWithWrongCatalog(): Unit = { + testUdf("wrong_catalog.default_database.") + } + + @Test(expected = classOf[ValidationException]) + def testUdfWithWrongDatabase(): Unit = { + testUdf("default_catalog.wrong_database.") + } + @Test def testInsertInto(): Unit = { val sourceData = List( diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala index 9fd90a3475a2..48685f92c608 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala @@ -22,7 +22,7 @@ import org.apache.flink.table.api.{TableConfig, TableException} import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog, GenericInMemoryCatalog} import org.apache.flink.table.expressions._ import org.apache.flink.table.expressions.utils.ApiExpressionUtils.intervalOfMillis -import org.apache.flink.table.functions.UserFunctionsTypeHelper +import org.apache.flink.table.functions.{FunctionIdentifier, UserFunctionsTypeHelper} import org.apache.flink.table.planner.calcite.FlinkRelBuilder.PlannerNamedWindowProperty import org.apache.flink.table.planner.calcite.{FlinkRelBuilder, FlinkTypeFactory} import org.apache.flink.table.planner.delegation.PlannerContext @@ -48,6 +48,7 @@ import org.apache.flink.table.runtime.operators.rank.{ConstantRankRange, RankTyp import org.apache.flink.table.types.AtomicDataType import org.apache.flink.table.types.logical.{BigIntType, DoubleType, IntType, LogicalType, TimestampKind, TimestampType, VarCharType} import org.apache.flink.table.types.utils.TypeConversions + import com.google.common.collect.{ImmutableList, Lists} import org.apache.calcite.jdbc.CalciteSchema import org.apache.calcite.plan._ @@ -59,7 +60,7 @@ import org.apache.calcite.rel.logical.{LogicalAggregate, LogicalProject, Logical import org.apache.calcite.rel.metadata.{JaninoRelMetadataProvider, RelMetadataQuery} import org.apache.calcite.rex._ import org.apache.calcite.schema.SchemaPlus -import org.apache.calcite.sql.SqlWindow +import org.apache.calcite.sql.{SqlIdentifier, SqlWindow} import org.apache.calcite.sql.`type`.{BasicSqlType, SqlTypeName} import org.apache.calcite.sql.`type`.SqlTypeName.{BIGINT, BOOLEAN, DATE, DOUBLE, FLOAT, TIME, TIMESTAMP, VARCHAR} import org.apache.calcite.sql.fun.SqlStdOperatorTable.{AND, CASE, DIVIDE, EQUALS, GREATER_THAN, LESS_THAN, MINUS, MULTIPLY, OR, PLUS} @@ -67,9 +68,9 @@ import org.apache.calcite.sql.fun.{SqlCountAggFunction, SqlStdOperatorTable} import org.apache.calcite.sql.parser.SqlParserPos import org.apache.calcite.util.{DateString, ImmutableBitSet, ImmutableIntList, TimeString, TimestampString} import org.junit.{Before, BeforeClass} + import java.math.BigDecimal import java.util - import org.apache.flink.table.module.ModuleManager import scala.collection.JavaConversions._ @@ -724,7 +725,14 @@ class FlinkRelMdHandlerTestBase { val relDataType = builder.build() AggregateCall.create( - AggSqlFunction("top3", "top3", new Top3, resultDataType, accDataType, typeFactory, false), + AggSqlFunction( + FunctionIdentifier.of("top3"), + "top3", + new Top3, + resultDataType, + accDataType, + typeFactory, + false), false, false, false, diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/PartitionPrunerTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/PartitionPrunerTest.scala index 6a53ab0c863c..13ee3fe66f52 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/PartitionPrunerTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/PartitionPrunerTest.scala @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.utils import org.apache.flink.table.api.{DataTypes, TableConfig} +import org.apache.flink.table.functions.FunctionIdentifier import org.apache.flink.table.planner.expressions.utils.Func1 import org.apache.flink.table.planner.functions.utils.ScalarSqlFunction @@ -78,7 +79,12 @@ class PartitionPrunerTest extends RexNodeTestBase { // amount val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 1) // MyUdf(amount) - val t1 = rexBuilder.makeCall(new ScalarSqlFunction("MyUdf", "MyUdf", Func1, typeFactory), t0) + val t1 = rexBuilder.makeCall(new ScalarSqlFunction( + FunctionIdentifier.of("MyUdf"), + "MyUdf", + Func1, + typeFactory), + t0) // 100 val t2 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) // MyUdf(amount) > 100 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 efe2f6e0998d..509f2b2373f2 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 @@ -23,7 +23,7 @@ import org.apache.flink.table.api.DataTypes import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog, GenericInMemoryCatalog} import org.apache.flink.table.expressions.utils.ApiExpressionUtils.{unresolvedCall, unresolvedRef, valueLiteral} import org.apache.flink.table.expressions.{Expression, ExpressionParser} -import org.apache.flink.table.functions.AggregateFunctionDefinition +import org.apache.flink.table.functions.{AggregateFunctionDefinition, FunctionIdentifier} import org.apache.flink.table.functions.BuiltInFunctionDefinitions.{EQUALS, GREATER_THAN, LESS_THAN, LESS_THAN_OR_EQUAL} import org.apache.flink.table.planner.expressions.utils.Func1 import org.apache.flink.table.planner.expressions.{EqualTo, ExpressionBridge, GreaterThan, Literal, PlannerExpression, PlannerExpressionConverter, Sum, UnresolvedFieldReference} @@ -31,9 +31,10 @@ import org.apache.flink.table.planner.functions.sql.FlinkSqlOperatorTable import org.apache.flink.table.planner.functions.utils.ScalarSqlFunction import org.apache.flink.table.planner.plan.utils.InputTypeBuilder.inputOf import org.apache.flink.table.planner.utils.{DateTimeTestUtil, IntSumAggFunction} + import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rex.{RexBuilder, RexNode} -import org.apache.calcite.sql.SqlPostfixOperator +import org.apache.calcite.sql.{SqlIdentifier, SqlPostfixOperator} import org.apache.calcite.sql.`type`.SqlTypeName import org.apache.calcite.sql.`type`.SqlTypeName.{BIGINT, INTEGER, VARCHAR} import org.apache.calcite.sql.fun.{SqlStdOperatorTable, SqlTrimFunction} @@ -41,12 +42,14 @@ import org.apache.calcite.util.{DateString, TimeString, TimestampString} import org.hamcrest.CoreMatchers.is import org.junit.Assert.{assertArrayEquals, assertEquals, assertThat, assertTrue} import org.junit.Test + import java.math.BigDecimal import java.sql.Timestamp import java.util.{TimeZone, List => JList} - import org.apache.flink.table.module.ModuleManager +import org.apache.calcite.sql.parser.SqlParserPos + import scala.collection.JavaConverters._ /** @@ -700,7 +703,8 @@ class RexNodeExtractorTest extends RexNodeTestBase { // amount val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) // my_udf(amount) - val t1 = rexBuilder.makeCall(new ScalarSqlFunction("myUdf", "myUdf", Func1, typeFactory), t0) + val t1 = rexBuilder.makeCall(new ScalarSqlFunction( + FunctionIdentifier.of("MyUdf"), "myUdf", Func1, typeFactory), t0) // 100 val t2 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) // my_udf(amount) > 100 @@ -899,7 +903,7 @@ class RexNodeExtractorTest extends RexNodeTestBase { rexBuilder: RexBuilder, catalog: FunctionCatalog): (Array[Expression], Array[RexNode]) = { RexNodeExtractor.extractConjunctiveConditions(expr, maxCnfNodeCount, - inputFieldNames, rexBuilder, catalog, TimeZone.getDefault) + inputFieldNames, rexBuilder, catalog, catalogManager, TimeZone.getDefault) } } From 8bd4b5cda4c4f9e1b00f9ff5fea4ae319d120ff4 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Wed, 6 Nov 2019 20:42:05 +0800 Subject: [PATCH 509/746] [FLINK-14262][table] FunctionIdentifier shouldn't normalize function name and use un-escaped identifier as toString The display name of functions shouldn't be normalized and escaped, otherwise, a lot of planner tests will fail. However, FunctionIdentifier enforce to normalize names when construction. This makes it impossible to get the original name. The normalization can happen out of FunctionIdentifier. --- .../pyflink/table/tests/test_aggregate.py | 3 +- flink-python/pyflink/table/tests/test_calc.py | 14 ++--- .../table/tests/test_column_operation.py | 8 +-- .../pyflink/table/tests/test_correlate.py | 2 +- flink-python/pyflink/table/tests/test_join.py | 8 +-- flink-python/pyflink/table/tests/test_sort.py | 2 +- .../flink/table/catalog/FunctionCatalog.java | 6 +- .../expressions/UnresolvedCallExpression.java | 2 +- .../table/expressions/CallExpression.java | 2 +- .../table/functions/FunctionIdentifier.java | 58 ++++++++++++------- .../catalog/FunctionCatalogOperatorTable.java | 9 +-- .../functions/utils/FunctionUtils.java | 15 ----- .../functions/utils/HiveTableSqlFunction.java | 8 +-- .../functions/utils/AggSqlFunction.scala | 17 +++--- .../functions/utils/ScalarSqlFunction.scala | 25 ++++---- .../functions/utils/TableSqlFunction.scala | 25 ++++---- .../rules/logical/LogicalUnnestRule.scala | 2 - 17 files changed, 99 insertions(+), 107 deletions(-) diff --git a/flink-python/pyflink/table/tests/test_aggregate.py b/flink-python/pyflink/table/tests/test_aggregate.py index d7700c790c70..f0e70b91e7f7 100644 --- a/flink-python/pyflink/table/tests/test_aggregate.py +++ b/flink-python/pyflink/table/tests/test_aggregate.py @@ -26,8 +26,7 @@ def test_group_by(self): result = t.group_by("c").select("a.sum, c as b") query_operation = result._j_table.getQueryOperation().getChildren().get(0) self.assertEqual("[c]", query_operation.getGroupingExpressions().toString()) - self.assertEqual("[`as`(" - "`sum`(a), 'EXPR$0')]", + self.assertEqual("[as(sum(a), 'EXPR$0')]", query_operation.getAggregateExpressions().toString()) diff --git a/flink-python/pyflink/table/tests/test_calc.py b/flink-python/pyflink/table/tests/test_calc.py index 2b788831e524..1580cc73ed61 100644 --- a/flink-python/pyflink/table/tests/test_calc.py +++ b/flink-python/pyflink/table/tests/test_calc.py @@ -33,7 +33,7 @@ def test_select(self): t = self.t_env.from_elements([(1, 'hi', 'hello')], ['a', 'b', 'c']) result = t.select("a + 1, b, c") query_operation = result._j_table.getQueryOperation() - self.assertEqual('[`plus`(a, 1), b, c]', + self.assertEqual('[plus(a, 1), b, c]', query_operation.getProjectList().toString()) def test_alias(self): @@ -47,18 +47,18 @@ def test_where(self): t = t_env.from_elements([(1, 'Hi', 'Hello')], ['a', 'b', 'c']) result = t.where("a > 1 && b = 'Hello'") query_operation = result._j_table.getQueryOperation() - self.assertEqual("`and`(" - "`greaterthan`(a, 1), " - "`equals`(b, 'Hello'))", + self.assertEqual("and(" + "greaterThan(a, 1), " + "equals(b, 'Hello'))", query_operation.getCondition().toString()) def test_filter(self): t = self.t_env.from_elements([(1, 'Hi', 'Hello')], ['a', 'b', 'c']) result = t.filter("a > 1 && b = 'Hello'") query_operation = result._j_table.getQueryOperation() - self.assertEqual("`and`(" - "`greaterthan`(a, 1), " - "`equals`(b, 'Hello'))", + self.assertEqual("and(" + "greaterThan(a, 1), " + "equals(b, 'Hello'))", query_operation.getCondition().toString()) def test_from_element(self): diff --git a/flink-python/pyflink/table/tests/test_column_operation.py b/flink-python/pyflink/table/tests/test_column_operation.py index d88cf8c7be6f..023d54617ddc 100644 --- a/flink-python/pyflink/table/tests/test_column_operation.py +++ b/flink-python/pyflink/table/tests/test_column_operation.py @@ -25,16 +25,16 @@ def test_add_columns(self): t = self.t_env.from_elements([(1, 'Hi', 'Hello')], ['a', 'b', 'c']) result = t.select("a").add_columns("a + 1 as b, a + 2 as c") query_operation = result._j_table.getQueryOperation() - self.assertEqual('[a, `plus`(a, 1), ' - '`plus`(a, 2)]', + self.assertEqual('[a, plus(a, 1), ' + 'plus(a, 2)]', query_operation.getProjectList().toString()) def test_add_or_replace_columns(self): t = self.t_env.from_elements([(1, 'Hi', 'Hello')], ['a', 'b', 'c']) result = t.select("a").add_or_replace_columns("a + 1 as b, a + 2 as a") query_operation = result._j_table.getQueryOperation() - self.assertEqual('[`plus`(a, 2), ' - '`plus`(a, 1)]', + self.assertEqual('[plus(a, 2), ' + 'plus(a, 1)]', query_operation.getProjectList().toString()) def test_rename_columns(self): diff --git a/flink-python/pyflink/table/tests/test_correlate.py b/flink-python/pyflink/table/tests/test_correlate.py index 3a9b57f32a52..ac34b1994b8c 100644 --- a/flink-python/pyflink/table/tests/test_correlate.py +++ b/flink-python/pyflink/table/tests/test_correlate.py @@ -42,7 +42,7 @@ def test_join_lateral_with_join_predicate(self): query_operation = result._j_table.getQueryOperation() self.assertEqual('INNER', query_operation.getJoinType().toString()) self.assertTrue(query_operation.isCorrelated()) - self.assertEqual('`equals`(id, word)', + self.assertEqual('equals(id, word)', query_operation.getCondition().toString()) def test_left_outer_join_lateral(self): diff --git a/flink-python/pyflink/table/tests/test_join.py b/flink-python/pyflink/table/tests/test_join.py index d1fc2cc75896..f05c9e7cb93a 100644 --- a/flink-python/pyflink/table/tests/test_join.py +++ b/flink-python/pyflink/table/tests/test_join.py @@ -29,7 +29,7 @@ def test_join_without_where(self): query_operation = result._j_table.getQueryOperation() self.assertEqual('INNER', query_operation.getJoinType().toString()) - self.assertEqual('`equals`(a, d)', + self.assertEqual('equals(a, d)', query_operation.getCondition().toString()) self.assertFalse(query_operation.isCorrelated()) @@ -52,7 +52,7 @@ def test_left_outer_join_without_where(self): query_operation = result._j_table.getQueryOperation() self.assertEqual('LEFT_OUTER', query_operation.getJoinType().toString()) - self.assertEqual('`equals`(a, d)', + self.assertEqual('equals(a, d)', query_operation.getCondition().toString()) self.assertFalse(query_operation.isCorrelated()) @@ -75,7 +75,7 @@ def test_right_outer_join(self): query_operation = result._j_table.getQueryOperation() self.assertEqual('RIGHT_OUTER', query_operation.getJoinType().toString()) - self.assertEqual('`equals`(a, d)', + self.assertEqual('equals(a, d)', query_operation.getCondition().toString()) self.assertFalse(query_operation.isCorrelated()) @@ -87,7 +87,7 @@ def test_full_outer_join(self): result = t1.full_outer_join(t2, "a = d") query_operation = result._j_table.getQueryOperation() self.assertEqual('FULL_OUTER', query_operation.getJoinType().toString()) - self.assertEqual('`equals`(a, d)', + self.assertEqual('equals(a, d)', query_operation.getCondition().toString()) self.assertFalse(query_operation.isCorrelated()) diff --git a/flink-python/pyflink/table/tests/test_sort.py b/flink-python/pyflink/table/tests/test_sort.py index f41f43e558f2..e4ef07880c06 100644 --- a/flink-python/pyflink/table/tests/test_sort.py +++ b/flink-python/pyflink/table/tests/test_sort.py @@ -28,7 +28,7 @@ def test_order_by_offset_fetch(self): query_operation = result._j_table.getQueryOperation() self.assertEqual(2, query_operation.getOffset()) self.assertEqual(2, query_operation.getFetch()) - self.assertEqual('[`desc`(a)]', + self.assertEqual('[desc(a)]', query_operation.getOrder().toString()) 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 90c0bcb3de8d..571438a24777 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 @@ -284,10 +284,12 @@ public Optional lookupFunction(FunctionIdentifier identif // precise function reference if (identifier.getIdentifier().isPresent()) { - return resolvePreciseFunctionReference(identifier.getIdentifier().get()); + ObjectIdentifier oi = FunctionIdentifier.normalizeObjectIdentifier(identifier.getIdentifier().get()); + return resolvePreciseFunctionReference(oi); } else { // ambiguous function reference - return resolveAmbiguousFunctionReference(identifier.getSimpleName().get()); + String funcName = FunctionIdentifier.normalizeName(identifier.getSimpleName().get()); + return resolveAmbiguousFunctionReference(funcName); } } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/UnresolvedCallExpression.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/UnresolvedCallExpression.java index b42c88d78fd5..1fe55a603201 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/UnresolvedCallExpression.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/UnresolvedCallExpression.java @@ -115,7 +115,7 @@ public String asSummaryString() { if (functionIdentifier == null) { functionName = functionDefinition.toString(); } else { - functionName = functionIdentifier.asSerializableString(); + functionName = functionIdentifier.asSummaryString(); } final String argList = args.stream() diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/CallExpression.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/CallExpression.java index aedaa2bba0b9..e3c5cb9104e0 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/CallExpression.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/expressions/CallExpression.java @@ -102,7 +102,7 @@ public String asSummaryString() { if (functionIdentifier == null) { functionName = functionDefinition.toString(); } else { - functionName = functionIdentifier.asSerializableString(); + functionName = functionIdentifier.asSummaryString(); } final String argList = args.stream() diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/FunctionIdentifier.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/FunctionIdentifier.java index dba2454ef9b3..eb244738724f 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/FunctionIdentifier.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/FunctionIdentifier.java @@ -22,11 +22,15 @@ import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.util.StringUtils; +import javax.annotation.Nullable; + import java.io.Serializable; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; import java.util.Objects; import java.util.Optional; -import static org.apache.flink.table.utils.EncodingUtils.escapeIdentifier; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -37,9 +41,11 @@ @PublicEvolving public final class FunctionIdentifier implements Serializable { - private final ObjectIdentifier objectIdentifier; + private static final long serialVersionUID = 1L; + + private final @Nullable ObjectIdentifier objectIdentifier; - private final String functionName; + private final @Nullable String functionName; public static FunctionIdentifier of(ObjectIdentifier oi){ return new FunctionIdentifier(oi); @@ -51,14 +57,14 @@ public static FunctionIdentifier of(String functionName){ private FunctionIdentifier(ObjectIdentifier objectIdentifier){ checkNotNull(objectIdentifier, "Object identifier cannot be null"); - this.objectIdentifier = normalizeObjectIdentifier(objectIdentifier); + this.objectIdentifier = objectIdentifier; this.functionName = null; } private FunctionIdentifier(String functionName){ checkArgument(!StringUtils.isNullOrWhitespaceOnly(functionName), "function name cannot be null or empty string"); - this.functionName = normalizeName(functionName); + this.functionName = functionName; this.objectIdentifier = null; } @@ -88,18 +94,33 @@ public Optional getSimpleName(){ } /** - * Returns a string that fully serializes this instance. The serialized string can be used for - * transmitting or persisting an object identifier. + * List of the component names of this function identifier. + */ + public List getNames() { + if (objectIdentifier != null) { + return Arrays.asList( + objectIdentifier.getCatalogName(), + objectIdentifier.getDatabaseName(), + objectIdentifier.getObjectName()); + } else if (functionName != null) { + return Collections.singletonList(functionName); + } else { + throw new IllegalStateException( + "functionName and objectIdentifier are both null which should never happen."); + } + } + + /** + * Returns a string that summarizes this instance for printing to a console or log. */ - public String asSerializableString() { + public String asSummaryString() { if (objectIdentifier != null) { - return String.format( - "%s.%s.%s", - escapeIdentifier(objectIdentifier.getCatalogName()), - escapeIdentifier(objectIdentifier.getDatabaseName()), - escapeIdentifier(objectIdentifier.getObjectName())); + return String.join(".", + objectIdentifier.getCatalogName(), + objectIdentifier.getDatabaseName(), + objectIdentifier.getObjectName()); } else { - return String.format("%s", escapeIdentifier(functionName)); + return functionName; } } @@ -113,11 +134,8 @@ public boolean equals(Object o) { } FunctionIdentifier that = (FunctionIdentifier) o; - if (getIdentifier() != null && getIdentifier().equals(that.getIdentifier())) { - return true; - } else { - return functionName.equals(that.functionName); - } + return Objects.equals(objectIdentifier, that.objectIdentifier) && + Objects.equals(functionName, that.functionName); } @Override @@ -127,6 +145,6 @@ public int hashCode() { @Override public String toString() { - return asSerializableString(); + return asSummaryString(); } } diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/FunctionCatalogOperatorTable.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/FunctionCatalogOperatorTable.java index cf17ff892733..d4637e9dc7b7 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/FunctionCatalogOperatorTable.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/FunctionCatalogOperatorTable.java @@ -23,17 +23,12 @@ import org.apache.flink.table.catalog.CatalogManager; import org.apache.flink.table.catalog.FunctionCatalog; import org.apache.flink.table.catalog.FunctionLookup; -import org.apache.flink.table.catalog.ObjectIdentifier; -import org.apache.flink.table.catalog.UnresolvedIdentifier; -import org.apache.flink.table.expressions.CallExpression; import org.apache.flink.table.functions.AggregateFunctionDefinition; import org.apache.flink.table.functions.FunctionDefinition; import org.apache.flink.table.functions.FunctionIdentifier; import org.apache.flink.table.functions.ScalarFunctionDefinition; import org.apache.flink.table.functions.TableFunctionDefinition; -import org.apache.flink.table.functions.UserDefinedFunction; import org.apache.flink.table.planner.calcite.FlinkTypeFactory; -import org.apache.flink.table.planner.functions.utils.FunctionUtils; import org.apache.flink.table.planner.functions.utils.HiveAggSqlFunction; import org.apache.flink.table.planner.functions.utils.HiveScalarSqlFunction; import org.apache.flink.table.planner.functions.utils.HiveTableSqlFunction; @@ -49,10 +44,8 @@ import org.apache.calcite.sql.SqlOperator; import org.apache.calcite.sql.SqlOperatorTable; import org.apache.calcite.sql.SqlSyntax; -import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.calcite.sql.validate.SqlNameMatcher; -import java.util.Arrays; import java.util.List; import java.util.Optional; @@ -143,7 +136,7 @@ private Optional convertToSqlFunction( returnType, typeFactory, new DeferredTypeFlinkTableFunction(def.getTableFunction(), returnType), - HiveTableSqlFunction.operandTypeChecker(identifier, def.getTableFunction()))); + HiveTableSqlFunction.operandTypeChecker(identifier.toString(), def.getTableFunction()))); } else { return convertTableFunction(identifier, (TableFunctionDefinition) functionDefinition); } diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/FunctionUtils.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/FunctionUtils.java index 01abf689d7a5..c8265e444fd8 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/FunctionUtils.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/FunctionUtils.java @@ -19,18 +19,11 @@ package org.apache.flink.table.planner.functions.utils; import org.apache.flink.table.catalog.CatalogManager; -import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.UnresolvedIdentifier; import org.apache.flink.table.expressions.CallExpression; import org.apache.flink.table.functions.FunctionIdentifier; import org.apache.flink.table.functions.UserDefinedFunction; -import org.apache.calcite.sql.SqlIdentifier; -import org.apache.calcite.sql.parser.SqlParserPos; - -import java.util.Arrays; -import java.util.Optional; - /** * Utils for sql functions. */ @@ -47,12 +40,4 @@ public static FunctionIdentifier toFunctionIdentifier(CallExpression call, UserD return call.getFunctionIdentifier() .orElse(FunctionIdentifier.of(function.functionIdentifier())); } - - public static SqlIdentifier toSqlIdentifier(FunctionIdentifier fi) { - Optional objectIdentifier = fi.getIdentifier(); - String[] names = objectIdentifier - .map(id -> new String[] {id.getCatalogName(), id.getDatabaseName(), id.getObjectName()}) - .orElseGet(() -> new String[]{fi.getSimpleName().get()}); - return new SqlIdentifier(Arrays.asList(names), SqlParserPos.ZERO); - } } diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/HiveTableSqlFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/HiveTableSqlFunction.java index e2133c5c4f96..f29cbaacb12e 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/HiveTableSqlFunction.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/HiveTableSqlFunction.java @@ -212,9 +212,9 @@ private static class NonLiteralException extends Exception { } public static HiveOperandTypeChecker operandTypeChecker( - FunctionIdentifier identifier, TableFunction udtf) { + String name, TableFunction udtf) { return new HiveOperandTypeChecker( - identifier, udtf, UserDefinedFunctionUtils.checkAndExtractMethods(udtf, "eval")); + name, udtf, UserDefinedFunctionUtils.checkAndExtractMethods(udtf, "eval")); } /** @@ -228,8 +228,8 @@ public static class HiveOperandTypeChecker extends OperandTypeChecker { private LogicalType[] previousArgTypes; private HiveOperandTypeChecker( - FunctionIdentifier identifier, TableFunction udtf, Method[] methods) { - super(identifier, udtf, methods); + String name, TableFunction udtf, Method[] methods) { + super(name, udtf, methods); } @Override 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 94516884c933..8727eb4692fc 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 @@ -22,16 +22,15 @@ import org.apache.flink.table.api.ValidationException import org.apache.flink.table.functions.{AggregateFunction, FunctionIdentifier, TableAggregateFunction, UserDefinedAggregateFunction} import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.functions.utils.AggSqlFunction.{createOperandTypeChecker, createOperandTypeInference, createReturnTypeInference} -import org.apache.flink.table.planner.functions.utils.FunctionUtils.toSqlIdentifier import org.apache.flink.table.planner.functions.utils.UserDefinedFunctionUtils._ import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType import org.apache.flink.table.types.DataType import org.apache.flink.table.types.logical.LogicalType - import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.sql._ import org.apache.calcite.sql.`type`.SqlOperandTypeChecker.Consistency import org.apache.calcite.sql.`type`._ +import org.apache.calcite.sql.parser.SqlParserPos import org.apache.calcite.sql.validate.SqlUserDefinedAggFunction import org.apache.calcite.util.Optionality @@ -58,11 +57,11 @@ class AggSqlFunction( requiresOver: Boolean, returnTypeInfer: Option[SqlReturnTypeInference] = None) extends SqlUserDefinedAggFunction( - toSqlIdentifier(identifier), + new SqlIdentifier(identifier.getNames, SqlParserPos.ZERO), returnTypeInfer.getOrElse(createReturnTypeInference( fromDataTypeToLogicalType(externalResultType), typeFactory)), - createOperandTypeInference(identifier, aggregateFunction, typeFactory, externalAccType), - createOperandTypeChecker(identifier, aggregateFunction, externalAccType), + createOperandTypeInference(displayName, aggregateFunction, typeFactory, externalAccType), + createOperandTypeChecker(displayName, 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, @@ -109,7 +108,7 @@ object AggSqlFunction { } private[flink] def createOperandTypeInference( - identifier: FunctionIdentifier, + name: String, aggregateFunction: UserDefinedAggregateFunction[_, _], typeFactory: FlinkTypeFactory, externalAccType: DataType): SqlOperandTypeInference = { @@ -128,7 +127,7 @@ object AggSqlFunction { val foundSignature = getAccumulateMethodSignature(aggregateFunction, operandLogicalType) .getOrElse( throw new ValidationException( - s"Given parameters of function '$identifier' do not match any signature. \n" + + s"Given parameters of function '$name' do not match any signature. \n" + s"Actual: ${signatureInternalToString(actualSignature)} \n" + s"Expected: ${signaturesToString(aggregateFunction, "accumulate")}")) @@ -161,7 +160,7 @@ object AggSqlFunction { } private[flink] def createOperandTypeChecker( - identifier: FunctionIdentifier, + name: String, aggregateFunction: UserDefinedAggregateFunction[_, _], externalAccType: DataType): SqlOperandTypeChecker = { @@ -210,7 +209,7 @@ object AggSqlFunction { if (foundSignature.isEmpty) { if (throwOnFailure) { throw new ValidationException( - s"Given parameters of function '$identifier' do not match any signature. \n" + + s"Given parameters of function '$name' do not match any signature. \n" + s"Actual: ${signatureInternalToString(actualSignature)} \n" + s"Expected: ${signaturesToString(aggregateFunction, "accumulate")}") } else { 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 1840d4199542..0acf3572c438 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 @@ -21,18 +21,17 @@ package org.apache.flink.table.planner.functions.utils import org.apache.flink.table.api.ValidationException import org.apache.flink.table.functions.{FunctionIdentifier, ScalarFunction} import org.apache.flink.table.planner.calcite.FlinkTypeFactory -import org.apache.flink.table.planner.functions.utils.FunctionUtils.toSqlIdentifier import org.apache.flink.table.planner.functions.utils.ScalarSqlFunction._ import org.apache.flink.table.planner.functions.utils.UserDefinedFunctionUtils.{getOperandType, _} import org.apache.flink.table.runtime.types.ClassLogicalTypeConverter.getDefaultExternalClassForType import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType import org.apache.flink.table.types.logical.LogicalType - import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.sql._ import org.apache.calcite.sql.`type`.SqlOperandTypeChecker.Consistency import org.apache.calcite.sql.`type`._ +import org.apache.calcite.sql.parser.SqlParserPos import scala.collection.JavaConverters._ @@ -51,10 +50,10 @@ class ScalarSqlFunction( typeFactory: FlinkTypeFactory, returnTypeInfer: Option[SqlReturnTypeInference] = None) extends SqlFunction( - toSqlIdentifier(identifier), - returnTypeInfer.getOrElse(createReturnTypeInference(identifier, scalarFunction, typeFactory)), - createOperandTypeInference(identifier, scalarFunction, typeFactory), - createOperandTypeChecker(identifier, scalarFunction), + new SqlIdentifier(identifier.getNames, SqlParserPos.ZERO), + returnTypeInfer.getOrElse(createReturnTypeInference(displayName, scalarFunction, typeFactory)), + createOperandTypeInference(displayName, scalarFunction, typeFactory), + createOperandTypeChecker(displayName, scalarFunction), null, SqlFunctionCategory.USER_DEFINED_FUNCTION) { @@ -73,7 +72,7 @@ class ScalarSqlFunction( object ScalarSqlFunction { private[flink] def createReturnTypeInference( - identifier: FunctionIdentifier, + name: String, scalarFunction: ScalarFunction, typeFactory: FlinkTypeFactory): SqlReturnTypeInference = { /** @@ -102,7 +101,7 @@ object ScalarSqlFunction { } private[flink] def createOperandTypeInference( - identifier: FunctionIdentifier, + name: String, scalarFunction: ScalarFunction, typeFactory: FlinkTypeFactory): SqlOperandTypeInference = { /** @@ -114,13 +113,13 @@ object ScalarSqlFunction { returnType: RelDataType, operandTypes: Array[RelDataType]): Unit = { ScalarSqlFunction.inferOperandTypes( - identifier, scalarFunction, typeFactory, callBinding, returnType, operandTypes) + name, scalarFunction, typeFactory, callBinding, returnType, operandTypes) } } } def inferOperandTypes( - identifier: FunctionIdentifier, + name: String, func: ScalarFunction, typeFactory: FlinkTypeFactory, callBinding: SqlCallBinding, @@ -128,7 +127,7 @@ object ScalarSqlFunction { operandTypes: Array[RelDataType]): Unit = { val parameters = getOperandType(callBinding).toArray if (getEvalUserDefinedMethod(func, parameters).isEmpty) { - throwValidationException(identifier.toString, func, parameters) + throwValidationException(name, func, parameters) } func.getParameterTypes(getEvalMethodSignature(func, parameters)) .map(fromTypeInfoToLogicalType) @@ -140,7 +139,7 @@ object ScalarSqlFunction { } private[flink] def createOperandTypeChecker( - identifier: FunctionIdentifier, + name: String, scalarFunction: ScalarFunction): SqlOperandTypeChecker = { val methods = checkAndExtractMethods(scalarFunction, "eval") @@ -186,7 +185,7 @@ object ScalarSqlFunction { if (foundMethod.isEmpty) { if (throwOnFailure) { throw new ValidationException( - s"Given parameters of function '$identifier' do not match any signature. \n" + + s"Given parameters of function '$name' do not match any signature. \n" + s"Actual: ${signatureInternalToString(operandTypeInfo)} \n" + s"Expected: ${signaturesToString(scalarFunction, "eval")}") } else { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/TableSqlFunction.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/TableSqlFunction.scala index 1a6020f7b5e4..16d0d9cbec81 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/TableSqlFunction.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/TableSqlFunction.scala @@ -21,18 +21,17 @@ package org.apache.flink.table.planner.functions.utils import org.apache.flink.table.api.ValidationException import org.apache.flink.table.functions.{FunctionIdentifier, TableFunction} import org.apache.flink.table.planner.calcite.FlinkTypeFactory -import org.apache.flink.table.planner.functions.utils.FunctionUtils.toSqlIdentifier import org.apache.flink.table.planner.functions.utils.TableSqlFunction._ import org.apache.flink.table.planner.functions.utils.UserDefinedFunctionUtils._ import org.apache.flink.table.planner.plan.schema.FlinkTableFunction import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType import org.apache.flink.table.types.DataType import org.apache.flink.table.types.logical.LogicalType - import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory} import org.apache.calcite.sql._ import org.apache.calcite.sql.`type`.SqlOperandTypeChecker.Consistency import org.apache.calcite.sql.`type`._ +import org.apache.calcite.sql.parser.SqlParserPos import org.apache.calcite.sql.validate.{SqlUserDefinedTableFunction, SqlUserDefinedTableMacro} import java.lang.reflect.Method @@ -57,12 +56,12 @@ class TableSqlFunction( functionImpl: FlinkTableFunction, operandTypeInfer: Option[SqlOperandTypeChecker] = None) extends SqlUserDefinedTableFunction( - toSqlIdentifier(identifier), + new SqlIdentifier(identifier.getNames, SqlParserPos.ZERO), ReturnTypes.CURSOR, // type inference has the UNKNOWN operand types. - createOperandTypeInference(identifier, udtf, typeFactory), + createOperandTypeInference(displayName, udtf, typeFactory), // only checker has the real operand types. - operandTypeInfer.getOrElse(createOperandTypeChecker(identifier, udtf)), + operandTypeInfer.getOrElse(createOperandTypeChecker(displayName, udtf)), null, functionImpl) { @@ -98,7 +97,7 @@ class TableSqlFunction( object TableSqlFunction { private[flink] def createOperandTypeInference( - identifier: FunctionIdentifier, + name: String, udtf: TableFunction[_], typeFactory: FlinkTypeFactory): SqlOperandTypeInference = { /** @@ -110,13 +109,13 @@ object TableSqlFunction { returnType: RelDataType, operandTypes: Array[RelDataType]): Unit = { inferOperandTypesInternal( - identifier, udtf, typeFactory, callBinding, returnType, operandTypes) + name, udtf, typeFactory, callBinding, returnType, operandTypes) } } } def inferOperandTypesInternal( - identifier: FunctionIdentifier, + name: String, func: TableFunction[_], typeFactory: FlinkTypeFactory, callBinding: SqlCallBinding, @@ -124,7 +123,7 @@ object TableSqlFunction { operandTypes: Array[RelDataType]): Unit = { val parameters = getOperandType(callBinding).toArray if (getEvalUserDefinedMethod(func, parameters).isEmpty) { - throwValidationException(identifier.toString, func, parameters) + throwValidationException(name, func, parameters) } func.getParameterTypes(getEvalMethodSignature(func, parameters)) .map(fromTypeInfoToLogicalType) @@ -136,9 +135,9 @@ object TableSqlFunction { } private[flink] def createOperandTypeChecker( - identifier: FunctionIdentifier, + name: String, udtf: TableFunction[_]): SqlOperandTypeChecker = { - new OperandTypeChecker(identifier, udtf, checkAndExtractMethods(udtf, "eval")) + new OperandTypeChecker(name, udtf, checkAndExtractMethods(udtf, "eval")) } } @@ -146,7 +145,7 @@ object TableSqlFunction { * Operand type checker based on [[TableFunction]] given information. */ class OperandTypeChecker( - identifier: FunctionIdentifier, + name: String, udtf: TableFunction[_], methods: Array[Method]) extends SqlOperandTypeChecker { @@ -183,7 +182,7 @@ class OperandTypeChecker( if (getEvalUserDefinedMethod(udtf, operandTypes).isEmpty) { if (throwOnFailure) { throw new ValidationException( - s"Given parameters of function '$identifier' do not match any signature. \n" + + s"Given parameters of function '$name' do not match any signature. \n" + s"Actual: ${signatureInternalToString(operandTypes)} \n" + s"Expected: ${signaturesToString(udtf, "eval")}") } else { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.scala index c99cdccf4d56..0a88f8ea379d 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.scala @@ -37,9 +37,7 @@ import org.apache.calcite.rel.RelNode import org.apache.calcite.rel.`type`.{RelDataTypeFieldImpl, RelRecordType, StructKind} import org.apache.calcite.rel.core.Uncollect import org.apache.calcite.rel.logical._ -import org.apache.calcite.sql.SqlIdentifier import org.apache.calcite.sql.`type`.{AbstractSqlType, ArraySqlType, MapSqlType, MultisetSqlType} -import org.apache.calcite.sql.parser.SqlParserPos import java.util.Collections From b0a9afdd24fb70131b1e80d46d0ca101235a4a36 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Thu, 7 Nov 2019 22:50:49 +0800 Subject: [PATCH 510/746] [hotfix][table] Avoid passing CatalogManager to FunctionCatalogOperatorTable This also changes the parameter of FunctionLookup#lookupFunction from FunctionIdentifier UnresolvedIdentifier. The qualifying happens in FunctionCatalog which holds CatalogManager. --- .../flink/table/catalog/FunctionCatalog.java | 43 +++++++++---------- .../flink/table/catalog/FunctionLookup.java | 4 +- .../resolver/LookupCallResolver.java | 4 +- .../table/catalog/FunctionCatalogTest.java | 26 ++++++----- .../catalog/FunctionCatalogOperatorTable.java | 10 ++--- .../planner/delegation/PlannerContext.java | 1 - .../expressions/SqlAggFunctionVisitor.java | 10 +++-- .../converter/ScalarFunctionConvertRule.java | 6 ++- .../functions/utils/FunctionUtils.java | 43 ------------------- .../planner/plan/utils/RexNodeExtractor.scala | 11 ++--- .../catalog/FunctionCatalogOperatorTable.java | 3 +- .../table/plan/util/RexProgramExtractor.scala | 6 +-- .../table/api/stream/sql/AggregateTest.scala | 8 ++-- 13 files changed, 66 insertions(+), 109 deletions(-) delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/FunctionUtils.java 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 571438a24777..9377ca3d3e69 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 @@ -252,10 +252,9 @@ public String[] getFunctions() { } private Set getUserDefinedFunctionNames() { - Set result = new HashSet<>(); // add temp system functions - result.addAll(tempSystemFunctions.keySet()); + Set result = new HashSet<>(tempSystemFunctions.keySet()); String currentCatalog = catalogManager.getCurrentCatalog(); String currentDatabase = catalogManager.getCurrentDatabase(); @@ -264,7 +263,7 @@ private Set getUserDefinedFunctionNames() { result.addAll(tempCatalogFunctions.keySet().stream() .filter(oi -> oi.getCatalogName().equals(currentCatalog) && oi.getDatabaseName().equals(currentDatabase)) - .map(oi -> oi.getObjectName()) + .map(ObjectIdentifier::getObjectName) .collect(Collectors.toSet()) ); @@ -280,16 +279,13 @@ private Set getUserDefinedFunctionNames() { } @Override - public Optional lookupFunction(FunctionIdentifier identifier) { - + public Optional lookupFunction(UnresolvedIdentifier identifier) { // precise function reference - if (identifier.getIdentifier().isPresent()) { - ObjectIdentifier oi = FunctionIdentifier.normalizeObjectIdentifier(identifier.getIdentifier().get()); - return resolvePreciseFunctionReference(oi); + if (identifier.getDatabaseName().isPresent()) { + return resolvePreciseFunctionReference(catalogManager.qualifyIdentifier(identifier)); } else { // ambiguous function reference - String funcName = FunctionIdentifier.normalizeName(identifier.getSimpleName().get()); - return resolveAmbiguousFunctionReference(funcName); + return resolveAmbiguousFunctionReference(identifier.getObjectName()); } } @@ -297,8 +293,8 @@ private Optional resolvePreciseFunctionReference(ObjectId // resolve order: // 1. Temporary functions // 2. Catalog functions - - FunctionDefinition potentialResult = tempCatalogFunctions.get(oi); + ObjectIdentifier normalizedIdentifier = FunctionIdentifier.normalizeObjectIdentifier(oi); + FunctionDefinition potentialResult = tempCatalogFunctions.get(normalizedIdentifier); if (potentialResult != null) { return Optional.of( @@ -342,21 +338,24 @@ private Optional resolveAmbiguousFunctionReference(String // 3. Temporary catalog functions // 4. Catalog functions - if (tempSystemFunctions.containsKey(funcName)) { + String normalizedName = FunctionIdentifier.normalizeName(funcName); + if (tempSystemFunctions.containsKey(normalizedName)) { return Optional.of( - new FunctionLookup.Result(FunctionIdentifier.of(funcName), tempSystemFunctions.get(funcName)) + new FunctionLookup.Result( + FunctionIdentifier.of(funcName), + tempSystemFunctions.get(normalizedName)) ); } - Optional candidate = moduleManager.getFunctionDefinition(funcName); - if (candidate.isPresent()) { - return Optional.of( - new FunctionLookup.Result(FunctionIdentifier.of(funcName), candidate.get()) - ); - } + Optional candidate = moduleManager.getFunctionDefinition(normalizedName); + ObjectIdentifier oi = ObjectIdentifier.of( + catalogManager.getCurrentCatalog(), + catalogManager.getCurrentDatabase(), + funcName); - return resolvePreciseFunctionReference( - ObjectIdentifier.of(catalogManager.getCurrentCatalog(), catalogManager.getCurrentDatabase(), funcName)); + return candidate.map(fd -> + Optional.of(new Result(FunctionIdentifier.of(funcName), fd) + )).orElseGet(() -> resolvePreciseFunctionReference(oi)); } @Override diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionLookup.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionLookup.java index 686b0337a88a..34937a46b462 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionLookup.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionLookup.java @@ -36,13 +36,13 @@ public interface FunctionLookup { /** * Lookup a function by function identifier. The lookup is case insensitive. */ - Optional lookupFunction(FunctionIdentifier identifier); + Optional lookupFunction(UnresolvedIdentifier identifier); /** * Helper method for looking up a built-in function. */ default Result lookupBuiltInFunction(BuiltInFunctionDefinition definition) { - return lookupFunction(FunctionIdentifier.of(definition.getName())) + return lookupFunction(UnresolvedIdentifier.of(definition.getName())) .orElseThrow(() -> new TableException( String.format( "Required built-in function [%s] could not be found in any catalog.", diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/LookupCallResolver.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/LookupCallResolver.java index ef4508fc92e3..48fea8270bf2 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/LookupCallResolver.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/LookupCallResolver.java @@ -21,11 +21,11 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.FunctionLookup; +import org.apache.flink.table.catalog.UnresolvedIdentifier; import org.apache.flink.table.expressions.Expression; import org.apache.flink.table.expressions.LookupCallExpression; import org.apache.flink.table.expressions.UnresolvedCallExpression; import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor; -import org.apache.flink.table.functions.FunctionIdentifier; import java.util.List; import java.util.stream.Collectors; @@ -43,7 +43,7 @@ public LookupCallResolver(FunctionLookup functionLookup) { } public Expression visit(LookupCallExpression lookupCall) { - final FunctionLookup.Result result = functionLookup.lookupFunction(FunctionIdentifier.of(lookupCall.getUnresolvedName())) + final FunctionLookup.Result result = functionLookup.lookupFunction(UnresolvedIdentifier.of(lookupCall.getUnresolvedName())) .orElseThrow(() -> new ValidationException("Undefined function: " + lookupCall.getUnresolvedName())); return new UnresolvedCallExpression( diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java index 9556d4cdbab2..b7cede645d7f 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java @@ -22,7 +22,6 @@ import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; import org.apache.flink.table.catalog.exceptions.FunctionAlreadyExistException; import org.apache.flink.table.functions.FunctionDefinition; -import org.apache.flink.table.functions.FunctionIdentifier; import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.functions.ScalarFunctionDefinition; import org.apache.flink.table.module.Module; @@ -77,17 +76,24 @@ public void testGetBuiltInFunctions() { @Test public void testPreciseFunctionReference() throws FunctionAlreadyExistException, DatabaseNotExistException { - ObjectIdentifier oi = ObjectIdentifier.of(testCatalogName, GenericInMemoryCatalog.DEFAULT_DB, TEST_FUNCTION_NAME); + ObjectIdentifier oi = ObjectIdentifier.of( + testCatalogName, + GenericInMemoryCatalog.DEFAULT_DB, + TEST_FUNCTION_NAME); + UnresolvedIdentifier identifier = UnresolvedIdentifier.of( + testCatalogName, + GenericInMemoryCatalog.DEFAULT_DB, + TEST_FUNCTION_NAME); // test no function is found - assertFalse(functionCatalog.lookupFunction(FunctionIdentifier.of(oi)).isPresent()); + assertFalse(functionCatalog.lookupFunction(identifier).isPresent()); // test catalog function is found catalog.createFunction( oi.toObjectPath(), new CatalogFunctionImpl(TestFunction1.class.getName(), Collections.emptyMap()), false); - FunctionLookup.Result result = functionCatalog.lookupFunction(FunctionIdentifier.of(oi)).get(); + FunctionLookup.Result result = functionCatalog.lookupFunction(identifier).get(); assertFalse(result.getFunctionIdentifier().getSimpleName().isPresent()); assertEquals(oi, result.getFunctionIdentifier().getIdentifier().get()); @@ -99,7 +105,7 @@ public void testPreciseFunctionReference() throws FunctionAlreadyExistException, new TestFunction2() ); - result = functionCatalog.lookupFunction(FunctionIdentifier.of(oi)).get(); + result = functionCatalog.lookupFunction(identifier).get(); assertFalse(result.getFunctionIdentifier().getSimpleName().isPresent()); assertEquals(oi, result.getFunctionIdentifier().getIdentifier().get()); @@ -114,14 +120,14 @@ public void testAmbiguousFunctionReference() throws FunctionAlreadyExistExceptio TEST_FUNCTION_NAME); // test no function is found - assertFalse(functionCatalog.lookupFunction(FunctionIdentifier.of(TEST_FUNCTION_NAME)).isPresent()); + assertFalse(functionCatalog.lookupFunction(UnresolvedIdentifier.of(TEST_FUNCTION_NAME)).isPresent()); // test catalog function is found catalog.createFunction( oi.toObjectPath(), new CatalogFunctionImpl(TestFunction1.class.getName(), Collections.emptyMap()), false); - FunctionLookup.Result result = functionCatalog.lookupFunction(FunctionIdentifier.of(TEST_FUNCTION_NAME)).get(); + FunctionLookup.Result result = functionCatalog.lookupFunction(UnresolvedIdentifier.of(TEST_FUNCTION_NAME)).get(); assertFalse(result.getFunctionIdentifier().getSimpleName().isPresent()); assertEquals(oi, result.getFunctionIdentifier().getIdentifier().get()); @@ -133,7 +139,7 @@ public void testAmbiguousFunctionReference() throws FunctionAlreadyExistExceptio new TestFunction2() ); - result = functionCatalog.lookupFunction(FunctionIdentifier.of(TEST_FUNCTION_NAME)).get(); + result = functionCatalog.lookupFunction(UnresolvedIdentifier.of(TEST_FUNCTION_NAME)).get(); assertFalse(result.getFunctionIdentifier().getSimpleName().isPresent()); assertEquals(oi, result.getFunctionIdentifier().getIdentifier().get()); @@ -142,7 +148,7 @@ public void testAmbiguousFunctionReference() throws FunctionAlreadyExistExceptio // test system function is found moduleManager.loadModule("test_module", new TestModule()); - result = functionCatalog.lookupFunction(FunctionIdentifier.of(TEST_FUNCTION_NAME)).get(); + result = functionCatalog.lookupFunction(UnresolvedIdentifier.of(TEST_FUNCTION_NAME)).get(); assertEquals(TEST_FUNCTION_NAME, result.getFunctionIdentifier().getSimpleName().get()); assertTrue(((ScalarFunctionDefinition) result.getFunctionDefinition()).getScalarFunction() instanceof TestFunction3); @@ -150,7 +156,7 @@ public void testAmbiguousFunctionReference() throws FunctionAlreadyExistExceptio // test temp system function is found functionCatalog.registerTempSystemScalarFunction(TEST_FUNCTION_NAME, new TestFunction4()); - result = functionCatalog.lookupFunction(FunctionIdentifier.of(TEST_FUNCTION_NAME)).get(); + result = functionCatalog.lookupFunction(UnresolvedIdentifier.of(TEST_FUNCTION_NAME)).get(); assertEquals(TEST_FUNCTION_NAME, result.getFunctionIdentifier().getSimpleName().get()); assertTrue(((ScalarFunctionDefinition) result.getFunctionDefinition()).getScalarFunction() instanceof TestFunction4); diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/FunctionCatalogOperatorTable.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/FunctionCatalogOperatorTable.java index d4637e9dc7b7..deb9553ad23e 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/FunctionCatalogOperatorTable.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/FunctionCatalogOperatorTable.java @@ -20,9 +20,9 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.java.typeutils.GenericTypeInfo; -import org.apache.flink.table.catalog.CatalogManager; import org.apache.flink.table.catalog.FunctionCatalog; import org.apache.flink.table.catalog.FunctionLookup; +import org.apache.flink.table.catalog.UnresolvedIdentifier; import org.apache.flink.table.functions.AggregateFunctionDefinition; import org.apache.flink.table.functions.FunctionDefinition; import org.apache.flink.table.functions.FunctionIdentifier; @@ -49,7 +49,6 @@ import java.util.List; import java.util.Optional; -import static org.apache.flink.table.planner.functions.utils.FunctionUtils.toFunctionIdentifier; import static org.apache.flink.table.planner.functions.utils.HiveFunctionUtils.isHiveFunc; import static org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType; @@ -60,15 +59,12 @@ public class FunctionCatalogOperatorTable implements SqlOperatorTable { private final FunctionCatalog functionCatalog; - private final CatalogManager catalogManager; private final FlinkTypeFactory typeFactory; public FunctionCatalogOperatorTable( FunctionCatalog functionCatalog, - CatalogManager catalogManager, FlinkTypeFactory typeFactory) { this.functionCatalog = functionCatalog; - this.catalogManager = catalogManager; this.typeFactory = typeFactory; } @@ -89,12 +85,12 @@ public void lookupOperatorOverloads( return; } - FunctionIdentifier identifier = toFunctionIdentifier(opName.names.toArray(new String[0]), catalogManager); + UnresolvedIdentifier identifier = UnresolvedIdentifier.of(opName.names.toArray(new String[0])); Optional candidateFunction = functionCatalog.lookupFunction(identifier); candidateFunction.flatMap(lookupResult -> - convertToSqlFunction(category, identifier, lookupResult.getFunctionDefinition()) + convertToSqlFunction(category, lookupResult.getFunctionIdentifier(), lookupResult.getFunctionDefinition()) ).ifPresent(operatorList::add); } diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java index 997ce39f4cf3..66603d17b9ff 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java @@ -264,7 +264,6 @@ private SqlOperatorTable getBuiltinSqlOperatorTable() { return ChainedSqlOperatorTable.of( new FunctionCatalogOperatorTable( context.getFunctionCatalog(), - context.getCatalogManager(), typeFactory), FlinkSqlOperatorTable.instance()); } diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/SqlAggFunctionVisitor.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/SqlAggFunctionVisitor.java index d1ec68478b70..c3633a6ea4bc 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/SqlAggFunctionVisitor.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/SqlAggFunctionVisitor.java @@ -26,6 +26,7 @@ import org.apache.flink.table.functions.AggregateFunctionDefinition; import org.apache.flink.table.functions.BuiltInFunctionDefinitions; import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.functions.FunctionIdentifier; import org.apache.flink.table.functions.FunctionRequirement; import org.apache.flink.table.functions.TableAggregateFunction; import org.apache.flink.table.functions.TableAggregateFunctionDefinition; @@ -41,7 +42,6 @@ import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.isFunctionOfKind; import static org.apache.flink.table.functions.FunctionKind.AGGREGATE; import static org.apache.flink.table.functions.FunctionKind.TABLE_AGGREGATE; -import static org.apache.flink.table.planner.functions.utils.FunctionUtils.toFunctionIdentifier; import static org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType; /** @@ -89,8 +89,10 @@ public SqlAggFunction visit(CallExpression call) { if (isFunctionOfKind(call, AGGREGATE)) { AggregateFunctionDefinition aggDef = (AggregateFunctionDefinition) def; AggregateFunction aggFunc = aggDef.getAggregateFunction(); + FunctionIdentifier identifier = call.getFunctionIdentifier() + .orElse(FunctionIdentifier.of(aggFunc.functionIdentifier())); return new AggSqlFunction( - toFunctionIdentifier(call, aggFunc), + identifier, aggFunc.toString(), aggFunc, fromLegacyInfoToDataType(aggDef.getResultTypeInfo()), @@ -101,8 +103,10 @@ public SqlAggFunction visit(CallExpression call) { } else { TableAggregateFunctionDefinition aggDef = (TableAggregateFunctionDefinition) def; TableAggregateFunction aggFunc = aggDef.getTableAggregateFunction(); + FunctionIdentifier identifier = call.getFunctionIdentifier() + .orElse(FunctionIdentifier.of(aggFunc.functionIdentifier())); return new AggSqlFunction( - toFunctionIdentifier(call, aggFunc), + identifier, aggFunc.toString(), aggFunc, fromLegacyInfoToDataType(aggDef.getResultTypeInfo()), diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ScalarFunctionConvertRule.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ScalarFunctionConvertRule.java index 7b5a8a4dd59b..9041e2f55d85 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ScalarFunctionConvertRule.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ScalarFunctionConvertRule.java @@ -20,6 +20,7 @@ import org.apache.flink.table.expressions.CallExpression; import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.functions.FunctionIdentifier; import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.functions.ScalarFunctionDefinition; import org.apache.flink.table.planner.functions.utils.UserDefinedFunctionUtils; @@ -30,7 +31,6 @@ import java.util.Optional; import static org.apache.flink.table.planner.expressions.converter.ExpressionConverter.toRexNodes; -import static org.apache.flink.table.planner.functions.utils.FunctionUtils.toFunctionIdentifier; /** * {@link CallExpressionConvertRule} to convert {@link ScalarFunctionDefinition}. @@ -42,8 +42,10 @@ public Optional convert(CallExpression call, ConvertContext context) { FunctionDefinition def = call.getFunctionDefinition(); if (def instanceof ScalarFunctionDefinition) { ScalarFunction scalaFunc = ((ScalarFunctionDefinition) def).getScalarFunction(); + FunctionIdentifier identifier = call.getFunctionIdentifier() + .orElse(FunctionIdentifier.of(scalaFunc.functionIdentifier())); SqlFunction sqlFunction = UserDefinedFunctionUtils.createScalarSqlFunction( - toFunctionIdentifier(call, scalaFunc), + identifier, scalaFunc.toString(), scalaFunc, context.getTypeFactory()); diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/FunctionUtils.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/FunctionUtils.java deleted file mode 100644 index c8265e444fd8..000000000000 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/utils/FunctionUtils.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.planner.functions.utils; - -import org.apache.flink.table.catalog.CatalogManager; -import org.apache.flink.table.catalog.UnresolvedIdentifier; -import org.apache.flink.table.expressions.CallExpression; -import org.apache.flink.table.functions.FunctionIdentifier; -import org.apache.flink.table.functions.UserDefinedFunction; - -/** - * Utils for sql functions. - */ -public class FunctionUtils { - - public static FunctionIdentifier toFunctionIdentifier(String[] names, CatalogManager catalogManager) { - return names.length == 1 ? - FunctionIdentifier.of(names[0]) : - FunctionIdentifier.of( - catalogManager.qualifyIdentifier(UnresolvedIdentifier.of(names))); - } - - public static FunctionIdentifier toFunctionIdentifier(CallExpression call, UserDefinedFunction function) { - return call.getFunctionIdentifier() - .orElse(FunctionIdentifier.of(function.functionIdentifier())); - } -} diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala index c4fc15fe0c99..f178eb1154c1 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala @@ -19,21 +19,18 @@ package org.apache.flink.table.planner.plan.utils import org.apache.flink.table.api.TableException -import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog, FunctionLookup} +import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog, FunctionLookup, UnresolvedIdentifier} import org.apache.flink.table.dataformat.DataFormatConverters.{LocalDateConverter, LocalDateTimeConverter, LocalTimeConverter} import org.apache.flink.table.expressions._ import org.apache.flink.table.expressions.utils.ApiExpressionUtils._ import org.apache.flink.table.functions.BuiltInFunctionDefinitions.{AND, CAST, OR} -import org.apache.flink.table.functions.FunctionIdentifier import org.apache.flink.table.planner.calcite.FlinkTypeFactory -import org.apache.flink.table.planner.functions.utils.FunctionUtils.toFunctionIdentifier import org.apache.flink.table.planner.utils.Logging import org.apache.flink.table.runtime.functions.SqlDateTimeUtils.unixTimestampToLocalDateTime import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType import org.apache.flink.table.types.DataType import org.apache.flink.table.types.logical.LogicalTypeRoot._ import org.apache.flink.util.Preconditions - import org.apache.calcite.plan.RelOptUtil import org.apache.calcite.rex._ import org.apache.calcite.sql.fun.{SqlStdOperatorTable, SqlTrimFunction} @@ -412,11 +409,11 @@ class RexNodeToExpressionConverter( case _: SqlFunction | _: SqlPostfixOperator => val names = new util.ArrayList[String](rexCall.getOperator.getNameAsId.names) names.set(names.size() - 1, replace(names.get(names.size() - 1))) - val id = toFunctionIdentifier(names.asScala.toArray, catalogManager) + val id = UnresolvedIdentifier.of(names.asScala.toArray: _*) lookupFunction(id, operands, outputType) case operator@_ => lookupFunction( - FunctionIdentifier.of(replace(s"${operator.getKind}")), + UnresolvedIdentifier.of(replace(s"${operator.getKind}")), operands, outputType) } @@ -440,7 +437,7 @@ class RexNodeToExpressionConverter( fieldRef: RexPatternFieldRef): Option[ResolvedExpression] = None private def lookupFunction( - identifier: FunctionIdentifier, + identifier: UnresolvedIdentifier, operands: Seq[ResolvedExpression], outputType: DataType): Option[ResolvedExpression] = { Try(functionCatalog.lookupFunction(identifier)) match { diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/FunctionCatalogOperatorTable.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/FunctionCatalogOperatorTable.java index b56abf00c2f0..490063a64396 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/FunctionCatalogOperatorTable.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/FunctionCatalogOperatorTable.java @@ -22,7 +22,6 @@ import org.apache.flink.table.calcite.FlinkTypeFactory; import org.apache.flink.table.functions.AggregateFunctionDefinition; import org.apache.flink.table.functions.FunctionDefinition; -import org.apache.flink.table.functions.FunctionIdentifier; import org.apache.flink.table.functions.ScalarFunctionDefinition; import org.apache.flink.table.functions.TableFunctionDefinition; import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils; @@ -73,7 +72,7 @@ public void lookupOperatorOverloads( String name = opName.getSimple(); Optional candidateFunction = functionCatalog.lookupFunction( - FunctionIdentifier.of(name)); + UnresolvedIdentifier.of(name)); candidateFunction.flatMap(lookupResult -> convertToSqlFunction(category, name, lookupResult.getFunctionDefinition()) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala index 7e754b015674..4839daaba67a 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala @@ -19,7 +19,6 @@ package org.apache.flink.table.plan.util import java.sql.{Date, Time, Timestamp} - import org.apache.calcite.plan.RelOptUtil import org.apache.calcite.rex._ import org.apache.calcite.sql.fun.SqlStdOperatorTable @@ -28,10 +27,9 @@ import org.apache.calcite.util.{DateString, TimeString, TimestampString} import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, SqlTimeTypeInfo} import org.apache.flink.table.api.TableException import org.apache.flink.table.calcite.FlinkTypeFactory -import org.apache.flink.table.catalog.FunctionCatalog +import org.apache.flink.table.catalog.{FunctionCatalog, UnresolvedIdentifier} import org.apache.flink.table.expressions.utils.ApiExpressionUtils.unresolvedCall import org.apache.flink.table.expressions._ -import org.apache.flink.table.functions.FunctionIdentifier import org.apache.flink.table.util.JavaScalaConversionUtil import org.apache.flink.util.Preconditions import org.slf4j.{Logger, LoggerFactory} @@ -280,7 +278,7 @@ class RexNodeToExpressionConverter( val expressionBridge = new ExpressionBridge[PlannerExpression]( functionCatalog, PlannerExpressionConverter.INSTANCE) - JavaScalaConversionUtil.toScala(functionCatalog.lookupFunction(FunctionIdentifier.of(name))) + JavaScalaConversionUtil.toScala(functionCatalog.lookupFunction(UnresolvedIdentifier.of(name))) .flatMap(result => Try(expressionBridge.bridge( unresolvedCall(result.getFunctionDefinition, operands: _*))).toOption diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/AggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/AggregateTest.scala index ae016b09c058..25f875ff33e5 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/AggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/stream/sql/AggregateTest.scala @@ -26,9 +26,9 @@ import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment import org.apache.flink.table.api.scala._ import org.apache.flink.table.api.scala.internal.StreamTableEnvironmentImpl import org.apache.flink.table.api.{TableConfig, Types} -import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog, GenericInMemoryCatalog} +import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog, GenericInMemoryCatalog, UnresolvedIdentifier} import org.apache.flink.table.delegation.{Executor, Planner} -import org.apache.flink.table.functions.{AggregateFunction, AggregateFunctionDefinition, FunctionIdentifier} +import org.apache.flink.table.functions.{AggregateFunction, AggregateFunctionDefinition} import org.apache.flink.table.module.ModuleManager import org.apache.flink.table.utils.TableTestUtil.{streamTableNode, term, unaryNode} import org.apache.flink.table.utils.{StreamTableTestUtil, TableTestBase} @@ -85,7 +85,7 @@ class AggregateTest extends TableTestBase { tablEnv.registerFunction("udag", new MyAgg) val aggFunctionDefinition = functionCatalog - .lookupFunction(FunctionIdentifier.of("udag")).get() + .lookupFunction(UnresolvedIdentifier.of("udag")).get() .getFunctionDefinition .asInstanceOf[AggregateFunctionDefinition] @@ -98,7 +98,7 @@ class AggregateTest extends TableTestBase { tablEnv.registerFunction("udag2", new MyAgg2) val aggFunctionDefinition2 = functionCatalog - .lookupFunction(FunctionIdentifier.of("udag2")).get() + .lookupFunction(UnresolvedIdentifier.of("udag2")).get() .getFunctionDefinition .asInstanceOf[AggregateFunctionDefinition] From c2c0fb681b469c47c85bde07fa6525b9b735c107 Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Fri, 8 Nov 2019 16:18:35 +0800 Subject: [PATCH 511/746] [hotfix][table] Fix the comments of Operation --- .../main/java/org/apache/flink/table/operations/Operation.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/Operation.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/Operation.java index 89d86ef4b353..eb40217c8405 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/Operation.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/Operation.java @@ -19,12 +19,13 @@ package org.apache.flink.table.operations; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.table.delegation.Parser; import org.apache.flink.table.delegation.Planner; /** * Covers all sort of Table operations such as queries(DQL), modifications(DML), definitions(DDL), * or control actions(DCL). This is the output of - * {@link Planner#parse(String)}. + * {@link Planner#getParser()} and {@link Parser#parse(String)}. * * @see QueryOperation * @see ModifyOperation From 47c277d85aca6ce288505b13cd3a3595911e6bfa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Marco=20Zu=CC=88hlke?= Date: Sat, 26 Oct 2019 00:00:21 +0200 Subject: [PATCH 512/746] [FLINK-14380][ScalaAPI] Passing inferred outputType directly to map and flatMap function This closes #9999 --- .../api/datastream/ConnectedStreams.java | 37 ++++++++++++++- .../streaming/api/datastream/DataStream.java | 45 ++++++++++++++++++- .../api/scala/ConnectedStreams.scala | 4 +- .../streaming/api/scala/DataStream.scala | 4 +- 4 files changed, 82 insertions(+), 8 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java index d4a34c96fed7..6060a297e245 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/ConnectedStreams.java @@ -239,8 +239,22 @@ public SingleOutputStreamOperator map(CoMapFunction coMapper Utils.getCallLocationName(), true); - return transform("Co-Map", outTypeInfo, new CoStreamMap<>(inputStream1.clean(coMapper))); + return map(coMapper, outTypeInfo); + } + /** + * Applies a CoMap transformation on a {@link ConnectedStreams} and maps + * the output to a common type. The transformation calls a + * {@link CoMapFunction#map1} for each element of the first input and + * {@link CoMapFunction#map2} for each element of the second input. Each + * CoMapFunction call returns exactly one element. + * + * @param coMapper The CoMapFunction used to jointly transform the two input DataStreams + * @param outputType {@link TypeInformation} for the result type of the function. + * @return The transformed {@link DataStream} + */ + public SingleOutputStreamOperator map(CoMapFunction coMapper, TypeInformation outputType) { + return transform("Co-Map", outputType, new CoStreamMap<>(inputStream1.clean(coMapper))); } /** @@ -271,7 +285,26 @@ public SingleOutputStreamOperator flatMap( Utils.getCallLocationName(), true); - return transform("Co-Flat Map", outTypeInfo, new CoStreamFlatMap<>(inputStream1.clean(coFlatMapper))); + return flatMap(coFlatMapper, outTypeInfo); + } + + /** + * Applies a CoFlatMap transformation on a {@link ConnectedStreams} and + * maps the output to a common type. The transformation calls a + * {@link CoFlatMapFunction#flatMap1} for each element of the first input + * and {@link CoFlatMapFunction#flatMap2} for each element of the second + * input. Each CoFlatMapFunction call returns any number of elements + * including none. + * + * @param coFlatMapper + * The CoFlatMapFunction used to jointly transform the two input + * DataStreams + * @param outputType {@link TypeInformation} for the result type of the function. + * + * @return The transformed {@link DataStream} + */ + public SingleOutputStreamOperator flatMap(CoFlatMapFunction coFlatMapper, TypeInformation outputType) { + return transform("Co-Flat Map", outputType, new CoStreamFlatMap<>(inputStream1.clean(coFlatMapper))); } /** diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java index bd12cb2a1aa3..b7bea55002cc 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java @@ -590,7 +590,27 @@ public SingleOutputStreamOperator map(MapFunction mapper) { TypeInformation outType = TypeExtractor.getMapReturnTypes(clean(mapper), getType(), Utils.getCallLocationName(), true); - return transform("Map", outType, new StreamMap<>(clean(mapper))); + return map(mapper, outType); + } + + /** + * Applies a Map transformation on a {@link DataStream}. The transformation + * calls a {@link MapFunction} for each element of the DataStream. Each + * MapFunction call returns exactly one element. The user can also extend + * {@link RichMapFunction} to gain access to other features provided by the + * {@link org.apache.flink.api.common.functions.RichFunction} interface. + * + * @param mapper + * The MapFunction that is called for each element of the + * DataStream. + * @param outputType {@link TypeInformation} for the result type of the function. + * + * @param + * output type + * @return The transformed {@link DataStream}. + */ + public SingleOutputStreamOperator map(MapFunction mapper, TypeInformation outputType) { + return transform("Map", outputType, new StreamMap<>(clean(mapper))); } /** @@ -614,7 +634,28 @@ public SingleOutputStreamOperator flatMap(FlatMapFunction flatMappe TypeInformation outType = TypeExtractor.getFlatMapReturnTypes(clean(flatMapper), getType(), Utils.getCallLocationName(), true); - return transform("Flat Map", outType, new StreamFlatMap<>(clean(flatMapper))); + return flatMap(flatMapper, outType); + } + + /** + * Applies a FlatMap transformation on a {@link DataStream}. The + * transformation calls a {@link FlatMapFunction} for each element of the + * DataStream. Each FlatMapFunction call can return any number of elements + * including none. The user can also extend {@link RichFlatMapFunction} to + * gain access to other features provided by the + * {@link org.apache.flink.api.common.functions.RichFunction} interface. + * + * @param flatMapper + * The FlatMapFunction that is called for each element of the + * DataStream + * @param outputType {@link TypeInformation} for the result type of the function. + * + * @param + * output type + * @return The transformed {@link DataStream}. + */ + public SingleOutputStreamOperator flatMap(FlatMapFunction flatMapper, TypeInformation outputType) { + return transform("Flat Map", outputType, new StreamFlatMap<>(clean(flatMapper))); } diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala index 68514b77b46c..caaf22dab167 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala @@ -97,7 +97,7 @@ class ConnectedStreams[IN1, IN2](javaStream: JavaCStream[IN1, IN2]) { } val outType : TypeInformation[R] = implicitly[TypeInformation[R]] - asScalaStream(javaStream.map(coMapper).returns(outType).asInstanceOf[JavaStream[R]]) + asScalaStream(javaStream.map(coMapper, outType).asInstanceOf[JavaStream[R]]) } /** @@ -176,7 +176,7 @@ class ConnectedStreams[IN1, IN2](javaStream: JavaCStream[IN1, IN2]) { } val outType : TypeInformation[R] = implicitly[TypeInformation[R]] - asScalaStream(javaStream.flatMap(coFlatMapper).returns(outType).asInstanceOf[JavaStream[R]]) + asScalaStream(javaStream.flatMap(coFlatMapper, outType).asInstanceOf[JavaStream[R]]) } /** diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala index 15dca2c9c075..c2ca15a2ab48 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala @@ -631,7 +631,7 @@ class DataStream[T](stream: JavaStream[T]) { } val outType : TypeInformation[R] = implicitly[TypeInformation[R]] - asScalaStream(stream.map(mapper).returns(outType).asInstanceOf[JavaStream[R]]) + asScalaStream(stream.map(mapper, outType).asInstanceOf[JavaStream[R]]) } /** @@ -644,7 +644,7 @@ class DataStream[T](stream: JavaStream[T]) { } val outType : TypeInformation[R] = implicitly[TypeInformation[R]] - asScalaStream(stream.flatMap(flatMapper).returns(outType).asInstanceOf[JavaStream[R]]) + asScalaStream(stream.flatMap(flatMapper, outType).asInstanceOf[JavaStream[R]]) } /** From 0b3372605cf46aed72dbe926fa8f10d4511147b9 Mon Sep 17 00:00:00 2001 From: joe <15692118798@163.com> Date: Fri, 25 Oct 2019 11:02:44 +0800 Subject: [PATCH 513/746] [FLINK-14481][runtime] Change Flink's port check to 0 to 65535 This closes #9992 --- .../org/apache/flink/runtime/io/network/netty/NettyConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConfig.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConfig.java index 9730907ef99e..7a42f67f05d7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConfig.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConfig.java @@ -63,7 +63,7 @@ public NettyConfig( this.serverAddress = checkNotNull(serverAddress); - checkArgument(serverPort >= 0 && serverPort <= 65536, "Invalid port number."); + checkArgument(serverPort >= 0 && serverPort <= 65535, "Invalid port number."); this.serverPort = serverPort; checkArgument(memorySegmentSize > 0, "Invalid memory segment size."); From ab7b433081589229385faf6207adb24ab75229be Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Thu, 7 Nov 2019 11:55:25 +0100 Subject: [PATCH 514/746] [hotfix] remove unused method in YarnConfigUtils --- .../java/org/apache/flink/yarn/cli/YarnConfigUtils.java | 9 --------- 1 file changed, 9 deletions(-) diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnConfigUtils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnConfigUtils.java index efd3d45cde30..3012d7cf4e6f 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnConfigUtils.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnConfigUtils.java @@ -22,7 +22,6 @@ import org.apache.flink.configuration.Configuration; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Objects; @@ -37,14 +36,6 @@ */ public class YarnConfigUtils { - public static void encodeListToConfig( - final Configuration configuration, - final ConfigOption> key, - final Collection value, - final Function mapper) { - encodeListToConfig(configuration, key, value.stream(), mapper); - } - public static void encodeListToConfig( final Configuration configuration, final ConfigOption> key, From bd7d745593e0f4ebdea718a09042b7187e2d03bd Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Wed, 6 Nov 2019 13:12:14 +0100 Subject: [PATCH 515/746] [hotfix] Port the DYNAMIC_PROPERTIES config option to new options API --- .../flink/yarn/configuration/YarnConfigOptionsInternal.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptionsInternal.java b/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptionsInternal.java index 4910809fb490..12cf4c61cfe8 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptionsInternal.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptionsInternal.java @@ -31,6 +31,7 @@ public class YarnConfigOptionsInternal { public static final ConfigOption DYNAMIC_PROPERTIES = key("$internal.yarn.dynamic-properties") + .stringType() .noDefaultValue() .withDescription("**DO NOT USE** Specify YARN dynamic properties."); From 93c4d49830f9d44d421136ff48fd5015023bd233 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Wed, 6 Nov 2019 13:09:30 +0100 Subject: [PATCH 516/746] [FLINK-14630] Make the yarn APPLICATION_LOG_CONFIG_FILE an internal option --- docs/_includes/generated/yarn_config_configuration.html | 5 ----- .../java/org/apache/flink/yarn/YarnClusterDescriptor.java | 2 +- .../java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java | 2 +- .../apache/flink/yarn/configuration/YarnConfigOptions.java | 6 ------ .../flink/yarn/configuration/YarnConfigOptionsInternal.java | 5 +++++ 5 files changed, 7 insertions(+), 13 deletions(-) diff --git a/docs/_includes/generated/yarn_config_configuration.html b/docs/_includes/generated/yarn_config_configuration.html index 98655bfd16c0..199f08cea36d 100644 --- a/docs/_includes/generated/yarn_config_configuration.html +++ b/docs/_includes/generated/yarn_config_configuration.html @@ -87,11 +87,6 @@ 5 Time between heartbeats with the ResourceManager in seconds. - -

    yarn.log-config-file
    - (none) - The location of the log config file, e.g. the path to your log4j.properties for log4j. -
    yarn.maximum-failed-containers
    (none) 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 f958b70b8f60..72f677884e1b 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 @@ -730,7 +730,7 @@ private ApplicationReport startAppMaster( systemShipFiles.add(file.getAbsoluteFile()); } - final String logConfigFilePath = configuration.getString(YarnConfigOptions.APPLICATION_LOG_CONFIG_FILE); + final String logConfigFilePath = configuration.getString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE); if (logConfigFilePath != null) { systemShipFiles.add(new File(logConfigFilePath)); } 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 0efa6103d6b0..af6e95373951 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 @@ -443,7 +443,7 @@ private void applyDescriptorOptionToConfig(final CommandLine commandLine, final } discoverLogConfigFile().ifPresent( - file -> configuration.setString(YarnConfigOptions.APPLICATION_LOG_CONFIG_FILE, file.getPath()) + file -> configuration.setString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE, file.getPath()) ); } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java b/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java index b4f924764ba0..1bd967b88b84 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptions.java @@ -208,12 +208,6 @@ public class YarnConfigOptions { // ----------------------- YARN CLI OPTIONS ------------------------------------ - public static final ConfigOption APPLICATION_LOG_CONFIG_FILE = - key("yarn.log-config-file") - .stringType() - .noDefaultValue() - .withDescription("The location of the log config file, e.g. the path to your log4j.properties for log4j."); - public static final ConfigOption> SHIP_DIRECTORIES = key("yarn.ship-directories") .stringType() diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptionsInternal.java b/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptionsInternal.java index 12cf4c61cfe8..8dbda0193cd5 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptionsInternal.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/configuration/YarnConfigOptionsInternal.java @@ -35,4 +35,9 @@ public class YarnConfigOptionsInternal { .noDefaultValue() .withDescription("**DO NOT USE** Specify YARN dynamic properties."); + public static final ConfigOption APPLICATION_LOG_CONFIG_FILE = + key("$internal.yarn.log-config-file") + .stringType() + .noDefaultValue() + .withDescription("**DO NOT USE** The location of the log config file, e.g. the path to your log4j.properties for log4j."); } From 18f101acef33c4375c1e21d133a9a16c9d333e03 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Wed, 6 Nov 2019 16:59:17 +0100 Subject: [PATCH 517/746] [FLINK-14630] Add test utility to create YarnClusterDescriptor with Logging --- .../org/apache/flink/yarn/YARNITCase.java | 12 +++++---- .../flink/yarn/YarnConfigurationITCase.java | 12 +++++---- .../org/apache/flink/yarn/YarnTestBase.java | 11 ++++---- .../flink/yarn/cli/FlinkYarnSessionCli.java | 6 +++-- .../flink/yarn/AbstractYarnClusterTest.java | 11 ++++---- .../flink/yarn/YarnClusterDescriptorTest.java | 22 +++++++++------- .../org/apache/flink/yarn/YarnTestUtils.java | 26 +++++++++++++++++++ 7 files changed, 68 insertions(+), 32 deletions(-) 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 6cf6fb62c64b..324e00e73c34 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 @@ -22,6 +22,7 @@ import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.AkkaOptions; +import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmaster.JobResult; @@ -68,11 +69,12 @@ public void testPerJobMode() throws Exception { configuration.setString(AkkaOptions.ASK_TIMEOUT, "30 s"); final YarnClient yarnClient = getYarnClient(); - try (final YarnClusterDescriptor yarnClusterDescriptor = new YarnClusterDescriptor( - configuration, - getYarnConfiguration(), - yarnClient, - true)) { + try (final YarnClusterDescriptor yarnClusterDescriptor = org.apache.flink.yarn.YarnTestUtils.createClusterDescriptorWithLogging( + System.getenv(ConfigConstants.ENV_FLINK_CONF_DIR), + configuration, + getYarnConfiguration(), + yarnClient, + true)) { yarnClusterDescriptor.setLocalJarPath(new Path(flinkUberjar.getAbsolutePath())); yarnClusterDescriptor.addShipFiles(Arrays.asList(flinkLibFolder.listFiles())); 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 2aa476a74fb9..dfab48ef3c96 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 @@ -19,6 +19,7 @@ package org.apache.flink.yarn; import org.apache.flink.api.common.time.Time; +import org.apache.flink.client.cli.CliFrontend; import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.PackagedProgram; @@ -94,11 +95,12 @@ public void testFlinkContainerMemory() throws Exception { configuration.setString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, String.valueOf(4L << 20)); final YarnConfiguration yarnConfiguration = getYarnConfiguration(); - final YarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor( - configuration, - yarnConfiguration, - yarnClient, - true); + final YarnClusterDescriptor clusterDescriptor = YarnTestUtils.createClusterDescriptorWithLogging( + CliFrontend.getConfigurationDirectoryFromEnv(), + configuration, + yarnConfiguration, + yarnClient, + true); clusterDescriptor.setLocalJarPath(new Path(flinkUberjar.getAbsolutePath())); clusterDescriptor.addShipFiles(Arrays.asList(flinkLibFolder.listFiles())); 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 80f6efc72560..d4984501eb4c 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 @@ -304,11 +304,12 @@ public static File findFile(String startAt, FilenameFilter fnf) { @Nonnull YarnClusterDescriptor createYarnClusterDescriptor(org.apache.flink.configuration.Configuration flinkConfiguration) { - final YarnClusterDescriptor yarnClusterDescriptor = new YarnClusterDescriptor( - flinkConfiguration, - YARN_CONFIGURATION, - yarnClient, - true); + final YarnClusterDescriptor yarnClusterDescriptor = YarnTestUtils.createClusterDescriptorWithLogging( + CliFrontend.getConfigurationDirectoryFromEnv(), + flinkConfiguration, + YARN_CONFIGURATION, + yarnClient, + true); yarnClusterDescriptor.setLocalJarPath(new Path(flinkUberjar.toURI())); yarnClusterDescriptor.addShipFiles(Collections.singletonList(flinkLibFolder)); return yarnClusterDescriptor; 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 af6e95373951..b8698cd065b4 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 @@ -18,6 +18,7 @@ package org.apache.flink.yarn.cli; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.client.cli.AbstractCustomCommandLine; import org.apache.flink.client.cli.CliArgsException; import org.apache.flink.client.cli.CliFrontend; @@ -442,12 +443,13 @@ private void applyDescriptorOptionToConfig(final CommandLine commandLine, final configuration.setString(YarnConfigOptions.NODE_LABEL, nodeLabelValue); } - discoverLogConfigFile().ifPresent( + discoverLogConfigFile(configurationDirectory).ifPresent( file -> configuration.setString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE, file.getPath()) ); } - private Optional discoverLogConfigFile() { + @VisibleForTesting + public static Optional discoverLogConfigFile(final String configurationDirectory) { Optional logConfigFile = Optional.empty(); final File log4jFile = new File(configurationDirectory + File.separator + CONFIG_FILE_LOG4J_NAME); diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java index 1ba20083178c..55538aebff82 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java @@ -64,11 +64,12 @@ public void testClusterClientRetrievalOfFinishedYarnApplication() throws Excepti yarnClient.init(yarnConfiguration); yarnClient.start(); - final YarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor( - new Configuration(), - yarnConfiguration, - yarnClient, - false); + final YarnClusterDescriptor clusterDescriptor = YarnTestUtils.createClusterDescriptorWithLogging( + temporaryFolder.newFolder().getAbsolutePath(), + new Configuration(), + yarnConfiguration, + yarnClient, + false); try { clusterDescriptor.retrieve(applicationId); 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 69af7c386e65..30ba407ecb51 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 @@ -522,11 +522,12 @@ public void testYarnClientShutDown() { closableYarnClient.init(yarnConfiguration); closableYarnClient.start(); - yarnClusterDescriptor = new YarnClusterDescriptor( - new Configuration(), - yarnConfiguration, - closableYarnClient, - false); + yarnClusterDescriptor = YarnTestUtils.createClusterDescriptorWithLogging( + temporaryFolder.getRoot().getAbsolutePath(), + new Configuration(), + yarnConfiguration, + closableYarnClient, + false); yarnClusterDescriptor.close(); @@ -538,10 +539,11 @@ private YarnClusterDescriptor createYarnClusterDescriptor() { } private YarnClusterDescriptor createYarnClusterDescriptor(Configuration configuration) { - return new YarnClusterDescriptor( - configuration, - yarnConfiguration, - yarnClient, - true); + return YarnTestUtils.createClusterDescriptorWithLogging( + temporaryFolder.getRoot().getAbsolutePath(), + configuration, + yarnConfiguration, + yarnClient, + true); } } diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnTestUtils.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnTestUtils.java index cffc333a7be3..d24433f8cb3a 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnTestUtils.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnTestUtils.java @@ -18,7 +18,13 @@ package org.apache.flink.yarn; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.yarn.cli.FlinkYarnSessionCli; +import org.apache.flink.yarn.configuration.YarnConfigOptionsInternal; + import org.apache.hadoop.util.VersionInfo; +import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import java.util.Arrays; @@ -36,4 +42,24 @@ static boolean isHadoopVersionGreaterThanOrEquals(final int major, final int min private YarnTestUtils() { throw new UnsupportedOperationException("This class should never be instantiated."); } + + public static YarnClusterDescriptor createClusterDescriptorWithLogging( + final String flinkConfDir, + final Configuration flinkConfiguration, + final YarnConfiguration yarnConfiguration, + final YarnClient yarnClient, + final boolean sharedYarnClient) { + final Configuration effectiveConfiguration = addLogDirToConfiguration(flinkConfiguration, flinkConfDir); + return new YarnClusterDescriptor(effectiveConfiguration, yarnConfiguration, yarnClient, sharedYarnClient); + } + + private static Configuration addLogDirToConfiguration(final Configuration flinkConfiguration, final String flinkConfDir) { + if (flinkConfiguration.getString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE) != null) { + return flinkConfiguration; + } + + FlinkYarnSessionCli.discoverLogConfigFile(flinkConfDir).ifPresent(file -> + flinkConfiguration.setString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE, file.getAbsolutePath())); + return flinkConfiguration; + } } From 91d6e76816debfd2553516872cf67c5bf1d23e66 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Thu, 7 Nov 2019 11:52:59 +0100 Subject: [PATCH 518/746] [FLINK-14630] Create separate util for setting the yarn log config option --- .../flink/yarn/cli/FlinkYarnSessionCli.java | 16 ++++++++++++---- .../org/apache/flink/yarn/YarnTestUtils.java | 13 +------------ 2 files changed, 13 insertions(+), 16 deletions(-) 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 b8698cd065b4..970fdeb5a2aa 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 @@ -443,13 +443,21 @@ private void applyDescriptorOptionToConfig(final CommandLine commandLine, final configuration.setString(YarnConfigOptions.NODE_LABEL, nodeLabelValue); } - discoverLogConfigFile(configurationDirectory).ifPresent( - file -> configuration.setString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE, file.getPath()) - ); + setLogConfigFileInConfig(configuration, configurationDirectory); } @VisibleForTesting - public static Optional discoverLogConfigFile(final String configurationDirectory) { + public static Configuration setLogConfigFileInConfig(final Configuration configuration, final String configurationDirectory) { + if (configuration.getString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE) != null) { + return configuration; + } + + FlinkYarnSessionCli.discoverLogConfigFile(configurationDirectory).ifPresent(file -> + configuration.setString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE, file.getPath())); + return configuration; + } + + private static Optional discoverLogConfigFile(final String configurationDirectory) { Optional logConfigFile = Optional.empty(); final File log4jFile = new File(configurationDirectory + File.separator + CONFIG_FILE_LOG4J_NAME); diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnTestUtils.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnTestUtils.java index d24433f8cb3a..4aef1debcb4f 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnTestUtils.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnTestUtils.java @@ -20,7 +20,6 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.yarn.cli.FlinkYarnSessionCli; -import org.apache.flink.yarn.configuration.YarnConfigOptionsInternal; import org.apache.hadoop.util.VersionInfo; import org.apache.hadoop.yarn.client.api.YarnClient; @@ -49,17 +48,7 @@ public static YarnClusterDescriptor createClusterDescriptorWithLogging( final YarnConfiguration yarnConfiguration, final YarnClient yarnClient, final boolean sharedYarnClient) { - final Configuration effectiveConfiguration = addLogDirToConfiguration(flinkConfiguration, flinkConfDir); + final Configuration effectiveConfiguration = FlinkYarnSessionCli.setLogConfigFileInConfig(flinkConfiguration, flinkConfDir); return new YarnClusterDescriptor(effectiveConfiguration, yarnConfiguration, yarnClient, sharedYarnClient); } - - private static Configuration addLogDirToConfiguration(final Configuration flinkConfiguration, final String flinkConfDir) { - if (flinkConfiguration.getString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE) != null) { - return flinkConfiguration; - } - - FlinkYarnSessionCli.discoverLogConfigFile(flinkConfDir).ifPresent(file -> - flinkConfiguration.setString(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE, file.getAbsolutePath())); - return flinkConfiguration; - } } From 63bf1df26570e2fcd6eb292b1c003c5a6c83e4ec Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Thu, 7 Nov 2019 13:01:52 +0100 Subject: [PATCH 519/746] [hotfix] Make logging flags explicit --- .../src/test/java/org/apache/flink/yarn/YarnTestBase.java | 2 +- .../java/org/apache/flink/yarn/YarnClusterDescriptor.java | 7 +++++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java index d4984501eb4c..c163c67db608 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 @@ -305,7 +305,7 @@ public static File findFile(String startAt, FilenameFilter fnf) { @Nonnull YarnClusterDescriptor createYarnClusterDescriptor(org.apache.flink.configuration.Configuration flinkConfiguration) { final YarnClusterDescriptor yarnClusterDescriptor = YarnTestUtils.createClusterDescriptorWithLogging( - CliFrontend.getConfigurationDirectoryFromEnv(), + tempConfPathForSecureRun.getAbsolutePath(), flinkConfiguration, YARN_CONFIGURATION, yarnClient, 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 72f677884e1b..ae59d9213e6b 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 @@ -950,10 +950,13 @@ private ApplicationReport startAppMaster( ""); } + final boolean hasLogback = logConfigFilePath != null && logConfigFilePath.endsWith(CONFIG_FILE_LOGBACK_NAME); + final boolean hasLog4j = logConfigFilePath != null && logConfigFilePath.endsWith(CONFIG_FILE_LOG4J_NAME); + final ContainerLaunchContext amContainer = setupApplicationMasterContainer( yarnClusterEntrypoint, - logConfigFilePath != null && logConfigFilePath.endsWith(CONFIG_FILE_LOGBACK_NAME), - logConfigFilePath != null && logConfigFilePath.endsWith(CONFIG_FILE_LOG4J_NAME), + hasLogback, + hasLog4j, hasKrb5, clusterSpecification.getMasterMemoryMB()); From 5acf09d62edb6070846d1d537f728ca89d9c3cc8 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Tue, 5 Nov 2019 15:29:13 +0800 Subject: [PATCH 520/746] [hotfix][table-planner-blink] Make sure TableSourceTable.catalogTable is not null --- .../flink/table/planner/catalog/DatabaseCalciteSchema.java | 2 +- .../flink/table/planner/plan/QueryOperationConverter.java | 3 ++- .../logical/PushPartitionIntoTableSourceScanRule.scala | 4 ++-- .../flink/table/planner/plan/schema/TableSourceTable.scala | 6 +++++- 4 files changed, 10 insertions(+), 5 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/DatabaseCalciteSchema.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/DatabaseCalciteSchema.java index 233c190f7ec6..9ccfbf1464fa 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/DatabaseCalciteSchema.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/DatabaseCalciteSchema.java @@ -185,7 +185,7 @@ private Table convertSourceTable( tableSource, isStreamingMode, FlinkStatistic.builder().tableStats(tableStats).build(), - null); + table); } private TableStats extractTableStats(ConnectorCatalogTable table, ObjectPath tablePath) { diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java index 752a63605d32..0607835a6a2a 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java @@ -22,6 +22,7 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.ConnectorCatalogTable; import org.apache.flink.table.catalog.FunctionLookup; import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.expressions.CallExpression; @@ -356,7 +357,7 @@ public RelNode visit(TableSourceQueryOperation tableSourceOperation) { } TableSourceTable tableSourceTable = new TableSourceTable<>( - tableSource, !isBatch, statistic, null); + tableSource, !isBatch, statistic, ConnectorCatalogTable.source(tableSource, isBatch)); FlinkRelOptTable table = FlinkRelOptTable.create( relBuilder.getRelOptSchema(), tableSourceTable.getRowType(relBuilder.getTypeFactory()), 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 a39ab6e98b45..c67bf5492b28 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 @@ -50,8 +50,8 @@ class PushPartitionIntoTableSourceScanRule extends RelOptRule( val scan: LogicalTableScan = call.rel(1) scan.getTable.unwrap(classOf[TableSourceTable[_]]) match { - case table: TableSourceTable[_] => table.catalogTable != null && - table.catalogTable.isPartitioned && + case table: TableSourceTable[_] => + table.catalogTable.isPartitioned && table.tableSource.isInstanceOf[PartitionableTableSource] case _ => false } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TableSourceTable.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TableSourceTable.scala index 9444107b1c28..14c577cc8d06 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TableSourceTable.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TableSourceTable.scala @@ -23,8 +23,8 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.stats.FlinkStatistic import org.apache.flink.table.planner.sources.TableSourceUtil import org.apache.flink.table.sources.TableSource - import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory} +import org.apache.flink.shaded.guava18.com.google.common.base.Preconditions /** * Abstract class which define the interfaces required to convert a [[TableSource]] to @@ -50,6 +50,10 @@ class TableSourceTable[T]( this(tableSource, isStreamingMode, statistic, None, catalogTable) } + Preconditions.checkNotNull(tableSource) + Preconditions.checkNotNull(statistic) + Preconditions.checkNotNull(catalogTable) + // TODO implements this // TableSourceUtil.validateTableSource(tableSource) From 35836df1ef616e8103eb367a470c54e0eea433bd Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Tue, 5 Nov 2019 16:53:20 +0800 Subject: [PATCH 521/746] [FLINK-14324][table-planner-blink] Convert SqlCreateTable with SqlWatermark to CatalogTable --- .../calcite/sql/validate/ParameterScope.java | 72 ++++++++++++++++ .../operations/SqlToOperationConverter.java | 84 +++++++++++-------- .../utils/JavaUserDefinedScalarFunctions.java | 44 ++++++++++ .../sqlexec/SqlToOperationConverterTest.java | 44 ++++++++++ 4 files changed, 209 insertions(+), 35 deletions(-) create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/sql/validate/ParameterScope.java diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/sql/validate/ParameterScope.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/sql/validate/ParameterScope.java new file mode 100644 index 000000000000..414c92891088 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/sql/validate/ParameterScope.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.calcite.sql.validate; + +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.sql.SqlCall; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlNode; + +import java.util.Map; + +// This class is copied from Calcite's org.apache.calcite.sql.validate.ParameterScope, +// can be removed after https://issues.apache.org/jira/browse/CALCITE-3476 is fixed. +// +// Modification: +// - L66~L69: override resolveColumn method + +/** + * A scope which contains nothing besides a few parameters. Like + * {@link EmptyScope} (which is its base class), it has no parent scope. + * + * @see ParameterNamespace + */ +public class ParameterScope extends EmptyScope { + //~ Instance fields -------------------------------------------------------- + + /** + * Map from the simple names of the parameters to types of the parameters + * ({@link RelDataType}). + */ + private final Map nameToTypeMap; + + //~ Constructors ----------------------------------------------------------- + + public ParameterScope( + SqlValidatorImpl validator, + Map nameToTypeMap) { + super(validator); + this.nameToTypeMap = nameToTypeMap; + } + + //~ Methods ---------------------------------------------------------------- + + public SqlQualified fullyQualify(SqlIdentifier identifier) { + return SqlQualified.create(this, 1, null, identifier); + } + + public SqlValidatorScope getOperandScope(SqlCall call) { + return this; + } + + @Override + public RelDataType resolveColumn(String name, SqlNode ctx) { + return nameToTypeMap.get(name); + } +} + +// End ParameterScope.java 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 596aa492e5a9..116951e518f4 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 @@ -36,14 +36,17 @@ import org.apache.flink.table.operations.ddl.DropTableOperation; import org.apache.flink.table.planner.calcite.FlinkPlannerImpl; import org.apache.flink.table.planner.calcite.FlinkTypeFactory; -import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.utils.TypeConversions; import org.apache.calcite.rel.RelRoot; import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlNodeList; +import org.apache.calcite.sql.validate.SqlValidator; import java.util.HashMap; import java.util.List; @@ -114,11 +117,6 @@ private Operation convertCreateTable(SqlCreateTable sqlCreateTable) { throw new SqlConversionException("Primary key and unique key are not supported yet."); } - if (sqlCreateTable.getWatermark().isPresent()) { - // TODO: FLINK-14320 - throw new SqlConversionException("Watermark statement is not supported yet."); - } - // set with properties Map properties = new HashMap<>(); sqlCreateTable.getPropertyList().getList().forEach(p -> @@ -187,16 +185,18 @@ private Operation convertSqlQuery(SqlNode node) { //~ Tools ------------------------------------------------------------------ /** - * Create a table schema from {@link SqlCreateTable}. This schema contains computed column - * fields, say, we have a create table DDL statement: + * Create a table schema from {@link SqlCreateTable}. This schema may contains computed column + * fields and watermark information, say, we have a create table DDL statement: *
    -	 *   create table t(
    -	 *     a int,
    -	 *     b varchar,
    -	 *     c as to_timestamp(b))
    -	 *   with (
    -	 *     'connector' = 'csv',
    -	 *     'k1' = 'v1')
    +	 *   CREATE TABLE myTable (
    +	 *     a INT,
    +	 *     b STRING,
    +	 *     c AS TO_TIMESTAMP(b),
    +	 *     WATERMARK FOR c AS c - INTERVAL '1' SECOND
    +	 *   ) WITH (
    +	 *     'connector.type' = 'csv',
    +	 *     ...
    +	 *   )
     	 * 
    * *

    The returned table schema contains columns (a:int, b:varchar, c:timestamp). @@ -205,29 +205,43 @@ private Operation convertSqlQuery(SqlNode node) { * @return TableSchema */ private TableSchema createTableSchema(SqlCreateTable sqlCreateTable) { - // setup table columns - SqlNodeList columnList = sqlCreateTable.getColumnList(); - TableSchema physicalSchema = null; - TableSchema.Builder builder = new TableSchema.Builder(); - // collect the physical table schema first. - final List physicalColumns = columnList.getList().stream() - .filter(n -> n instanceof SqlTableColumn).collect(Collectors.toList()); - for (SqlNode node : physicalColumns) { - SqlTableColumn column = (SqlTableColumn) node; - final RelDataType relType = column.getType() - .deriveType( - flinkPlanner.getOrCreateSqlValidator(), - column.getType().getNullable()); - builder.field(column.getName().getSimple(), - LogicalTypeDataTypeConverter.fromLogicalTypeToDataType( - FlinkTypeFactory.toLogicalType(relType))); - physicalSchema = builder.build(); - } - assert physicalSchema != null; if (sqlCreateTable.containsComputedColumn()) { throw new SqlConversionException("Computed columns for DDL is not supported yet!"); } - return physicalSchema; + TableSchema.Builder builder = new TableSchema.Builder(); + SqlValidator validator = flinkPlanner.getOrCreateSqlValidator(); + // setup table columns + SqlNodeList columnList = sqlCreateTable.getColumnList(); + Map nameToTypeMap = new HashMap<>(); + for (SqlNode node : columnList.getList()) { + if (node instanceof SqlTableColumn) { + SqlTableColumn column = (SqlTableColumn) node; + RelDataType relType = column.getType() + .deriveType(validator, column.getType().getNullable()); + String name = column.getName().getSimple(); + nameToTypeMap.put(name, relType); + DataType dataType = TypeConversions.fromLogicalToDataType( + FlinkTypeFactory.toLogicalType(relType)); + builder.field(name, dataType); + } else if (node instanceof SqlBasicCall) { + // TODO: computed column ... + } + } + + // put watermark information into TableSchema + sqlCreateTable.getWatermark().ifPresent(watermark -> { + String rowtimeAttribute = watermark.getEventTimeColumnName().toString(); + SqlNode expression = watermark.getWatermarkStrategy(); + // this will validate and expand function identifiers. + SqlNode validated = validator.validateParameterizedExpression(expression, nameToTypeMap); + RelDataType validatedType = validator.getValidatedNodeType(validated); + DataType exprDataType = TypeConversions.fromLogicalToDataType( + FlinkTypeFactory.toLogicalType(validatedType)); + // use the qualified SQL expression string + builder.watermark(rowtimeAttribute, validated.toString(), exprDataType); + }); + + return builder.build(); } private PlannerQueryOperation toQueryOperation(FlinkPlannerImpl planner, SqlNode validated) { diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedScalarFunctions.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedScalarFunctions.java index 93481d09dc0d..40fc91d88729 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedScalarFunctions.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedScalarFunctions.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.table.functions.AggregateFunction; import org.apache.flink.table.functions.FunctionContext; @@ -27,8 +28,12 @@ import org.apache.flink.table.functions.python.PythonEnv; import org.apache.flink.table.functions.python.PythonFunction; +import java.sql.Timestamp; import java.util.Arrays; import java.util.Random; +import java.util.TimeZone; + +import static org.junit.Assert.fail; /** * Test scalar functions. @@ -117,6 +122,45 @@ public String eval(Integer[] a, String[] b) { } } + /** + * A UDF minus Timestamp with the specified offset. + * This UDF also ensures open and close are called. + */ + public static class JavaFunc5 extends ScalarFunction { + // these fields must be reset to false at the beginning of tests, + // otherwise the static fields will be changed by several tests concurrently + public static boolean openCalled = false; + public static boolean closeCalled = false; + + @Override + public void open(FunctionContext context) { + openCalled = true; + } + + public Timestamp eval(Long l, Integer offset) { + if (!openCalled) { + fail("Open was not called before run."); + } + if (l == null || offset == null) { + return null; + } else { + long ts = l - offset; + int tzOffset = TimeZone.getDefault().getOffset(ts); + return new Timestamp(ts - tzOffset); + } + } + + @Override + public TypeInformation getResultType(Class[] signature) { + return Types.SQL_TIMESTAMP; + } + + @Override + public void close() { + closeCalled = true; + } + } + /** * Testing open method is called. */ diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java index a94d5412b990..826b9b328b87 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java @@ -24,6 +24,8 @@ import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.catalog.CatalogFunction; +import org.apache.flink.table.catalog.CatalogFunctionImpl; import org.apache.flink.table.catalog.CatalogManager; import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.CatalogTableImpl; @@ -31,6 +33,7 @@ import org.apache.flink.table.catalog.GenericInMemoryCatalog; import org.apache.flink.table.catalog.ObjectPath; import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.FunctionAlreadyExistException; import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; import org.apache.flink.table.catalog.exceptions.TableNotExistException; import org.apache.flink.table.module.ModuleManager; @@ -43,6 +46,7 @@ import org.apache.flink.table.planner.delegation.PlannerContext; import org.apache.flink.table.planner.operations.SqlConversionException; import org.apache.flink.table.planner.operations.SqlToOperationConverter; +import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions; import org.apache.flink.table.types.DataType; import org.apache.calcite.sql.SqlNode; @@ -54,6 +58,7 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -188,6 +193,45 @@ public void testCreateTableWithMinusInOptionKey() { assertEquals(expected, sortedProperties.toString()); } + @Test + public void testCreateTableWithWatermark() throws FunctionAlreadyExistException, DatabaseNotExistException { + CatalogFunction cf = new CatalogFunctionImpl( + JavaUserDefinedScalarFunctions.JavaFunc5.class.getName(), + Collections.emptyMap()); + catalog.createFunction(ObjectPath.fromString("default.myfunc"), cf, true); + + final String sql = "create table source_table(\n" + + " a int,\n" + + " b bigint,\n" + + " c timestamp(3),\n" + + " watermark for `c` as myfunc(c, 1) - interval '5' second\n" + + ") with (\n" + + " 'connector.type' = 'kafka')\n"; + final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); + final CalciteParser parser = getParserBySqlDialect(SqlDialect.DEFAULT); + SqlNode node = parser.parse(sql); + assert node instanceof SqlCreateTable; + Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node).get(); + assert operation instanceof CreateTableOperation; + CreateTableOperation op = (CreateTableOperation) operation; + CatalogTable catalogTable = op.getCatalogTable(); + Map properties = catalogTable.toProperties(); + Map expected = new HashMap<>(); + expected.put("schema.0.name", "a"); + expected.put("schema.0.type", "INT"); + expected.put("schema.1.name", "b"); + expected.put("schema.1.type", "BIGINT"); + expected.put("schema.2.name", "c"); + expected.put("schema.2.type", "TIMESTAMP"); + expected.put("schema.watermark.0.rowtime", "c"); + expected.put( + "schema.watermark.0.strategy.expr", + "`builtin`.`default`.`myfunc`(`c`, 1) - INTERVAL '5' SECOND"); + expected.put("schema.watermark.0.strategy.datatype", "TIMESTAMP(3)"); + expected.put("connector.type", "kafka"); + assertEquals(expected, properties); + } + @Test public void testSqlInsertWithStaticPartition() { final String sql = "insert into t1 partition(a=1) select b, c, d from t2"; From d98ad19e030fb4b6c3e26fd2b96726e2be77ef5c Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Wed, 6 Nov 2019 14:15:48 +0800 Subject: [PATCH 522/746] [FLINK-14326][table-planner-blink] Introduce WatermarkGenerator interface to generate watermark from current row --- .../stream/StreamExecWatermarkAssigner.scala | 8 +- .../runtime/generated/WatermarkGenerator.java | 41 +++++++ .../BoundedOutOfOrderWatermarkGenerator.java | 51 ++++++++ .../WatermarkAssignerOperator.java | 64 ++++++---- .../WatermarkAssignerOperatorTest.java | 113 +++++++++++++++++- .../WatermarkAssignerOperatorTestBase.java | 2 +- 6 files changed, 248 insertions(+), 31 deletions(-) create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/WatermarkGenerator.java create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/BoundedOutOfOrderWatermarkGenerator.java diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecWatermarkAssigner.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecWatermarkAssigner.scala index 871d54ee1254..56391da123f0 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecWatermarkAssigner.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecWatermarkAssigner.scala @@ -29,10 +29,9 @@ import org.apache.flink.table.planner.plan.`trait`.{MiniBatchIntervalTraitDef, M import org.apache.flink.table.planner.plan.nodes.calcite.WatermarkAssigner import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, StreamExecNode} import org.apache.flink.table.planner.utils.TableConfigUtils.getMillisecondFromConfigDuration -import org.apache.flink.table.runtime.operators.wmassigners.{MiniBatchAssignerOperator, MiniBatchedWatermarkAssignerOperator, WatermarkAssignerOperator} +import org.apache.flink.table.runtime.operators.wmassigners.{BoundedOutOfOrderWatermarkGenerator, MiniBatchAssignerOperator, MiniBatchedWatermarkAssignerOperator, WatermarkAssignerOperator} import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo import org.apache.flink.util.Preconditions - import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.{RelNode, RelWriter} @@ -126,7 +125,10 @@ class StreamExecWatermarkAssigner( // 1. redundant watermark definition in DDL // 2. existing window aggregate // 3. operator requiring watermark, but minibatch is not enabled - new WatermarkAssignerOperator(rowtimeFieldIndex.get, watermarkDelay.get, idleTimeout) + new WatermarkAssignerOperator( + rowtimeFieldIndex.get, + new BoundedOutOfOrderWatermarkGenerator(rowtimeFieldIndex.get, watermarkDelay.get), + idleTimeout) } else if (inferredInterval.mode == MiniBatchMode.ProcTime) { new MiniBatchAssignerOperator(inferredInterval.interval) } else { diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/WatermarkGenerator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/WatermarkGenerator.java new file mode 100644 index 000000000000..c2fc75626125 --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/WatermarkGenerator.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.table.runtime.generated; + +import org.apache.flink.api.common.functions.AbstractRichFunction; +import org.apache.flink.table.dataformat.BaseRow; + +import javax.annotation.Nullable; + +/** + * The {@link WatermarkGenerator} is used to generate watermark based the input elements. + */ +public abstract class WatermarkGenerator extends AbstractRichFunction { + + private static final long serialVersionUID = 1L; + + /** + * Returns the watermark for the current row or null if no watermark should be generated. + * + * @param row The current row. + * @return The watermark for this row or null if no watermark should be generated. + */ + @Nullable + public abstract Long currentWatermark(BaseRow row) throws Exception; +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/BoundedOutOfOrderWatermarkGenerator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/BoundedOutOfOrderWatermarkGenerator.java new file mode 100644 index 000000000000..30421c194e96 --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/BoundedOutOfOrderWatermarkGenerator.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.table.runtime.operators.wmassigners; + +import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.table.runtime.generated.WatermarkGenerator; + +import javax.annotation.Nullable; + +/** + * A watermark generator for rowtime attributes which are out-of-order by a bounded time interval. + * + *

    Emits watermarks which are the observed timestamp minus the specified delay. + */ +public class BoundedOutOfOrderWatermarkGenerator extends WatermarkGenerator { + + private static final long serialVersionUID = 1L; + private final long delay; + private final int rowtimeIndex; + + /** + * @param rowtimeIndex the field index of rowtime attribute, the value of rowtime should never be null. + * @param delay The delay by which watermarks are behind the observed timestamp. + */ + public BoundedOutOfOrderWatermarkGenerator(int rowtimeIndex, long delay) { + this.delay = delay; + this.rowtimeIndex = rowtimeIndex; + } + + @Nullable + @Override + public Long currentWatermark(BaseRow row) { + return row.getLong(rowtimeIndex) - delay; + } +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperator.java index 049790f18419..d924602f140e 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperator.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperator.java @@ -18,7 +18,8 @@ package org.apache.flink.table.runtime.operators.wmassigners; -import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.api.common.functions.util.FunctionUtils; +import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; @@ -28,6 +29,7 @@ import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeCallback; import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.table.runtime.generated.WatermarkGenerator; /** * A stream operator that extracts timestamps from stream elements and @@ -41,29 +43,32 @@ public class WatermarkAssignerOperator private final int rowtimeFieldIndex; - private final long watermarkDelay; - private final long idleTimeout; + private final WatermarkGenerator watermarkGenerator; + + private transient long lastWatermark; + private transient long watermarkInterval; private transient long currentWatermark; - private transient long currentMaxTimestamp; - private transient long lastRecordTime; private transient StreamStatusMaintainer streamStatusMaintainer; + /** Flag to prevent duplicate function.close() calls in close() and dispose(). */ + private transient boolean functionsClosed = false; + /** * Create a watermark assigner operator. * @param rowtimeFieldIndex the field index to extract event timestamp - * @param watermarkDelay the delay by which watermarks are behind the maximum observed timestamp. + * @param watermarkGenerator the watermark generator * @param idleTimeout (idleness checking timeout) */ - public WatermarkAssignerOperator(int rowtimeFieldIndex, long watermarkDelay, long idleTimeout) { + public WatermarkAssignerOperator(int rowtimeFieldIndex, WatermarkGenerator watermarkGenerator, long idleTimeout) { this.rowtimeFieldIndex = rowtimeFieldIndex; - this.watermarkDelay = watermarkDelay; + this.watermarkGenerator = watermarkGenerator; this.idleTimeout = idleTimeout; this.chainingStrategy = ChainingStrategy.ALWAYS; @@ -75,7 +80,6 @@ public void open() throws Exception { // watermark and timestamp should start from 0 this.currentWatermark = 0; - this.currentMaxTimestamp = 0; this.watermarkInterval = getExecutionConfig().getAutoWatermarkInterval(); this.lastRecordTime = getProcessingTimeService().getCurrentProcessingTime(); this.streamStatusMaintainer = getContainingTask().getStreamStatusMaintainer(); @@ -84,6 +88,9 @@ public void open() throws Exception { long now = getProcessingTimeService().getCurrentProcessingTime(); getProcessingTimeService().registerTimer(now + watermarkInterval, this); } + + FunctionUtils.setFunctionRuntimeContext(watermarkGenerator, getRuntimeContext()); + FunctionUtils.openFunction(watermarkGenerator, new Configuration()); } @Override @@ -98,24 +105,25 @@ public void processElement(StreamRecord element) throws Exception { throw new RuntimeException("RowTime field should not be null," + " please convert it to a non-null long value."); } - long ts = row.getLong(rowtimeFieldIndex); - currentMaxTimestamp = Math.max(currentMaxTimestamp, ts); + Long watermark = watermarkGenerator.currentWatermark(row); + if (watermark != null) { + currentWatermark = Math.max(currentWatermark, watermark); + } // forward element output.collect(element); - // eagerly emit watermark to avoid period timer not called - // current_ts - last_ts > interval - if (currentMaxTimestamp - (currentWatermark + watermarkDelay) > watermarkInterval) { + // eagerly emit watermark to avoid period timer not called (this often happens when cpu load is high) + // current_wm - last_wm > interval + if (currentWatermark - lastWatermark > watermarkInterval) { advanceWatermark(); } } private void advanceWatermark() { - long newWatermark = currentMaxTimestamp - watermarkDelay; - if (newWatermark > currentWatermark) { - currentWatermark = newWatermark; + if (currentWatermark > lastWatermark) { + lastWatermark = currentWatermark; // emit watermark - output.emitWatermark(new Watermark(newWatermark)); + output.emitWatermark(new Watermark(currentWatermark)); } } @@ -138,8 +146,7 @@ public void onProcessingTime(long timestamp) throws Exception { /** * Override the base implementation to completely ignore watermarks propagated from - * upstream (we rely only on the {@link AssignerWithPeriodicWatermarks} to emit - * watermarks from here). + * upstream (we rely only on the {@link WatermarkGenerator} to emit watermarks from here). */ @Override public void processWatermark(Watermark mark) throws Exception { @@ -155,16 +162,23 @@ public void processWatermark(Watermark mark) throws Exception { } } - public void endInput() throws Exception { - processWatermark(Watermark.MAX_WATERMARK); - } - @Override public void close() throws Exception { - endInput(); // TODO after introduce endInput super.close(); // emit a final watermark advanceWatermark(); + + functionsClosed = true; + FunctionUtils.closeFunction(watermarkGenerator); + } + + @Override + public void dispose() throws Exception { + super.dispose(); + if (!functionsClosed) { + functionsClosed = true; + FunctionUtils.closeFunction(watermarkGenerator); + } } } diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperatorTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperatorTest.java index 820dc980c9b9..28f1d41b7b1c 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperatorTest.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperatorTest.java @@ -19,31 +19,39 @@ package org.apache.flink.table.runtime.operators.wmassigners; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.streamstatus.StreamStatus; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.dataformat.BaseRow; import org.apache.flink.table.dataformat.GenericRow; +import org.apache.flink.table.runtime.generated.WatermarkGenerator; import org.junit.Test; +import javax.annotation.Nullable; + +import java.util.ArrayList; import java.util.List; import java.util.concurrent.ConcurrentLinkedQueue; import static junit.framework.TestCase.assertTrue; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.fail; /** * Tests of {@link WatermarkAssignerOperator}. */ public class WatermarkAssignerOperatorTest extends WatermarkAssignerOperatorTestBase { + private static final WatermarkGenerator WATERMARK_GENERATOR = new BoundedOutOfOrderWatermarkGenerator(0, 1); + @Test public void testWatermarkAssignerWithIdleSource() throws Exception { // with timeout 1000 ms - final WatermarkAssignerOperator operator = new WatermarkAssignerOperator(0, 1, 1000); + final WatermarkAssignerOperator operator = new WatermarkAssignerOperator(0, WATERMARK_GENERATOR, 1000); OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness<>(operator); testHarness.getExecutionConfig().setAutoWatermarkInterval(50); @@ -83,7 +91,7 @@ public void testWatermarkAssignerWithIdleSource() throws Exception { @Test public void testWatermarkAssignerOperator() throws Exception { - final WatermarkAssignerOperator operator = new WatermarkAssignerOperator(0, 1, -1); + final WatermarkAssignerOperator operator = new WatermarkAssignerOperator(0, WATERMARK_GENERATOR, -1); OneInputStreamOperatorTestHarness testHarness = new OneInputStreamOperatorTestHarness<>(operator); @@ -162,4 +170,105 @@ public void testWatermarkAssignerOperator() throws Exception { assertEquals(Long.MAX_VALUE, ((Watermark) testHarness.getOutput().poll()).getTimestamp()); } + @Test + public void testCustomizedWatermarkGenerator() throws Exception { + MyWatermarkGenerator.openCalled = false; + MyWatermarkGenerator.closeCalled = false; + WatermarkGenerator generator = new MyWatermarkGenerator(1); + WatermarkAssignerOperator operator = new WatermarkAssignerOperator(0, generator, -1); + + OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(operator); + + testHarness.getExecutionConfig().setAutoWatermarkInterval(5); + + long currentTime = 0; + List expected = new ArrayList<>(); + + testHarness.open(); + + testHarness.processElement(new StreamRecord<>(GenericRow.of(1L, 0L))); + testHarness.processElement(new StreamRecord<>(GenericRow.of(2L, 1L))); + testHarness.processWatermark(new Watermark(2)); // this watermark should be ignored + testHarness.processElement(new StreamRecord<>(GenericRow.of(3L, 1L))); + currentTime = currentTime + 5; + testHarness.setProcessingTime(currentTime); + expected.add(new Watermark(1L)); + + testHarness.processElement(new StreamRecord<>(GenericRow.of(4L, 2L))); + testHarness.processElement(new StreamRecord<>(GenericRow.of(2L, 1L))); + testHarness.processElement(new StreamRecord<>(GenericRow.of(1L, 0L))); + testHarness.processElement(new StreamRecord<>(GenericRow.of(6L, null))); + currentTime = currentTime + 5; + testHarness.setProcessingTime(currentTime); + expected.add(new Watermark(2L)); + + testHarness.processElement(new StreamRecord<>(GenericRow.of(9L, 8L))); + expected.add(new Watermark(8L)); + + // no watermark output + testHarness.processElement(new StreamRecord<>(GenericRow.of(8L, 7L))); + testHarness.processElement(new StreamRecord<>(GenericRow.of(10L, null))); + testHarness.processElement(new StreamRecord<>(GenericRow.of(11L, 10L))); + currentTime = currentTime + 5; + testHarness.setProcessingTime(currentTime); + expected.add(new Watermark(10L)); + + testHarness.close(); + + // num_watermark + num_records + assertEquals(expected.size() + 11, testHarness.getOutput().size()); + List results = extractWatermarks(testHarness.getOutput()); + assertEquals(expected, results); + assertTrue(MyWatermarkGenerator.openCalled); + assertTrue(MyWatermarkGenerator.closeCalled); + } + + /** + * The special watermark generator will generate null watermarks and + * also checks open&close are called. + */ + private static final class MyWatermarkGenerator extends WatermarkGenerator { + + private static final long serialVersionUID = 1L; + private static boolean openCalled = false; + private static boolean closeCalled = false; + + private final int watermarkFieldIndex; + + private MyWatermarkGenerator(int watermarkFieldIndex) { + this.watermarkFieldIndex = watermarkFieldIndex; + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + if (closeCalled) { + fail("Close called before open."); + } + openCalled = true; + } + + @Nullable + @Override + public Long currentWatermark(BaseRow row) throws Exception { + if (!openCalled) { + fail("Open was not called before run."); + } + if (row.isNullAt(watermarkFieldIndex)) { + return null; + } else { + return row.getLong(watermarkFieldIndex); + } + } + + @Override + public void close() throws Exception { + super.close(); + if (!openCalled) { + fail("Open was not called before close."); + } + closeCalled = true; + } + } } diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperatorTestBase.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperatorTestBase.java index cbaa1decf31e..cb98f550d878 100644 --- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperatorTestBase.java +++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperatorTestBase.java @@ -33,7 +33,7 @@ /** * Base class for watermark assigner operator test. */ -public class WatermarkAssignerOperatorTestBase { +public abstract class WatermarkAssignerOperatorTestBase { protected Tuple2 validateElement(Object element, long nextElementValue, long currentWatermark) { if (element instanceof StreamRecord) { From bf27bdd1ffe4f28b14f4ee14cc0863e8dc70422b Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Wed, 6 Nov 2019 14:16:30 +0800 Subject: [PATCH 523/746] [FLINK-14326][table-planner-blink] Support code generate a WatermarkGenerator from RexNode --- .../calcite/SqlExprToRexConverter.java | 38 ++++ .../calcite/SqlExprToRexConverterImpl.java | 166 ++++++++++++++++++ .../planner/calcite/FlinkPlannerImpl.scala | 19 +- .../WatermarkGeneratorCodeGenerator.scala | 89 ++++++++++ .../WatermarkGeneratorCodeGenTest.scala | 138 +++++++++++++++ .../GeneratedWatermarkGenerator.java | 31 ++++ 6 files changed, 475 insertions(+), 6 deletions(-) create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/calcite/SqlExprToRexConverter.java create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/calcite/SqlExprToRexConverterImpl.java create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/WatermarkGeneratorCodeGenerator.scala create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/codegen/WatermarkGeneratorCodeGenTest.scala create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedWatermarkGenerator.java diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/calcite/SqlExprToRexConverter.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/calcite/SqlExprToRexConverter.java new file mode 100644 index 000000000000..19524abefcc7 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/calcite/SqlExprToRexConverter.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.calcite; + +import org.apache.calcite.rex.RexNode; + +/** + * Converts SQL expressions to {@link RexNode}. + */ +public interface SqlExprToRexConverter { + + /** + * Converts a SQL expression to a {@link RexNode} expression. + */ + RexNode convertToRexNode(String expr); + + /** + * Converts an array of SQL expressions to an array of {@link RexNode} expressions. + */ + RexNode[] convertToRexNodes(String[] exprs); + +} diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/calcite/SqlExprToRexConverterImpl.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/calcite/SqlExprToRexConverterImpl.java new file mode 100644 index 000000000000..dbcfafe766ef --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/calcite/SqlExprToRexConverterImpl.java @@ -0,0 +1,166 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.calcite; + +import org.apache.flink.table.planner.plan.FlinkCalciteCatalogReader; + +import org.apache.calcite.config.CalciteConnectionConfig; +import org.apache.calcite.config.CalciteConnectionConfigImpl; +import org.apache.calcite.config.CalciteConnectionProperty; +import org.apache.calcite.jdbc.CalciteSchema; +import org.apache.calcite.jdbc.CalciteSchemaBuilder; +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.prepare.CalciteCatalogReader; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.TableScan; +import org.apache.calcite.rel.logical.LogicalProject; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.schema.Table; +import org.apache.calcite.schema.impl.AbstractSchema; +import org.apache.calcite.schema.impl.AbstractTable; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.tools.FrameworkConfig; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; + +/** + * Standard implementation of {@link SqlExprToRexConverter}. + */ +public class SqlExprToRexConverterImpl implements SqlExprToRexConverter { + + private static final String TEMPORARY_TABLE_NAME = "__temp_table__"; + private static final String QUERY_FORMAT = "SELECT %s FROM " + TEMPORARY_TABLE_NAME; + + private final FlinkPlannerImpl planner; + + public SqlExprToRexConverterImpl( + FrameworkConfig config, + FlinkTypeFactory typeFactory, + RelOptCluster cluster, + RelDataType tableRowType) { + this.planner = new FlinkPlannerImpl( + config, + (isLenient) -> createSingleTableCatalogReader(isLenient, config, typeFactory, tableRowType), + typeFactory, + cluster + ); + } + + @Override + public RexNode convertToRexNode(String expr) { + return convertToRexNodes(new String[]{expr})[0]; + } + + @Override + public RexNode[] convertToRexNodes(String[] exprs) { + String query = String.format(QUERY_FORMAT, String.join(",", exprs)); + SqlNode parsed = planner.parser().parse(query); + SqlNode validated = planner.validate(parsed); + RelNode rel = planner.rel(validated).rel; + // The plan should in the following tree + // LogicalProject + // +- TableScan + if (rel instanceof LogicalProject + && rel.getInput(0) != null + && rel.getInput(0) instanceof TableScan) { + return ((LogicalProject) rel).getProjects().toArray(new RexNode[0]); + } else { + throw new IllegalStateException("The root RelNode should be LogicalProject, but is " + rel.toString()); + } + } + + // ------------------------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------------------------ + + /** + * Creates a catalog reader that contains a single {@link Table} with temporary table name + * and specified {@code rowType}. + * + * @param rowType table row type + * @return the {@link CalciteCatalogReader} instance + */ + private static CalciteCatalogReader createSingleTableCatalogReader( + boolean lenientCaseSensitivity, + FrameworkConfig config, + FlinkTypeFactory typeFactory, + RelDataType rowType) { + + // connection properties + boolean caseSensitive = !lenientCaseSensitivity && config.getParserConfig().caseSensitive(); + Properties properties = new Properties(); + properties.put( + CalciteConnectionProperty.CASE_SENSITIVE.camelName(), + String.valueOf(caseSensitive)); + CalciteConnectionConfig connectionConfig = new CalciteConnectionConfigImpl(properties); + + // prepare root schema + final RowTypeSpecifiedTable table = new RowTypeSpecifiedTable(rowType); + final Map tableMap = Collections.singletonMap(TEMPORARY_TABLE_NAME, table); + CalciteSchema schema = CalciteSchemaBuilder.asRootSchema(new TableSpecifiedSchema(tableMap)); + + return new FlinkCalciteCatalogReader( + schema, + new ArrayList<>(new ArrayList<>()), + typeFactory, + connectionConfig); + } + + // ------------------------------------------------------------------------------------------ + // Inner Class + // ------------------------------------------------------------------------------------------ + + /** + * A {@link AbstractTable} that can specify the row type explicitly. + */ + private static class RowTypeSpecifiedTable extends AbstractTable { + private final RelDataType rowType; + + RowTypeSpecifiedTable(RelDataType rowType) { + this.rowType = Objects.requireNonNull(rowType); + } + + @Override + public RelDataType getRowType(RelDataTypeFactory typeFactory) { + return this.rowType; + } + } + + /** + * A {@link AbstractSchema} that can specify the table map explicitly. + */ + private static class TableSpecifiedSchema extends AbstractSchema { + private final Map tableMap; + + TableSpecifiedSchema(Map tableMap) { + this.tableMap = Objects.requireNonNull(tableMap); + } + + @Override + protected Map getTableMap() { + return tableMap; + } + } +} diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala index 6e6d45ae0ade..064ccf3e0a5c 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala @@ -28,7 +28,6 @@ import org.apache.calcite.prepare.CalciteCatalogReader import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.{RelFieldCollation, RelRoot} import org.apache.calcite.sql.advise.{SqlAdvisor, SqlAdvisorValidator} -import org.apache.calcite.sql.validate.SqlValidator import org.apache.calcite.sql.{SqlKind, SqlNode, SqlOperatorTable} import org.apache.calcite.sql2rel.{RelDecorrelator, SqlRexConvertletTable, SqlToRelConverter} import org.apache.calcite.tools.{FrameworkConfig, RelConversionException} @@ -144,6 +143,14 @@ class FlinkPlannerImpl( } } + /** + * Creates a new instance of [[SqlExprToRexConverter]] to convert SQL expression + * to RexNode. + */ + def createSqlExprToRexConverter(tableRowType: RelDataType): SqlExprToRexConverter = { + new SqlExprToRexConverterImpl(config, typeFactory, cluster, tableRowType) + } + /** Implements [[org.apache.calcite.plan.RelOptTable.ViewExpander]] * interface for [[org.apache.calcite.tools.Planner]]. */ class ViewExpanderImpl extends ViewExpander { @@ -154,14 +161,14 @@ class FlinkPlannerImpl( schemaPath: util.List[String], viewPath: util.List[String]): RelRoot = { - val sqlNode: SqlNode = parser.parse(queryString) - val catalogReader: CalciteCatalogReader = catalogReaderSupplier.apply(false) + val sqlNode = parser.parse(queryString) + val catalogReader = catalogReaderSupplier.apply(false) .withSchemaPath(schemaPath) - val validator: SqlValidator = + val validator = new FlinkCalciteSqlValidator(operatorTable, catalogReader, typeFactory) validator.setIdentifierExpansion(true) - val validatedSqlNode: SqlNode = validator.validate(sqlNode) - val sqlToRelConverter: SqlToRelConverter = new SqlToRelConverter( + val validatedSqlNode = validator.validate(sqlNode) + val sqlToRelConverter = new SqlToRelConverter( new ViewExpanderImpl, validator, catalogReader, diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/WatermarkGeneratorCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/WatermarkGeneratorCodeGenerator.scala new file mode 100644 index 000000000000..35bed3892e95 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/WatermarkGeneratorCodeGenerator.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.codegen + +import org.apache.flink.configuration.Configuration +import org.apache.flink.table.api.TableConfig +import org.apache.flink.table.planner.calcite.FlinkTypeFactory +import org.apache.flink.table.planner.codegen.CodeGenUtils.{BASE_ROW, newName} +import org.apache.flink.table.planner.codegen.Indenter.toISC +import org.apache.flink.table.runtime.generated.{GeneratedWatermarkGenerator, WatermarkGenerator} +import org.apache.flink.table.types.logical.{LogicalTypeRoot, RowType} +import org.apache.calcite.rex.RexNode + +/** + * A code generator for generating [[WatermarkGenerator]]s. + */ +object WatermarkGeneratorCodeGenerator { + + def generateWatermarkGenerator( + config: TableConfig, + inputType: RowType, + watermarkExpr: RexNode): GeneratedWatermarkGenerator = { + // validation + val watermarkOutputType = FlinkTypeFactory.toLogicalType(watermarkExpr.getType) + if (watermarkOutputType.getTypeRoot != LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE) { + throw new CodeGenException( + "WatermarkGenerator only accepts output data type of TIMESTAMP," + + " but is " + watermarkOutputType) + } + val funcName = newName("WatermarkGenerator") + val ctx = CodeGeneratorContext(config) + val generator = new ExprCodeGenerator(ctx, false) + .bindInput(inputType, inputTerm = "row") + val generatedExpr = generator.generateExpression(watermarkExpr) + + val funcCode = + j""" + public final class $funcName + extends ${classOf[WatermarkGenerator].getCanonicalName} { + + ${ctx.reuseMemberCode()} + + public $funcName(Object[] references) throws Exception { + ${ctx.reuseInitCode()} + } + + @Override + public void open(${classOf[Configuration].getCanonicalName} parameters) throws Exception { + ${ctx.reuseOpenCode()} + } + + @Override + public Long currentWatermark($BASE_ROW row) throws Exception { + ${ctx.reusePerRecordCode()} + ${ctx.reuseLocalVariableCode()} + ${ctx.reuseInputUnboxingCode()} + ${generatedExpr.code} + if (${generatedExpr.nullTerm}) { + return null; + } else { + return ${generatedExpr.resultTerm}; + } + } + + @Override + public void close() throws Exception { + ${ctx.reuseCloseCode()} + } + } + """.stripMargin + new GeneratedWatermarkGenerator(funcName, funcCode, ctx.references.toArray) + } +} diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/codegen/WatermarkGeneratorCodeGenTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/codegen/WatermarkGeneratorCodeGenTest.scala new file mode 100644 index 000000000000..dad1c7ddd321 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/codegen/WatermarkGeneratorCodeGenTest.scala @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.codegen + +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.util.MockStreamingRuntimeContext +import org.apache.flink.table.api.TableConfig +import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog, GenericInMemoryCatalog, ObjectIdentifier} +import org.apache.flink.table.dataformat.GenericRow +import org.apache.flink.table.module.ModuleManager +import org.apache.flink.table.planner.calcite.{FlinkPlannerImpl, FlinkTypeFactory} +import org.apache.flink.table.planner.catalog.CatalogManagerCalciteSchema +import org.apache.flink.table.planner.delegation.PlannerContext +import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.JavaFunc5 +import org.apache.flink.table.runtime.generated.WatermarkGenerator +import org.apache.flink.table.types.logical.{IntType, TimestampType} +import org.apache.calcite.jdbc.CalciteSchemaBuilder.asRootSchema + +import org.junit.Assert.{assertEquals, assertTrue} +import org.junit.Test + +import java.lang.{Integer => JInt, Long => JLong} +import java.util.Collections + +/** + * Tests the generated [[WatermarkGenerator]] from [[WatermarkGeneratorCodeGenerator]]. + */ +class WatermarkGeneratorCodeGenTest { + + // mock FlinkPlannerImpl to avoid discovering TableEnvironment and Executor. + val catalog = new GenericInMemoryCatalog("MockCatalog", "default") + val catalogManager = new CatalogManager("builtin", catalog) + val functionCatalog = new FunctionCatalog(catalogManager, new ModuleManager) + val plannerContext = new PlannerContext( + new TableConfig, + functionCatalog, + catalogManager, + asRootSchema(new CatalogManagerCalciteSchema(catalogManager, false)), + Collections.emptyList()) + val planner: FlinkPlannerImpl = plannerContext.createFlinkPlanner( + catalogManager.getCurrentCatalog, + catalogManager.getCurrentDatabase) + + val data = List( + GenericRow.of(JLong.valueOf(1000L), JInt.valueOf(5)), + GenericRow.of(null, JInt.valueOf(4)), + GenericRow.of(JLong.valueOf(3000L), null), + GenericRow.of(JLong.valueOf(5000L), JInt.valueOf(3)), + GenericRow.of(JLong.valueOf(4000L), JInt.valueOf(10)), + GenericRow.of(JLong.valueOf(6000L), JInt.valueOf(8)) + ) + + @Test + def testAscendingWatermark(): Unit = { + val generator = generateWatermarkGenerator("ts - INTERVAL '0.001' SECOND") + val results = data.map(d => generator.currentWatermark(d)) + val expected = List( + JLong.valueOf(999L), + null, + JLong.valueOf(2999), + JLong.valueOf(4999), + JLong.valueOf(3999), + JLong.valueOf(5999)) + assertEquals(expected, results) + } + + @Test + def testBoundedOutOfOrderWatermark(): Unit = { + val generator = generateWatermarkGenerator("ts - INTERVAL '5' SECOND") + val results = data.map(d => generator.currentWatermark(d)) + val expected = List( + JLong.valueOf(-4000L), + null, + JLong.valueOf(-2000L), + JLong.valueOf(0L), + JLong.valueOf(-1000L), + JLong.valueOf(1000L)) + assertEquals(expected, results) + } + + @Test + def testCustomizedWatermark(): Unit = { + JavaFunc5.openCalled = false + JavaFunc5.closeCalled = false + functionCatalog.registerTempCatalogScalarFunction( + ObjectIdentifier.of("builtin", "default", "myFunc"), + new JavaFunc5 + ) + val generator = generateWatermarkGenerator("myFunc(ts, `offset`)") + // mock open and close invoking + generator.setRuntimeContext(new MockStreamingRuntimeContext(false, 1, 1)) + generator.open(new Configuration()) + val results = data.map(d => generator.currentWatermark(d)) + generator.close() + val expected = List( + JLong.valueOf(995L), + null, + null, + JLong.valueOf(4997L), + JLong.valueOf(3990L), + JLong.valueOf(5992L)) + assertEquals(expected, results) + assertTrue(JavaFunc5.openCalled) + assertTrue(JavaFunc5.closeCalled) + } + + private def generateWatermarkGenerator(expr: String): WatermarkGenerator = { + val tableRowType = plannerContext.getTypeFactory.buildRelNodeRowType( + Seq("ts", "offset"), + Seq( + new TimestampType(3), + new IntType() + )) + val rowType = FlinkTypeFactory.toLogicalRowType(tableRowType) + val converter = planner.createSqlExprToRexConverter(tableRowType) + val rexNode = converter.convertToRexNode(expr) + val generated = WatermarkGeneratorCodeGenerator + .generateWatermarkGenerator(new TableConfig(), rowType, rexNode) + generated.newInstance(Thread.currentThread().getContextClassLoader) + } + +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedWatermarkGenerator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedWatermarkGenerator.java new file mode 100644 index 000000000000..18bd45a5b543 --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedWatermarkGenerator.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.generated; + +/** + * Describes a generated {@link WatermarkGenerator}. + */ +public class GeneratedWatermarkGenerator extends GeneratedClass { + + private static final long serialVersionUID = 1L; + + public GeneratedWatermarkGenerator(String className, String code, Object[] references) { + super(className, code, references); + } +} From d0901dc70205d171fe0a777f99e800929e2231a6 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Wed, 6 Nov 2019 14:18:26 +0800 Subject: [PATCH 524/746] [FLINK-14326][table-planner-blink] Support to generate and apply watermark assigner in translateToPlan --- .../apache/flink/table/api/WatermarkSpec.java | 6 + .../physical/PhysicalTableSourceScan.scala | 8 +- .../stream/StreamExecTableSourceScan.scala | 111 +++++++++--- .../plan/schema/TableSourceTable.scala | 43 ++++- .../planner/sources/TableSourceUtil.scala | 28 ++- .../planner/plan/batch/sql/TableScanTest.xml | 24 +++ .../planner/plan/stream/sql/TableScanTest.xml | 37 +++- .../plan/batch/sql/TableScanTest.scala | 20 ++- .../plan/stream/sql/TableScanTest.scala | 19 +- .../stream/sql/TimeAttributeITCase.scala | 163 ++++++++++++++++++ .../table/planner/utils/TableTestBase.scala | 17 +- .../WatermarkAssignerOperatorFactory.java | 77 +++++++++ 12 files changed, 496 insertions(+), 57 deletions(-) create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TimeAttributeITCase.scala create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperatorFactory.java diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/WatermarkSpec.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/WatermarkSpec.java index f997badbfbe0..a3b22d2f7d00 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/WatermarkSpec.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/WatermarkSpec.java @@ -89,4 +89,10 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(rowtimeAttribute, watermarkExpressionString, watermarkExprOutputType); } + + @Override + public String toString() { + return "rowtime: '" + rowtimeAttribute + '\'' + + ", watermark: '" + watermarkExpressionString + '\''; + } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/PhysicalTableSourceScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/PhysicalTableSourceScan.scala index 63aa63a8492e..49ac5773a0a0 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/PhysicalTableSourceScan.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/PhysicalTableSourceScan.scala @@ -58,7 +58,13 @@ abstract class PhysicalTableSourceScan( } override def explainTerms(pw: RelWriter): RelWriter = { - super.explainTerms(pw).item("fields", getRowType.getFieldNames.asScala.mkString(", ")) + val hasWatermark = tableSourceTable.watermarkSpec.isDefined && tableSourceTable.isStreamingMode + val rowtime = tableSourceTable.watermarkSpec.map(_.getRowtimeAttribute).orNull + val watermark = tableSourceTable.watermarkSpec.map(_.getWatermarkExpressionString).orNull + super.explainTerms(pw) + .item("fields", getRowType.getFieldNames.asScala.mkString(", ")) + .itemIf("rowtime", rowtime, hasWatermark) + .itemIf("watermark", watermark, hasWatermark) } def createInput[IN]( diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecTableSourceScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecTableSourceScan.scala index f44c8daf9260..92ce7b48192c 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecTableSourceScan.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecTableSourceScan.scala @@ -26,10 +26,10 @@ import org.apache.flink.streaming.api.datastream.DataStream import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import org.apache.flink.streaming.api.functions.{AssignerWithPeriodicWatermarks, AssignerWithPunctuatedWatermarks} import org.apache.flink.streaming.api.watermark.Watermark -import org.apache.flink.table.api.{DataTypes, TableException} +import org.apache.flink.table.api.{DataTypes, TableConfig, TableException, WatermarkSpec} import org.apache.flink.table.dataformat.DataFormatConverters.DataFormatConverter import org.apache.flink.table.dataformat.{BaseRow, DataFormatConverters} -import org.apache.flink.table.planner.codegen.CodeGeneratorContext +import org.apache.flink.table.planner.codegen.{CodeGeneratorContext, WatermarkGeneratorCodeGenerator} import org.apache.flink.table.planner.codegen.OperatorCodeGenerator._ import org.apache.flink.table.planner.delegation.StreamPlanner import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, StreamExecNode} @@ -43,11 +43,14 @@ import org.apache.flink.table.sources.wmstrategies.{PeriodicWatermarkAssigner, P import org.apache.flink.table.sources.{RowtimeAttributeDescriptor, StreamTableSource} import org.apache.flink.table.types.{DataType, FieldsDataType} import org.apache.flink.types.Row - import org.apache.calcite.plan._ import org.apache.calcite.rel.RelNode import org.apache.calcite.rel.metadata.RelMetadataQuery import org.apache.calcite.rex.RexNode +import org.apache.flink.table.planner.calcite.{FlinkTypeFactory, SqlExprToRexConverter} +import org.apache.flink.table.runtime.operators.wmassigners.WatermarkAssignerOperatorFactory +import org.apache.flink.table.types.logical.RowType +import org.apache.flink.table.types.utils.TypeConversions import java.util @@ -153,33 +156,91 @@ class StreamExecTableSourceScan( val ingestedTable = new DataStream(planner.getExecEnv, streamTransformation) - // generate watermarks for rowtime indicator - val rowtimeDesc: Option[RowtimeAttributeDescriptor] = - TableSourceUtil.getRowtimeAttributeDescriptor(tableSource, tableSourceTable.selectedFields) - - val withWatermarks = if (rowtimeDesc.isDefined) { - val rowtimeFieldIdx = getRowType.getFieldNames.indexOf(rowtimeDesc.get.getAttributeName) - val watermarkStrategy = rowtimeDesc.get.getWatermarkStrategy - watermarkStrategy match { - case p: PeriodicWatermarkAssigner => - val watermarkGenerator = new PeriodicWatermarkAssignerWrapper(rowtimeFieldIdx, p) - ingestedTable.assignTimestampsAndWatermarks(watermarkGenerator) - case p: PunctuatedWatermarkAssigner => - val watermarkGenerator = - new PunctuatedWatermarkAssignerWrapper(rowtimeFieldIdx, p, producedDataType) - ingestedTable.assignTimestampsAndWatermarks(watermarkGenerator) - case _: PreserveWatermarks => - // The watermarks have already been provided by the underlying DataStream. - ingestedTable - } + val withWatermarks = if (tableSourceTable.watermarkSpec.isDefined) { + // generate watermarks for rowtime indicator from WatermarkSpec + val rowType = tableSourceTable.getRowType(planner.getTypeFactory) + val converter = planner.createFlinkPlanner.createSqlExprToRexConverter(rowType) + applyWatermarkByWatermarkSpec( + config, + ingestedTable, + FlinkTypeFactory.toLogicalRowType(rowType), + tableSourceTable.watermarkSpec.get, + converter) } else { - // No need to generate watermarks if no rowtime attribute is specified. - ingestedTable + val rowtimeDesc: Option[RowtimeAttributeDescriptor] = + TableSourceUtil.getRowtimeAttributeDescriptor(tableSource, tableSourceTable.selectedFields) + + // generate watermarks for rowtime indicator from DefinedRowtimeAttributes + if (rowtimeDesc.isDefined) { + applyWatermarkByRowtimeAttributeDescriptor( + ingestedTable, + rowtimeDesc.get, + producedDataType) + } else { + // No need to generate watermarks if no rowtime attribute is specified. + ingestedTable + } } + withWatermarks.getTransformation } - def needInternalConversion: Boolean = { + private def applyWatermarkByWatermarkSpec( + config: TableConfig, + input: DataStream[BaseRow], + inputType: RowType, + watermarkSpec: WatermarkSpec, + converter: SqlExprToRexConverter): DataStream[BaseRow] = { + // TODO: [FLINK-14473] support nested field as the rowtime attribute in the future + val rowtime = watermarkSpec.getRowtimeAttribute + if (rowtime.contains(".")) { + throw new TableException( + s"Nested field '$rowtime' as rowtime attribute is not supported right now.") + } + val rowtimeFieldIndex = getRowType.getFieldNames.indexOf(rowtime) + val watermarkExpr = converter.convertToRexNode(watermarkSpec.getWatermarkExpressionString) + val watermarkResultType = TypeConversions.fromLogicalToDataType( + FlinkTypeFactory.toLogicalType(watermarkExpr.getType)) + // check the derived datatype equals to the datatype in WatermarkSpec in TableSchema. + if (!watermarkResultType.equals(watermarkSpec.getWatermarkExprOutputType)) { + throw new TableException( + s"The derived data type '$watermarkResultType' of watermark expression doesn't equal to " + + s"the data type '${watermarkSpec.getWatermarkExprOutputType}' in WatermarkSpec. " + + "Please pass in correct result type of watermark expression when constructing " + + "TableSchema.") + } + val watermarkGenerator = WatermarkGeneratorCodeGenerator.generateWatermarkGenerator( + config, + inputType, + watermarkExpr) + val operatorFactory = new WatermarkAssignerOperatorFactory( + rowtimeFieldIndex, + 0L, + watermarkGenerator) + input.transform(s"WatermarkAssigner($watermarkSpec)", input.getType, operatorFactory) + } + + private def applyWatermarkByRowtimeAttributeDescriptor( + input: DataStream[BaseRow], + rowtimeDesc: RowtimeAttributeDescriptor, + producedDataType: DataType): DataStream[BaseRow] = { + val rowtimeFieldIdx = getRowType.getFieldNames.indexOf(rowtimeDesc.getAttributeName) + val watermarkStrategy = rowtimeDesc.getWatermarkStrategy + watermarkStrategy match { + case p: PeriodicWatermarkAssigner => + val watermarkGenerator = new PeriodicWatermarkAssignerWrapper(rowtimeFieldIdx, p) + input.assignTimestampsAndWatermarks(watermarkGenerator) + case p: PunctuatedWatermarkAssigner => + val watermarkGenerator = + new PunctuatedWatermarkAssignerWrapper(rowtimeFieldIdx, p, producedDataType) + input.assignTimestampsAndWatermarks(watermarkGenerator) + case _: PreserveWatermarks => + // The watermarks have already been provided by the underlying DataStream. + input + } + } + + private def needInternalConversion: Boolean = { val fieldIndexes = TableSourceUtil.computeIndexMapping( tableSource, isStreamTable = true, diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TableSourceTable.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TableSourceTable.scala index 14c577cc8d06..50f88dcd9e97 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TableSourceTable.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TableSourceTable.scala @@ -22,9 +22,13 @@ import org.apache.flink.table.catalog.CatalogTable import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.stats.FlinkStatistic import org.apache.flink.table.planner.sources.TableSourceUtil -import org.apache.flink.table.sources.TableSource +import org.apache.flink.table.sources.{TableSource, TableSourceValidation} import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory} import org.apache.flink.shaded.guava18.com.google.common.base.Preconditions +import org.apache.flink.table.api.{TableException, WatermarkSpec} +import org.apache.flink.table.types.logical.{TimestampKind, TimestampType} + +import scala.collection.JavaConverters._ /** * Abstract class which define the interfaces required to convert a [[TableSource]] to @@ -54,15 +58,46 @@ class TableSourceTable[T]( Preconditions.checkNotNull(statistic) Preconditions.checkNotNull(catalogTable) + val watermarkSpec: Option[WatermarkSpec] = catalogTable + .getSchema + // we only support single watermark currently + .getWatermarkSpecs.asScala.headOption + + if (TableSourceValidation.hasRowtimeAttribute(tableSource) && watermarkSpec.isDefined) { + throw new TableException( + "If watermark is specified in DDL, the underlying TableSource of connector shouldn't" + + " return an non-empty list of RowtimeAttributeDescriptor" + + " via DefinedRowtimeAttributes interface.") + } + // TODO implements this // TableSourceUtil.validateTableSource(tableSource) override def getRowType(typeFactory: RelDataTypeFactory): RelDataType = { - TableSourceUtil.getRelDataType( + val factory = typeFactory.asInstanceOf[FlinkTypeFactory] + val (fieldNames, fieldTypes) = TableSourceUtil.getFieldNamesTypes( tableSource, selectedFields, - streaming = isStreamingMode, - typeFactory.asInstanceOf[FlinkTypeFactory]) + streaming = isStreamingMode) + // patch rowtime field according to WatermarkSpec + val patchedTypes = if (isStreamingMode && watermarkSpec.isDefined) { + // TODO: [FLINK-14473] we only support top-level rowtime attribute right now + val rowtime = watermarkSpec.get.getRowtimeAttribute + if (rowtime.contains(".")) { + throw new TableException( + s"Nested field '$rowtime' as rowtime attribute is not supported right now.") + } + val idx = fieldNames.indexOf(rowtime) + val originalType = fieldTypes(idx).asInstanceOf[TimestampType] + val rowtimeType = new TimestampType( + originalType.isNullable, + TimestampKind.ROWTIME, + originalType.getPrecision) + fieldTypes.patch(idx, Seq(rowtimeType), 1) + } else { + fieldTypes + } + factory.buildRelNodeRowType(fieldNames, patchedTypes) } /** diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sources/TableSourceUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sources/TableSourceUtil.scala index a7be5613a2e8..514c08d424c8 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sources/TableSourceUtil.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sources/TableSourceUtil.scala @@ -33,7 +33,6 @@ import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTyp import org.apache.flink.table.sources.{DefinedFieldMapping, DefinedProctimeAttribute, DefinedRowtimeAttributes, RowtimeAttributeDescriptor, TableSource} import org.apache.flink.table.types.logical.{LogicalType, TimestampKind, TimestampType, TinyIntType} import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo - import com.google.common.collect.ImmutableList import org.apache.calcite.plan.RelOptCluster import org.apache.calcite.rel.RelNode @@ -135,19 +134,17 @@ object TableSourceUtil { } /** - * Returns the Calcite schema of a [[TableSource]]. + * Returns schema of the selected fields of the given [[TableSource]]. * * @param tableSource The [[TableSource]] for which the Calcite schema is generated. * @param selectedFields The indices of all selected fields. None, if all fields are selected. * @param streaming Flag to determine whether the schema of a stream or batch table is created. - * @param typeFactory The type factory to create the schema. - * @return The Calcite schema for the selected fields of the given [[TableSource]]. + * @return The schema for the selected fields of the given [[TableSource]]. */ - def getRelDataType( + def getFieldNamesTypes( tableSource: TableSource[_], selectedFields: Option[Array[Int]], - streaming: Boolean, - typeFactory: FlinkTypeFactory): RelDataType = { + streaming: Boolean): (Seq[String], Seq[LogicalType]) = { val fieldNames = tableSource.getTableSchema.getFieldNames var fieldTypes = tableSource.getTableSchema.getFieldDataTypes @@ -173,17 +170,12 @@ object TableSourceUtil { fieldTypes = fieldTypes.patch(idx, Seq(proctimeType), 1) } } - val (selectedFieldNames, selectedFieldTypes) = - if (selectedFields.isDefined) { - // filter field names and types by selected fields - ( - selectedFields.get.map(fieldNames(_)), - selectedFields.get.map(fieldTypes(_)) - ) - } else { - (fieldNames, fieldTypes) - } - typeFactory.buildRelNodeRowType(selectedFieldNames, selectedFieldTypes) + if (selectedFields.isDefined) { + // filter field names and types by selected fields + (selectedFields.get.map(fieldNames(_)), selectedFields.get.map(fieldTypes(_))) + } else { + (fieldNames, fieldTypes) + } } /** diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.xml index 2b2070c59bcf..1d53e32b0533 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.xml @@ -19,17 +19,41 @@ limitations under the License. + + + + + + + + 1]]> + + + + ($1, 1)]) + +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [CollectionTableSource(ts, a, b)]]]) +]]> + + + + (a, 1)]) ++- TableSourceScan(table=[[default_catalog, default_database, src, source: [CollectionTableSource(ts, a, b)]]], fields=[ts, a, b]) +]]> + diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml index 97e43b04aadb..1385cd0f10a9 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml @@ -16,36 +16,63 @@ See the License for the specific language governing permissions and limitations under the License. --> + + + + + + + + + + + + + + + + + - + - + 1]]> + ($1, 1)]) + +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [CollectionTableSource(ts, a, b)]]]) ]]> + (a, 1)]) ++- TableSourceScan(table=[[default_catalog, default_database, src, source: [CollectionTableSource(ts, a, b)]]], fields=[ts, a, b], rowtime=[ts], watermark=[`ts` - INTERVAL '0.001' SECOND]) ]]> + diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.scala index 7b79176646b2..edfbdbeb60ea 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.scala @@ -20,8 +20,8 @@ package org.apache.flink.table.planner.plan.batch.sql import org.apache.flink.api.scala._ import org.apache.flink.table.api.scala._ +import org.apache.flink.table.planner.factories.utils.TestCollectionTableFactory import org.apache.flink.table.planner.utils.TableTestBase - import org.junit.Test class TableScanTest extends TableTestBase { @@ -34,4 +34,22 @@ class TableScanTest extends TableTestBase { util.verifyPlan("SELECT * FROM MyTable") } + + @Test + def testDDLTableScan(): Unit = { + TestCollectionTableFactory.isStreaming = false + util.addTable( + """ + |CREATE TABLE src ( + | ts TIMESTAMP(3), + | a INT, + | b DOUBLE, + | WATERMARK FOR ts AS ts - INTERVAL '0.001' SECOND + |) WITH ( + | 'connector' = 'COLLECTION' + |) + """.stripMargin) + util.verifyPlan("SELECT * FROM src WHERE a > 1") + } + } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala index d04390d38c31..8912c9e75a2d 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala @@ -20,8 +20,8 @@ package org.apache.flink.table.planner.plan.stream.sql import org.apache.flink.api.scala._ import org.apache.flink.table.api.scala._ +import org.apache.flink.table.planner.factories.utils.TestCollectionTableFactory import org.apache.flink.table.planner.utils.TableTestBase - import org.junit.Test class TableScanTest extends TableTestBase { @@ -39,4 +39,21 @@ class TableScanTest extends TableTestBase { util.addDataStream[(Int, Long, String)]("DataStreamTable", 'a, 'b, 'c) util.verifyPlan("SELECT * FROM DataStreamTable") } + + @Test + def testDDLTableScan(): Unit = { + TestCollectionTableFactory.isStreaming = true + util.addTable( + """ + |CREATE TABLE src ( + | ts TIMESTAMP(3), + | a INT, + | b DOUBLE, + | WATERMARK FOR ts AS ts - INTERVAL '0.001' SECOND + |) WITH ( + | 'connector' = 'COLLECTION' + |) + """.stripMargin) + util.verifyPlan("SELECT * FROM src WHERE a > 1") + } } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TimeAttributeITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TimeAttributeITCase.scala new file mode 100644 index 000000000000..ca1fc5f903a8 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TimeAttributeITCase.scala @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.runtime.stream.sql + +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.factories.utils.TestCollectionTableFactory +import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.JavaFunc5 +import org.apache.flink.table.planner.runtime.utils.{StreamingTestBase, TestingAppendSink} +import org.apache.flink.types.Row +import org.junit.Assert.{assertEquals, assertTrue} +import org.junit.Test + +import java.sql.Timestamp +import java.util.TimeZone + +import scala.collection.JavaConverters._ + +/** + * Integration tests for time attributes defined in DDL. + */ +class TimeAttributeITCase extends StreamingTestBase { + + val data = List( + row(utcTimestamp(1L), 1, 1d), + row(utcTimestamp(2L), 1, 2d), + row(utcTimestamp(3L), 1, 2d), + row(utcTimestamp(4L), 1, 5d), + row(utcTimestamp(7L), 1, 3d), + row(utcTimestamp(8L), 1, 3d), + row(utcTimestamp(16L), 1, 4d)) + TestCollectionTableFactory.reset() + TestCollectionTableFactory.initData(data.asJava) + TestCollectionTableFactory.isStreaming = true + + @Test + def testWindowAggregateOnWatermark(): Unit = { + val ddl = + """ + |CREATE TABLE src ( + | ts TIMESTAMP(3), + | a INT, + | b DOUBLE, + | WATERMARK FOR ts AS ts - INTERVAL '0.001' SECOND + |) WITH ( + | 'connector' = 'COLLECTION' + |) + """.stripMargin + val query = + """ + |SELECT TUMBLE_END(ts, INTERVAL '0.003' SECOND), COUNT(ts), SUM(b) + |FROM src + |GROUP BY TUMBLE(ts, INTERVAL '0.003' SECOND) + """.stripMargin + tEnv.sqlUpdate(ddl) + val sink = new TestingAppendSink() + tEnv.sqlQuery(query).toAppendStream[Row].addSink(sink) + tEnv.execute("SQL JOB") + + val expected = Seq( + "1970-01-01T00:00:00.003,2,3.0", + "1970-01-01T00:00:00.006,2,7.0", + "1970-01-01T00:00:00.009,2,6.0", + "1970-01-01T00:00:00.018,1,4.0") + assertEquals(expected.sorted, sink.getAppendResults.sorted) + } + + @Test + def testWindowAggregateOnCustomizedWatermark(): Unit = { + JavaFunc5.openCalled = false + JavaFunc5.closeCalled = false + tEnv.registerFunction("myFunc", new JavaFunc5) + val ddl = + """ + |CREATE TABLE src ( + | ts TIMESTAMP(3), + | a INT, + | b DOUBLE, + | WATERMARK FOR ts AS myFunc(ts, a) + |) WITH ( + | 'connector' = 'COLLECTION' + |) + """.stripMargin + val query = + """ + |SELECT TUMBLE_END(ts, INTERVAL '0.003' SECOND), COUNT(ts), SUM(b) + |FROM src + |GROUP BY TUMBLE(ts, INTERVAL '0.003' SECOND) + """.stripMargin + tEnv.sqlUpdate(ddl) + val sink = new TestingAppendSink() + tEnv.sqlQuery(query).toAppendStream[Row].addSink(sink) + tEnv.execute("SQL JOB") + + val expected = Seq( + "1970-01-01T00:00:00.003,2,3.0", + "1970-01-01T00:00:00.006,2,7.0", + "1970-01-01T00:00:00.009,2,6.0", + "1970-01-01T00:00:00.018,1,4.0") + assertEquals(expected.sorted, sink.getAppendResults.sorted) + assertTrue(JavaFunc5.openCalled) + assertTrue(JavaFunc5.closeCalled) + } + + @Test + def testWindowAggregateOnNestedRowtime(): Unit = { + val ddl = + """ + |CREATE TABLE src ( + | col ROW< + | ts TIMESTAMP(3), + | a INT, + | b DOUBLE>, + | WATERMARK FOR col.ts AS col.ts - INTERVAL '0.001' SECOND + |) WITH ( + | 'connector' = 'COLLECTION' + |) + """.stripMargin + val query = + """ + |SELECT TUMBLE_END(col.ts, INTERVAL '0.003' SECOND), COUNT(*) + |FROM src + |GROUP BY TUMBLE(col.ts, INTERVAL '0.003' SECOND) + """.stripMargin + tEnv.sqlUpdate(ddl) + expectedException.expect(classOf[ValidationException]) + expectedException.expectMessage( + "Nested field 'col.ts' as rowtime attribute is not supported right now.") + tEnv.sqlQuery(query) + } + + // ------------------------------------------------------------------------------------------ + + private def utcTimestamp(ts: Long): Timestamp = { + new Timestamp(ts - TimeZone.getDefault.getOffset(ts)) + } + + private def row(args: Any*):Row = { + val row = new Row(args.length) + 0 until args.length foreach { + i => row.setField(i, args(i)) + } + row + } + +} 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 ec2b8896e821..fb986dc4a647 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 @@ -33,7 +33,7 @@ import org.apache.flink.table.api.java.internal.{StreamTableEnvironmentImpl => J import org.apache.flink.table.api.java.{StreamTableEnvironment => JavaStreamTableEnv} import org.apache.flink.table.api.scala.internal.{StreamTableEnvironmentImpl => ScalaStreamTableEnvImpl} import org.apache.flink.table.api.scala.{StreamTableEnvironment => ScalaStreamTableEnv} -import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog, GenericInMemoryCatalog, UnresolvedIdentifier} +import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog, GenericInMemoryCatalog} import org.apache.flink.table.dataformat.BaseRow import org.apache.flink.table.delegation.{Executor, ExecutorFactory, PlannerFactory} import org.apache.flink.table.expressions.Expression @@ -63,9 +63,10 @@ import org.apache.commons.lang3.SystemUtils import org.junit.Assert.{assertEquals, assertTrue} import org.junit.Rule import org.junit.rules.{ExpectedException, TestName} -import _root_.java.util +import _root_.java.util import org.apache.flink.table.module.ModuleManager +import org.apache.flink.table.operations.ddl.CreateTableOperation import _root_.scala.collection.JavaConversions._ import _root_.scala.io.Source @@ -142,6 +143,13 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) tableEnv.insertInto(sinkName, table) } + /** + * Creates a table with the given DDL SQL string. + */ + def addTable(ddl: String): Unit = { + getTableEnv.sqlUpdate(ddl) + } + /** * Create a [[DataStream]] with the given schema, * and registers this DataStream under given name into the TableEnvironment's catalog. @@ -1005,6 +1013,11 @@ class TestingTableEnvironment private( } else { buffer(modifyOperations) } + case createOperation: CreateTableOperation => + catalogManager.createTable( + createOperation.getCatalogTable, + createOperation.getTableIdentifier, + createOperation.isIgnoreIfExists) case _ => throw new TableException( "Unsupported SQL query! sqlUpdate() only accepts a single SQL statements of type INSERT.") } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperatorFactory.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperatorFactory.java new file mode 100644 index 000000000000..6b50cbe39245 --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperatorFactory.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.wmassigners; + +import org.apache.flink.streaming.api.graph.StreamConfig; +import org.apache.flink.streaming.api.operators.ChainingStrategy; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.runtime.tasks.StreamTask; +import org.apache.flink.table.dataformat.BaseRow; +import org.apache.flink.table.runtime.generated.GeneratedWatermarkGenerator; +import org.apache.flink.table.runtime.generated.WatermarkGenerator; + +/** + * The factory of {@link WatermarkAssignerOperator}. + */ +public class WatermarkAssignerOperatorFactory implements OneInputStreamOperatorFactory { + private static final long serialVersionUID = 1L; + + private final int rowtimeFieldIndex; + + private final long idleTimeout; + + private final GeneratedWatermarkGenerator generatedWatermarkGenerator; + + private ChainingStrategy strategy = ChainingStrategy.HEAD; + + public WatermarkAssignerOperatorFactory( + int rowtimeFieldIndex, + long idleTimeout, + GeneratedWatermarkGenerator generatedWatermarkGenerator) { + this.rowtimeFieldIndex = rowtimeFieldIndex; + this.idleTimeout = idleTimeout; + this.generatedWatermarkGenerator = generatedWatermarkGenerator; + } + + @SuppressWarnings("unchecked") + @Override + public StreamOperator createStreamOperator(StreamTask containingTask, StreamConfig config, Output output) { + WatermarkGenerator watermarkGenerator = generatedWatermarkGenerator.newInstance(containingTask.getUserCodeClassLoader()); + WatermarkAssignerOperator operator = new WatermarkAssignerOperator(rowtimeFieldIndex, watermarkGenerator, idleTimeout); + operator.setup(containingTask, config, output); + return operator; + } + + @Override + public void setChainingStrategy(ChainingStrategy strategy) { + this.strategy = strategy; + } + + @Override + public ChainingStrategy getChainingStrategy() { + return strategy; + } + + @Override + public Class getStreamOperatorClass(ClassLoader classLoader) { + return WatermarkAssignerOperator.class; + } +} From 937356f786b5edac28a7ff44fc31410761c6d7f0 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Fri, 8 Nov 2019 13:55:43 +0800 Subject: [PATCH 525/746] [hotfix][table-planner-blink] Move SqlToOperationConverterTest from sqlexec package to operations package SqlToOperationConverter is in org.apache.flink.table.planner.operations package. --- .../operations}/SqlToOperationConverterTest.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) rename flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/{sqlexec => planner/operations}/SqlToOperationConverterTest.java (98%) diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/operations/SqlToOperationConverterTest.java similarity index 98% rename from flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java rename to flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/operations/SqlToOperationConverterTest.java index 826b9b328b87..b8fba2f42dd2 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/operations/SqlToOperationConverterTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.sqlexec; +package org.apache.flink.table.planner.operations; import org.apache.flink.sql.parser.ddl.SqlCreateTable; import org.apache.flink.table.api.DataTypes; @@ -44,8 +44,6 @@ import org.apache.flink.table.planner.calcite.FlinkPlannerImpl; import org.apache.flink.table.planner.catalog.CatalogManagerCalciteSchema; import org.apache.flink.table.planner.delegation.PlannerContext; -import org.apache.flink.table.planner.operations.SqlConversionException; -import org.apache.flink.table.planner.operations.SqlToOperationConverter; import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions; import org.apache.flink.table.types.DataType; @@ -70,7 +68,7 @@ import static org.junit.Assert.assertEquals; /** - * Test cases for {@link org.apache.flink.table.planner.operations.SqlToOperationConverter}. + * Test cases for {@link SqlToOperationConverter}. */ public class SqlToOperationConverterTest { private final TableConfig tableConfig = new TableConfig(); From 3a096659297b4b61260d1799b5fe3f7151089e19 Mon Sep 17 00:00:00 2001 From: Jiayi Date: Fri, 8 Nov 2019 00:54:56 +0800 Subject: [PATCH 526/746] [FLINK-14646] Add non-null checks to KeyGroupRangeAssignment This closes #10120. --- .../org/apache/flink/runtime/state/KeyGroupRangeAssignment.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupRangeAssignment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupRangeAssignment.java index 430765b74e1f..d467ca64abf7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupRangeAssignment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupRangeAssignment.java @@ -46,6 +46,7 @@ private KeyGroupRangeAssignment() { * @return the index of the parallel operator to which the given key should be routed. */ public static int assignKeyToParallelOperator(Object key, int maxParallelism, int parallelism) { + Preconditions.checkNotNull(key, "Assigned key must not be null!"); return computeOperatorIndexForKeyGroup(maxParallelism, parallelism, assignToKeyGroup(key, maxParallelism)); } @@ -57,6 +58,7 @@ public static int assignKeyToParallelOperator(Object key, int maxParallelism, in * @return the key-group to which the given key is assigned */ public static int assignToKeyGroup(Object key, int maxParallelism) { + Preconditions.checkNotNull(key, "Assigned key must not be null!"); return computeKeyGroupForKeyHash(key.hashCode(), maxParallelism); } From c1affda675fbb0c5823bd9de29d32c2892481892 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 8 Nov 2019 10:49:13 +0100 Subject: [PATCH 527/746] [hotfix] Fix checkstyle violations in KeyGroupRangeAssignment --- .../apache/flink/runtime/state/KeyGroupRangeAssignment.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupRangeAssignment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupRangeAssignment.java index d467ca64abf7..b5676ba7365e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupRangeAssignment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupRangeAssignment.java @@ -30,7 +30,7 @@ public final class KeyGroupRangeAssignment { */ public static final int DEFAULT_LOWER_BOUND_MAX_PARALLELISM = 1 << 7; - /** The (inclusive) upper bound for max parallelism */ + /** The (inclusive) upper bound for max parallelism. */ public static final int UPPER_BOUND_MAX_PARALLELISM = Transformation.UPPER_BOUND_MAX_PARALLELISM; private KeyGroupRangeAssignment() { @@ -77,7 +77,7 @@ public static int computeKeyGroupForKeyHash(int keyHash, int maxParallelism) { * Computes the range of key-groups that are assigned to a given operator under the given parallelism and maximum * parallelism. * - * IMPORTANT: maxParallelism must be <= Short.MAX_VALUE to avoid rounding problems in this method. If we ever want + *

    IMPORTANT: maxParallelism must be <= Short.MAX_VALUE to avoid rounding problems in this method. If we ever want * to go beyond this boundary, this method must perform arithmetic on long values. * * @param maxParallelism Maximal parallelism that the job was initially created with. @@ -105,7 +105,7 @@ public static KeyGroupRange computeKeyGroupRangeForOperatorIndex( * Computes the index of the operator to which a key-group belongs under the given parallelism and maximum * parallelism. * - * IMPORTANT: maxParallelism must be <= Short.MAX_VALUE to avoid rounding problems in this method. If we ever want + *

    IMPORTANT: maxParallelism must be <= Short.MAX_VALUE to avoid rounding problems in this method. If we ever want * to go beyond this boundary, this method must perform arithmetic on long values. * * @param maxParallelism Maximal parallelism that the job was initially created with. From 2df3cccd0c16639e13348b1de7c90196c02af441 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 25 Oct 2019 14:02:26 +0200 Subject: [PATCH 528/746] [FLINK-14530][coordination] Replace PartitionTable with PartitionTracker --- .../org/apache/flink/util/CollectionUtil.java | 10 ++++++ .../JobMasterPartitionTrackerImpl.java | 13 ++----- .../partition/TaskExecutorPartitionInfo.java | 24 +++++++++++++ .../TaskExecutorPartitionTracker.java | 33 ++++++++++++++++++ .../TaskExecutorPartitionTrackerImpl.java | 34 +++++++++++++++++++ .../runtime/taskexecutor/TaskExecutor.java | 28 ++++++++------- .../taskexecutor/TaskManagerRunner.java | 4 +-- .../TaskExecutorPartitionLifecycleTest.java | 24 +++++++------ .../taskexecutor/TaskExecutorTest.java | 6 ++-- .../TaskSubmissionTestEnvironment.java | 4 +-- .../taskexecutor/TestingTaskExecutor.java | 7 ++-- 11 files changed, 143 insertions(+), 44 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionInfo.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTracker.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImpl.java diff --git a/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java b/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java index 11ba0c5ee279..45ceaa448069 100644 --- a/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java +++ b/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java @@ -27,8 +27,11 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; +import java.util.function.Function; import java.util.stream.Stream; +import static java.util.stream.Collectors.toList; + /** * Simple utility to work with Java collections. */ @@ -72,4 +75,11 @@ public static Collection> partition(Collection elements, int numB return buckets.values(); } + + public static Collection project(Collection collection, Function projector) { + return collection + .stream() + .map(projector) + .collect(toList()); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImpl.java index 2b2af3c82ebb..4da8ed31ba9e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/JobMasterPartitionTrackerImpl.java @@ -21,6 +21,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.shuffle.ShuffleMaster; +import org.apache.flink.util.CollectionUtil; import org.apache.flink.util.Preconditions; import java.util.Collection; @@ -28,7 +29,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -94,7 +94,7 @@ public void stopTrackingAndReleasePartitionsFor(ResourceID producingTaskExecutor Preconditions.checkNotNull(producingTaskExecutorId); Collection resultPartitionIds = - project(stopTrackingPartitionsFor(producingTaskExecutorId), PartitionTrackerEntry::getMetaInfo); + CollectionUtil.project(stopTrackingPartitionsFor(producingTaskExecutorId), PartitionTrackerEntry::getMetaInfo); internalReleasePartitions(producingTaskExecutorId, resultPartitionIds); } @@ -104,7 +104,7 @@ public void stopTrackingAndReleaseOrPromotePartitionsFor(ResourceID producingTas Preconditions.checkNotNull(producingTaskExecutorId); Collection resultPartitionIds = - project(stopTrackingPartitionsFor(producingTaskExecutorId), PartitionTrackerEntry::getMetaInfo); + CollectionUtil.project(stopTrackingPartitionsFor(producingTaskExecutorId), PartitionTrackerEntry::getMetaInfo); internalReleaseOrPromotePartitions(producingTaskExecutorId, resultPartitionIds); } @@ -177,13 +177,6 @@ private void internalReleasePartitionsOnShuffleMaster(Stream Collection project(Collection collection, Function projector) { - return collection - .stream() - .map(projector) - .collect(toList()); - } - private static boolean isPartitionWithLocalResources(ResultPartitionDeploymentDescriptor resultPartitionDeploymentDescriptor) { return resultPartitionDeploymentDescriptor.getShuffleDescriptor().storesLocalResourcesOn().isPresent(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionInfo.java new file mode 100644 index 000000000000..96f788386fdf --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionInfo.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.partition; + +/** + * Encapsulates meta-information the TaskExecutor requires to be kept for each partition. + */ +public final class TaskExecutorPartitionInfo { +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTracker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTracker.java new file mode 100644 index 000000000000..8d581a80a582 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTracker.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.partition; + +import org.apache.flink.api.common.JobID; + +/** + * Utility for tracking partitions. + */ +public interface TaskExecutorPartitionTracker extends PartitionTracker { + + /** + * Starts the tracking of the given partition for the given job. + * + * @param producingJobId ID of job by which the partition is produced + */ + void startTrackingPartition(JobID producingJobId, ResultPartitionID resultPartitionId); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImpl.java new file mode 100644 index 000000000000..86ac25d072e0 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTrackerImpl.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.runtime.io.network.partition; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.util.Preconditions; + +/** + * Utility for tracking partitions and issuing release calls to task executors and shuffle masters. + */ +public class TaskExecutorPartitionTrackerImpl extends AbstractPartitionTracker implements TaskExecutorPartitionTracker { + + public void startTrackingPartition(JobID producingJobId, ResultPartitionID resultPartitionId) { + Preconditions.checkNotNull(producingJobId); + Preconditions.checkNotNull(resultPartitionId); + + startTrackingPartition(producingJobId, resultPartitionId, new TaskExecutorPartitionInfo()); + } +} 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 5f22ef62125c..8192dfdd79d0 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 @@ -52,8 +52,10 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.instance.HardwareDescription; import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.io.network.partition.PartitionTrackerEntry; import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.TaskExecutorPartitionTracker; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.jobmaster.AllocatedSlotInfo; import org.apache.flink.runtime.jobmaster.AllocatedSlotReport; @@ -89,7 +91,6 @@ import org.apache.flink.runtime.taskexecutor.exceptions.TaskException; import org.apache.flink.runtime.taskexecutor.exceptions.TaskManagerException; import org.apache.flink.runtime.taskexecutor.exceptions.TaskSubmissionException; -import org.apache.flink.runtime.taskexecutor.partition.PartitionTable; import org.apache.flink.runtime.taskexecutor.rpc.RpcCheckpointResponder; import org.apache.flink.runtime.taskexecutor.rpc.RpcGlobalAggregateManager; import org.apache.flink.runtime.taskexecutor.rpc.RpcInputSplitProvider; @@ -108,6 +109,7 @@ import org.apache.flink.runtime.taskmanager.TaskManagerActions; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.types.SerializableOptional; +import org.apache.flink.util.CollectionUtil; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; @@ -210,7 +212,7 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { /** The heartbeat manager for resource manager in the task manager. */ private final HeartbeatManager resourceManagerHeartbeatManager; - private final PartitionTable partitionTable; + private final TaskExecutorPartitionTracker partitionTracker; // --------- resource manager -------- @@ -240,7 +242,7 @@ public TaskExecutor( @Nullable String metricQueryServiceAddress, BlobCacheService blobCacheService, FatalErrorHandler fatalErrorHandler, - PartitionTable partitionTable) { + TaskExecutorPartitionTracker partitionTracker) { super(rpcService, AkkaRpcServiceUtils.createRandomName(TASK_MANAGER_NAME)); @@ -250,7 +252,7 @@ public TaskExecutor( this.taskExecutorServices = checkNotNull(taskExecutorServices); this.haServices = checkNotNull(haServices); this.fatalErrorHandler = checkNotNull(fatalErrorHandler); - this.partitionTable = partitionTable; + this.partitionTracker = partitionTracker; this.taskManagerMetricGroup = checkNotNull(taskManagerMetricGroup); this.blobCacheService = checkNotNull(blobCacheService); this.metricQueryServiceAddress = metricQueryServiceAddress; @@ -624,15 +626,15 @@ private void setupResultPartitionBookkeeping( JobID jobId, Collection producedResultPartitions, CompletableFuture terminationFuture) { - final List partitionsRequiringRelease = filterPartitionsRequiringRelease(producedResultPartitions); + final Set partitionsRequiringRelease = filterPartitionsRequiringRelease(producedResultPartitions); - partitionTable.startTrackingPartitions(jobId, partitionsRequiringRelease); + partitionsRequiringRelease.forEach(resultPartitionId -> partitionTracker.startTrackingPartition(jobId, resultPartitionId)); final CompletableFuture taskTerminationWithResourceCleanupFuture = terminationFuture.thenApplyAsync( executionState -> { if (executionState != ExecutionState.FINISHED) { - partitionTable.stopTrackingPartitions(jobId, partitionsRequiringRelease); + partitionTracker.stopTrackingPartitions(partitionsRequiringRelease); } return executionState; }, @@ -650,7 +652,7 @@ private void setupResultPartitionBookkeeping( }); } - private List filterPartitionsRequiringRelease(Collection producedResultPartitions) { + private Set filterPartitionsRequiringRelease(Collection producedResultPartitions) { return producedResultPartitions.stream() // only blocking partitions require explicit release call .filter(d -> d.getPartitionType().isBlocking()) @@ -658,7 +660,7 @@ private List filterPartitionsRequiringRelease(Collection d.storesLocalResourcesOn().isPresent()) .map(ShuffleDescriptor::getResultPartitionID) - .collect(Collectors.toList()); + .collect(Collectors.toSet()); } @Override @@ -727,7 +729,7 @@ public CompletableFuture updatePartitions( @Override public void releaseOrPromotePartitions(JobID jobId, Set partitionToRelease, Set partitionsToPromote) { try { - partitionTable.stopTrackingPartitions(jobId, partitionToRelease); + partitionTracker.stopTrackingPartitions(partitionToRelease); shuffleEnvironment.releasePartitionsLocally(partitionToRelease); closeJobManagerConnectionIfNoAllocatedResources(jobId); } catch (Throwable t) { @@ -1369,7 +1371,9 @@ private void scheduleResultPartitionCleanup(JobID jobId) { if (taskTerminationFutures != null) { FutureUtils.waitForAll(taskTerminationFutures) .thenRunAsync(() -> { - Collection partitionsForJob = partitionTable.stopTrackingPartitions(jobId); + Collection partitionsForJob = CollectionUtil.project( + partitionTracker.stopTrackingPartitionsFor(jobId), + PartitionTrackerEntry::getResultPartitionId); shuffleEnvironment.releasePartitionsLocally(partitionsForJob); }, getMainThreadExecutor()); } @@ -1496,7 +1500,7 @@ private void freeSlotInternal(AllocationID allocationId, Throwable cause) { private void closeJobManagerConnectionIfNoAllocatedResources(JobID jobId) { // check whether we still have allocated slots for the same job - if (taskSlotTable.getAllocationIdsPerJob(jobId).isEmpty() && !partitionTable.hasTrackedPartitions(jobId)) { + if (taskSlotTable.getAllocationIdsPerJob(jobId).isEmpty() && !partitionTracker.isTrackingPartitionsFor(jobId)) { // we can remove the job from the job leader service try { jobLeaderService.removeJob(jobId); 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 6493decef993..b3d52455ff8e 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 @@ -39,6 +39,7 @@ import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; +import org.apache.flink.runtime.io.network.partition.TaskExecutorPartitionTrackerImpl; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalException; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; @@ -51,7 +52,6 @@ import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils; import org.apache.flink.runtime.security.SecurityConfiguration; import org.apache.flink.runtime.security.SecurityUtils; -import org.apache.flink.runtime.taskexecutor.partition.PartitionTable; import org.apache.flink.runtime.taskmanager.MemoryLogger; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.util.EnvironmentInformation; @@ -387,7 +387,7 @@ public static TaskExecutor startTaskManager( metricQueryServiceAddress, blobCacheService, fatalErrorHandler, - new PartitionTable<>()); + new TaskExecutorPartitionTrackerImpl()); } /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java index e6ed4ae4b5b1..02483e3b2d5c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java @@ -46,6 +46,8 @@ import org.apache.flink.runtime.io.network.partition.ResultPartition; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; +import org.apache.flink.runtime.io.network.partition.TaskExecutorPartitionTracker; +import org.apache.flink.runtime.io.network.partition.TaskExecutorPartitionTrackerImpl; import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobmaster.JMTMRegistrationSuccess; @@ -61,7 +63,6 @@ import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.shuffle.ShuffleIOOwnerContext; import org.apache.flink.runtime.state.TaskExecutorLocalStateStoresManager; -import org.apache.flink.runtime.taskexecutor.partition.PartitionTable; import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable; import org.apache.flink.runtime.taskexecutor.slot.TimerService; import org.apache.flink.runtime.taskmanager.NoOpTaskManagerActions; @@ -150,10 +151,11 @@ public void testConnectionTerminationAfterExternalRelease() throws Exception { .setTaskSlotTable(createTaskSlotTable()) .build(); - final PartitionTable partitionTable = new PartitionTable<>(); - final ResultPartitionID resultPartitionId = new ResultPartitionID(); + final TaskExecutorPartitionTracker partitionTracker = new TaskExecutorPartitionTrackerImpl(); + final ResultPartitionDeploymentDescriptor resultPartitionDeploymentDescriptor = PartitionTestUtils.createPartitionDeploymentDescriptor(ResultPartitionType.BLOCKING); + final ResultPartitionID resultPartitionId = resultPartitionDeploymentDescriptor.getShuffleDescriptor().getResultPartitionID(); - final TestingTaskExecutor taskExecutor = createTestingTaskExecutor(taskManagerServices, partitionTable); + final TestingTaskExecutor taskExecutor = createTestingTaskExecutor(taskManagerServices, partitionTracker); try { taskExecutor.start(); @@ -164,7 +166,7 @@ public void testConnectionTerminationAfterExternalRelease() throws Exception { // baseline, jobmanager was added in test setup runInTaskExecutorThreadAndWait(taskExecutor, () -> assertTrue(jobManagerTable.contains(jobId))); - runInTaskExecutorThreadAndWait(taskExecutor, () -> partitionTable.startTrackingPartitions(jobId, Collections.singletonList(resultPartitionId))); + runInTaskExecutorThreadAndWait(taskExecutor, () -> partitionTracker.startTrackingPartition(jobId, resultPartitionId)); final CompletableFuture> firstReleasePartitionsCallFuture = new CompletableFuture<>(); runInTaskExecutorThreadAndWait(taskExecutor, () -> shuffleEnvironment.releasePartitionsLocallyFuture = firstReleasePartitionsCallFuture); @@ -302,9 +304,9 @@ private void testPartitionRelease( }) .build(); - final PartitionTable partitionTable = new PartitionTable<>(); + final TaskExecutorPartitionTracker partitionTracker = new TaskExecutorPartitionTrackerImpl(); - final TestingTaskExecutor taskExecutor = createTestingTaskExecutor(taskManagerServices, partitionTable); + final TestingTaskExecutor taskExecutor = createTestingTaskExecutor(taskManagerServices, partitionTracker); final CompletableFuture initialSlotReportFuture = new CompletableFuture<>(); @@ -374,7 +376,7 @@ private void testPartitionRelease( // the task is still running => the partition is in in-progress runInTaskExecutorThreadAndWait( taskExecutor, - () -> assertThat(partitionTable.hasTrackedPartitions(jobId), is(resultPartitionType.isBlocking()))); + () -> assertThat(partitionTracker.isTrackingPartitionsFor(jobId), is(resultPartitionType.isBlocking()))); TestingInvokable.sync.releaseBlocker(); taskFinishedFuture.get(timeout.getSize(), timeout.getUnit()); @@ -382,7 +384,7 @@ private void testPartitionRelease( // the task is finished => the partition should be finished now runInTaskExecutorThreadAndWait( taskExecutor, - () -> assertThat(partitionTable.hasTrackedPartitions(jobId), is(resultPartitionType.isBlocking()))); + () -> assertThat(partitionTracker.isTrackingPartitionsFor(jobId), is(resultPartitionType.isBlocking()))); final CompletableFuture> releasePartitionsFuture = new CompletableFuture<>(); runInTaskExecutorThreadAndWait( @@ -424,7 +426,7 @@ public void invoke() throws Exception { } } - private TestingTaskExecutor createTestingTaskExecutor(TaskManagerServices taskManagerServices, PartitionTable partitionTable) throws IOException { + private TestingTaskExecutor createTestingTaskExecutor(TaskManagerServices taskManagerServices, TaskExecutorPartitionTracker partitionTracker) throws IOException { return new TestingTaskExecutor( RPC, TaskManagerConfiguration.fromConfiguration(new Configuration()), @@ -438,7 +440,7 @@ private TestingTaskExecutor createTestingTaskExecutor(TaskManagerServices taskMa new VoidBlobStore(), null), new TestingFatalErrorHandler(), - partitionTable); + partitionTracker); } private static TaskSlotTable createTaskSlotTable() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index 6574620a3896..a68eb35b3555 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -57,6 +57,7 @@ import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; import org.apache.flink.runtime.io.network.partition.NoOpResultPartitionConsumableNotifier; +import org.apache.flink.runtime.io.network.partition.TaskExecutorPartitionTrackerImpl; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobmaster.AllocatedSlotInfo; @@ -87,7 +88,6 @@ import org.apache.flink.runtime.taskexecutor.TaskSubmissionTestEnvironment.Builder; import org.apache.flink.runtime.taskexecutor.exceptions.RegistrationTimeoutException; import org.apache.flink.runtime.taskexecutor.exceptions.TaskManagerException; -import org.apache.flink.runtime.taskexecutor.partition.PartitionTable; import org.apache.flink.runtime.taskexecutor.slot.SlotNotFoundException; import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable; @@ -1919,7 +1919,7 @@ private TaskExecutor createTaskExecutor(TaskManagerServices taskManagerServices, null, dummyBlobCacheService, testingFatalErrorHandler, - new PartitionTable<>()); + new TaskExecutorPartitionTrackerImpl()); } private TestingTaskExecutor createTestingTaskExecutor(TaskManagerServices taskManagerServices) { @@ -1937,7 +1937,7 @@ private TestingTaskExecutor createTestingTaskExecutor(TaskManagerServices taskMa null, dummyBlobCacheService, testingFatalErrorHandler, - new PartitionTable<>()); + new TaskExecutorPartitionTrackerImpl()); } private static final class StartStopNotifyingLeaderRetrievalService implements LeaderRetrievalService { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java index 47b2dbabb4ef..de8a78a07daa 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java @@ -35,6 +35,7 @@ import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; +import org.apache.flink.runtime.io.network.partition.TaskExecutorPartitionTrackerImpl; import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.io.network.netty.NettyConfig; import org.apache.flink.runtime.jobmaster.JobMasterGateway; @@ -47,7 +48,6 @@ import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.state.TaskExecutorLocalStateStoresManager; -import org.apache.flink.runtime.taskexecutor.partition.PartitionTable; import org.apache.flink.runtime.taskexecutor.rpc.RpcResultPartitionConsumableNotifier; import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable; import org.apache.flink.runtime.taskexecutor.slot.TimerService; @@ -207,7 +207,7 @@ private TestingTaskExecutor createTaskExecutor(TaskManagerServices taskManagerSe null, blobCacheService, testingFatalErrorHandler, - new PartitionTable<>() + new TaskExecutorPartitionTrackerImpl() ); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutor.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutor.java index 1aa708bd27c1..cf8daa4fe475 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutor.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutor.java @@ -18,14 +18,13 @@ package org.apache.flink.runtime.taskexecutor; -import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.blob.BlobCacheService; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.io.network.partition.TaskExecutorPartitionTracker; import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.taskexecutor.partition.PartitionTable; import javax.annotation.Nullable; @@ -47,7 +46,7 @@ public TestingTaskExecutor( @Nullable String metricQueryServiceAddress, BlobCacheService blobCacheService, FatalErrorHandler fatalErrorHandler, - PartitionTable partitionTable) { + TaskExecutorPartitionTracker partitionTracker) { super( rpcService, taskManagerConfiguration, @@ -58,7 +57,7 @@ public TestingTaskExecutor( metricQueryServiceAddress, blobCacheService, fatalErrorHandler, - partitionTable); + partitionTracker); } @Override From d788c6ff024fbe8a699e987d3df6a00160d22a06 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 25 Oct 2019 14:51:45 +0200 Subject: [PATCH 529/746] [FLINK-15430][coordination] Store IntermediateDatasetID --- .../partition/TaskExecutorPartitionInfo.java | 12 ++++++++++++ .../partition/TaskExecutorPartitionTracker.java | 4 +++- .../TaskExecutorPartitionTrackerImpl.java | 7 +++++-- .../flink/runtime/taskexecutor/TaskExecutor.java | 16 ++++++++-------- .../TaskExecutorPartitionLifecycleTest.java | 5 ++++- 5 files changed, 32 insertions(+), 12 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionInfo.java index 96f788386fdf..82e774a2e2e6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionInfo.java @@ -17,8 +17,20 @@ package org.apache.flink.runtime.io.network.partition; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; + /** * Encapsulates meta-information the TaskExecutor requires to be kept for each partition. */ public final class TaskExecutorPartitionInfo { + + private final IntermediateDataSetID intermediateDataSetId; + + public TaskExecutorPartitionInfo(IntermediateDataSetID intermediateDataSetId) { + this.intermediateDataSetId = intermediateDataSetId; + } + + public IntermediateDataSetID getIntermediateDataSetId() { + return intermediateDataSetId; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTracker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTracker.java index 8d581a80a582..f8231085d1f2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTracker.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/TaskExecutorPartitionTracker.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.io.network.partition; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; /** * Utility for tracking partitions. @@ -28,6 +29,7 @@ public interface TaskExecutorPartitionTracker extends PartitionTracker implements TaskExecutorPartitionTracker { - public void startTrackingPartition(JobID producingJobId, ResultPartitionID resultPartitionId) { + @Override + public void startTrackingPartition(JobID producingJobId, ResultPartitionID resultPartitionId, IntermediateDataSetID intermediateDataSetId) { Preconditions.checkNotNull(producingJobId); Preconditions.checkNotNull(resultPartitionId); + Preconditions.checkNotNull(intermediateDataSetId); - startTrackingPartition(producingJobId, resultPartitionId, new TaskExecutorPartitionInfo()); + startTrackingPartition(producingJobId, resultPartitionId, new TaskExecutorPartitionInfo(intermediateDataSetId)); } } 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 8192dfdd79d0..4b15ebc2499e 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 @@ -139,6 +139,7 @@ import java.util.concurrent.TimeoutException; import java.util.function.BiConsumer; import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -626,9 +627,11 @@ private void setupResultPartitionBookkeeping( JobID jobId, Collection producedResultPartitions, CompletableFuture terminationFuture) { - final Set partitionsRequiringRelease = filterPartitionsRequiringRelease(producedResultPartitions); - - partitionsRequiringRelease.forEach(resultPartitionId -> partitionTracker.startTrackingPartition(jobId, resultPartitionId)); + final Set partitionsRequiringRelease = filterPartitionsRequiringRelease(producedResultPartitions) + .peek(rpdd -> partitionTracker.startTrackingPartition(jobId, rpdd.getShuffleDescriptor().getResultPartitionID(), rpdd.getResultId())) + .map(ResultPartitionDeploymentDescriptor::getShuffleDescriptor) + .map(ShuffleDescriptor::getResultPartitionID) + .collect(Collectors.toSet()); final CompletableFuture taskTerminationWithResourceCleanupFuture = terminationFuture.thenApplyAsync( @@ -652,15 +655,12 @@ private void setupResultPartitionBookkeeping( }); } - private Set filterPartitionsRequiringRelease(Collection producedResultPartitions) { + private Stream filterPartitionsRequiringRelease(Collection producedResultPartitions) { return producedResultPartitions.stream() // only blocking partitions require explicit release call .filter(d -> d.getPartitionType().isBlocking()) - .map(ResultPartitionDeploymentDescriptor::getShuffleDescriptor) // partitions without local resources don't store anything on the TaskExecutor - .filter(d -> d.storesLocalResourcesOn().isPresent()) - .map(ShuffleDescriptor::getResultPartitionID) - .collect(Collectors.toSet()); + .filter(d -> d.getShuffleDescriptor().storesLocalResourcesOn().isPresent()); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java index 02483e3b2d5c..6646fb0ceb55 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java @@ -166,7 +166,10 @@ public void testConnectionTerminationAfterExternalRelease() throws Exception { // baseline, jobmanager was added in test setup runInTaskExecutorThreadAndWait(taskExecutor, () -> assertTrue(jobManagerTable.contains(jobId))); - runInTaskExecutorThreadAndWait(taskExecutor, () -> partitionTracker.startTrackingPartition(jobId, resultPartitionId)); + runInTaskExecutorThreadAndWait(taskExecutor, () -> partitionTracker.startTrackingPartition( + jobId, + resultPartitionId, + resultPartitionDeploymentDescriptor.getResultId())); final CompletableFuture> firstReleasePartitionsCallFuture = new CompletableFuture<>(); runInTaskExecutorThreadAndWait(taskExecutor, () -> shuffleEnvironment.releasePartitionsLocallyFuture = firstReleasePartitionsCallFuture); From 5927c54c6b4342159ec1334ba2e7b4281bc1275e Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Wed, 6 Nov 2019 14:45:49 +0100 Subject: [PATCH 530/746] [FLINK-14636][runtime] Enable scheduling for ScheduleMode.LAZY_FROM_SOURCES_WITH_BATCH_SLOT_REQUEST This enables scheduling of jobs with ScheduleMode.LAZY_FROM_SOURCES_WITH_BATCH_SLOT_REQUEST with DefaultScheduler. This closes #10132. --- .../apache/flink/runtime/scheduler/DefaultSchedulerFactory.java | 1 + 1 file changed, 1 insertion(+) 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 82b949502f3a..3058ac0c3295 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 @@ -112,6 +112,7 @@ private SchedulingStrategyFactory createSchedulingStrategyFactory(final Schedule switch (scheduleMode) { case EAGER: return new EagerSchedulingStrategy.Factory(); + case LAZY_FROM_SOURCES_WITH_BATCH_SLOT_REQUEST: case LAZY_FROM_SOURCES: return new LazyFromSourcesSchedulingStrategy.Factory(); default: From 2125ea2178863cf5e2faed929b95faadcae10a8d Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Wed, 6 Nov 2019 10:12:28 +0800 Subject: [PATCH 531/746] [FLINK-14611][runtime] Move allVerticesInSameSlotSharingGroupByDefault setting from ExecutionConfig to StreamGraph --- .../flink/api/common/ExecutionConfig.java | 35 ++----------------- .../streaming/api/graph/StreamGraph.java | 22 ++++++++++++ .../api/graph/StreamingJobGraphGenerator.java | 6 ++-- .../graph/StreamingJobGraphGeneratorTest.java | 4 +-- .../planner/delegation/BatchExecutor.java | 2 +- .../planner/delegation/BatchExecutorTest.java | 2 +- 6 files changed, 30 insertions(+), 41 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java index 05d105667336..7321fc4e2110 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java @@ -160,9 +160,6 @@ public class ExecutionConfig implements Serializable, Archiveable streamNodes; private Set sources; private Set sinks; @@ -212,6 +215,25 @@ public void setBlockingConnectionsBetweenChains(boolean blockingConnectionsBetwe this.blockingConnectionsBetweenChains = blockingConnectionsBetweenChains; } + /** + * Set whether to put all vertices into the same slot sharing group by default. + * + * @param allVerticesInSameSlotSharingGroupByDefault indicates whether to put all vertices + * into the same slot sharing group by default. + */ + public void setAllVerticesInSameSlotSharingGroupByDefault(boolean allVerticesInSameSlotSharingGroupByDefault) { + this.allVerticesInSameSlotSharingGroupByDefault = allVerticesInSameSlotSharingGroupByDefault; + } + + /** + * Gets whether to put all vertices into the same slot sharing group by default. + * + * @return whether to put all vertices into the same slot sharing group by default. + */ + public boolean isAllVerticesInSameSlotSharingGroupByDefault() { + return allVerticesInSameSlotSharingGroupByDefault; + } + // Checkpointing public boolean isChainingEnabled() { 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 56d71883f27b..3d08264f9f2f 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 @@ -18,7 +18,6 @@ package org.apache.flink.streaming.api.graph; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.operators.ResourceSpec; @@ -630,15 +629,14 @@ private void setSlotSharing() { /** * Maps a vertex to its region slot sharing group. - * If {@link ExecutionConfig#isAllVerticesInSameSlotSharingGroupByDefault()} + * If {@link StreamGraph#isAllVerticesInSameSlotSharingGroupByDefault()} * returns true, all regions will be in the same slot sharing group. */ private Map buildVertexRegionSlotSharingGroups() { final Map vertexRegionSlotSharingGroups = new HashMap<>(); final SlotSharingGroup defaultSlotSharingGroup = new SlotSharingGroup(); - final boolean allRegionsInSameSlotSharingGroup = streamGraph.getExecutionConfig() - .isAllVerticesInSameSlotSharingGroupByDefault(); + final boolean allRegionsInSameSlotSharingGroup = streamGraph.isAllVerticesInSameSlotSharingGroupByDefault(); final Set regions = new DefaultLogicalTopology(jobGraph).getLogicalPipelinedRegions(); for (LogicalPipelinedRegion region : regions) { 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 fc21a03f13fc..64a2e1d599d0 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 @@ -666,7 +666,7 @@ public void testSlotSharingOnAllVerticesInSameSlotSharingGroupByDefaultEnabled() .findFirst() .get() .setSlotSharingGroup("testSlotSharingGroup"); - streamGraph.getExecutionConfig().enableAllVerticesInSameSlotSharingGroupByDefault(); + streamGraph.setAllVerticesInSameSlotSharingGroupByDefault(true); final JobGraph jobGraph = StreamingJobGraphGenerator.createJobGraph(streamGraph); final List verticesSorted = jobGraph.getVerticesSortedTopologicallyFromSources(); @@ -686,7 +686,7 @@ public void testSlotSharingOnAllVerticesInSameSlotSharingGroupByDefaultEnabled() @Test public void testSlotSharingOnAllVerticesInSameSlotSharingGroupByDefaultDisabled() { final StreamGraph streamGraph = createStreamGraphForSlotSharingTest(); - streamGraph.getExecutionConfig().disableAllVerticesInSameSlotSharingGroupByDefault(); + streamGraph.setAllVerticesInSameSlotSharingGroupByDefault(false); final JobGraph jobGraph = StreamingJobGraphGenerator.createJobGraph(streamGraph); final List verticesSorted = jobGraph.getVerticesSortedTopologicallyFromSources(); diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/BatchExecutor.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/BatchExecutor.java index f9cc9265ab0d..0b12e1a3d65f 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/BatchExecutor.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/BatchExecutor.java @@ -67,7 +67,6 @@ private void setBatchProperties(StreamExecutionEnvironment execEnv) { if (isShuffleModeAllBatch()) { executionConfig.setDefaultInputDependencyConstraint(InputDependencyConstraint.ALL); } - executionConfig.disableAllVerticesInSameSlotSharingGroupByDefault(); } @Override @@ -85,6 +84,7 @@ public StreamGraph generateStreamGraph(List> transformations, } }); streamGraph.setChaining(true); + streamGraph.setAllVerticesInSameSlotSharingGroupByDefault(false); streamGraph.setScheduleMode(ScheduleMode.LAZY_FROM_SOURCES_WITH_BATCH_SLOT_REQUEST); streamGraph.setStateBackend(null); if (streamGraph.getCheckpointConfig().isCheckpointingEnabled()) { diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/delegation/BatchExecutorTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/delegation/BatchExecutorTest.java index eeca304f732a..501a5b47e4b4 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/delegation/BatchExecutorTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/delegation/BatchExecutorTest.java @@ -66,6 +66,6 @@ public void cancel() { @Test public void testAllVerticesInSameSlotSharingGroupByDefaultIsDisabled() { - assertFalse(streamGraph.getExecutionConfig().isAllVerticesInSameSlotSharingGroupByDefault()); + assertFalse(streamGraph.isAllVerticesInSameSlotSharingGroupByDefault()); } } From 501f640454f2841166841cdd809b5c29893caffd Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Tue, 5 Nov 2019 11:08:58 -0800 Subject: [PATCH 532/746] [FLINK-14579][sql cli] enable SQL CLI to configure modules via yaml config enable SQL CLI to configure modules via yaml config. This closes #10093. --- .../conf/sql-client-defaults.yaml | 9 +++ .../table/client/config/Environment.java | 34 ++++++++++ .../client/config/entries/ModuleEntry.java | 68 +++++++++++++++++++ .../gateway/local/ExecutionContext.java | 23 +++++++ .../client/gateway/local/DependencyTest.java | 36 ++++++++++ .../client/gateway/local/EnvironmentTest.java | 21 ++++++ .../gateway/local/ExecutionContextTest.java | 28 ++++++++ ....apache.flink.table.factories.TableFactory | 1 + .../resources/test-sql-client-modules.yaml | 52 ++++++++++++++ 9 files changed, 272 insertions(+) create mode 100644 flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/config/entries/ModuleEntry.java create mode 100644 flink-table/flink-sql-client/src/test/resources/test-sql-client-modules.yaml diff --git a/flink-table/flink-sql-client/conf/sql-client-defaults.yaml b/flink-table/flink-sql-client/conf/sql-client-defaults.yaml index f6e83405dca1..e8511ee70bb4 100644 --- a/flink-table/flink-sql-client/conf/sql-client-defaults.yaml +++ b/flink-table/flink-sql-client/conf/sql-client-defaults.yaml @@ -78,6 +78,15 @@ catalogs: [] # empty list # hive-conf-dir: /opt/hive_conf/ # default-database: ... +#============================================================================== +# Modules +#============================================================================== + +# Define modules here. + +#modules: # note the following modules will be of the order they are specified +# - name: core +# type: core #============================================================================== # Execution properties diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java index 39bced99481b..065787ba4808 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java @@ -24,6 +24,7 @@ import org.apache.flink.table.client.config.entries.DeploymentEntry; import org.apache.flink.table.client.config.entries.ExecutionEntry; import org.apache.flink.table.client.config.entries.FunctionEntry; +import org.apache.flink.table.client.config.entries.ModuleEntry; import org.apache.flink.table.client.config.entries.TableEntry; import org.apache.flink.table.client.config.entries.ViewEntry; @@ -53,6 +54,8 @@ public class Environment { public static final String DEPLOYMENT_ENTRY = "deployment"; + private Map modules; + private Map catalogs; private Map tables; @@ -66,6 +69,7 @@ public class Environment { private DeploymentEntry deployment; public Environment() { + this.modules = Collections.emptyMap(); this.catalogs = Collections.emptyMap(); this.tables = Collections.emptyMap(); this.functions = Collections.emptyMap(); @@ -74,6 +78,24 @@ public Environment() { this.deployment = DeploymentEntry.DEFAULT_INSTANCE; } + public Map getModules() { + return modules; + } + + public void setModules(List> modules) { + this.modules = new HashMap<>(modules.size()); + + modules.forEach(config -> { + final ModuleEntry entry = ModuleEntry.create(config); + if (this.modules.containsKey(entry.getName())) { + throw new SqlClientException( + String.format("Cannot register module '%s' because a module with this name is already registered.", + entry.getName())); + } + this.modules.put(entry.getName(), entry); + }); + } + public Map getCatalogs() { return catalogs; } @@ -153,6 +175,11 @@ public DeploymentEntry getDeployment() { @Override public String toString() { final StringBuilder sb = new StringBuilder(); + sb.append("===================== Modules =====================\n"); + modules.forEach((name, module) -> { + sb.append("- ").append(ModuleEntry.MODULE_NAME).append(": ").append(name).append("\n"); + module.asMap().forEach((k, v) -> sb.append(" ").append(k).append(": ").append(v).append('\n')); + }); sb.append("===================== Catalogs =====================\n"); catalogs.forEach((name, catalog) -> { sb.append("- ").append(CatalogEntry.CATALOG_NAME).append(": ").append(name).append("\n"); @@ -207,6 +234,11 @@ public static Environment parse(String content) throws IOException { public static Environment merge(Environment env1, Environment env2) { final Environment mergedEnv = new Environment(); + // merge modules + final Map modules = new HashMap<>(env1.getModules()); + modules.putAll(env2.getModules()); + mergedEnv.modules = modules; + // merge catalogs final Map catalogs = new HashMap<>(env1.getCatalogs()); catalogs.putAll(env2.getCatalogs()); @@ -243,6 +275,8 @@ public static Environment enrich( Map views) { final Environment enrichedEnv = new Environment(); + enrichedEnv.modules = new LinkedHashMap<>(env.getModules()); + // merge catalogs enrichedEnv.catalogs = new LinkedHashMap<>(env.getCatalogs()); diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/config/entries/ModuleEntry.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/config/entries/ModuleEntry.java new file mode 100644 index 000000000000..94c2717d78dd --- /dev/null +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/config/entries/ModuleEntry.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.table.client.config.entries; + +import org.apache.flink.table.client.config.ConfigUtil; +import org.apache.flink.table.descriptors.DescriptorProperties; + +import java.util.Collections; +import java.util.Map; + +import static org.apache.flink.table.descriptors.ModuleDescriptorValidator.MODULE_TYPE; + +/** + * Describes a module configuration entry. + */ +public class ModuleEntry extends ConfigEntry { + + public static final String MODULE_NAME = "name"; + + private final String name; + + protected ModuleEntry(String name, DescriptorProperties properties) { + super(properties); + this.name = name; + } + + public String getName() { + return name; + } + + @Override + protected void validate(DescriptorProperties properties) { + properties.validateString(MODULE_TYPE, false, 1); + + // further validation is performed by the discovered factory + } + + public static ModuleEntry create(Map config) { + return create(ConfigUtil.normalizeYaml(config)); + } + + private static ModuleEntry create(DescriptorProperties properties) { + properties.validateString(MODULE_NAME, false, 1); + + final String name = properties.getString(MODULE_NAME); + + final DescriptorProperties cleanedProperties = + properties.withoutKeys(Collections.singletonList(MODULE_NAME)); + + return new ModuleEntry(name, cleanedProperties); + } +} 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 1fb0a3941aca..692ebee431c5 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 @@ -69,10 +69,12 @@ import org.apache.flink.table.delegation.ExecutorFactory; import org.apache.flink.table.delegation.Planner; import org.apache.flink.table.delegation.PlannerFactory; +import org.apache.flink.table.descriptors.CoreModuleDescriptorValidator; import org.apache.flink.table.factories.BatchTableSinkFactory; import org.apache.flink.table.factories.BatchTableSourceFactory; import org.apache.flink.table.factories.CatalogFactory; import org.apache.flink.table.factories.ComponentFactoryService; +import org.apache.flink.table.factories.ModuleFactory; import org.apache.flink.table.factories.TableFactoryService; import org.apache.flink.table.factories.TableSinkFactory; import org.apache.flink.table.factories.TableSourceFactory; @@ -81,6 +83,7 @@ import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.functions.TableFunction; import org.apache.flink.table.functions.UserDefinedFunction; +import org.apache.flink.table.module.Module; import org.apache.flink.table.module.ModuleManager; import org.apache.flink.table.planner.delegation.ExecutorBase; import org.apache.flink.table.sinks.TableSink; @@ -113,6 +116,7 @@ public class ExecutionContext { private final Environment mergedEnv; private final List dependencies; private final ClassLoader classLoader; + private final Map modules; private final Map catalogs; private final Map> tableSources; private final Map> tableSinks; @@ -141,6 +145,12 @@ public ExecutionContext(Environment defaultEnvironment, SessionContext sessionCo dependencies.toArray(new URL[dependencies.size()]), this.getClass().getClassLoader()); + // create modules + modules = new LinkedHashMap<>(); + mergedEnv.getModules().forEach((name, entry) -> + modules.put(name, createModule(entry.asMap(), classLoader)) + ); + // create catalogs catalogs = new LinkedHashMap<>(); mergedEnv.getCatalogs().forEach((name, entry) -> @@ -262,6 +272,12 @@ private static ExecutionConfigAccessor createExecutionParameterProvider(CommandL } } + private Module createModule(Map moduleProperties, ClassLoader classLoader) { + final ModuleFactory factory = + TableFactoryService.find(ModuleFactory.class, moduleProperties, classLoader); + return factory.createModule(moduleProperties); + } + private Catalog createCatalog(String name, Map catalogProperties, ClassLoader classLoader) { final CatalogFactory factory = TableFactoryService.find(CatalogFactory.class, catalogProperties, classLoader); @@ -382,6 +398,13 @@ private EnvironmentInstance() { mergedEnv.getConfiguration().asMap().forEach((k, v) -> tableEnv.getConfig().getConfiguration().setString(k, v)); + // load modules + if (!modules.isEmpty()) { + // unload core module first to respect whatever users configure + tableEnv.unloadModule(CoreModuleDescriptorValidator.MODULE_TYPE_CORE); + modules.forEach(tableEnv::loadModule); + } + // register catalogs catalogs.forEach(tableEnv::registerCatalog); diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java index 64f52331ad90..3b12b95b6c91 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java @@ -46,6 +46,8 @@ import org.apache.flink.table.client.gateway.utils.TestTableSourceFactoryBase; import org.apache.flink.table.descriptors.DescriptorProperties; import org.apache.flink.table.factories.CatalogFactory; +import org.apache.flink.table.factories.ModuleFactory; +import org.apache.flink.table.module.Module; import org.apache.flink.table.types.DataType; import org.junit.Test; @@ -61,6 +63,7 @@ import static org.apache.flink.table.descriptors.CatalogDescriptorValidator.CATALOG_DEFAULT_DATABASE; import static org.apache.flink.table.descriptors.CatalogDescriptorValidator.CATALOG_TYPE; +import static org.apache.flink.table.descriptors.ModuleDescriptorValidator.MODULE_TYPE; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -73,6 +76,7 @@ public class DependencyTest { public static final String TEST_PROPERTY = "test-property"; public static final String CATALOG_TYPE_TEST = "DependencyTest"; + public static final String MODULE_TYPE_TEST = "ModuleDependencyTest"; private static final String FACTORY_ENVIRONMENT_FILE = "test-sql-client-factory.yaml"; private static final String TABLE_FACTORY_JAR_FILE = "table-factories-test-jar.jar"; @@ -129,6 +133,38 @@ public TestTableSinkFactory() { } } + /** + * Module that can be discovered if classloading is correct. + */ + public static class TestModuleFactory implements ModuleFactory { + + @Override + public Module createModule(Map properties) { + return new TestModule(); + } + + @Override + public Map requiredContext() { + final Map context = new HashMap<>(); + context.put(MODULE_TYPE, MODULE_TYPE_TEST); + return context; + } + + @Override + public List supportedProperties() { + final List properties = new ArrayList<>(); + properties.add("test"); + return properties; + } + } + + /** + * Test module. + */ + public static class TestModule implements Module { + + } + /** * Catalog that can be discovered if classloading is correct. */ diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/EnvironmentTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/EnvironmentTest.java index 94d6fa270ec6..013cb3002f2d 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/EnvironmentTest.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/EnvironmentTest.java @@ -33,7 +33,9 @@ import java.util.Set; import static org.apache.flink.table.client.config.entries.CatalogEntry.CATALOG_NAME; +import static org.apache.flink.table.client.config.entries.ModuleEntry.MODULE_NAME; import static org.apache.flink.table.descriptors.CatalogDescriptorValidator.CATALOG_TYPE; +import static org.apache.flink.table.descriptors.ModuleDescriptorValidator.MODULE_TYPE; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -97,6 +99,16 @@ public void testDuplicateCatalog() { createCatalog("catalog2", "test"))); } + @Test + public void testDuplicateModules() { + exception.expect(SqlClientException.class); + Environment env = new Environment(); + env.setModules(Arrays.asList( + createModule("module1", "test"), + createModule("module2", "test"), + createModule("module2", "test"))); + } + private static Map createCatalog(String name, String type) { Map prop = new HashMap<>(); @@ -105,4 +117,13 @@ private static Map createCatalog(String name, String type) { return prop; } + + private static Map createModule(String name, String type) { + Map prop = new HashMap<>(); + + prop.put(MODULE_NAME, name); + prop.put(MODULE_TYPE, type); + + return prop; + } } diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/ExecutionContextTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/ExecutionContextTest.java index e8f114541352..b3277a69466b 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/ExecutionContextTest.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/ExecutionContextTest.java @@ -60,6 +60,7 @@ public class ExecutionContextTest { private static final String DEFAULTS_ENVIRONMENT_FILE = "test-sql-client-defaults.yaml"; + private static final String MODULES_ENVIRONMENT_FILE = "test-sql-client-modules.yaml"; private static final String CATALOGS_ENVIRONMENT_FILE = "test-sql-client-catalogs.yaml"; private static final String STREAMING_ENVIRONMENT_FILE = "test-sql-client-streaming.yaml"; private static final String CONFIGURATION_ENVIRONMENT_FILE = "test-sql-client-configuration.yaml"; @@ -80,6 +81,23 @@ public void testExecutionConfig() throws Exception { assertEquals(1_000, failureRateStrategy.getDelayBetweenAttemptsInterval().toMilliseconds()); } + @Test + public void testModules() throws Exception { + final ExecutionContext context = createModuleExecutionContext(); + final TableEnvironment tableEnv = context.createEnvironmentInstance().getTableEnvironment(); + + Set allModules = new HashSet<>(Arrays.asList(tableEnv.listModules())); + assertEquals(2, allModules.size()); + assertEquals( + new HashSet<>( + Arrays.asList( + "core", + "mymodule") + ), + allModules + ); + } + @Test public void testCatalogs() throws Exception { final String inmemoryCatalog = "inmemorycatalog"; @@ -294,6 +312,16 @@ private ExecutionContext createDefaultExecutionContext() throws Exception return createExecutionContext(DEFAULTS_ENVIRONMENT_FILE, replaceVars); } + private ExecutionContext createModuleExecutionContext() throws Exception { + final Map replaceVars = new HashMap<>(); + replaceVars.put("$VAR_PLANNER", "old"); + replaceVars.put("$VAR_EXECUTION_TYPE", "streaming"); + replaceVars.put("$VAR_RESULT_MODE", "changelog"); + replaceVars.put("$VAR_UPDATE_MODE", "update-mode: append"); + replaceVars.put("$VAR_MAX_ROWS", "100"); + return createExecutionContext(MODULES_ENVIRONMENT_FILE, replaceVars); + } + private ExecutionContext createCatalogExecutionContext() throws Exception { final Map replaceVars = new HashMap<>(); replaceVars.put("$VAR_PLANNER", "old"); diff --git a/flink-table/flink-sql-client/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink-table/flink-sql-client/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory index 5ba6b0b0b0a6..0b5ae7b26f9b 100644 --- a/flink-table/flink-sql-client/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory +++ b/flink-table/flink-sql-client/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory @@ -18,3 +18,4 @@ org.apache.flink.table.client.gateway.utils.DummyTableSourceFactory org.apache.flink.table.client.gateway.utils.SimpleCatalogFactory org.apache.flink.table.client.gateway.local.DependencyTest$TestCatalogFactory org.apache.flink.table.client.gateway.local.DependencyTest$TestHiveCatalogFactory +org.apache.flink.table.client.gateway.local.DependencyTest$TestModuleFactory diff --git a/flink-table/flink-sql-client/src/test/resources/test-sql-client-modules.yaml b/flink-table/flink-sql-client/src/test/resources/test-sql-client-modules.yaml new file mode 100644 index 000000000000..0e1902893a92 --- /dev/null +++ b/flink-table/flink-sql-client/src/test/resources/test-sql-client-modules.yaml @@ -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. +################################################################################ + +#============================================================================== +# TEST ENVIRONMENT FILE +# General purpose default environment file. +#============================================================================== + +# this file has variables that can be filled with content by replacing $VAR_XXX + +execution: + planner: "$VAR_PLANNER" + type: "$VAR_EXECUTION_TYPE" + time-characteristic: event-time + periodic-watermarks-interval: 99 + parallelism: 1 + max-parallelism: 16 + min-idle-state-retention: 0 + max-idle-state-retention: 0 + result-mode: "$VAR_RESULT_MODE" + max-table-result-rows: "$VAR_MAX_ROWS" + restart-strategy: + type: failure-rate + max-failures-per-interval: 10 + failure-rate-interval: 99000 + delay: 1000 + +deployment: + response-timeout: 5000 + +modules: + - name: core + type: core + - name: mymodule + type: ModuleDependencyTest + test: test + From 1806a373fcec1e981601bd755d3d3652dc61219a Mon Sep 17 00:00:00 2001 From: Rui Li Date: Fri, 8 Nov 2019 19:22:22 +0800 Subject: [PATCH 533/746] [FLINK-14673][hive] Shouldn't expect HMS client to throw NoSuchObjectException for non-existing function Always to check MetaException when getting function with HMS client. This closes #10133. --- .../client/HiveMetastoreClientWrapper.java | 15 +++++++++++++-- .../table/catalog/hive/client/HiveShim.java | 14 -------------- .../catalog/hive/client/HiveShimV100.java | 19 ------------------- .../catalog/hive/client/HiveShimV230.java | 7 ------- 4 files changed, 13 insertions(+), 42 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveMetastoreClientWrapper.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveMetastoreClientWrapper.java index 77d80390af6d..3da18751aa51 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveMetastoreClientWrapper.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveMetastoreClientWrapper.java @@ -221,8 +221,19 @@ private IMetaStoreClient createMetastoreClient() { } public Function getFunction(String databaseName, String functionName) throws MetaException, TException { - HiveShim hiveShim = HiveShimLoader.loadHiveShim(hiveVersion); - return hiveShim.getFunction(client, databaseName, functionName); + try { + // Hive may not throw NoSuchObjectException if function doesn't exist, instead it throws a MetaException + return client.getFunction(databaseName, functionName); + } catch (MetaException e) { + // need to check the cause and message of this MetaException to decide whether it should actually be a NoSuchObjectException + if (e.getCause() instanceof NoSuchObjectException) { + throw (NoSuchObjectException) e.getCause(); + } + if (e.getMessage().startsWith(NoSuchObjectException.class.getSimpleName())) { + throw new NoSuchObjectException(e.getMessage()); + } + throw e; + } } public void alter_table(String databaseName, String tableName, Table table) diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShim.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShim.java index 0b58724285e4..929d90e53d39 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShim.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShim.java @@ -28,10 +28,8 @@ import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData; import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.Function; import org.apache.hadoop.hive.metastore.api.InvalidOperationException; import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.api.UnknownDBException; @@ -76,18 +74,6 @@ public interface HiveShim extends Serializable { */ List getViews(IMetaStoreClient client, String databaseName) throws UnknownDBException, TException; - /** - * Gets a function from a database with the given HMS client. - * - * @param client the Hive Metastore client - * @param dbName name of the database - * @param functionName name of the function - * @return the Function under the specified name - * @throws NoSuchObjectException if the function doesn't exist - * @throws TException for any other generic exceptions caused by Thrift - */ - Function getFunction(IMetaStoreClient client, String dbName, String functionName) throws NoSuchObjectException, TException; - /** * Moves a particular file or directory to trash. * The file/directory can potentially be deleted (w/o going to trash) if purge is set to true, or if it cannot diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV100.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV100.java index e2f79d7c25b8..e5f3b6770786 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV100.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV100.java @@ -37,10 +37,8 @@ import org.apache.hadoop.hive.metastore.Warehouse; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData; import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.Function; import org.apache.hadoop.hive.metastore.api.InvalidOperationException; import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.api.UnknownDBException; @@ -115,23 +113,6 @@ public List getViews(IMetaStoreClient client, String databaseName) throw return views; } - @Override - public Function getFunction(IMetaStoreClient client, String dbName, String functionName) throws NoSuchObjectException, TException { - try { - // hive-1.x doesn't throw NoSuchObjectException if function doesn't exist, instead it throws a MetaException - return client.getFunction(dbName, functionName); - } catch (MetaException e) { - // need to check the cause and message of this MetaException to decide whether it should actually be a NoSuchObjectException - if (e.getCause() instanceof NoSuchObjectException) { - throw (NoSuchObjectException) e.getCause(); - } - if (e.getMessage().startsWith(NoSuchObjectException.class.getSimpleName())) { - throw new NoSuchObjectException(e.getMessage()); - } - throw e; - } - } - @Override public boolean moveToTrash(FileSystem fs, Path path, Configuration conf, boolean purge) throws IOException { try { diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV230.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV230.java index ac76cbab4f60..c373b6964efc 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV230.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV230.java @@ -28,10 +28,8 @@ import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.metastore.RetryingMetaStoreClient; import org.apache.hadoop.hive.metastore.TableType; -import org.apache.hadoop.hive.metastore.api.Function; import org.apache.hadoop.hive.metastore.api.InvalidOperationException; import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.api.UnknownDBException; import org.apache.hadoop.hive.ql.udf.generic.SimpleGenericUDAFParameterInfo; @@ -77,11 +75,6 @@ public List getViews(IMetaStoreClient client, String databaseName) throw } } - @Override - public Function getFunction(IMetaStoreClient client, String dbName, String functionName) throws NoSuchObjectException, TException { - return client.getFunction(dbName, functionName); - } - @Override public boolean moveToTrash(FileSystem fs, Path path, Configuration conf, boolean purge) throws IOException { try { From c1e9aefc2449a4ea0ff3fa590cf1eb6c1cb484a2 Mon Sep 17 00:00:00 2001 From: GuoWei Ma Date: Mon, 9 Sep 2019 19:02:28 +0800 Subject: [PATCH 534/746] [FLINK-14465] Let StandaloneJobClusterEntryPoint use the user code class loader [FLINK-14465] The PackageProgram's constructor does not throw excpetion any more when jarFile is null. Introducing this change is because there may be no jarFile in perjob mode. All jars the user code depends on are in the classpaths. [FLINK-14465] ClassPathJobGraphRetriever creates PackagesProgram with user class paths. [FLINK-14465] StandaloneJobClusterEntryPoint uses "FLINK_HOME/usrlib" as the job's class path. The environment variable FLINK_HOME is set at Dockerfile. Link the FLINK_JOB_ARTIFACTS_DIR to the FLINK_HOME/job, which makes the FlinkUserClassloader load the user class in the standalone perjob mode. This closes #10076. --- .../apache/flink/client/cli/CliFrontend.java | 9 +- .../flink/client/program/PackagedProgram.java | 255 ++++++++---------- .../client/program/PackagedProgramUtils.java | 8 + .../cli/CliFrontendPackageProgramTest.java | 1 + .../flink/client/program/ClientTest.java | 15 +- .../program/ExecutionPlanCreationTest.java | 5 +- .../client/program/PackagedProgramTest.java | 17 +- flink-container/docker/Dockerfile | 3 +- flink-container/docker/docker-entrypoint.sh | 2 +- flink-container/pom.xml | 60 +++++ .../ClassPathJobGraphRetriever.java | 149 ++++++++-- .../StandaloneJobClusterEntryPoint.java | 10 +- ...assembly-test-user-classloader-job-jar.xml | 35 +++ ...mbly-test-user-classloader-job-lib-jar.xml | 35 +++ .../ClassPathJobGraphRetrieverTest.java | 181 ++++++++++--- .../entrypoint/testjar/TestJobInfo.java | 33 +++ .../testjar/TestUserClassLoaderJob.java | 41 +++ .../testjar/TestUserClassLoaderJobLib.java | 32 +++ .../flink/configuration/ConfigConstants.java | 3 + .../java/org/apache/flink/util/FileUtils.java | 28 ++ .../examples/wordcount/WordCount.java | 1 - .../avro/AvroExternalJarProgramITCase.java | 5 +- .../handlers/EntryClassQueryParameter.java | 4 +- .../webmonitor/handlers/JarListHandler.java | 2 +- .../handlers/ProgramArgQueryParameter.java | 4 +- .../handlers/ProgramArgsQueryParameter.java | 4 +- .../handlers/utils/JarHandlerUtils.java | 9 +- .../runtime/entrypoint/ClusterEntrypoint.java | 2 +- .../runtime/util/ClusterEntrypointUtils.java | 62 +++++ .../test/classloading/ClassLoaderITCase.java | 58 ++-- .../flink/yarn/YarnConfigurationITCase.java | 2 +- 31 files changed, 813 insertions(+), 262 deletions(-) create mode 100644 flink-container/src/test/assembly/test-assembly-test-user-classloader-job-jar.xml create mode 100644 flink-container/src/test/assembly/test-assembly-test-user-classloader-job-lib-jar.xml create mode 100644 flink-container/src/test/java/org/apache/flink/container/entrypoint/testjar/TestJobInfo.java create mode 100644 flink-container/src/test/java/org/apache/flink/container/entrypoint/testjar/TestUserClassLoaderJob.java create mode 100644 flink-container/src/test/java/org/apache/flink/container/entrypoint/testjar/TestUserClassLoaderJobLib.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/ClusterEntrypointUtils.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 6f6e93326538..9552978c3546 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 @@ -796,9 +796,12 @@ PackagedProgram buildProgram( jarFile = getJarFile(jarFilePath); } - PackagedProgram program = entryPointClass == null ? - new PackagedProgram(jarFile, classpaths, programArgs) : - new PackagedProgram(jarFile, classpaths, entryPointClass, programArgs); + PackagedProgram program = PackagedProgram.newBuilder() + .setJarFile(jarFile) + .setUserClassPaths(classpaths) + .setEntryPointClassName(entryPointClass) + .setArguments(programArgs) + .build(); program.setSavepointRestoreSettings(executionParameters.getSavepointRestoreSettings()); 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 2f765b14ed7e..0a09367d3ae2 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 @@ -54,6 +54,10 @@ import java.util.jar.JarFile; import java.util.jar.Manifest; +import static org.apache.flink.client.program.PackagedProgramUtils.isPython; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * This class encapsulates represents a program, packaged in a jar file. It supplies * functionality to extract nested libraries, search for the program entry point, and extract @@ -92,88 +96,27 @@ public class PackagedProgram { */ private final boolean isPython; - /** - * Creates an instance that wraps the plan defined in the jar file using the given - * argument. - * - * @param jarFile - * The jar file which contains the plan and a Manifest which defines - * the program-class - * @param args - * Optional. The arguments used to create the pact plan, depend on - * implementation of the pact plan. See getDescription(). - * @throws ProgramInvocationException - * This invocation is thrown if the Program can't be properly loaded. Causes - * may be a missing / wrong class or manifest files. - */ - public PackagedProgram(File jarFile, String... args) throws ProgramInvocationException { - this(jarFile, Collections.emptyList(), null, args); - } - - /** - * Creates an instance that wraps the plan defined in the jar file using the given - * argument. - * - * @param jarFile - * The jar file which contains the plan and a Manifest which defines - * the program-class - * @param classpaths - * Additional classpath URLs needed by the Program. - * @param args - * Optional. The arguments used to create the pact plan, depend on - * implementation of the pact plan. See getDescription(). - * @throws ProgramInvocationException - * This invocation is thrown if the Program can't be properly loaded. Causes - * may be a missing / wrong class or manifest files. - */ - public PackagedProgram(File jarFile, List classpaths, String... args) throws ProgramInvocationException { - this(jarFile, classpaths, null, args); - } - /** * Creates an instance that wraps the plan defined in the jar file using the given * arguments. For generating the plan the class defined in the className parameter * is used. * - * @param jarFile - * The jar file which contains the plan. - * @param entryPointClassName - * Name of the class which generates the plan. Overrides the class defined - * in the jar file manifest - * @param args - * Optional. The arguments used to create the pact plan, depend on - * implementation of the pact plan. See getDescription(). - * @throws ProgramInvocationException - * This invocation is thrown if the Program can't be properly loaded. Causes - * may be a missing / wrong class or manifest files. + * @param jarFile The jar file which contains the plan. + * @param classpaths Additional classpath URLs needed by the Program. + * @param entryPointClassName Name of the class which generates the plan. Overrides the class defined + * in the jar file manifest + * @param args Optional. The arguments used to create the pact plan, depend on + * implementation of the pact plan. See getDescription(). + * @throws ProgramInvocationException This invocation is thrown if the Program can't be properly loaded. Causes + * may be a missing / wrong class or manifest files. */ - public PackagedProgram(File jarFile, @Nullable String entryPointClassName, String... args) throws ProgramInvocationException { - this(jarFile, Collections.emptyList(), entryPointClassName, args); - } + private PackagedProgram(@Nullable File jarFile, List classpaths, @Nullable String entryPointClassName, String... args) throws ProgramInvocationException { + checkNotNull(classpaths); + checkNotNull(args); + checkArgument(jarFile != null || entryPointClassName != null, "Either the jarFile or the entryPointClassName needs to be non-null."); - /** - * Creates an instance that wraps the plan defined in the jar file using the given - * arguments. For generating the plan the class defined in the className parameter - * is used. - * - * @param jarFile - * The jar file which contains the plan. - * @param classpaths - * Additional classpath URLs needed by the Program. - * @param entryPointClassName - * Name of the class which generates the plan. Overrides the class defined - * in the jar file manifest - * @param args - * Optional. The arguments used to create the pact plan, depend on - * implementation of the pact plan. See getDescription(). - * @throws ProgramInvocationException - * This invocation is thrown if the Program can't be properly loaded. Causes - * may be a missing / wrong class or manifest files. - */ - public PackagedProgram(File jarFile, List classpaths, @Nullable String entryPointClassName, String... args) throws ProgramInvocationException { // Whether the job is a Python job. - isPython = entryPointClassName != null && (entryPointClassName.equals("org.apache.flink.client.python.PythonDriver") - || entryPointClassName.equals("org.apache.flink.client.python.PythonGatewayServer")); + isPython = isPython(entryPointClassName); URL jarFileUrl = null; if (jarFile != null) { @@ -184,12 +127,10 @@ public PackagedProgram(File jarFile, List classpaths, @Nullable String entr } checkJarFile(jarFileUrl); - } else if (!isPython) { - throw new IllegalArgumentException("The jar file must not be null."); } this.jarFile = jarFileUrl; - this.args = args == null ? new String[0] : args; + this.args = args; // if no entryPointClassName name was given, we try and look one up through the manifest if (entryPointClassName == null) { @@ -209,23 +150,6 @@ public PackagedProgram(File jarFile, List classpaths, @Nullable String entr } } - public PackagedProgram(Class entryPointClass, String... args) throws ProgramInvocationException { - this.jarFile = null; - this.args = args == null ? new String[0] : args; - - this.extractedTempLibraries = Collections.emptyList(); - this.classpaths = Collections.emptyList(); - this.userCodeClassLoader = entryPointClass.getClassLoader(); - - // load the entry point class - this.mainClass = entryPointClass; - isPython = entryPointClass.getCanonicalName().equals("org.apache.flink.client.python.PythonDriver"); - - if (!hasMainMethod(mainClass)) { - throw new ProgramInvocationException("The given program class does not have a main(String[]) method."); - } - } - public void setSavepointRestoreSettings(SavepointRestoreSettings savepointSettings) { this.savepointSettings = savepointSettings; } @@ -247,9 +171,8 @@ public String getMainClassName() { * may contain a description of the plan itself and its arguments. * * @return The description of the PactProgram's input parameters. - * @throws ProgramInvocationException - * This invocation is thrown if the Program can't be properly loaded. Causes - * may be a missing / wrong class or manifest files. + * @throws ProgramInvocationException This invocation is thrown if the Program can't be properly loaded. Causes + * may be a missing / wrong class or manifest files. */ @Nullable public String getDescription() throws ProgramInvocationException { @@ -258,17 +181,16 @@ public String getDescription() throws ProgramInvocationException { ProgramDescription descr; try { descr = InstantiationUtil.instantiate( - this.mainClass.asSubclass(ProgramDescription.class), ProgramDescription.class); + this.mainClass.asSubclass(ProgramDescription.class), ProgramDescription.class); } catch (Throwable t) { return null; } try { return descr.getDescription(); - } - catch (Throwable t) { + } catch (Throwable t) { throw new ProgramInvocationException("Error while getting the program description" + - (t.getMessage() == null ? "." : ": " + t.getMessage()), t); + (t.getMessage() == null ? "." : ": " + t.getMessage()), t); } } else { @@ -280,7 +202,7 @@ public String getDescription() throws ProgramInvocationException { * This method assumes that the context environment is prepared, or the execution * will be a local execution by default. */ - public void invokeInteractiveModeForExecution() throws ProgramInvocationException{ + public void invokeInteractiveModeForExecution() throws ProgramInvocationException { callMainMethod(mainClass, args); } @@ -314,8 +236,7 @@ public List getAllLibraries() { for (File tmpLib : this.extractedTempLibraries) { try { libs.add(tmpLib.getAbsoluteFile().toURI().toURL()); - } - catch (MalformedURLException e) { + } catch (MalformedURLException e) { throw new RuntimeException("URL is invalid. This should not happen.", e); } } @@ -367,10 +288,9 @@ private static boolean hasMainMethod(Class entryClass) { mainMethod = entryClass.getMethod("main", String[].class); } catch (NoSuchMethodException e) { return false; - } - catch (Throwable t) { + } catch (Throwable t) { throw new RuntimeException("Could not look up the main(String[]) method from the class " + - entryClass.getName() + ": " + t.getMessage(), t); + entryClass.getName() + ": " + t.getMessage(), t); } return Modifier.isStatic(mainMethod.getModifiers()) && Modifier.isPublic(mainMethod.getModifiers()); @@ -386,10 +306,9 @@ private static void callMainMethod(Class entryClass, String[] args) throws Pr mainMethod = entryClass.getMethod("main", String[].class); } catch (NoSuchMethodException e) { throw new ProgramInvocationException("The class " + entryClass.getName() + " has no main(String[]) method."); - } - catch (Throwable t) { + } catch (Throwable t) { throw new ProgramInvocationException("Could not look up the main(String[]) method from the class " + - entryClass.getName() + ": " + t.getMessage(), t); + entryClass.getName() + ": " + t.getMessage(), t); } if (!Modifier.isStatic(mainMethod.getModifiers())) { @@ -401,14 +320,11 @@ private static void callMainMethod(Class entryClass, String[] args) throws Pr try { mainMethod.invoke(null, (Object) args); - } - catch (IllegalArgumentException e) { + } catch (IllegalArgumentException e) { throw new ProgramInvocationException("Could not invoke the main method, arguments are not matching.", e); - } - catch (IllegalAccessException e) { + } catch (IllegalAccessException e) { throw new ProgramInvocationException("Access to the main method was denied: " + e.getMessage(), e); - } - catch (InvocationTargetException e) { + } catch (InvocationTargetException e) { Throwable exceptionInMethod = e.getTargetException(); if (exceptionInMethod instanceof Error) { throw (Error) exceptionInMethod; @@ -419,8 +335,7 @@ private static void callMainMethod(Class entryClass, String[] args) throws Pr } else { throw new ProgramInvocationException("The main method caused an error: " + exceptionInMethod.getMessage(), exceptionInMethod); } - } - catch (Throwable t) { + } catch (Throwable t) { throw new ProgramInvocationException("An error occurred while invoking the program's main method: " + t.getMessage(), t); } } @@ -468,10 +383,9 @@ private static String getEntryPointClassNameFromJar(URL jarFile) throws ProgramI return className; } else { throw new ProgramInvocationException("Neither a '" + MANIFEST_ATTRIBUTE_MAIN_CLASS + "', nor a '" + - MANIFEST_ATTRIBUTE_ASSEMBLER_CLASS + "' entry was found in the jar file."); + MANIFEST_ATTRIBUTE_ASSEMBLER_CLASS + "' entry was found in the jar file."); } - } - finally { + } finally { try { jar.close(); } catch (Throwable t) { @@ -486,20 +400,16 @@ private static Class loadMainClass(String className, ClassLoader cl) throws P contextCl = Thread.currentThread().getContextClassLoader(); Thread.currentThread().setContextClassLoader(cl); return Class.forName(className, false, cl); - } - catch (ClassNotFoundException e) { + } catch (ClassNotFoundException e) { throw new ProgramInvocationException("The program's entry point class '" + className + "' was not found in the jar file.", e); - } - catch (ExceptionInInitializerError e) { + } catch (ExceptionInInitializerError e) { throw new ProgramInvocationException("The program's entry point class '" + className + "' threw an error during initialization.", e); - } - catch (LinkageError e) { + } catch (LinkageError e) { throw new ProgramInvocationException("The program's entry point class '" + className + "' could not be loaded due to a linkage failure.", e); - } - catch (Throwable t) { + } catch (Throwable t) { throw new ProgramInvocationException("The program's entry point class '" + className + "' caused an exception during initialization: " + t.getMessage(), t); } finally { @@ -537,8 +447,7 @@ public static List extractContainedLibraries(URL jarFile) throws ProgramIn if (containedJarFileEntries.isEmpty()) { return Collections.emptyList(); - } - else { + } else { // go over all contained jar files final List extractedTempLibraries = new ArrayList(containedJarFileEntries.size()); final byte[] buffer = new byte[4096]; @@ -557,11 +466,10 @@ public static List extractContainedLibraries(URL jarFile) throws ProgramIn try { tempFile = File.createTempFile(rnd.nextInt(Integer.MAX_VALUE) + "_", name); tempFile.deleteOnExit(); - } - catch (IOException e) { + } catch (IOException e) { throw new ProgramInvocationException( "An I/O error occurred while creating temporary file to extract nested library '" + - entry.getName() + "'.", e); + entry.getName() + "'.", e); } extractedTempLibraries.add(tempFile); @@ -578,12 +486,10 @@ public static List extractContainedLibraries(URL jarFile) throws ProgramIn while ((numRead = in.read(buffer)) != -1) { out.write(buffer, 0, numRead); } - } - catch (IOException e) { + } catch (IOException e) { throw new ProgramInvocationException("An I/O error occurred while extracting nested library '" - + entry.getName() + "' to temporary file '" + tempFile.getAbsolutePath() + "'."); - } - finally { + + entry.getName() + "' to temporary file '" + tempFile.getAbsolutePath() + "'."); + } finally { if (out != null) { out.close(); } @@ -594,8 +500,7 @@ public static List extractContainedLibraries(URL jarFile) throws ProgramIn } incomplete = false; - } - finally { + } finally { if (incomplete) { deleteExtractedLibraries(extractedTempLibraries); } @@ -603,15 +508,14 @@ public static List extractContainedLibraries(URL jarFile) throws ProgramIn return extractedTempLibraries; } - } - catch (Throwable t) { + } catch (Throwable t) { throw new ProgramInvocationException("Unknown I/O error while extracting contained jar files.", t); - } - finally { + } finally { if (jar != null) { try { jar.close(); - } catch (Throwable t) {} + } catch (Throwable t) { + } } } } @@ -625,13 +529,64 @@ public static void deleteExtractedLibraries(List tempLibraries) { private static void checkJarFile(URL jarfile) throws ProgramInvocationException { try { ClientUtils.checkJarFile(jarfile); - } - catch (IOException e) { + } catch (IOException e) { throw new ProgramInvocationException(e.getMessage(), e); - } - catch (Throwable t) { + } catch (Throwable t) { throw new ProgramInvocationException("Cannot access jar file" + (t.getMessage() == null ? "." : ": " + t.getMessage()), t); } } + /** + * A Builder For {@link PackagedProgram}. + */ + public static class Builder { + + @Nullable + private File jarFile; + + @Nullable + private String entryPointClassName; + + private String[] args = new String[0]; + + private List userClassPaths = Collections.emptyList(); + + public Builder setJarFile(@Nullable File jarFile) { + this.jarFile = jarFile; + return this; + } + + public Builder setUserClassPaths(List userClassPaths) { + this.userClassPaths = userClassPaths; + return this; + } + + public Builder setEntryPointClassName(@Nullable String entryPointClassName) { + this.entryPointClassName = entryPointClassName; + return this; + } + + public Builder setArguments(String... args) { + this.args = args; + return this; + } + + public PackagedProgram build() throws ProgramInvocationException { + if (jarFile == null && entryPointClassName == null) { + throw new IllegalArgumentException("The jarFile and entryPointClassName can not be null at the same time."); + } + return new PackagedProgram( + jarFile, + userClassPaths, + entryPointClassName, + args); + } + + private Builder() { + } + } + + public static Builder newBuilder() { + return new Builder(); + } } 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 fa9f8b0486d1..2f2719318a77 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 @@ -33,6 +33,9 @@ */ public class PackagedProgramUtils { + private static final String PYTHON_DRIVER_CLASS_NAME = "org.apache.flink.client.python.PythonDriver"; + + private static final String PYTHON_GATEWAY_CLASS_NAME = "org.apache.flink.client.python.PythonGatewayServer"; /** * Creates a {@link JobGraph} with a specified {@link JobID} * from the given {@link PackagedProgram}. @@ -102,5 +105,10 @@ public static Pipeline getPipelineFromProgram(PackagedProgram prog, int parallel } } + public static Boolean isPython(String entryPointClassName) { + return (entryPointClassName != null) && + (entryPointClassName.equals(PYTHON_DRIVER_CLASS_NAME) || entryPointClassName.equals(PYTHON_GATEWAY_CLASS_NAME)); + } + 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 2efb8ca9bc25..2b64c8238fb3 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 @@ -97,6 +97,7 @@ public void testFileNotJarFile() throws Exception { ProgramOptions programOptions = mock(ProgramOptions.class); ExecutionConfigAccessor executionOptions = mock(ExecutionConfigAccessor.class); when(executionOptions.getJarFilePath()).thenReturn(getNonJarFilePath()); + when(programOptions.getProgramArgs()).thenReturn(new String[0]); try { frontend.buildProgram(programOptions, executionOptions); 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 6c6ec1dfc85a..78023d7cd2e0 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 @@ -99,7 +99,7 @@ public void setUp() throws Exception { public void testDetachedMode() throws Exception{ final ClusterClient clusterClient = new MiniClusterClient(new Configuration(), MINI_CLUSTER_RESOURCE.getMiniCluster()); try { - PackagedProgram prg = new PackagedProgram(TestExecuteTwice.class); + PackagedProgram prg = PackagedProgram.newBuilder().setEntryPointClassName(TestExecuteTwice.class.getName()).build(); ClientUtils.executeProgram(clusterClient, prg, 1, true); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { @@ -109,7 +109,7 @@ public void testDetachedMode() throws Exception{ } try { - PackagedProgram prg = new PackagedProgram(TestEager.class); + PackagedProgram prg = PackagedProgram.newBuilder().setEntryPointClassName(TestEager.class.getName()).build(); ClientUtils.executeProgram(clusterClient, prg, 1, true); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { @@ -119,7 +119,7 @@ public void testDetachedMode() throws Exception{ } try { - PackagedProgram prg = new PackagedProgram(TestGetRuntime.class); + PackagedProgram prg = PackagedProgram.newBuilder().setEntryPointClassName(TestGetRuntime.class.getName()).build(); ClientUtils.executeProgram(clusterClient, prg, 1, true); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { @@ -129,7 +129,7 @@ public void testDetachedMode() throws Exception{ } try { - PackagedProgram prg = new PackagedProgram(TestGetAccumulator.class); + PackagedProgram prg = PackagedProgram.newBuilder().setEntryPointClassName(TestGetAccumulator.class.getName()).build(); ClientUtils.executeProgram(clusterClient, prg, 1, true); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { @@ -139,7 +139,7 @@ public void testDetachedMode() throws Exception{ } try { - PackagedProgram prg = new PackagedProgram(TestGetAllAccumulator.class); + PackagedProgram prg = PackagedProgram.newBuilder().setEntryPointClassName(TestGetAllAccumulator.class.getName()).build(); ClientUtils.executeProgram(clusterClient, prg, 1, true); fail(FAIL_MESSAGE); } catch (ProgramInvocationException e) { @@ -194,7 +194,10 @@ public Void answer(InvocationOnMock invocation) throws Throwable { @Test public void testGetExecutionPlan() throws ProgramInvocationException { - PackagedProgram prg = new PackagedProgram(TestOptimizerPlan.class, "/dev/random", "/tmp"); + PackagedProgram prg = PackagedProgram.newBuilder() + .setEntryPointClassName(TestOptimizerPlan.class.getName()) + .setArguments("/dev/random", "/tmp") + .build(); Optimizer optimizer = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), config); Plan plan = (Plan) PackagedProgramUtils.getPipelineFromProgram(prg, 1); 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 1b52f377c749..6343a2f22277 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 @@ -49,7 +49,10 @@ public class ExecutionPlanCreationTest { @Test public void testGetExecutionPlan() { try { - PackagedProgram prg = new PackagedProgram(TestOptimizerPlan.class, "/dev/random", "/tmp"); + PackagedProgram prg = PackagedProgram.newBuilder() + .setEntryPointClassName(TestOptimizerPlan.class.getName()) + .setArguments("/dev/random", "/tmp") + .build(); InetAddress mockAddress = InetAddress.getLocalHost(); InetSocketAddress mockJmAddress = new InetSocketAddress(mockAddress, 12345); diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/PackagedProgramTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/PackagedProgramTest.java index 7a7cf64a1ac3..c30d3f518c8e 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/PackagedProgramTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/PackagedProgramTest.java @@ -18,6 +18,7 @@ package org.apache.flink.client.program; +import org.apache.flink.client.cli.CliFrontendTestUtils; import org.apache.flink.configuration.ConfigConstants; import org.junit.Assert; @@ -28,10 +29,13 @@ import java.io.File; import java.io.FileOutputStream; import java.nio.file.Files; +import java.util.Collections; import java.util.List; import java.util.zip.ZipEntry; import java.util.zip.ZipOutputStream; +import static org.apache.flink.client.cli.CliFrontendTestUtils.TEST_JAR_MAIN_CLASS; + /** * Tests for the {@link PackagedProgram}. */ @@ -57,8 +61,15 @@ public void testExtractContainedLibraries() throws Exception { Assert.assertArrayEquals(nestedJarContent, Files.readAllBytes(files.iterator().next().toPath())); } - private static final class NullOutputStream extends java.io.OutputStream { - @Override - public void write(int b) {} + @Test + public void testNotThrowExceptionWhenJarFileIsNull() throws Exception { + PackagedProgram.newBuilder() + .setUserClassPaths(Collections.singletonList(new File(CliFrontendTestUtils.getTestJarPath()).toURI().toURL())) + .setEntryPointClassName(TEST_JAR_MAIN_CLASS); + } + + @Test(expected = IllegalArgumentException.class) + public void testBuilderThrowExceptionIfjarFileAndEntryPointClassNameAreBothNull() throws ProgramInvocationException { + PackagedProgram.newBuilder().build(); } } diff --git a/flink-container/docker/Dockerfile b/flink-container/docker/Dockerfile index a68835edff20..a0d3d8fb1bce 100644 --- a/flink-container/docker/Dockerfile +++ b/flink-container/docker/Dockerfile @@ -28,6 +28,7 @@ ENV FLINK_LIB_DIR $FLINK_HOME/lib ENV FLINK_PLUGINS_DIR $FLINK_HOME/plugins ENV FLINK_OPT_DIR $FLINK_HOME/opt ENV FLINK_JOB_ARTIFACTS_DIR $FLINK_INSTALL_PATH/artifacts +ENV FLINK_USR_LIB_DIR $FLINK_HOME/usrlib ENV PATH $PATH:$FLINK_HOME/bin # flink-dist can point to a directory or a tarball on the local system @@ -51,7 +52,7 @@ ADD $job_artifacts/* $FLINK_JOB_ARTIFACTS_DIR/ RUN set -x && \ ln -s $FLINK_INSTALL_PATH/flink-[0-9]* $FLINK_HOME && \ - for jar in $FLINK_JOB_ARTIFACTS_DIR/*.jar; do [ -f "$jar" ] || continue; ln -s $jar $FLINK_LIB_DIR; done && \ + ln -s $FLINK_JOB_ARTIFACTS_DIR $FLINK_USR_LIB_DIR && \ if [ -n "$python_version" ]; then ln -s $FLINK_OPT_DIR/flink-python*.jar $FLINK_LIB_DIR; fi && \ if [ -f ${FLINK_INSTALL_PATH}/flink-shaded-hadoop* ]; then ln -s ${FLINK_INSTALL_PATH}/flink-shaded-hadoop* $FLINK_LIB_DIR; fi && \ addgroup -S flink && adduser -D -S -H -G flink -h $FLINK_HOME flink && \ diff --git a/flink-container/docker/docker-entrypoint.sh b/flink-container/docker/docker-entrypoint.sh index 0bf7c04fa93c..0c1df000acb9 100755 --- a/flink-container/docker/docker-entrypoint.sh +++ b/flink-container/docker/docker-entrypoint.sh @@ -19,7 +19,7 @@ ################################################################################ ### If unspecified, the hostname of the container is taken as the JobManager address -FLINK_HOME=${FLINK_HOME:-"/opt/flink/bin"} +FLINK_HOME=${FLINK_HOME:-"/opt/flink"} JOB_CLUSTER="job-cluster" TASK_MANAGER="task-manager" diff --git a/flink-container/pom.xml b/flink-container/pom.xml index 78dfe3d909c5..e94579b39d64 100644 --- a/flink-container/pom.xml +++ b/flink-container/pom.xml @@ -95,6 +95,66 @@ under the License. + + create-test-dependency-user-jar + process-test-classes + + single + + + + + org.apache.flink.container.entrypoint.testjar.TestUserClassLoaderJob + + + maven + false + + src/test/assembly/test-assembly-test-user-classloader-job-jar.xml + + + + + create-test-dependency-user-jar-depend + process-test-classes + + single + + + maven + false + + src/test/assembly/test-assembly-test-user-classloader-job-lib-jar.xml + + + + + + + + maven-clean-plugin + 2.5 + + + remove-externaltestclasses + process-test-classes + + clean + + + true + + + ${project.build.testOutputDirectory} + + **/testjar/TestUser*.class + + + + + diff --git a/flink-container/src/main/java/org/apache/flink/container/entrypoint/ClassPathJobGraphRetriever.java b/flink-container/src/main/java/org/apache/flink/container/entrypoint/ClassPathJobGraphRetriever.java index 57808e332850..9ed109a49023 100644 --- a/flink-container/src/main/java/org/apache/flink/container/entrypoint/ClassPathJobGraphRetriever.java +++ b/flink-container/src/main/java/org/apache/flink/container/entrypoint/ClassPathJobGraphRetriever.java @@ -26,9 +26,12 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.container.entrypoint.JarManifestParser.JarFileWithEntryClass; +import org.apache.flink.runtime.entrypoint.component.AbstractUserClassPathJobGraphRetriever; import org.apache.flink.runtime.entrypoint.component.JobGraphRetriever; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FileUtils; import org.apache.flink.util.FlinkException; import org.slf4j.Logger; @@ -39,9 +42,14 @@ import java.io.File; import java.io.IOException; +import java.net.URL; +import java.util.ArrayList; import java.util.Arrays; import java.util.NoSuchElementException; import java.util.function.Supplier; +import java.util.jar.JarEntry; +import java.util.jar.JarFile; +import java.util.regex.Pattern; import java.util.stream.Collectors; import static java.util.Objects.requireNonNull; @@ -50,7 +58,7 @@ * {@link JobGraphRetriever} which creates the {@link JobGraph} from a class * on the class path. */ -class ClassPathJobGraphRetriever implements JobGraphRetriever { +class ClassPathJobGraphRetriever extends AbstractUserClassPathJobGraphRetriever { private static final Logger LOG = LoggerFactory.getLogger(ClassPathJobGraphRetriever.class); @@ -69,26 +77,23 @@ class ClassPathJobGraphRetriever implements JobGraphRetriever { @Nonnull private final Supplier> jarsOnClassPath; - ClassPathJobGraphRetriever( - @Nonnull JobID jobId, - @Nonnull SavepointRestoreSettings savepointRestoreSettings, - @Nonnull String[] programArguments, - @Nullable String jobClassName) { - this(jobId, savepointRestoreSettings, programArguments, jobClassName, JarsOnClassPath.INSTANCE); - } + @Nullable + private final File userLibDirectory; - @VisibleForTesting - ClassPathJobGraphRetriever( - @Nonnull JobID jobId, - @Nonnull SavepointRestoreSettings savepointRestoreSettings, - @Nonnull String[] programArguments, - @Nullable String jobClassName, - @Nonnull Supplier> jarsOnClassPath) { + private ClassPathJobGraphRetriever( + @Nonnull JobID jobId, + @Nonnull SavepointRestoreSettings savepointRestoreSettings, + @Nonnull String[] programArguments, + @Nullable String jobClassName, + @Nonnull Supplier> jarsOnClassPath, + @Nullable File userLibDirectory) throws IOException { + super(userLibDirectory); + this.userLibDirectory = userLibDirectory; this.jobId = requireNonNull(jobId, "jobId"); this.savepointRestoreSettings = requireNonNull(savepointRestoreSettings, "savepointRestoreSettings"); this.programArguments = requireNonNull(programArguments, "programArguments"); this.jobClassName = jobClassName; - this.jarsOnClassPath = requireNonNull(jarsOnClassPath, "jarsOnClassPath"); + this.jarsOnClassPath = requireNonNull(jarsOnClassPath); } @Override @@ -112,15 +117,28 @@ public JobGraph retrieveJobGraph(Configuration configuration) throws FlinkExcept private PackagedProgram createPackagedProgram() throws FlinkException { final String entryClass = getJobClassNameOrScanClassPath(); try { - final Class mainClass = getClass().getClassLoader().loadClass(entryClass); - return new PackagedProgram(mainClass, programArguments); - } catch (ClassNotFoundException | ProgramInvocationException e) { + return PackagedProgram.newBuilder() + .setUserClassPaths(new ArrayList<>(getUserClassPaths())) + .setEntryPointClassName(entryClass) + .setArguments(programArguments) + .build(); + } catch (ProgramInvocationException e) { throw new FlinkException("Could not load the provided entrypoint class.", e); } } private String getJobClassNameOrScanClassPath() throws FlinkException { if (jobClassName != null) { + if (userLibDirectory != null) { + // check that we find the entrypoint class in the user lib directory. + if (!userClassPathContainsJobClass(jobClassName)) { + throw new FlinkException( + String.format( + "Could not find the provided job class (%s) in the user lib directory (%s).", + jobClassName, + userLibDirectory)); + } + } return jobClassName; } @@ -131,10 +149,47 @@ private String getJobClassNameOrScanClassPath() throws FlinkException { } } + private boolean userClassPathContainsJobClass(String jobClassName) { + for (URL userClassPath : getUserClassPaths()) { + try (final JarFile jarFile = new JarFile(userClassPath.getFile())) { + if (jarContainsJobClass(jobClassName, jarFile)) { + return true; + } + } catch (IOException e) { + ExceptionUtils.rethrow( + e, + String.format( + "Failed to open user class path %s. Make sure that all files on the user class path can be accessed.", + userClassPath)); + } + } + return false; + } + + private boolean jarContainsJobClass(String jobClassName, JarFile jarFile) { + return jarFile + .stream() + .map(JarEntry::getName) + .filter(fileName -> fileName.endsWith(FileUtils.CLASS_FILE_EXTENSION)) + .map(FileUtils::stripFileExtension) + .map(fileName -> fileName.replaceAll(Pattern.quote(File.separator), FileUtils.PACKAGE_SEPARATOR)) + .anyMatch(name -> name.equals(jobClassName)); + } + private String scanClassPathForJobJar() throws IOException { - LOG.info("Scanning class path for job JAR"); - JarFileWithEntryClass jobJar = JarManifestParser.findOnlyEntryClass(jarsOnClassPath.get()); + final Iterable jars; + if (userLibDirectory == null) { + LOG.info("Scanning system class path for job JAR"); + jars = jarsOnClassPath.get(); + } else { + LOG.info("Scanning user class path for job JAR"); + jars = getUserClassPaths() + .stream() + .map(url -> new File(url.getFile())) + .collect(Collectors.toList()); + } + final JarFileWithEntryClass jobJar = JarManifestParser.findOnlyEntryClass(jars); LOG.info("Using {} as job jar", jobJar); return jobJar.getEntryClass(); } @@ -164,4 +219,56 @@ private static boolean notNullAndNotEmpty(String string) { } } + static class Builder { + + private final JobID jobId; + + private final SavepointRestoreSettings savepointRestoreSettings; + + private final String[] programArguments; + + @Nullable + private String jobClassName; + + @Nullable + private File userLibDirectory; + + private Supplier> jarsOnClassPath = JarsOnClassPath.INSTANCE; + + private Builder(JobID jobId, SavepointRestoreSettings savepointRestoreSettings, String[] programArguments) { + this.jobId = requireNonNull(jobId); + this.savepointRestoreSettings = requireNonNull(savepointRestoreSettings); + this.programArguments = requireNonNull(programArguments); + } + + Builder setJobClassName(@Nullable String jobClassName) { + this.jobClassName = jobClassName; + return this; + } + + Builder setUserLibDirectory(File userLibDirectory) { + this.userLibDirectory = userLibDirectory; + return this; + } + + Builder setJarsOnClassPath(Supplier> jarsOnClassPath) { + this.jarsOnClassPath = jarsOnClassPath; + return this; + } + + ClassPathJobGraphRetriever build() throws IOException { + return new ClassPathJobGraphRetriever( + jobId, + savepointRestoreSettings, + programArguments, + jobClassName, + jarsOnClassPath, + userLibDirectory); + } + } + + static Builder newBuilder(JobID jobId, SavepointRestoreSettings savepointRestoreSettings, String[] programArguments) { + return new Builder(jobId, savepointRestoreSettings, programArguments); + } + } diff --git a/flink-container/src/main/java/org/apache/flink/container/entrypoint/StandaloneJobClusterEntryPoint.java b/flink-container/src/main/java/org/apache/flink/container/entrypoint/StandaloneJobClusterEntryPoint.java index 8935f358d523..b5be5c415123 100644 --- a/flink-container/src/main/java/org/apache/flink/container/entrypoint/StandaloneJobClusterEntryPoint.java +++ b/flink-container/src/main/java/org/apache/flink/container/entrypoint/StandaloneJobClusterEntryPoint.java @@ -36,9 +36,11 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; +import java.io.IOException; import java.util.Optional; import static java.util.Objects.requireNonNull; +import static org.apache.flink.runtime.util.ClusterEntrypointUtils.tryFindUserLibDirectory; /** * {@link JobClusterEntrypoint} which is started with a job in a predefined @@ -74,10 +76,14 @@ private StandaloneJobClusterEntryPoint( } @Override - protected DispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration) { + protected DispatcherResourceManagerComponentFactory createDispatcherResourceManagerComponentFactory(Configuration configuration) throws IOException { + final ClassPathJobGraphRetriever.Builder classPathJobGraphRetrieverBuilder = ClassPathJobGraphRetriever.newBuilder(jobId, savepointRestoreSettings, programArguments) + .setJobClassName(jobClassName); + tryFindUserLibDirectory().ifPresent(classPathJobGraphRetrieverBuilder::setUserLibDirectory); + return DefaultDispatcherResourceManagerComponentFactory.createJobComponentFactory( StandaloneResourceManagerFactory.INSTANCE, - new ClassPathJobGraphRetriever(jobId, savepointRestoreSettings, programArguments, jobClassName)); + classPathJobGraphRetrieverBuilder.build()); } public static void main(String[] args) { diff --git a/flink-container/src/test/assembly/test-assembly-test-user-classloader-job-jar.xml b/flink-container/src/test/assembly/test-assembly-test-user-classloader-job-jar.xml new file mode 100644 index 000000000000..ac08f684ead3 --- /dev/null +++ b/flink-container/src/test/assembly/test-assembly-test-user-classloader-job-jar.xml @@ -0,0 +1,35 @@ + + + + test-user-classloader-job-jar + + jar + + false + + + ${project.build.testOutputDirectory} + / + + org/apache/flink/container/entrypoint/testjar/TestUserClassLoaderJob.class + + + + diff --git a/flink-container/src/test/assembly/test-assembly-test-user-classloader-job-lib-jar.xml b/flink-container/src/test/assembly/test-assembly-test-user-classloader-job-lib-jar.xml new file mode 100644 index 000000000000..b1513219044a --- /dev/null +++ b/flink-container/src/test/assembly/test-assembly-test-user-classloader-job-lib-jar.xml @@ -0,0 +1,35 @@ + + + + test-user-classloader-job-lib-jar + + jar + + false + + + ${project.build.testOutputDirectory} + / + + org/apache/flink/container/entrypoint/testjar/TestUserClassLoaderJobLib.class + + + + diff --git a/flink-container/src/test/java/org/apache/flink/container/entrypoint/ClassPathJobGraphRetrieverTest.java b/flink-container/src/test/java/org/apache/flink/container/entrypoint/ClassPathJobGraphRetrieverTest.java index 433263dff4d2..92228fd09a0a 100644 --- a/flink-container/src/test/java/org/apache/flink/container/entrypoint/ClassPathJobGraphRetrieverTest.java +++ b/flink-container/src/test/java/org/apache/flink/container/entrypoint/ClassPathJobGraphRetrieverTest.java @@ -22,21 +22,34 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.container.entrypoint.ClassPathJobGraphRetriever.JarsOnClassPath; +import org.apache.flink.container.entrypoint.testjar.TestJobInfo; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FileUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; +import org.apache.flink.util.function.FunctionUtils; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; import java.io.File; -import java.io.FileNotFoundException; import java.io.IOException; +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasItem; @@ -44,6 +57,7 @@ import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; /** * Tests for the {@link ClassPathJobGraphRetriever}. @@ -53,20 +67,69 @@ public class ClassPathJobGraphRetrieverTest extends TestLogger { @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + @ClassRule + public static final TemporaryFolder JOB_DIRS = new TemporaryFolder(); + private static final String[] PROGRAM_ARGUMENTS = {"--arg", "suffix"}; + /* + * The directory structure used to test + * + * userDirHasEntryClass/ + * |_jarWithEntryClass + * |_jarWithoutEntryClass + * |_textFile + * + * userDirHasNotEntryClass/ + * |_jarWithoutEntryClass + * |_textFile + */ + + private static final Collection expectedURLs = new ArrayList<>(); + + private static File userDirHasEntryClass; + + private static File userDirHasNotEntryClass; + + @BeforeClass + public static void init() throws IOException { + final String textFileName = "test.txt"; + final String userDirHasEntryClassName = "_test_user_dir_has_entry_class"; + final String userDirHasNotEntryClassName = "_test_user_dir_has_not_entry_class"; + + userDirHasEntryClass = JOB_DIRS.newFolder(userDirHasEntryClassName); + final Path userJarPath = userDirHasEntryClass.toPath().resolve(TestJobInfo.JOB_JAR_PATH.toFile().getName()); + final Path userLibJarPath = + userDirHasEntryClass.toPath().resolve(TestJobInfo.JOB_LIB_JAR_PATH.toFile().getName()); + userDirHasNotEntryClass = JOB_DIRS.newFolder(userDirHasNotEntryClassName); + + //create files + Files.copy(TestJobInfo.JOB_JAR_PATH, userJarPath); + Files.copy(TestJobInfo.JOB_LIB_JAR_PATH, userLibJarPath); + Files.createFile(userDirHasEntryClass.toPath().resolve(textFileName)); + + Files.copy(TestJobInfo.JOB_LIB_JAR_PATH, userDirHasNotEntryClass.toPath().resolve(TestJobInfo.JOB_LIB_JAR_PATH.toFile().getName())); + Files.createFile(userDirHasNotEntryClass.toPath().resolve(textFileName)); + + final Path workingDirectory = FileUtils.getCurrentWorkingDirectory(); + Arrays.asList(userJarPath, userLibJarPath) + .stream() + .map(path -> FileUtils.relativizePath(workingDirectory, path)) + .map(FunctionUtils.uncheckedFunction(FileUtils::toURL)) + .forEach(expectedURLs::add); + } + @Test - public void testJobGraphRetrieval() throws FlinkException { + public void testJobGraphRetrieval() throws FlinkException, IOException { final int parallelism = 42; final Configuration configuration = new Configuration(); configuration.setInteger(CoreOptions.DEFAULT_PARALLELISM, parallelism); final JobID jobId = new JobID(); - final ClassPathJobGraphRetriever classPathJobGraphRetriever = new ClassPathJobGraphRetriever( - jobId, - SavepointRestoreSettings.none(), - PROGRAM_ARGUMENTS, - TestJob.class.getCanonicalName()); + final ClassPathJobGraphRetriever classPathJobGraphRetriever = + ClassPathJobGraphRetriever.newBuilder(jobId, SavepointRestoreSettings.none(), PROGRAM_ARGUMENTS) + .setJobClassName(TestJob.class.getCanonicalName()) + .build(); final JobGraph jobGraph = classPathJobGraphRetriever.retrieveJobGraph(configuration); @@ -76,15 +139,12 @@ public void testJobGraphRetrieval() throws FlinkException { } @Test - public void testJobGraphRetrievalFromJar() throws FlinkException, FileNotFoundException { + public void testJobGraphRetrievalFromJar() throws FlinkException, IOException { final File testJar = TestJob.getTestJobJar(); - final ClassPathJobGraphRetriever classPathJobGraphRetriever = new ClassPathJobGraphRetriever( - new JobID(), - SavepointRestoreSettings.none(), - PROGRAM_ARGUMENTS, - // No class name specified, but the test JAR "is" on the class path - null, - () -> Collections.singleton(testJar)); + final ClassPathJobGraphRetriever classPathJobGraphRetriever = + ClassPathJobGraphRetriever.newBuilder(new JobID(), SavepointRestoreSettings.none(), PROGRAM_ARGUMENTS) + .setJarsOnClassPath(() -> Collections.singleton(testJar)) + .build(); final JobGraph jobGraph = classPathJobGraphRetriever.retrieveJobGraph(new Configuration()); @@ -92,17 +152,16 @@ public void testJobGraphRetrievalFromJar() throws FlinkException, FileNotFoundEx } @Test - public void testJobGraphRetrievalJobClassNameHasPrecedenceOverClassPath() throws FlinkException, FileNotFoundException { + public void testJobGraphRetrievalJobClassNameHasPrecedenceOverClassPath() throws FlinkException, IOException { final File testJar = new File("non-existing"); - final ClassPathJobGraphRetriever classPathJobGraphRetriever = new ClassPathJobGraphRetriever( - new JobID(), - SavepointRestoreSettings.none(), - PROGRAM_ARGUMENTS, - // Both a class name is specified and a JAR "is" on the class path - // The class name should have precedence. - TestJob.class.getCanonicalName(), - () -> Collections.singleton(testJar)); + final ClassPathJobGraphRetriever classPathJobGraphRetriever = + ClassPathJobGraphRetriever.newBuilder(new JobID(), SavepointRestoreSettings.none(), PROGRAM_ARGUMENTS) + // Both a class name is specified and a JAR "is" on the class path + // The class name should have precedence. + .setJobClassName(TestJob.class.getCanonicalName()) + .setJarsOnClassPath(() -> Collections.singleton(testJar)) + .build(); final JobGraph jobGraph = classPathJobGraphRetriever.retrieveJobGraph(new Configuration()); @@ -110,16 +169,15 @@ public void testJobGraphRetrievalJobClassNameHasPrecedenceOverClassPath() throws } @Test - public void testSavepointRestoreSettings() throws FlinkException { + public void testSavepointRestoreSettings() throws FlinkException, IOException { final Configuration configuration = new Configuration(); final SavepointRestoreSettings savepointRestoreSettings = SavepointRestoreSettings.forPath("foobar", true); final JobID jobId = new JobID(); - final ClassPathJobGraphRetriever classPathJobGraphRetriever = new ClassPathJobGraphRetriever( - jobId, - savepointRestoreSettings, - PROGRAM_ARGUMENTS, - TestJob.class.getCanonicalName()); + final ClassPathJobGraphRetriever classPathJobGraphRetriever = + ClassPathJobGraphRetriever.newBuilder(jobId, savepointRestoreSettings, PROGRAM_ARGUMENTS) + .setJobClassName(TestJob.class.getCanonicalName()) + .build(); final JobGraph jobGraph = classPathJobGraphRetriever.retrieveJobGraph(configuration); @@ -160,6 +218,68 @@ public void testJarFromClassPathSupplier() throws IOException { assertThat(jarFiles, contains(file1, file2)); } + @Test + public void testJobGraphRetrievalFailIfJobDirDoesNotHaveEntryClass() throws IOException { + final File testJar = TestJob.getTestJobJar(); + final ClassPathJobGraphRetriever classPathJobGraphRetriever = + ClassPathJobGraphRetriever.newBuilder(new JobID(), SavepointRestoreSettings.none(), PROGRAM_ARGUMENTS) + .setJarsOnClassPath(() -> Collections.singleton(testJar)) + .setUserLibDirectory(userDirHasNotEntryClass) + .build(); + try { + classPathJobGraphRetriever.retrieveJobGraph(new Configuration()); + Assert.fail("This case should throw exception !"); + } catch (FlinkException e) { + assertTrue(ExceptionUtils + .findThrowableWithMessage(e, "Failed to find job JAR on class path") + .isPresent()); + } + } + + @Test + public void testJobGraphRetrievalFailIfDoesNotFindTheEntryClassInTheJobDir() throws IOException { + final ClassPathJobGraphRetriever classPathJobGraphRetriever = + ClassPathJobGraphRetriever.newBuilder(new JobID(), SavepointRestoreSettings.none(), PROGRAM_ARGUMENTS) + .setJobClassName(TestJobInfo.JOB_CLASS) + .setJarsOnClassPath(Collections::emptyList) + .setUserLibDirectory(userDirHasNotEntryClass) + .build(); + try { + classPathJobGraphRetriever.retrieveJobGraph(new Configuration()); + Assert.fail("This case should throw class not found exception!!"); + } catch (FlinkException e) { + assertTrue(ExceptionUtils + .findThrowableWithMessage(e, "Could not find the provided job class") + .isPresent()); + } + + } + + @Test + public void testRetrieveCorrectUserClasspathsWithoutSpecifiedEntryClass() throws IOException, FlinkException { + final ClassPathJobGraphRetriever classPathJobGraphRetriever = + ClassPathJobGraphRetriever.newBuilder(new JobID(), SavepointRestoreSettings.none(), PROGRAM_ARGUMENTS) + .setJarsOnClassPath(Collections::emptyList) + .setUserLibDirectory(userDirHasEntryClass) + .build(); + final JobGraph jobGraph = classPathJobGraphRetriever.retrieveJobGraph(new Configuration()); + + assertThat(jobGraph.getClasspaths(), containsInAnyOrder(expectedURLs.toArray())); + } + + @Test + public void testRetrieveCorrectUserClasspathsWithSpecifiedEntryClass() throws IOException, FlinkException { + final ClassPathJobGraphRetriever classPathJobGraphRetriever = + ClassPathJobGraphRetriever.newBuilder(new JobID(), SavepointRestoreSettings.none(), PROGRAM_ARGUMENTS) + .setJobClassName(TestJobInfo.JOB_CLASS) + .setJarsOnClassPath(Collections::emptyList) + .setUserLibDirectory(userDirHasEntryClass) + .build(); + final JobGraph jobGraph = classPathJobGraphRetriever.retrieveJobGraph(new Configuration()); + + assertThat(jobGraph.getClasspaths(), containsInAnyOrder(expectedURLs.toArray())); + } + private static String javaClassPath(String... entries) { String pathSeparator = System.getProperty(JarsOnClassPath.PATH_SEPARATOR); return String.join(pathSeparator, entries); @@ -175,5 +295,4 @@ private static Iterable setClassPathAndGetJarsOnClassPath(String classPath System.setProperty(JarsOnClassPath.JAVA_CLASS_PATH, originalClassPath); } } - } diff --git a/flink-container/src/test/java/org/apache/flink/container/entrypoint/testjar/TestJobInfo.java b/flink-container/src/test/java/org/apache/flink/container/entrypoint/testjar/TestJobInfo.java new file mode 100644 index 000000000000..d682eccebc9a --- /dev/null +++ b/flink-container/src/test/java/org/apache/flink/container/entrypoint/testjar/TestJobInfo.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.container.entrypoint.testjar; + +import java.nio.file.Path; +import java.nio.file.Paths; + +/** + * The test job information. + */ +public class TestJobInfo { + + public static final String JOB_CLASS = "org.apache.flink.container.entrypoint.testjar.TestUserClassLoaderJob"; + public static final String JOB_LIB_CLASS = "org.apache.flink.container.entrypoint.testjar.TestUserClassLoaderJobLib"; + public static final Path JOB_JAR_PATH = Paths.get("target", "maven-test-user-classloader-job-jar.jar"); + public static final Path JOB_LIB_JAR_PATH = Paths.get("target", "maven-test-user-classloader-job-lib-jar.jar"); +} diff --git a/flink-container/src/test/java/org/apache/flink/container/entrypoint/testjar/TestUserClassLoaderJob.java b/flink-container/src/test/java/org/apache/flink/container/entrypoint/testjar/TestUserClassLoaderJob.java new file mode 100644 index 000000000000..1f68db037fff --- /dev/null +++ b/flink-container/src/test/java/org/apache/flink/container/entrypoint/testjar/TestUserClassLoaderJob.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.container.entrypoint.testjar; + +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.DiscardingSink; + +/** + * This class can used to test situation that the jar is not in the system classpath. + */ +public class TestUserClassLoaderJob { + public static void main(String[] args) throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + final DataStreamSource source = env.fromElements(new TestUserClassLoaderJobLib().getValue(), 1, 2, 3, 4); + final SingleOutputStreamOperator mapper = source.map(element -> 2 * element); + mapper.addSink(new DiscardingSink<>()); + + ParameterTool parameterTool = ParameterTool.fromArgs(args); + env.execute(TestUserClassLoaderJob.class.getCanonicalName() + "-" + parameterTool.getRequired("arg")); + } +} diff --git a/flink-container/src/test/java/org/apache/flink/container/entrypoint/testjar/TestUserClassLoaderJobLib.java b/flink-container/src/test/java/org/apache/flink/container/entrypoint/testjar/TestUserClassLoaderJobLib.java new file mode 100644 index 000000000000..82f5a29b33d8 --- /dev/null +++ b/flink-container/src/test/java/org/apache/flink/container/entrypoint/testjar/TestUserClassLoaderJobLib.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.container.entrypoint.testjar; + +/** + * This class is depended by {@link TestUserClassLoaderJob}. + */ +class TestUserClassLoaderJobLib { + + int getValue() { + return 0; + } + + public static void main(String[] args) { + } +} 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 d84b5626000c..b156ec844e55 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 @@ -2021,6 +2021,9 @@ public final class ConfigConstants { /** The environment variable name which contains the Flink installation root directory. */ public static final String ENV_FLINK_HOME_DIR = "FLINK_HOME"; + /** The user lib directory name. */ + public static final String DEFAULT_FLINK_USR_LIB_DIR = "usrlib"; + // ---------------------------- Encoding ------------------------------ public static final Charset DEFAULT_CHARSET = StandardCharsets.UTF_8; diff --git a/flink-core/src/main/java/org/apache/flink/util/FileUtils.java b/flink-core/src/main/java/org/apache/flink/util/FileUtils.java index 08ea86281d2a..37150edc5e64 100644 --- a/flink-core/src/main/java/org/apache/flink/util/FileUtils.java +++ b/flink-core/src/main/java/org/apache/flink/util/FileUtils.java @@ -84,6 +84,11 @@ public final class FileUtils { private static final String JAR_FILE_EXTENSION = "jar"; + public static final String CLASS_FILE_EXTENSION = "class"; + + public static final String PACKAGE_SEPARATOR = "."; + + // ------------------------------------------------------------------------ public static void writeCompletely(WritableByteChannel channel, ByteBuffer src) throws IOException { @@ -595,6 +600,16 @@ public static java.nio.file.Path getCurrentWorkingDirectory() { return Paths.get(System.getProperty("user.dir")); } + /** + * Checks whether the given file has a class extension. + * + * @param file to check + * @return true if the file has a class extension, otherwise false + */ + public static boolean isClassFile(java.nio.file.Path file) { + return CLASS_FILE_EXTENSION.equals(org.apache.flink.shaded.guava18.com.google.common.io.Files.getFileExtension(file.toString())); + } + /** * Checks whether the given file has a jar extension. * @@ -605,6 +620,19 @@ public static boolean isJarFile(java.nio.file.Path file) { return JAR_FILE_EXTENSION.equals(org.apache.flink.shaded.guava18.com.google.common.io.Files.getFileExtension(file.toString())); } + /** + * Remove the extension of the file name. + * @param fileName to strip + * @return the file name without extension + */ + public static String stripFileExtension(String fileName) { + final String extension = org.apache.flink.shaded.guava18.com.google.common.io.Files.getFileExtension(fileName); + if (!extension.isEmpty()) { + return fileName.substring(0, fileName.lastIndexOf(extension) - 1); + } + return fileName; + } + /** * Converts the given {@link java.nio.file.Path} into a file {@link URL}. The resulting url is * relative iff the given path is relative. diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java index ad34d7138033..56eb9b110c67 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java @@ -84,7 +84,6 @@ public static void main(String[] args) throws Exception { System.out.println("Printing result to stdout. Use --output to specify output path."); counts.print(); } - // execute program env.execute("Streaming WordCount"); } 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 fa79a26e23fb..281fb56ddc48 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 @@ -81,7 +81,10 @@ public void testExternalProgram() throws Exception { String testData = getClass().getResource(TEST_DATA_FILE).toString(); - PackagedProgram program = new PackagedProgram(new File(jarFile), new String[]{testData}); + PackagedProgram program = PackagedProgram.newBuilder() + .setJarFile(new File(jarFile)) + .setArguments(new String[]{testData}) + .build(); program.invokeInteractiveModeForExecution(); } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/EntryClassQueryParameter.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/EntryClassQueryParameter.java index e1c7c44e362f..70935bb199da 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/EntryClassQueryParameter.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/EntryClassQueryParameter.java @@ -18,11 +18,9 @@ package org.apache.flink.runtime.webmonitor.handlers; -import java.io.File; - /** * Query parameter specifying the name of the entry point class. - * @see org.apache.flink.client.program.PackagedProgram#PackagedProgram(File, String, String...) + * @see org.apache.flink.client.program.PackagedProgram.Builder#setEntryPointClassName(String) */ public class EntryClassQueryParameter extends StringQueryParameter { public EntryClassQueryParameter() { diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java index 359b65543cde..dc3b0beb02e9 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java @@ -129,7 +129,7 @@ protected CompletableFuture handleRequest(@Nonnull HandlerRequest tryFindUserLibDirectory() { + final File flinkHomeDirectory = deriveFlinkHomeDirectoryFromLibDirectory(); + final File usrLibDirectory = new File(flinkHomeDirectory, ConfigConstants.DEFAULT_FLINK_USR_LIB_DIR); + + if (!usrLibDirectory.isDirectory()) { + return Optional.empty(); + } + return Optional.of(usrLibDirectory); + } + + @Nullable + private static File deriveFlinkHomeDirectoryFromLibDirectory() { + final String libDirectory = System.getenv().get(ConfigConstants.ENV_FLINK_LIB_DIR); + + if (libDirectory == null) { + return null; + } else { + return new File(libDirectory).getParentFile(); + } + } +} 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 2f755193844d..23280f0d54a6 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 @@ -140,7 +140,9 @@ public void tearDown() { @Test public void testCustomSplitJobWithCustomClassLoaderJar() throws ProgramInvocationException { - PackagedProgram inputSplitTestProg = new PackagedProgram(new File(INPUT_SPLITS_PROG_JAR_FILE)); + PackagedProgram inputSplitTestProg = PackagedProgram.newBuilder() + .setJarFile(new File(INPUT_SPLITS_PROG_JAR_FILE)) + .build(); TestEnvironment.setAsContext( miniClusterResource.getMiniCluster(), @@ -153,7 +155,9 @@ public void testCustomSplitJobWithCustomClassLoaderJar() throws ProgramInvocatio @Test public void testStreamingCustomSplitJobWithCustomClassLoader() throws ProgramInvocationException { - PackagedProgram streamingInputSplitTestProg = new PackagedProgram(new File(STREAMING_INPUT_SPLITS_PROG_JAR_FILE)); + PackagedProgram streamingInputSplitTestProg = PackagedProgram.newBuilder() + .setJarFile(new File(STREAMING_INPUT_SPLITS_PROG_JAR_FILE)) + .build(); TestStreamEnvironment.setAsContext( miniClusterResource.getMiniCluster(), @@ -167,7 +171,9 @@ public void testStreamingCustomSplitJobWithCustomClassLoader() throws ProgramInv @Test public void testCustomSplitJobWithCustomClassLoaderPath() throws IOException, ProgramInvocationException { URL classpath = new File(INPUT_SPLITS_PROG_JAR_FILE).toURI().toURL(); - PackagedProgram inputSplitTestProg2 = new PackagedProgram(new File(INPUT_SPLITS_PROG_JAR_FILE)); + PackagedProgram inputSplitTestProg2 = PackagedProgram.newBuilder() + .setJarFile(new File(INPUT_SPLITS_PROG_JAR_FILE)) + .build(); TestEnvironment.setAsContext( miniClusterResource.getMiniCluster(), @@ -181,7 +187,7 @@ public void testCustomSplitJobWithCustomClassLoaderPath() throws IOException, Pr @Test public void testStreamingClassloaderJobWithCustomClassLoader() throws ProgramInvocationException { // regular streaming job - PackagedProgram streamingProg = new PackagedProgram(new File(STREAMING_PROG_JAR_FILE)); + PackagedProgram streamingProg = PackagedProgram.newBuilder().setJarFile(new File(STREAMING_PROG_JAR_FILE)).build(); TestStreamEnvironment.setAsContext( miniClusterResource.getMiniCluster(), @@ -196,7 +202,9 @@ public void testStreamingClassloaderJobWithCustomClassLoader() throws ProgramInv 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)); + PackagedProgram streamingCheckpointedProg = PackagedProgram.newBuilder() + .setJarFile(new File(STREAMING_CHECKPOINTED_PROG_JAR_FILE)) + .build(); TestStreamEnvironment.setAsContext( miniClusterResource.getMiniCluster(), @@ -230,13 +238,13 @@ public void testCheckpointedStreamingClassloaderJobWithCustomClassLoader() throw @Test public void testKMeansJobWithCustomClassLoader() throws ProgramInvocationException { - PackagedProgram kMeansProg = new PackagedProgram( - new File(KMEANS_JAR_PATH), - new String[] { + PackagedProgram kMeansProg = PackagedProgram.newBuilder() + .setJarFile(new File(KMEANS_JAR_PATH)) + .setArguments(new String[] { KMeansData.DATAPOINTS, KMeansData.INITIAL_CENTERS, - "25" - }); + "25"}) + .build(); TestEnvironment.setAsContext( miniClusterResource.getMiniCluster(), @@ -249,7 +257,9 @@ public void testKMeansJobWithCustomClassLoader() throws ProgramInvocationExcepti @Test public void testUserCodeTypeJobWithCustomClassLoader() throws ProgramInvocationException { - PackagedProgram userCodeTypeProg = new PackagedProgram(new File(USERCODETYPE_JAR_PATH)); + PackagedProgram userCodeTypeProg = PackagedProgram.newBuilder() + .setJarFile(new File(USERCODETYPE_JAR_PATH)) + .build(); TestEnvironment.setAsContext( miniClusterResource.getMiniCluster(), @@ -265,12 +275,10 @@ public void testCheckpointingCustomKvStateJobWithCustomClassLoader() throws IOEx File checkpointDir = FOLDER.newFolder(); File outputDir = FOLDER.newFolder(); - final PackagedProgram program = new PackagedProgram( - new File(CHECKPOINTING_CUSTOM_KV_STATE_JAR_PATH), - new String[] { - checkpointDir.toURI().toString(), - outputDir.toURI().toString() - }); + final PackagedProgram program = PackagedProgram.newBuilder() + .setJarFile(new File(CHECKPOINTING_CUSTOM_KV_STATE_JAR_PATH)) + .setArguments(new String[] { checkpointDir.toURI().toString(), outputDir.toURI().toString()}) + .build(); TestStreamEnvironment.setAsContext( miniClusterResource.getMiniCluster(), @@ -298,14 +306,14 @@ public void testDisposeSavepointWithCustomKvState() throws Exception { File checkpointDir = FOLDER.newFolder(); File outputDir = FOLDER.newFolder(); - final PackagedProgram program = new PackagedProgram( - new File(CUSTOM_KV_STATE_JAR_PATH), - new String[] { - String.valueOf(parallelism), - checkpointDir.toURI().toString(), - "5000", - outputDir.toURI().toString() - }); + final PackagedProgram program = PackagedProgram.newBuilder() + .setJarFile(new File(CUSTOM_KV_STATE_JAR_PATH)) + .setArguments(new String[] { + String.valueOf(parallelism), + checkpointDir.toURI().toString(), + "5000", + outputDir.toURI().toString()}) + .build(); TestStreamEnvironment.setAsContext( miniClusterResource.getMiniCluster(), 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 dfab48ef3c96..76370a22ba0b 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 @@ -108,7 +108,7 @@ public void testFlinkContainerMemory() throws Exception { final File streamingWordCountFile = getTestJarPath("WindowJoin.jar"); - final PackagedProgram packagedProgram = new PackagedProgram(streamingWordCountFile); + final PackagedProgram packagedProgram = PackagedProgram.newBuilder().setJarFile(streamingWordCountFile).build(); final JobGraph jobGraph = PackagedProgramUtils.createJobGraph(packagedProgram, configuration, 1); try { From ef4055ab989db431e29e0071573619d267960000 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Thu, 7 Nov 2019 15:43:23 +0100 Subject: [PATCH 535/746] [FLINK-14657] Generalize and move config utils from flink-yarn to flink-core --- .../client/cli/ExecutionConfigAccessor.java | 30 +++-- .../cli/ExecutionConfigurationUtils.java | 86 ------------- .../flink/configuration/ConfigUtils.java | 118 ++++++++++++++++++ .../flink/yarn/YarnClusterDescriptor.java | 4 +- .../flink/yarn/cli/FlinkYarnSessionCli.java | 3 +- .../flink/yarn/cli/YarnConfigUtils.java | 76 ----------- 6 files changed, 141 insertions(+), 176 deletions(-) delete mode 100644 flink-clients/src/main/java/org/apache/flink/client/cli/ExecutionConfigurationUtils.java create mode 100644 flink-core/src/main/java/org/apache/flink/configuration/ConfigUtils.java delete mode 100644 flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnConfigUtils.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/ExecutionConfigAccessor.java b/flink-clients/src/main/java/org/apache/flink/client/cli/ExecutionConfigAccessor.java index ba5341481c22..72a338a7b246 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/ExecutionConfigAccessor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/ExecutionConfigAccessor.java @@ -19,6 +19,8 @@ package org.apache.flink.client.cli; import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.DeploymentOptions; @@ -63,7 +65,10 @@ public static ExecutionConfigAccessor fromProgramOptions(final ProgramOptions op configuration.setBoolean(DeploymentOptions.ATTACHED, !options.getDetachedMode()); configuration.setBoolean(DeploymentOptions.SHUTDOWN_IF_ATTACHED, options.isShutdownOnAttachedExit()); - parseClasspathURLsToConfig(options.getClasspaths(), configuration); + if (options.getClasspaths() != null) { + ConfigUtils.encodeStreamToConfig(configuration, PipelineOptions.CLASSPATHS, options.getClasspaths().stream(), URL::toString); + } + parseJarURLToConfig(options.getJarFilePath(), configuration); SavepointRestoreSettings.toConfiguration(options.getSavepointRestoreSettings(), configuration); @@ -71,13 +76,6 @@ public static ExecutionConfigAccessor fromProgramOptions(final ProgramOptions op return new ExecutionConfigAccessor(configuration); } - private static void parseClasspathURLsToConfig(final List classpathURLs, final Configuration configuration) { - ExecutionConfigurationUtils.urlListToConfig( - classpathURLs, - configuration, - PipelineOptions.CLASSPATHS); - } - private static void parseJarURLToConfig(final String jarFile, final Configuration configuration) { if (jarFile == null) { return; @@ -86,7 +84,7 @@ private static void parseJarURLToConfig(final String jarFile, final Configuratio try { final URL jarUrl = new File(jarFile).getAbsoluteFile().toURI().toURL(); final List jarUrlSingleton = Collections.singletonList(jarUrl); - ExecutionConfigurationUtils.urlListToConfig(jarUrlSingleton, configuration, PipelineOptions.JARS); + ConfigUtils.encodeStreamToConfig(configuration, PipelineOptions.JARS, jarUrlSingleton.stream(), URL::toString); } catch (MalformedURLException e) { throw new IllegalArgumentException("JAR file path invalid", e); } @@ -97,7 +95,7 @@ public Configuration getConfiguration() { } public String getJarFilePath() { - final List jarURL = ExecutionConfigurationUtils.urlListFromConfig(configuration, PipelineOptions.JARS); + final List jarURL = decodeUrlList(configuration, PipelineOptions.JARS); if (jarURL != null && !jarURL.isEmpty()) { return jarURL.get(0).getPath(); } @@ -105,7 +103,17 @@ public String getJarFilePath() { } public List getClasspaths() { - return ExecutionConfigurationUtils.urlListFromConfig(configuration, PipelineOptions.CLASSPATHS); + return decodeUrlList(configuration, PipelineOptions.CLASSPATHS); + } + + private List decodeUrlList(final Configuration configuration, final ConfigOption> configOption) { + return ConfigUtils.decodeListFromConfig(configuration, configOption, url -> { + try { + return new URL(url); + } catch (MalformedURLException e) { + throw new IllegalArgumentException("Invalid URL", e); + } + }); } public int getParallelism() { diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/ExecutionConfigurationUtils.java b/flink-clients/src/main/java/org/apache/flink/client/cli/ExecutionConfigurationUtils.java deleted file mode 100644 index e25c129a4dc2..000000000000 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/ExecutionConfigurationUtils.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.client.cli; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.Configuration; - -import java.net.MalformedURLException; -import java.net.URL; -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * Utilities for parsing parameters in the {@link ExecutionConfigAccessor}. - */ -@Internal -class ExecutionConfigurationUtils { - - /** - * Parses a list of {@link URL URLs} to a string and puts it in the provided {@code configuration} as the value of the provided {@code option}. - * @param urls the list of URLs to parse - * @param configuration the configuration object to put the list - * @param option the {@link ConfigOption option} to serve as the key for the list in the configuration - * @return the produced list of strings to be put in the configuration. - */ - static List urlListToConfig( - final List urls, - final Configuration configuration, - final ConfigOption> option) { - - checkNotNull(urls); - checkNotNull(configuration); - checkNotNull(option); - - final List str = urls.stream().map(URL::toString).collect(Collectors.toList()); - configuration.set(option, str); - return str; - } - - /** - * Parses a string into a list of {@link URL URLs} from a given {@link Configuration}. - * @param configuration the configuration containing the string-ified list of URLs - * @param option the {@link ConfigOption option} whose value is the list of URLs - * @return the produced list of URLs. - */ - static List urlListFromConfig( - final Configuration configuration, - final ConfigOption> option) { - - checkNotNull(configuration); - checkNotNull(option); - - final List urls = configuration.get(option); - if (urls == null || urls.isEmpty()) { - return Collections.emptyList(); - } - - return urls.stream().map(str -> { - try { - return new URL(str); - } catch (MalformedURLException e) { - throw new IllegalArgumentException("Invalid URL", e); - } - }).collect(Collectors.toList()); - } -} diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigUtils.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigUtils.java new file mode 100644 index 000000000000..be9504119a4f --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigUtils.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.configuration; + +import org.apache.flink.annotation.Internal; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * General utilities for parsing values to configuration options. + */ +@Internal +public class ConfigUtils { + + /** + * Puts an array of values of type {@code IN} in a {@link WritableConfig} + * as a {@link ConfigOption} of type {@link List} of type {@code OUT}. + * + * @param configuration the configuration object to put the list in + * @param key the {@link ConfigOption option} to serve as the key for the list in the configuration + * @param value the array of values to put as value for the {@code key} + * @param mapper the transformation function from {@code IN} to {@code OUT}. + */ + public static void encodeArrayToConfig( + final WritableConfig configuration, + final ConfigOption> key, + final IN[] value, + final Function mapper) { + if (value == null) { + return; + } + encodeStreamToConfig(configuration, key, Arrays.stream(value), mapper); + } + + /** + * Puts a {@link Stream} of values of type {@code IN} in a {@link WritableConfig} + * as a {@link ConfigOption} of type {@link List} of type {@code OUT}. + * + * @param configuration the configuration object to put the list in + * @param key the {@link ConfigOption option} to serve as the key for the list in the configuration + * @param values the stream of values to put as value for the {@code key} + * @param mapper the transformation function from {@code IN} to {@code OUT}. + */ + public static void encodeStreamToConfig( + final WritableConfig configuration, + final ConfigOption> key, + final Stream values, + final Function mapper) { + + checkNotNull(configuration); + checkNotNull(key); + checkNotNull(mapper); + + if (values == null) { + return; + } + + final List encodedOption = values + .map(mapper) + .filter(Objects::nonNull) + .collect(Collectors.toList()); + + if (!encodedOption.isEmpty()) { + configuration.set(key, encodedOption); + } + } + + /** + * Gets a {@link List} of values of type {@code IN} from a {@link ReadableConfig} + * and transforms it to a {@link List} of type {@code OUT} based on the provided {@code mapper} function. + * + * @param configuration the configuration object to get the value out of + * @param key the {@link ConfigOption option} to serve as the key for the list in the configuration + * @param mapper the transformation function from {@code IN} to {@code OUT}. + * @return the transformed values in a list of type {@code OUT}. + */ + public static List decodeListFromConfig( + final ReadableConfig configuration, + final ConfigOption> key, + final Function mapper) { + + checkNotNull(configuration); + checkNotNull(key); + checkNotNull(mapper); + + final List encodedString = configuration.get(key); + return encodedString != null + ? encodedString.stream().map(mapper).collect(Collectors.toList()) + : Collections.emptyList(); + } + + private ConfigUtils() { + } +} 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 ae59d9213e6b..aad6d3600ddd 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 @@ -27,6 +27,7 @@ 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.ConfigUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.DeploymentOptions; @@ -48,7 +49,6 @@ import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; import org.apache.flink.util.ShutdownHookUtil; -import org.apache.flink.yarn.cli.YarnConfigUtils; import org.apache.flink.yarn.configuration.YarnConfigOptions; import org.apache.flink.yarn.configuration.YarnConfigOptionsInternal; import org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint; @@ -185,7 +185,7 @@ public YarnClusterDescriptor( private Optional> decodeDirsToShipToCluster(final Configuration configuration) { checkNotNull(configuration); - final List files = YarnConfigUtils.decodeListFromConfig(configuration, YarnConfigOptions.SHIP_DIRECTORIES, File::new); + final List files = ConfigUtils.decodeListFromConfig(configuration, YarnConfigOptions.SHIP_DIRECTORIES, File::new); return files.isEmpty() ? Optional.empty() : Optional.of(files); } 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 970fdeb5a2aa..3fb8dfa1ad29 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 @@ -27,6 +27,7 @@ import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.deployment.DefaultClusterClientServiceLoader; import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.configuration.ConfigUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.DeploymentOptions; @@ -285,7 +286,7 @@ private void encodeDirsToShipToCluster(final Configuration configuration, final checkNotNull(configuration); if (cmd.hasOption(shipPath.getOpt())) { - YarnConfigUtils.encodeListToConfig( + ConfigUtils.encodeArrayToConfig( configuration, YarnConfigOptions.SHIP_DIRECTORIES, cmd.getOptionValues(this.shipPath.getOpt()), diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnConfigUtils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnConfigUtils.java deleted file mode 100644 index 3012d7cf4e6f..000000000000 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/YarnConfigUtils.java +++ /dev/null @@ -1,76 +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.cli; - -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.Configuration; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Objects; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * Utilities for parsing {@link org.apache.flink.configuration.ConfigOption configuration options}. - */ -public class YarnConfigUtils { - - public static void encodeListToConfig( - final Configuration configuration, - final ConfigOption> key, - final T[] value, - final Function mapper) { - encodeListToConfig(configuration, key, Arrays.stream(value), mapper); - } - - private static void encodeListToConfig( - final Configuration configuration, - final ConfigOption> key, - final Stream values, - final Function mapper) { - - checkNotNull(values); - checkNotNull(key); - checkNotNull(configuration); - - final List encodedString = values.map(mapper).filter(Objects::nonNull).collect(Collectors.toList()); - if (!encodedString.isEmpty()) { - configuration.set(key, encodedString); - } - } - - public static List decodeListFromConfig( - final Configuration configuration, - final ConfigOption> key, - final Function mapper) { - - checkNotNull(configuration); - checkNotNull(key); - - final List encodedString = configuration.get(key); - return encodedString != null - ? encodedString.stream().map(mapper).collect(Collectors.toList()) - : Collections.emptyList(); - } -} From 5aa00fb875b3a941b92e8c0aa658395a2d529abd Mon Sep 17 00:00:00 2001 From: yanghua Date: Tue, 22 Oct 2019 20:58:02 +0800 Subject: [PATCH 536/746] [FLINK-14395] Refactor ES 7 connectors to make them keep consistency with es 6 connectors --- .../flink-connector-elasticsearch7/pom.xml | 14 ++++- .../Elasticsearch7UpsertTableSink.java | 4 +- .../Elasticsearch7UpsertTableSinkFactory.java | 2 +- ....apache.flink.table.factories.TableFactory | 2 +- ...sticsearch7UpsertTableSinkFactoryTest.java | 3 +- .../pom.xml | 54 ------------------- .../src/main/resources/META-INF/NOTICE | 1 + .../src/test/resources/log4j-test.properties | 24 --------- 8 files changed, 18 insertions(+), 86 deletions(-) rename flink-connectors/{flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql => flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming}/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java (97%) rename flink-connectors/{flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql => flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming}/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java (97%) rename flink-connectors/{flink-sql-connector-elasticsearch7 => flink-connector-elasticsearch7}/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory (89%) rename flink-connectors/{flink-sql-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/sql => flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming}/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java (98%) delete mode 100644 flink-connectors/flink-sql-connector-elasticsearch7/src/test/resources/log4j-test.properties diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index 3c68eac4fe47..0b5fda5edefb 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -71,6 +71,19 @@ under the License. ${elasticsearch.version} + + + + org.apache.logging.log4j + log4j-to-slf4j + 2.11.1 + + @@ -174,7 +187,6 @@ under the License. org.apache.maven.plugins maven-surefire-plugin - 2.12.2 true 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-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java similarity index 97% rename from flink-connectors/flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java rename to flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java index b6cf75c9ed80..7795fb8ef5ca 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.sql.connectors.elasticsearch7; +package org.apache.flink.streaming.connectors.elasticsearch7; import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; @@ -27,8 +27,6 @@ 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; 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-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java similarity index 97% rename from flink-connectors/flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java rename to flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java index ca2f54499169..29208c80e73b 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.sql.connectors.elasticsearch7; +package org.apache.flink.streaming.connectors.elasticsearch7; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.SerializationSchema; 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-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory similarity index 89% rename from flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory rename to flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory index fe10b08d2bbc..5bcb91f38f7a 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.streaming.sql.connectors.elasticsearch7.Elasticsearch7UpsertTableSinkFactory +org.apache.flink.streaming.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-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java similarity index 98% rename from flink-connectors/flink-sql-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java rename to flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java index a047b02f9bdb..1b9845237b86 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.sql.connectors.elasticsearch7; +package org.apache.flink.streaming.connectors.elasticsearch7; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.serialization.SerializationSchema; @@ -37,7 +37,6 @@ 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; diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml index 63aa212dcc0a..096f4126da63 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml @@ -36,65 +36,11 @@ under the License. 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 - 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 80866c7c2e0b..7cc11198b0ee 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 @@ -17,6 +17,7 @@ This project bundles the following dependencies under the Apache Software Licens - org.apache.httpcomponents:httpcore:4.4.6 - org.apache.httpcomponents:httpcore-nio:4.4.11 - org.apache.logging.log4j:log4j-api:2.11.1 +- org.apache.logging.log4j:log4j-to-slf4j: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 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 deleted file mode 100644 index fcd865466684..000000000000 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/test/resources/log4j-test.properties +++ /dev/null @@ -1,24 +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. -################################################################################ - -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 From 73193a825594c18e92c0bc27f6151d43a6506042 Mon Sep 17 00:00:00 2001 From: Pablo Valtuille <43780129+pablovalt@users.noreply.github.com> Date: Fri, 8 Nov 2019 15:17:06 +0000 Subject: [PATCH 537/746] [hotfix] Update Javadoc in FlinkKafkaProducer.java fix number of options --- .../flink/streaming/connectors/kafka/FlinkKafkaProducer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java index 7bf191309b7b..232177bc66e4 100644 --- a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java +++ b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java @@ -118,7 +118,7 @@ public enum Semantic { * case any subsequent {@link FlinkKafkaProducer#snapshotState(FunctionSnapshotContext)} requests will fail * and {@link FlinkKafkaProducer} will keep using the {@link FlinkKafkaInternalProducer} * from the previous checkpoint. - * To decrease the chance of failing checkpoints there are three options: + * To decrease the chance of failing checkpoints there are four options: *

  • decrease number of max concurrent checkpoints
  • *
  • make checkpoints more reliable (so that they complete faster)
  • *
  • increase the delay between checkpoints
  • From 6dbfd26f5a5038b718d061b99cbb20ae98cc6808 Mon Sep 17 00:00:00 2001 From: yanghua Date: Tue, 29 Oct 2019 17:45:38 +0800 Subject: [PATCH 538/746] Fixed word typo : environment and refactored load savepoint exception message --- .../src/main/java/org/apache/flink/state/api/Savepoint.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/Savepoint.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/Savepoint.java index 96e4005c99c6..e5599f4c481e 100644 --- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/Savepoint.java +++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/Savepoint.java @@ -47,7 +47,7 @@ private Savepoint() {} * Loads an existing savepoint. Useful if you want to query, modify, or extend * the state of an existing application. * - * @param env The execution enviornment used to transform the savepoint. + * @param env The execution environment used to transform the savepoint. * @param path The path to an existing savepoint on disk. * @param stateBackend The state backend of the savepoint. */ @@ -59,7 +59,7 @@ public static ExistingSavepoint load(ExecutionEnvironment env, String path, Stat .stream() .map(OperatorState::getMaxParallelism) .max(Comparator.naturalOrder()) - .orElseThrow(() -> new RuntimeException("Savepoint's must contain at least one operator")); + .orElseThrow(() -> new RuntimeException("Savepoint must contain at least one operator state.")); SavepointMetadata metadata = new SavepointMetadata(maxParallelism, savepoint.getMasterStates(), savepoint.getOperatorStates()); return new ExistingSavepoint(env, metadata, stateBackend); From 11dc0daca71a40d49b3d338e212d55a28960359e Mon Sep 17 00:00:00 2001 From: yanghua Date: Wed, 30 Oct 2019 15:25:27 +0800 Subject: [PATCH 539/746] [hotfix] Rename TimeReaderFunction to TimerReaderFunction --- .../flink/state/api/input/KeyedStateInputFormatTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/input/KeyedStateInputFormatTest.java b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/input/KeyedStateInputFormatTest.java index 132751968495..e036bc888a60 100644 --- a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/input/KeyedStateInputFormatTest.java +++ b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/input/KeyedStateInputFormatTest.java @@ -145,10 +145,10 @@ public void testReadTime() throws Exception { OperatorState operatorState = new OperatorState(operatorID, 1, 128); operatorState.putState(0, state); - KeyedStateInputFormat format = new KeyedStateInputFormat<>(operatorState, new MemoryStateBackend(), Types.INT, new TimeReaderFunction()); + KeyedStateInputFormat format = new KeyedStateInputFormat<>(operatorState, new MemoryStateBackend(), Types.INT, new TimerReaderFunction()); KeyGroupRangeInputSplit split = format.createInputSplits(1)[0]; - KeyedStateReaderFunction userFunction = new TimeReaderFunction(); + KeyedStateReaderFunction userFunction = new TimerReaderFunction(); List data = readInputSplit(split, userFunction); @@ -269,7 +269,7 @@ public void processElement(Integer value, Context ctx, Collector out) thro } } - static class TimeReaderFunction extends KeyedStateReaderFunction { + static class TimerReaderFunction extends KeyedStateReaderFunction { ValueState state; @Override From a815b7bfed52e9aec5f189c1fe94f13d6a3ac38d Mon Sep 17 00:00:00 2001 From: stayhsfLee Date: Tue, 29 Oct 2019 17:09:54 +0800 Subject: [PATCH 540/746] [hotfix][JavaDocs] Correct docs in MemoryStateBackend --- .../apache/flink/runtime/state/memory/MemoryStateBackend.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java index b2ff0e8af2a4..c0fbb9ae7597 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java @@ -66,7 +66,7 @@ * main memory, reducing operational stability. * For any other setup, the {@link org.apache.flink.runtime.state.filesystem.FsStateBackend FsStateBackend} * should be used. The {@code FsStateBackend} holds the working state on the TaskManagers in the same way, but - * checkpoints state directly to files rather then to the JobManager's memory, thus supporting + * checkpoints state directly to files rather than to the JobManager's memory, thus supporting * large state sizes. * *

    State Size Considerations

    From 9e0b2499e768213d19f2d8d1ffdd9a64293df5b5 Mon Sep 17 00:00:00 2001 From: "yuzhao.cyz" Date: Wed, 6 Nov 2019 13:31:13 +0800 Subject: [PATCH 541/746] [FLINK-14665][table-planner-blink] Support computed column in blink-planner This closes #10123 --- .../planner/calcite/FlinkToRelContext.java | 44 +++ .../QueryOperationCatalogViewTable.java | 4 +- .../planner/delegation/PlannerContext.java | 2 +- .../operations/SqlToOperationConverter.java | 43 ++- .../planner/calcite/FlinkPlannerImpl.scala | 60 ++-- .../logical/FlinkLogicalTableSourceScan.scala | 6 +- .../physical/PhysicalTableSourceScan.scala | 6 +- .../plan/schema/FlinkRelOptTable.scala | 57 +++- .../plan/schema/TableSourceTable.scala | 3 +- .../planner/sources/TableSourceUtil.scala | 15 +- .../SqlToOperationConverterTest.java | 45 +++ .../testGetDigestWithDynamicFunctionView.out | 3 - .../flink/table/api/stream/ExplainTest.xml | 162 +++++---- .../plan/batch/sql/DagOptimizationTest.xml | 154 ++++----- .../table/planner/plan/batch/sql/SinkTest.xml | 18 +- .../plan/schema/ComputedColumnToRelTest.xml | 67 ++++ .../plan/stream/sql/DagOptimizationTest.xml | 192 +++++------ .../stream/sql/MiniBatchIntervalInferTest.xml | 138 ++++---- .../table/api/TableEnvironmentTest.scala | 7 +- .../planner/catalog/CatalogTableITCase.scala | 308 ++++++++++++++++-- .../WatermarkGeneratorCodeGenTest.scala | 5 +- .../utils/TestCollectionTableFactory.scala | 14 +- .../plan/schema/ComputedColumnToRelTest.scala | 191 +++++++++++ .../plan/utils/FlinkRelOptUtilTest.scala | 19 +- .../batch/sql/PartitionableSinkITCase.scala | 2 +- .../batch/sql/agg/AggregateITCaseBase.scala | 4 +- .../sql/agg/DistinctAggregateITCaseBase.scala | 2 +- .../batch/sql/agg/GroupingSetsITCase.scala | 10 +- .../batch/sql/join/InnerJoinITCase.scala | 8 +- .../planner/runtime/utils/TestData.scala | 48 +-- 30 files changed, 1120 insertions(+), 517 deletions(-) create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/calcite/FlinkToRelContext.java create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/schema/ComputedColumnToRelTest.xml create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/schema/ComputedColumnToRelTest.scala diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/calcite/FlinkToRelContext.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/calcite/FlinkToRelContext.java new file mode 100644 index 000000000000..401178e7d746 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/calcite/FlinkToRelContext.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.calcite; + +import org.apache.calcite.plan.RelOptTable; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.tools.RelBuilder; + +/** + * A ToRelContext impl that takes the context variables + * used for sql expression transformation. + */ +public interface FlinkToRelContext extends RelOptTable.ToRelContext { + + /** + * Creates a new instance of {@link SqlExprToRexConverter} to convert sql statements + * to {@link org.apache.calcite.rex.RexNode}. + * + *

    See {@link org.apache.flink.table.planner.plan.schema.FlinkRelOptTable#toRel} + * for details. + */ + SqlExprToRexConverter createSqlExprToRexConverter(RelDataType tableRowType); + + /** + * Creates a new instance of {@link RelBuilder} to build relational expressions. + */ + RelBuilder createRelBuilder(); +} diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/QueryOperationCatalogViewTable.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/QueryOperationCatalogViewTable.java index 1251dc4abfed..43908b2c98c3 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/QueryOperationCatalogViewTable.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/QueryOperationCatalogViewTable.java @@ -30,7 +30,6 @@ import org.apache.flink.table.types.logical.LogicalType; import org.apache.calcite.plan.RelOptTable; -import org.apache.calcite.plan.RelOptUtil; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeFactory; @@ -81,8 +80,7 @@ private QueryOperationCatalogViewTable( public RelNode toRel(RelOptTable.ToRelContext context, RelOptTable relOptTable) { FlinkRelBuilder relBuilder = FlinkRelBuilder.of(context.getCluster(), relOptTable); - RelNode relNode = relBuilder.queryOperation(catalogView.getQueryOperation()).build(); - return RelOptUtil.createCastRel(relNode, rowType.apply(relBuilder.getTypeFactory()), false); + return relBuilder.queryOperation(catalogView.getQueryOperation()).build(); } @Override diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java index 66603d17b9ff..9093d84718bc 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java @@ -236,7 +236,7 @@ private SqlToRelConverter.Config getSqlToRelConverterConfig(CalciteConfig calcit return JavaScalaConversionUtil.toJava(calciteConfig.getSqlToRelConverterConfig()).orElseGet( () -> SqlToRelConverter.configBuilder() .withTrimUnusedFields(false) - .withConvertTableAccess(false) + .withConvertTableAccess(true) .withInSubQueryThreshold(Integer.MAX_VALUE) .withExpand(false) .build() 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 116951e518f4..9a9da5a4f2e5 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 @@ -201,30 +201,45 @@ private Operation convertSqlQuery(SqlNode node) { * *

    The returned table schema contains columns (a:int, b:varchar, c:timestamp). * - * @param sqlCreateTable sql create table node. + * @param sqlCreateTable sql create table node * @return TableSchema */ private TableSchema createTableSchema(SqlCreateTable sqlCreateTable) { - if (sqlCreateTable.containsComputedColumn()) { - throw new SqlConversionException("Computed columns for DDL is not supported yet!"); - } - TableSchema.Builder builder = new TableSchema.Builder(); - SqlValidator validator = flinkPlanner.getOrCreateSqlValidator(); - // setup table columns + // Setup table columns. SqlNodeList columnList = sqlCreateTable.getColumnList(); - Map nameToTypeMap = new HashMap<>(); + // Collect the physical fields info first. + Map nameToType = new HashMap<>(); + final SqlValidator validator = flinkPlanner.getOrCreateSqlValidator(); for (SqlNode node : columnList.getList()) { if (node instanceof SqlTableColumn) { SqlTableColumn column = (SqlTableColumn) node; RelDataType relType = column.getType() .deriveType(validator, column.getType().getNullable()); String name = column.getName().getSimple(); - nameToTypeMap.put(name, relType); - DataType dataType = TypeConversions.fromLogicalToDataType( - FlinkTypeFactory.toLogicalType(relType)); - builder.field(name, dataType); + nameToType.put(name, relType); + } + } + final TableSchema.Builder builder = new TableSchema.Builder(); + // Build the table schema. + for (SqlNode node : columnList) { + if (node instanceof SqlTableColumn) { + SqlTableColumn column = (SqlTableColumn) node; + final String fieldName = column.getName().getSimple(); + assert nameToType.containsKey(fieldName); + builder.field(fieldName, + TypeConversions.fromLogicalToDataType( + FlinkTypeFactory.toLogicalType(nameToType.get(fieldName)))); } else if (node instanceof SqlBasicCall) { - // TODO: computed column ... + SqlBasicCall call = (SqlBasicCall) node; + SqlNode validatedExpr = validator + .validateParameterizedExpression(call.operand(0), nameToType); + final RelDataType validatedType = validator.getValidatedNodeType(validatedExpr); + builder.field(call.operand(1).toString(), + TypeConversions.fromLogicalToDataType( + FlinkTypeFactory.toLogicalType(validatedType)), + validatedExpr.toString()); + } else { + throw new TableException("Unexpected table column type!"); } } @@ -233,7 +248,7 @@ private TableSchema createTableSchema(SqlCreateTable sqlCreateTable) { String rowtimeAttribute = watermark.getEventTimeColumnName().toString(); SqlNode expression = watermark.getWatermarkStrategy(); // this will validate and expand function identifiers. - SqlNode validated = validator.validateParameterizedExpression(expression, nameToTypeMap); + SqlNode validated = validator.validateParameterizedExpression(expression, nameToType); RelDataType validatedType = validator.getValidatedNodeType(validated); DataType exprDataType = TypeConversions.fromLogicalToDataType( FlinkTypeFactory.toLogicalType(validatedType)); diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala index 064ccf3e0a5c..5105b350bc98 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala @@ -22,7 +22,6 @@ import org.apache.flink.sql.parser.ExtendedSqlNode import org.apache.flink.table.api.{TableException, ValidationException} import org.apache.calcite.config.NullCollation -import org.apache.calcite.plan.RelOptTable.ViewExpander import org.apache.calcite.plan._ import org.apache.calcite.prepare.CalciteCatalogReader import org.apache.calcite.rel.`type`.RelDataType @@ -30,7 +29,7 @@ import org.apache.calcite.rel.{RelFieldCollation, RelRoot} import org.apache.calcite.sql.advise.{SqlAdvisor, SqlAdvisorValidator} import org.apache.calcite.sql.{SqlKind, SqlNode, SqlOperatorTable} import org.apache.calcite.sql2rel.{RelDecorrelator, SqlRexConvertletTable, SqlToRelConverter} -import org.apache.calcite.tools.{FrameworkConfig, RelConversionException} +import org.apache.calcite.tools.{FrameworkConfig, RelBuilder, RelConversionException} import java.lang.{Boolean => JBoolean} import java.util @@ -48,7 +47,7 @@ class FlinkPlannerImpl( config: FrameworkConfig, catalogReaderSupplier: JFunction[JBoolean, CalciteCatalogReader], typeFactory: FlinkTypeFactory, - cluster: RelOptCluster) { + cluster: RelOptCluster) extends FlinkToRelContext { val operatorTable: SqlOperatorTable = config.getOperatorTable val parser: CalciteParser = new CalciteParser(config.getParserConfig) @@ -122,7 +121,7 @@ class FlinkPlannerImpl( try { assert(validatedSqlNode != null) val sqlToRelConverter: SqlToRelConverter = new SqlToRelConverter( - new ViewExpanderImpl, + this, getOrCreateSqlValidator(), catalogReaderSupplier.apply(false), cluster, @@ -151,37 +150,36 @@ class FlinkPlannerImpl( new SqlExprToRexConverterImpl(config, typeFactory, cluster, tableRowType) } - /** Implements [[org.apache.calcite.plan.RelOptTable.ViewExpander]] - * interface for [[org.apache.calcite.tools.Planner]]. */ - class ViewExpanderImpl extends ViewExpander { + override def getCluster: RelOptCluster = cluster - override def expandView( - rowType: RelDataType, - queryString: String, - schemaPath: util.List[String], - viewPath: util.List[String]): RelRoot = { + override def expandView( + rowType: RelDataType, + queryString: String, + schemaPath: util.List[String], + viewPath: util.List[String]): RelRoot = { - val sqlNode = parser.parse(queryString) - val catalogReader = catalogReaderSupplier.apply(false) - .withSchemaPath(schemaPath) - val validator = - new FlinkCalciteSqlValidator(operatorTable, catalogReader, typeFactory) - validator.setIdentifierExpansion(true) - val validatedSqlNode = validator.validate(sqlNode) - val sqlToRelConverter = new SqlToRelConverter( - new ViewExpanderImpl, - validator, - catalogReader, - cluster, - convertletTable, - sqlToRelConverterConfig) - root = sqlToRelConverter.convertQuery(validatedSqlNode, true, false) - root = root.withRel(sqlToRelConverter.flattenTypes(root.project(), true)) - root = root.withRel(RelDecorrelator.decorrelateQuery(root.project())) - FlinkPlannerImpl.this.root - } + val sqlNode = parser.parse(queryString) + val catalogReader = catalogReaderSupplier.apply(false) + .withSchemaPath(schemaPath) + val validator = + new FlinkCalciteSqlValidator(operatorTable, catalogReader, typeFactory) + validator.setIdentifierExpansion(true) + val validatedSqlNode = validator.validate(sqlNode) + val sqlToRelConverter = new SqlToRelConverter( + this, + validator, + catalogReader, + cluster, + convertletTable, + sqlToRelConverterConfig) + var root: RelRoot = sqlToRelConverter.convertQuery(validatedSqlNode, true, false) + root = root.withRel(sqlToRelConverter.flattenTypes(root.project(), true)) + root.withRel(RelDecorrelator.decorrelateQuery(root.project())) } + override def createRelBuilder(): RelBuilder = { + sqlToRelConverterConfig.getRelBuilderFactory.create(cluster, null) + } } object FlinkPlannerImpl { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalTableSourceScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalTableSourceScan.scala index 04306d26065c..07389886538d 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalTableSourceScan.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalTableSourceScan.scala @@ -18,7 +18,6 @@ package org.apache.flink.table.planner.plan.nodes.logical -import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.FlinkConventions import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableSourceScan.isTableSourceScan import org.apache.flink.table.planner.plan.schema.{FlinkRelOptTable, TableSourceTable} @@ -63,8 +62,9 @@ class FlinkLogicalTableSourceScan( } override def deriveRowType(): RelDataType = { - val flinkTypeFactory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory] - tableSourceTable.getRowType(flinkTypeFactory) + // TableScan row type should always keep same with its + // interval RelOptTable's row type. + relOptTable.getRowType } override def computeSelfCost(planner: RelOptPlanner, mq: RelMetadataQuery): RelOptCost = { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/PhysicalTableSourceScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/PhysicalTableSourceScan.scala index 49ac5773a0a0..1338f9c20a00 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/PhysicalTableSourceScan.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/PhysicalTableSourceScan.scala @@ -23,7 +23,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.dag.Transformation import org.apache.flink.core.io.InputSplit import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment -import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.schema.{FlinkRelOptTable, TableSourceTable} import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter.fromDataTypeToTypeInfo import org.apache.flink.table.sources.{InputFormatTableSource, StreamTableSource, TableSource} @@ -53,8 +52,9 @@ abstract class PhysicalTableSourceScan( protected[flink] val tableSource: TableSource[_] = tableSourceTable.tableSource override def deriveRowType(): RelDataType = { - val flinkTypeFactory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory] - tableSourceTable.getRowType(flinkTypeFactory) + // TableScan row type should always keep same with its + // interval RelOptTable's row type. + relOptTable.getRowType } override def explainTerms(pw: RelWriter): RelWriter = { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/FlinkRelOptTable.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/FlinkRelOptTable.scala index e240c65c505b..fbbfdf29b99b 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/FlinkRelOptTable.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/FlinkRelOptTable.scala @@ -19,22 +19,20 @@ package org.apache.flink.table.planner.plan.schema import org.apache.flink.table.operations.TableSourceQueryOperation +import org.apache.flink.table.planner.calcite.FlinkToRelContext import org.apache.flink.table.planner.catalog.QueryOperationCatalogViewTable import org.apache.flink.table.planner.plan.`trait`.FlinkRelDistributionTraitDef import org.apache.flink.table.planner.plan.stats.FlinkStatistic import org.apache.flink.table.sources.TableSource import com.google.common.collect.{ImmutableList, ImmutableSet} -import org.apache.calcite.adapter.enumerable.EnumerableTableScan import org.apache.calcite.linq4j.tree.Expression import org.apache.calcite.plan.RelOptTable.ToRelContext import org.apache.calcite.plan.{RelOptCluster, RelOptSchema} import org.apache.calcite.prepare.Prepare.AbstractPreparingTable -import org.apache.calcite.prepare.{CalcitePrepareImpl, RelOptTableImpl} import org.apache.calcite.rel._ import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.logical.LogicalTableScan -import org.apache.calcite.runtime.Hook import org.apache.calcite.schema._ import org.apache.calcite.sql.SqlAccessType import org.apache.calcite.sql.validate.{SqlModality, SqlMonotonicity} @@ -44,6 +42,7 @@ import org.apache.calcite.util.{ImmutableBitSet, Util} import java.util.{List => JList, Set => JSet} import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ /** * [[FlinkRelOptTable]] wraps a [[FlinkTable]] @@ -65,6 +64,16 @@ class FlinkRelOptTable protected( // Sets a bigger default value to avoid broadcast join. val DEFAULT_ROWCOUNT: Double = 1E8 + lazy val columnExprs: Map[String, String] = table match { + case tableSourceTable : TableSourceTable[_] => + tableSourceTable.catalogTable.getSchema + .getTableColumns + .filter(column => column.isGenerated) + .map(column => (column.getName, column.getExpr.get())) + .toMap + case _ => Map() + } + // unique keySets of current table. lazy val uniqueKeysSet: Option[JSet[ImmutableBitSet]] = { table.getStatistic match { @@ -221,12 +230,42 @@ class FlinkRelOptTable protected( val cluster: RelOptCluster = context.getCluster if (table.isInstanceOf[TranslatableTable]) { table.asInstanceOf[TranslatableTable].toRel(context, this) - } else if (Hook.ENABLE_BINDABLE.get(false)) { - LogicalTableScan.create(cluster, this) - } else if (CalcitePrepareImpl.ENABLE_ENUMERABLE) { - EnumerableTableScan.create(cluster, this) } else { - throw new AssertionError + if (!context.isInstanceOf[FlinkToRelContext]) { + // If the transform comes from a RelOptRule, + // returns the scan directly. + LogicalTableScan.create(cluster, this) + } else { + // Get row type of physical fields. + val physicalFields = getRowType + .getFieldList + .filter(f => !columnExprs.contains(f.getName)) + .toList + val scanRowType = cluster.getTypeFactory.createStructType(physicalFields) + val scan = LogicalTableScan.create(cluster, copy(table, scanRowType)) + if (columnExprs.isEmpty) { + // There is no virtual columns, returns the scan. + scan + } else { + val toRelContext = context.asInstanceOf[FlinkToRelContext] + val relBuilder = toRelContext.createRelBuilder() + val fieldNames = rowType.getFieldNames.asScala + val fieldExprs = fieldNames + .map { name => + if (columnExprs.contains(name)) { + columnExprs(name) + } else { + name + } + }.toArray + val rexNodes = toRelContext + .createSqlExprToRexConverter(scanRowType) + .convertToRexNodes(fieldExprs) + relBuilder.push(scan) + .projectNamed(rexNodes.toList, fieldNames, true) + .build() + } + } } } @@ -313,8 +352,6 @@ class FlinkRelOptTable protected( ordinal: Int, initializerContext: InitializerContext): Boolean = false - override def getColumnStrategies: JList[ColumnStrategy] = RelOptTableImpl.columnStrategies(this) - override def extend(extendedTable: Table) = throw new RuntimeException("Extending column not supported") diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TableSourceTable.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TableSourceTable.scala index 50f88dcd9e97..d12806917b32 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TableSourceTable.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TableSourceTable.scala @@ -75,7 +75,8 @@ class TableSourceTable[T]( override def getRowType(typeFactory: RelDataTypeFactory): RelDataType = { val factory = typeFactory.asInstanceOf[FlinkTypeFactory] - val (fieldNames, fieldTypes) = TableSourceUtil.getFieldNamesTypes( + val (fieldNames, fieldTypes) = TableSourceUtil.getFieldNameType( + catalogTable.getSchema, tableSource, selectedFields, streaming = isStreamingMode) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sources/TableSourceUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sources/TableSourceUtil.scala index 514c08d424c8..3a2e39256875 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sources/TableSourceUtil.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sources/TableSourceUtil.scala @@ -20,7 +20,7 @@ package org.apache.flink.table.planner.sources import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.common.typeutils.CompositeType -import org.apache.flink.table.api.{DataTypes, ValidationException} +import org.apache.flink.table.api.{DataTypes, TableSchema, ValidationException} import org.apache.flink.table.expressions.utils.ApiExpressionUtils.{typeLiteral, valueLiteral} import org.apache.flink.table.expressions.{CallExpression, ResolvedExpression, ResolvedFieldReference} import org.apache.flink.table.functions.BuiltInFunctionDefinitions @@ -33,6 +33,7 @@ import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTyp import org.apache.flink.table.sources.{DefinedFieldMapping, DefinedProctimeAttribute, DefinedRowtimeAttributes, RowtimeAttributeDescriptor, TableSource} import org.apache.flink.table.types.logical.{LogicalType, TimestampKind, TimestampType, TinyIntType} import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo + import com.google.common.collect.ImmutableList import org.apache.calcite.plan.RelOptCluster import org.apache.calcite.rel.RelNode @@ -136,18 +137,22 @@ object TableSourceUtil { /** * Returns schema of the selected fields of the given [[TableSource]]. * - * @param tableSource The [[TableSource]] for which the Calcite schema is generated. + * @param tableSchema The [[TableSchema]] to derive the names and data types. + * This table schema should include all the columns, say, + * computed columns should also be included. + * @param tableSource The [[TableSource]] to derive time attributes. * @param selectedFields The indices of all selected fields. None, if all fields are selected. * @param streaming Flag to determine whether the schema of a stream or batch table is created. * @return The schema for the selected fields of the given [[TableSource]]. */ - def getFieldNamesTypes( + def getFieldNameType( + tableSchema: TableSchema, tableSource: TableSource[_], selectedFields: Option[Array[Int]], streaming: Boolean): (Seq[String], Seq[LogicalType]) = { - val fieldNames = tableSource.getTableSchema.getFieldNames - var fieldTypes = tableSource.getTableSchema.getFieldDataTypes + val fieldNames = tableSchema.getFieldNames + var fieldTypes = tableSchema.getFieldDataTypes .map(LogicalTypeDataTypeConverter.fromDataTypeToLogicalType) if (streaming) { diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/operations/SqlToOperationConverterTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/operations/SqlToOperationConverterTest.java index b8fba2f42dd2..7fa149e22684 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/operations/SqlToOperationConverterTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/operations/SqlToOperationConverterTest.java @@ -21,6 +21,7 @@ import org.apache.flink.sql.parser.ddl.SqlCreateTable; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.SqlDialect; +import org.apache.flink.table.api.TableColumn; import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.catalog.Catalog; @@ -31,6 +32,7 @@ import org.apache.flink.table.catalog.CatalogTableImpl; import org.apache.flink.table.catalog.FunctionCatalog; import org.apache.flink.table.catalog.GenericInMemoryCatalog; +import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.ObjectPath; import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; import org.apache.flink.table.catalog.exceptions.FunctionAlreadyExistException; @@ -44,6 +46,7 @@ import org.apache.flink.table.planner.calcite.FlinkPlannerImpl; import org.apache.flink.table.planner.catalog.CatalogManagerCalciteSchema; import org.apache.flink.table.planner.delegation.PlannerContext; +import org.apache.flink.table.planner.expressions.utils.Func0$; import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions; import org.apache.flink.table.types.DataType; @@ -380,6 +383,48 @@ public void testCreateTableWithFullDataTypes() { assertArrayEquals(expectedDataTypes, schema.getFieldDataTypes()); } + @Test + public void testCreateTableWithComputedColumn() { + final String sql = "CREATE TABLE tbl1 (\n" + + " a int,\n" + + " b varchar, \n" + + " c as a - 1, \n" + + " d as b || '$$', \n" + + " e as my_catalog.my_database.my_udf(a)\n" + + ")\n" + + " with (\n" + + " 'connector' = 'kafka', \n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + functionCatalog.registerTempCatalogScalarFunction( + ObjectIdentifier.of("my_catalog", "my_database", "my_udf"), + Func0$.MODULE$); + FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); + Operation operation = parse(sql, planner, getParserBySqlDialect(SqlDialect.DEFAULT)); + assert operation instanceof CreateTableOperation; + CreateTableOperation op = (CreateTableOperation) operation; + CatalogTable catalogTable = op.getCatalogTable(); + assertArrayEquals( + new String[] {"a", "b", "c", "d", "e"}, + catalogTable.getSchema().getFieldNames()); + assertArrayEquals( + new DataType[]{ + DataTypes.INT(), + DataTypes.STRING(), + DataTypes.INT(), + DataTypes.STRING(), + DataTypes.INT().notNull()}, + catalogTable.getSchema().getFieldDataTypes()); + String[] columnExpressions = + catalogTable.getSchema().getTableColumns().stream() + .filter(TableColumn::isGenerated) + .map(c -> c.getExpr().orElse(null)) + .toArray(String[]::new); + assertArrayEquals( + new String[] {"`a` - 1", "`b` || '$$'", "`my_catalog`.`my_database`.`my_udf`(`a`)"}, + columnExpressions); + } + //~ Tool Methods ---------------------------------------------------------- private static TestItem createTestItem(Object... args) { diff --git a/flink-table/flink-table-planner-blink/src/test/resources/digest/testGetDigestWithDynamicFunctionView.out b/flink-table/flink-table-planner-blink/src/test/resources/digest/testGetDigestWithDynamicFunctionView.out index c965d302f9ca..ac3b44b7a5f2 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/digest/testGetDigestWithDynamicFunctionView.out +++ b/flink-table/flink-table-planner-blink/src/test/resources/digest/testGetDigestWithDynamicFunctionView.out @@ -1,17 +1,14 @@ LogicalIntersect(all=[false],rowType=[RecordType(INTEGER random)]) LogicalIntersect(all=[false],rowType=[RecordType(INTEGER random)]) LogicalProject(random=[$0],rowType=[RecordType(INTEGER random)]) -LogicalProject(random=[$0],rowType=[RecordType(INTEGER random)]) LogicalSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[1],rowType=[RecordType(INTEGER random, DOUBLE EXPR$1)]) LogicalProject(random=[$1], EXPR$1=[RAND()],rowType=[RecordType(INTEGER random, DOUBLE EXPR$1)]) LogicalTableScan(table=[[default_catalog, default_database, MyTable]],rowType=[RecordType(VARCHAR(2147483647) first, INTEGER id, DOUBLE score, VARCHAR(2147483647) last)]) LogicalProject(random=[$0],rowType=[RecordType(INTEGER random)]) -LogicalProject(random=[$0],rowType=[RecordType(INTEGER random)]) LogicalSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[1],rowType=[RecordType(INTEGER random, DOUBLE EXPR$1)]) LogicalProject(random=[$1], EXPR$1=[RAND()],rowType=[RecordType(INTEGER random, DOUBLE EXPR$1)]) LogicalTableScan(table=[[default_catalog, default_database, MyTable]],rowType=[RecordType(VARCHAR(2147483647) first, INTEGER id, DOUBLE score, VARCHAR(2147483647) last)]) LogicalProject(random=[$0],rowType=[RecordType(INTEGER random)]) -LogicalProject(random=[$0],rowType=[RecordType(INTEGER random)]) LogicalSort(sort0=[$1], dir0=[ASC-nulls-first], fetch=[1],rowType=[RecordType(INTEGER random, DOUBLE EXPR$1)]) LogicalProject(random=[$1], EXPR$1=[RAND()],rowType=[RecordType(INTEGER random, DOUBLE EXPR$1)]) LogicalTableScan(table=[[default_catalog, default_database, MyTable]],rowType=[RecordType(VARCHAR(2147483647) first, INTEGER id, DOUBLE score, VARCHAR(2147483647) last)]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml index 46661c62c59f..9eef1f6079b9 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml @@ -638,47 +638,44 @@ SortLimit(orderBy=[a ASC], offset=[0], fetch=[5], updateAsRetraction=[false], ac LogicalSink(name=[`default_catalog`.`default_database`.`appendSink1`], fields=[a, b]) +- LogicalProject(id1=[$0], EXPR$1=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$1=[LISTAGG($2, $3)]) - +- LogicalProject(id1=[$0], $f1=[TUMBLE($1, 8000:INTERVAL SECOND)], text=[$2], $f3=[_UTF-16LE'#']) - +- LogicalProject(id1=[$0], ts=[$2], text=[$1]) - +- LogicalFilter(condition=[AND(=($0, $3), >($2, -($7, 300000:INTERVAL MINUTE)), <($2, +($7, 180000:INTERVAL MINUTE)))]) - +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalWatermarkAssigner(fields=[id1, text, rowtime], rowtimeField=[rowtime], watermarkDelay=[0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) - +- LogicalWatermarkAssigner(fields=[id2, cnt, name, goods, rowtime], rowtimeField=[rowtime], watermarkDelay=[0]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalProject(id1=[$0], $f1=[TUMBLE($2, 8000:INTERVAL SECOND)], text=[$1], $f3=[_UTF-16LE'#']) + +- LogicalFilter(condition=[AND(=($0, $3), >($2, -($7, 300000:INTERVAL MINUTE)), <($2, +($7, 180000:INTERVAL MINUTE)))]) + +- LogicalJoin(condition=[true], joinType=[inner]) + :- LogicalWatermarkAssigner(fields=[id1, text, rowtime], rowtimeField=[rowtime], watermarkDelay=[0]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + +- LogicalWatermarkAssigner(fields=[id2, cnt, name, goods, rowtime], rowtimeField=[rowtime], watermarkDelay=[0]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) LogicalSink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a, b]) +- LogicalProject(id1=[$0], EXPR$1=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$1=[LISTAGG($2, $3)]) - +- LogicalProject(id1=[$0], $f1=[HOP($1, 12000:INTERVAL SECOND, 6000:INTERVAL SECOND)], text=[$2], $f3=[_UTF-16LE'*']) - +- LogicalProject(id1=[$0], ts=[$2], text=[$1]) - +- LogicalFilter(condition=[AND(=($0, $3), >($2, -($7, 300000:INTERVAL MINUTE)), <($2, +($7, 180000:INTERVAL MINUTE)))]) - +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalWatermarkAssigner(fields=[id1, text, rowtime], rowtimeField=[rowtime], watermarkDelay=[0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) - +- LogicalWatermarkAssigner(fields=[id2, cnt, name, goods, rowtime], rowtimeField=[rowtime], watermarkDelay=[0]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalProject(id1=[$0], $f1=[HOP($2, 12000:INTERVAL SECOND, 6000:INTERVAL SECOND)], text=[$1], $f3=[_UTF-16LE'*']) + +- LogicalFilter(condition=[AND(=($0, $3), >($2, -($7, 300000:INTERVAL MINUTE)), <($2, +($7, 180000:INTERVAL MINUTE)))]) + +- LogicalJoin(condition=[true], joinType=[inner]) + :- LogicalWatermarkAssigner(fields=[id1, text, rowtime], rowtimeField=[rowtime], watermarkDelay=[0]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + +- LogicalWatermarkAssigner(fields=[id2, cnt, name, goods, rowtime], rowtimeField=[rowtime], watermarkDelay=[0]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) == Optimized Logical Plan == -Calc(select=[id1, rowtime AS ts, text], updateAsRetraction=[true], accMode=[Acc], reuse_id=[1]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} -+- WindowJoin(joinType=[InnerJoin], windowBounds=[isRowTime=true, leftLowerBound=-299999, leftUpperBound=179999, leftTimeIndex=2, rightTimeIndex=4], where=[AND(=(id1, id2), >(CAST(rowtime), -(CAST(rowtime0), 300000:INTERVAL MINUTE)), <(CAST(rowtime), +(CAST(rowtime0), 180000:INTERVAL MINUTE)))], select=[id1, text, rowtime, id2, cnt, name, goods, rowtime0], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} - :- Exchange(distribution=[hash[id1]], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} - : +- WatermarkAssigner(fields=[id1, text, rowtime], rowtimeField=[rowtime], watermarkDelay=[0], miniBatchInterval=[None], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} - : +- DataStreamScan(table=[[default_catalog, default_database, T1]], fields=[id1, text, rowtime], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} - +- Exchange(distribution=[hash[id2]], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} - +- WatermarkAssigner(fields=[id2, cnt, name, goods, rowtime], rowtimeField=[rowtime], watermarkDelay=[0], miniBatchInterval=[None], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} - +- DataStreamScan(table=[[default_catalog, default_database, T2]], fields=[id2, cnt, name, goods, rowtime], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +WindowJoin(joinType=[InnerJoin], windowBounds=[isRowTime=true, leftLowerBound=-299999, leftUpperBound=179999, leftTimeIndex=2, rightTimeIndex=4], where=[AND(=(id1, id2), >(CAST(rowtime), -(CAST(rowtime0), 300000:INTERVAL MINUTE)), <(CAST(rowtime), +(CAST(rowtime0), 180000:INTERVAL MINUTE)))], select=[id1, text, rowtime, id2, cnt, name, goods, rowtime0], updateAsRetraction=[true], accMode=[Acc], reuse_id=[1]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +:- Exchange(distribution=[hash[id1]], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +: +- WatermarkAssigner(fields=[id1, text, rowtime], rowtimeField=[rowtime], watermarkDelay=[0], miniBatchInterval=[None], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +: +- DataStreamScan(table=[[default_catalog, default_database, T1]], fields=[id1, text, rowtime], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} ++- Exchange(distribution=[hash[id2]], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} + +- WatermarkAssigner(fields=[id2, cnt, name, goods, rowtime], rowtimeField=[rowtime], watermarkDelay=[0], miniBatchInterval=[None], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} + +- DataStreamScan(table=[[default_catalog, default_database, T2]], fields=[id2, cnt, name, goods, rowtime], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} Sink(name=[`default_catalog`.`default_database`.`appendSink1`], fields=[a, b], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} -+- GroupWindowAggregate(groupBy=[id1], window=[TumblingGroupWindow('w$, ts, 8000)], select=[id1, LISTAGG(text, $f3) AS EXPR$1], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} ++- GroupWindowAggregate(groupBy=[id1], window=[TumblingGroupWindow('w$, rowtime, 8000)], select=[id1, LISTAGG(text, $f3) AS EXPR$1], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- Exchange(distribution=[hash[id1]], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} - +- Calc(select=[id1, ts, text, _UTF-16LE'#' AS $f3], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} + +- Calc(select=[id1, rowtime, text, _UTF-16LE'#' AS $f3], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- Reused(reference_id=[1]) Sink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a, b], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} -+- GroupWindowAggregate(groupBy=[id1], window=[SlidingGroupWindow('w$, ts, 6000, 12000)], select=[id1, LISTAGG(text, $f3) AS EXPR$1], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} ++- GroupWindowAggregate(groupBy=[id1], window=[SlidingGroupWindow('w$, rowtime, 6000, 12000)], select=[id1, LISTAGG(text, $f3) AS EXPR$1], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- Exchange(distribution=[hash[id1]], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} - +- Calc(select=[id1, ts, text, _UTF-16LE'*' AS $f3], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} + +- Calc(select=[id1, rowtime, text, _UTF-16LE'*' AS $f3], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- Reused(reference_id=[1]) == Physical Execution Plan == @@ -709,41 +706,37 @@ Sink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a, b], u ship_strategy : HASH : Operator - content : Calc(select=[id1, rowtime AS ts, text]) + content : Calc(select=[id1, rowtime, text, _UTF-16LE'#' AS $f3]) ship_strategy : FORWARD : Operator - content : Calc(select=[id1, ts, text, _UTF-16LE'#' AS $f3]) - ship_strategy : FORWARD + content : GroupWindowAggregate(groupBy=[id1], window=[TumblingGroupWindow('w$, rowtime, 8000)], select=[id1, LISTAGG(text, $f3) AS EXPR$1]) + ship_strategy : HASH : Operator - content : GroupWindowAggregate(groupBy=[id1], window=[TumblingGroupWindow('w$, ts, 8000)], select=[id1, LISTAGG(text, $f3) AS EXPR$1]) - ship_strategy : HASH + content : SinkConversionToRow + ship_strategy : FORWARD : Operator - content : SinkConversionToRow + content : Calc(select=[id1, rowtime, text, _UTF-16LE'*' AS $f3]) ship_strategy : FORWARD : Operator - content : Calc(select=[id1, ts, text, _UTF-16LE'*' AS $f3]) - ship_strategy : FORWARD + content : GroupWindowAggregate(groupBy=[id1], window=[SlidingGroupWindow('w$, rowtime, 6000, 12000)], select=[id1, LISTAGG(text, $f3) AS EXPR$1]) + ship_strategy : HASH : Operator - content : GroupWindowAggregate(groupBy=[id1], window=[SlidingGroupWindow('w$, ts, 6000, 12000)], select=[id1, LISTAGG(text, $f3) AS EXPR$1]) - ship_strategy : HASH + content : SinkConversionToRow + ship_strategy : FORWARD - : Operator - content : SinkConversionToRow + : Data Sink + content : Sink: TestingAppendTableSink ship_strategy : FORWARD : Data Sink content : Sink: TestingAppendTableSink ship_strategy : FORWARD - : Data Sink - content : Sink: TestingAppendTableSink - ship_strategy : FORWARD - ]]> @@ -785,47 +778,44 @@ Union(all=[true], union=[a, b, c]) LogicalSink(name=[`default_catalog`.`default_database`.`appendSink1`], fields=[a, b]) +- LogicalProject(id1=[$0], EXPR$1=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$1=[LISTAGG($2, $3)]) - +- LogicalProject(id1=[$0], $f1=[TUMBLE($1, 8000:INTERVAL SECOND)], text=[$2], $f3=[_UTF-16LE'#']) - +- LogicalProject(id1=[$0], ts=[$2], text=[$1]) - +- LogicalFilter(condition=[AND(=($0, $3), >($2, -($7, 300000:INTERVAL MINUTE)), <($2, +($7, 180000:INTERVAL MINUTE)))]) - +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalWatermarkAssigner(fields=[id1, text, rowtime], rowtimeField=[rowtime], watermarkDelay=[0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) - +- LogicalWatermarkAssigner(fields=[id2, cnt, name, goods, rowtime], rowtimeField=[rowtime], watermarkDelay=[0]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalProject(id1=[$0], $f1=[TUMBLE($2, 8000:INTERVAL SECOND)], text=[$1], $f3=[_UTF-16LE'#']) + +- LogicalFilter(condition=[AND(=($0, $3), >($2, -($7, 300000:INTERVAL MINUTE)), <($2, +($7, 180000:INTERVAL MINUTE)))]) + +- LogicalJoin(condition=[true], joinType=[inner]) + :- LogicalWatermarkAssigner(fields=[id1, text, rowtime], rowtimeField=[rowtime], watermarkDelay=[0]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + +- LogicalWatermarkAssigner(fields=[id2, cnt, name, goods, rowtime], rowtimeField=[rowtime], watermarkDelay=[0]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) LogicalSink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a, b]) +- LogicalProject(id1=[$0], EXPR$1=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$1=[LISTAGG($2, $3)]) - +- LogicalProject(id1=[$0], $f1=[HOP($1, 12000:INTERVAL SECOND, 6000:INTERVAL SECOND)], text=[$2], $f3=[_UTF-16LE'*']) - +- LogicalProject(id1=[$0], ts=[$2], text=[$1]) - +- LogicalFilter(condition=[AND(=($0, $3), >($2, -($7, 300000:INTERVAL MINUTE)), <($2, +($7, 180000:INTERVAL MINUTE)))]) - +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalWatermarkAssigner(fields=[id1, text, rowtime], rowtimeField=[rowtime], watermarkDelay=[0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) - +- LogicalWatermarkAssigner(fields=[id2, cnt, name, goods, rowtime], rowtimeField=[rowtime], watermarkDelay=[0]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalProject(id1=[$0], $f1=[HOP($2, 12000:INTERVAL SECOND, 6000:INTERVAL SECOND)], text=[$1], $f3=[_UTF-16LE'*']) + +- LogicalFilter(condition=[AND(=($0, $3), >($2, -($7, 300000:INTERVAL MINUTE)), <($2, +($7, 180000:INTERVAL MINUTE)))]) + +- LogicalJoin(condition=[true], joinType=[inner]) + :- LogicalWatermarkAssigner(fields=[id1, text, rowtime], rowtimeField=[rowtime], watermarkDelay=[0]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + +- LogicalWatermarkAssigner(fields=[id2, cnt, name, goods, rowtime], rowtimeField=[rowtime], watermarkDelay=[0]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) == Optimized Logical Plan == -Calc(select=[id1, rowtime AS ts, text], reuse_id=[1]) -+- WindowJoin(joinType=[InnerJoin], windowBounds=[isRowTime=true, leftLowerBound=-299999, leftUpperBound=179999, leftTimeIndex=2, rightTimeIndex=4], where=[AND(=(id1, id2), >(CAST(rowtime), -(CAST(rowtime0), 300000:INTERVAL MINUTE)), <(CAST(rowtime), +(CAST(rowtime0), 180000:INTERVAL MINUTE)))], select=[id1, text, rowtime, id2, cnt, name, goods, rowtime0]) - :- Exchange(distribution=[hash[id1]]) - : +- WatermarkAssigner(fields=[id1, text, rowtime], rowtimeField=[rowtime], watermarkDelay=[0], miniBatchInterval=[None]) - : +- DataStreamScan(table=[[default_catalog, default_database, T1]], fields=[id1, text, rowtime]) - +- Exchange(distribution=[hash[id2]]) - +- WatermarkAssigner(fields=[id2, cnt, name, goods, rowtime], rowtimeField=[rowtime], watermarkDelay=[0], miniBatchInterval=[None]) - +- DataStreamScan(table=[[default_catalog, default_database, T2]], fields=[id2, cnt, name, goods, rowtime]) +WindowJoin(joinType=[InnerJoin], windowBounds=[isRowTime=true, leftLowerBound=-299999, leftUpperBound=179999, leftTimeIndex=2, rightTimeIndex=4], where=[AND(=(id1, id2), >(CAST(rowtime), -(CAST(rowtime0), 300000:INTERVAL MINUTE)), <(CAST(rowtime), +(CAST(rowtime0), 180000:INTERVAL MINUTE)))], select=[id1, text, rowtime, id2, cnt, name, goods, rowtime0], reuse_id=[1]) +:- Exchange(distribution=[hash[id1]]) +: +- WatermarkAssigner(fields=[id1, text, rowtime], rowtimeField=[rowtime], watermarkDelay=[0], miniBatchInterval=[None]) +: +- DataStreamScan(table=[[default_catalog, default_database, T1]], fields=[id1, text, rowtime]) ++- Exchange(distribution=[hash[id2]]) + +- WatermarkAssigner(fields=[id2, cnt, name, goods, rowtime], rowtimeField=[rowtime], watermarkDelay=[0], miniBatchInterval=[None]) + +- DataStreamScan(table=[[default_catalog, default_database, T2]], fields=[id2, cnt, name, goods, rowtime]) Sink(name=[`default_catalog`.`default_database`.`appendSink1`], fields=[a, b]) -+- GroupWindowAggregate(groupBy=[id1], window=[TumblingGroupWindow('w$, ts, 8000)], select=[id1, LISTAGG(text, $f3) AS EXPR$1]) ++- GroupWindowAggregate(groupBy=[id1], window=[TumblingGroupWindow('w$, rowtime, 8000)], select=[id1, LISTAGG(text, $f3) AS EXPR$1]) +- Exchange(distribution=[hash[id1]]) - +- Calc(select=[id1, ts, text, _UTF-16LE'#' AS $f3]) + +- Calc(select=[id1, rowtime, text, _UTF-16LE'#' AS $f3]) +- Reused(reference_id=[1]) Sink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a, b]) -+- GroupWindowAggregate(groupBy=[id1], window=[SlidingGroupWindow('w$, ts, 6000, 12000)], select=[id1, LISTAGG(text, $f3) AS EXPR$1]) ++- GroupWindowAggregate(groupBy=[id1], window=[SlidingGroupWindow('w$, rowtime, 6000, 12000)], select=[id1, LISTAGG(text, $f3) AS EXPR$1]) +- Exchange(distribution=[hash[id1]]) - +- Calc(select=[id1, ts, text, _UTF-16LE'*' AS $f3]) + +- Calc(select=[id1, rowtime, text, _UTF-16LE'*' AS $f3]) +- Reused(reference_id=[1]) == Physical Execution Plan == @@ -856,41 +846,37 @@ Sink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a, b]) ship_strategy : HASH : Operator - content : Calc(select=[id1, rowtime AS ts, text]) + content : Calc(select=[id1, rowtime, text, _UTF-16LE'#' AS $f3]) ship_strategy : FORWARD : Operator - content : Calc(select=[id1, ts, text, _UTF-16LE'#' AS $f3]) - ship_strategy : FORWARD + content : GroupWindowAggregate(groupBy=[id1], window=[TumblingGroupWindow('w$, rowtime, 8000)], select=[id1, LISTAGG(text, $f3) AS EXPR$1]) + ship_strategy : HASH : Operator - content : GroupWindowAggregate(groupBy=[id1], window=[TumblingGroupWindow('w$, ts, 8000)], select=[id1, LISTAGG(text, $f3) AS EXPR$1]) - ship_strategy : HASH + content : SinkConversionToRow + ship_strategy : FORWARD : Operator - content : SinkConversionToRow + content : Calc(select=[id1, rowtime, text, _UTF-16LE'*' AS $f3]) ship_strategy : FORWARD : Operator - content : Calc(select=[id1, ts, text, _UTF-16LE'*' AS $f3]) - ship_strategy : FORWARD + content : GroupWindowAggregate(groupBy=[id1], window=[SlidingGroupWindow('w$, rowtime, 6000, 12000)], select=[id1, LISTAGG(text, $f3) AS EXPR$1]) + ship_strategy : HASH : Operator - content : GroupWindowAggregate(groupBy=[id1], window=[SlidingGroupWindow('w$, ts, 6000, 12000)], select=[id1, LISTAGG(text, $f3) AS EXPR$1]) - ship_strategy : HASH + content : SinkConversionToRow + ship_strategy : FORWARD - : Operator - content : SinkConversionToRow + : Data Sink + content : Sink: TestingAppendTableSink ship_strategy : FORWARD : Data Sink content : Sink: TestingAppendTableSink ship_strategy : FORWARD - : Data Sink - content : Sink: TestingAppendTableSink - ship_strategy : FORWARD - ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.xml index 6e5c56e70a83..7fe88c8613a7 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.xml @@ -31,13 +31,11 @@ LogicalSink(name=[`default_catalog`.`default_database`.`sink2`], fields=[b, cnt] +- LogicalProject(b=[$1], a=[$0]) +- LogicalUnion(all=[true]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) LogicalSink(name=[`default_catalog`.`default_database`.`sink3`], fields=[b, cnt]) +- LogicalProject(b=[$0], cnt=[$1]) @@ -46,13 +44,11 @@ LogicalSink(name=[`default_catalog`.`default_database`.`sink3`], fields=[b, cnt] +- LogicalProject(b=[$1], a=[$0]) +- LogicalUnion(all=[true]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ]]> @@ -89,19 +85,17 @@ Sink(name=[`default_catalog`.`default_database`.`sink3`], fields=[b, cnt]) @@ -133,32 +127,28 @@ Sink(name=[`default_catalog`.`default_database`.`sink2`], fields=[total_min]) LogicalSink(name=[`default_catalog`.`default_database`.`sink1`], fields=[a, b1]) +- LogicalUnion(all=[true]) :- LogicalProject(a1=[$0], b1=[$1]) - : +- LogicalProject(a1=[$0], b1=[$1]) - : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalProject(a=[$0], b1=[$1]) - +- LogicalProject(a=[$2], b1=[$1]) - +- LogicalFilter(condition=[=($2, $0)]) - +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalProject(a1=[$0], b1=[$1]) - : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + : +- LogicalFilter(condition=[<=($0, 10)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalProject(a=[$2], b1=[$1]) + +- LogicalFilter(condition=[=($2, $0)]) + +- LogicalJoin(condition=[true], joinType=[inner]) + :- LogicalProject(a1=[$0], b1=[$1]) + : +- LogicalFilter(condition=[<=($0, 10)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) LogicalSink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a, b1]) +- LogicalUnion(all=[true]) :- LogicalProject(a1=[$0], b1=[$1]) - : +- LogicalProject(a1=[$0], b1=[$1]) - : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalProject(a=[$0], b1=[$1]) - +- LogicalProject(a=[$2], b1=[$1]) - +- LogicalFilter(condition=[=($2, $0)]) - +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalProject(a1=[$0], b1=[$1]) - : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + : +- LogicalFilter(condition=[<=($0, 10)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalProject(a=[$2], b1=[$1]) + +- LogicalFilter(condition=[=($2, $0)]) + +- LogicalJoin(condition=[true], joinType=[inner]) + :- LogicalProject(a1=[$0], b1=[$1]) + : +- LogicalFilter(condition=[<=($0, 10)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) ]]> @@ -199,37 +189,35 @@ LogicalSink(name=[`default_catalog`.`default_database`.`sink1`], fields=[a, b1]) LogicalSink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a, b1]) +- LogicalUnion(all=[true]) :- LogicalProject(a1=[$0], b1=[$1]) - : +- LogicalProject(a1=[$0], b1=[$1]) - : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalProject(a=[$0], b1=[$1]) - +- LogicalProject(a=[$2], b1=[$1]) - +- LogicalFilter(condition=[=($2, $0)]) - +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalProject(a1=[$0], b1=[$1]) - : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + : +- LogicalFilter(condition=[<=($0, 10)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalProject(a=[$2], b1=[$1]) + +- LogicalFilter(condition=[=($2, $0)]) + +- LogicalJoin(condition=[true], joinType=[inner]) + :- LogicalProject(a1=[$0], b1=[$1]) + : +- LogicalFilter(condition=[<=($0, 10)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) ]]> @@ -309,15 +297,13 @@ Sink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a1, b, c1]) @@ -629,9 +615,8 @@ LogicalSink(name=[`default_catalog`.`default_database`.`sink1`], fields=[a, tota : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) : +- LogicalTableFunctionScan(invocation=[split($cor0.c)], rowType=[RecordType(VARCHAR(2147483647) f0)], elementType=[class [Ljava.lang.Object;]) +- LogicalProject(a=[$0], total_c=[0]) - +- LogicalProject(a=[$0], b=[-($1, MOD($1, 300))], c=[$2]) - +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) LogicalSink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a, total_c]) +- LogicalProject(a=[$0], total_c=[$1]) @@ -647,9 +632,8 @@ LogicalSink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a, tota : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) : +- LogicalTableFunctionScan(invocation=[split($cor0.c)], rowType=[RecordType(VARCHAR(2147483647) f0)], elementType=[class [Ljava.lang.Object;]) +- LogicalProject(a=[$0], total_c=[0]) - +- LogicalProject(a=[$0], b=[-($1, MOD($1, 300))], c=[$2]) - +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ]]> @@ -757,17 +741,15 @@ Sink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[a1, b, c1 LogicalSink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[a1, b1]) +- LogicalUnion(all=[true]) :- LogicalProject(a1=[$0], b1=[$1]) - : +- LogicalProject(a1=[$0], b1=[$1]) - : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalProject(a=[$0], b1=[$1]) - +- LogicalProject(a=[$2], b1=[$1]) - +- LogicalFilter(condition=[=($2, $0)]) - +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalProject(a1=[$0], b1=[$1]) - : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) + : +- LogicalFilter(condition=[<=($0, 10)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalProject(a=[$2], b1=[$1]) + +- LogicalFilter(condition=[=($2, $0)]) + +- LogicalJoin(condition=[true], joinType=[inner]) + :- LogicalProject(a1=[$0], b1=[$1]) + : +- LogicalFilter(condition=[<=($0, 10)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SinkTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SinkTest.xml index 97c98fc19fa3..0a7a587aaa42 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SinkTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SinkTest.xml @@ -43,19 +43,17 @@ Sink(name=[`default_catalog`.`default_database`.`sink`], fields=[a]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/schema/ComputedColumnToRelTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/schema/ComputedColumnToRelTest.xml new file mode 100644 index 000000000000..9f64e144f8db --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/schema/ComputedColumnToRelTest.xml @@ -0,0 +1,67 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.xml index fb2aa4d5b00d..98675c0ea3f0 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.xml @@ -31,13 +31,11 @@ LogicalSink(name=[`default_catalog`.`default_database`.`retractSink`], fields=[b +- LogicalProject(b=[$1], a=[$0]) +- LogicalUnion(all=[true]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) LogicalSink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[b, cnt]) +- LogicalProject(b=[$0], cnt=[$1]) @@ -46,13 +44,11 @@ LogicalSink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[b, +- LogicalProject(b=[$1], a=[$0]) +- LogicalUnion(all=[true]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ]]> @@ -88,19 +84,17 @@ Sink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[b, cnt], @@ -128,44 +122,40 @@ Sink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[total_mi LogicalSink(name=[`default_catalog`.`default_database`.`appendSink1`], fields=[a, b1]) +- LogicalUnion(all=[true]) :- LogicalProject(a1=[$0], b1=[$1]) - : +- LogicalProject(a1=[$0], b1=[$1]) - : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalProject(a=[$0], b1=[$1]) - +- LogicalProject(a=[$2], b1=[$1]) - +- LogicalFilter(condition=[=($2, $0)]) - +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalProject(a1=[$0], b1=[$1]) - : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + : +- LogicalFilter(condition=[<=($0, 10)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalProject(a=[$2], b1=[$1]) + +- LogicalFilter(condition=[=($2, $0)]) + +- LogicalJoin(condition=[true], joinType=[inner]) + :- LogicalProject(a1=[$0], b1=[$1]) + : +- LogicalFilter(condition=[<=($0, 10)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) LogicalSink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a, b1]) +- LogicalUnion(all=[true]) :- LogicalProject(a1=[$0], b1=[$1]) - : +- LogicalProject(a1=[$0], b1=[$1]) - : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalProject(a=[$0], b1=[$1]) - +- LogicalProject(a=[$2], b1=[$1]) - +- LogicalFilter(condition=[=($2, $0)]) - +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalProject(a1=[$0], b1=[$1]) - : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) + : +- LogicalFilter(condition=[<=($0, 10)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalProject(a=[$2], b1=[$1]) + +- LogicalFilter(condition=[=($2, $0)]) + +- LogicalJoin(condition=[true], joinType=[inner]) + :- LogicalProject(a1=[$0], b1=[$1]) + : +- LogicalFilter(condition=[<=($0, 10)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]]) ]]> @@ -301,15 +290,13 @@ Sink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a1, b, c @@ -345,9 +332,8 @@ LogicalSink(name=[`default_catalog`.`default_database`.`appendSink1`], fields=[a : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) : +- LogicalTableFunctionScan(invocation=[split($cor0.c)], rowType=[RecordType(VARCHAR(2147483647) f0)], elementType=[class [Ljava.lang.Object;]) +- LogicalProject(a=[$0], total_c=[0]) - +- LogicalProject(a=[$0], b=[-($1, MOD($1, 300))], c=[$2]) - +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) LogicalSink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a, total_c]) +- LogicalProject(a=[$0], total_c=[$1]) @@ -363,9 +349,8 @@ LogicalSink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) : +- LogicalTableFunctionScan(invocation=[split($cor0.c)], rowType=[RecordType(VARCHAR(2147483647) f0)], elementType=[class [Ljava.lang.Object;]) +- LogicalProject(a=[$0], total_c=[0]) - +- LogicalProject(a=[$0], b=[-($1, MOD($1, 300))], c=[$2]) - +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ]]> @@ -634,13 +619,11 @@ LogicalSink(name=[`default_catalog`.`default_database`.`retractSink1`], fields=[ +- LogicalFilter(condition=[>=($1, 5)]) +- LogicalUnion(all=[true]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) LogicalSink(name=[`default_catalog`.`default_database`.`retractSink2`], fields=[b, cnt]) +- LogicalProject(b=[$0], cnt=[$1]) @@ -649,13 +632,11 @@ LogicalSink(name=[`default_catalog`.`default_database`.`retractSink2`], fields=[ +- LogicalProject(b=[$1], a=[$0]) +- LogicalUnion(all=[true]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) LogicalSink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[b, cnt]) +- LogicalProject(b=[$0], cnt=[$1]) @@ -664,13 +645,11 @@ LogicalSink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[b, +- LogicalProject(b=[$1], a=[$0]) +- LogicalUnion(all=[true]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalProject(a=[$0], b=[$1], c=[$2]) - : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + : +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) ]]> @@ -779,17 +758,15 @@ Sink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[a1, b, c1 LogicalSink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[a1, b1]) +- LogicalUnion(all=[true]) :- LogicalProject(a1=[$0], b1=[$1]) - : +- LogicalProject(a1=[$0], b1=[$1]) - : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalProject(a=[$0], b1=[$1]) - +- LogicalProject(a=[$2], b1=[$1]) - +- LogicalFilter(condition=[=($2, $0)]) - +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalProject(a1=[$0], b1=[$1]) - : +- LogicalFilter(condition=[<=($0, 10)]) - : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) + : +- LogicalFilter(condition=[<=($0, 10)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalProject(a=[$2], b1=[$1]) + +- LogicalFilter(condition=[=($2, $0)]) + +- LogicalJoin(condition=[true], joinType=[inner]) + :- LogicalProject(a1=[$0], b1=[$1]) + : +- LogicalFilter(condition=[<=($0, 10)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> @@ -994,8 +971,7 @@ LogicalSink(name=[`default_catalog`.`default_database`.`retractSink`], fields=[a +- LogicalProject(a=[$0], b=[$1], c=[$2], rank_num=[$3]) +- LogicalFilter(condition=[<=($3, 10)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rank_num=[ROW_NUMBER() OVER (PARTITION BY $1 ORDER BY $2 DESC NULLS LAST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)]) - +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) LogicalSink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[a, b]) +- LogicalProject(a=[$0], b=[$1]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.xml index 29f795f65d7b..6bb8f50826ac 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.xml @@ -221,23 +221,32 @@ Calc(select=[a, b]) LogicalSink(name=[`default_catalog`.`default_database`.`appendSink1`], fields=[a, b]) +- LogicalProject(id1=[$1], EXPR$1=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$1=[LISTAGG($2, $3)]) - +- LogicalProject($f0=[HOP($2, 12000:INTERVAL SECOND, 4000:INTERVAL SECOND)], id1=[$0], text=[$1], $f3=[_UTF-16LE'*']) - +- LogicalProject(id1=[$1], text=[$2], ts=[TUMBLE_ROWTIME($0)]) - +- LogicalAggregate(group=[{0, 1}], text=[LISTAGG($2, $3)]) - +- LogicalProject($f0=[TUMBLE($1, 6000:INTERVAL SECOND)], id1=[$0], text=[$2], $f3=[_UTF-16LE'#']) - +- LogicalProject(id1=[$0], ts=[$1], text=[$2]) - +- LogicalFilter(condition=[AND(=($0, $3), >($1, -($4, 300000:INTERVAL MINUTE)), <($1, +($4, 180000:INTERVAL MINUTE)))]) - +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalWatermarkAssigner(fields=[id1, rowtime, text], rowtimeField=[rowtime], watermarkDelay=[0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) - +- LogicalWatermarkAssigner(fields=[id2, rowtime, cnt, name, goods], rowtimeField=[rowtime], watermarkDelay=[0]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +- LogicalProject($f0=[HOP(TUMBLE_ROWTIME($0), 12000:INTERVAL SECOND, 4000:INTERVAL SECOND)], id1=[$1], text=[$2], $f3=[_UTF-16LE'*']) + +- LogicalAggregate(group=[{0, 1}], text=[LISTAGG($2, $3)]) + +- LogicalProject($f0=[TUMBLE($1, 6000:INTERVAL SECOND)], id1=[$0], text=[$2], $f3=[_UTF-16LE'#']) + +- LogicalFilter(condition=[AND(=($0, $3), >($1, -($4, 300000:INTERVAL MINUTE)), <($1, +($4, 180000:INTERVAL MINUTE)))]) + +- LogicalJoin(condition=[true], joinType=[inner]) + :- LogicalWatermarkAssigner(fields=[id1, rowtime, text], rowtimeField=[rowtime], watermarkDelay=[0]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + +- LogicalWatermarkAssigner(fields=[id2, rowtime, cnt, name, goods], rowtimeField=[rowtime], watermarkDelay=[0]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) LogicalSink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a, b]) +- LogicalProject(id1=[$1], EXPR$1=[$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$1=[LISTAGG($2, $3)]) +- LogicalProject($f0=[TUMBLE($1, 9000:INTERVAL SECOND)], id1=[$0], text=[$2], $f3=[_UTF-16LE'-']) - +- LogicalProject(id1=[$0], ts=[$1], text=[$2]) + +- LogicalFilter(condition=[AND(=($0, $3), >($1, -($4, 300000:INTERVAL MINUTE)), <($1, +($4, 180000:INTERVAL MINUTE)))]) + +- LogicalJoin(condition=[true], joinType=[inner]) + :- LogicalWatermarkAssigner(fields=[id1, rowtime, text], rowtimeField=[rowtime], watermarkDelay=[0]) + : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) + +- LogicalWatermarkAssigner(fields=[id2, rowtime, cnt, name, goods], rowtimeField=[rowtime], watermarkDelay=[0]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) + +LogicalSink(name=[`default_catalog`.`default_database`.`appendSink3`], fields=[a, b]) ++- LogicalAggregate(group=[{0}], EXPR$1=[COUNT($1)]) + +- LogicalProject(id1=[$1], text=[$2]) + +- LogicalAggregate(group=[{0, 1}], text=[LISTAGG($2, $3)]) + +- LogicalProject($f0=[TUMBLE($1, 6000:INTERVAL SECOND)], id1=[$0], text=[$2], $f3=[_UTF-16LE'#']) +- LogicalFilter(condition=[AND(=($0, $3), >($1, -($4, 300000:INTERVAL MINUTE)), <($1, +($4, 180000:INTERVAL MINUTE)))]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalWatermarkAssigner(fields=[id1, rowtime, text], rowtimeField=[rowtime], watermarkDelay=[0]) @@ -245,52 +254,37 @@ LogicalSink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a +- LogicalWatermarkAssigner(fields=[id2, rowtime, cnt, name, goods], rowtimeField=[rowtime], watermarkDelay=[0]) +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) -LogicalSink(name=[`default_catalog`.`default_database`.`appendSink3`], fields=[a, b]) -+- LogicalAggregate(group=[{0}], EXPR$1=[COUNT($1)]) - +- LogicalProject(id1=[$0], text=[$1]) - +- LogicalProject(id1=[$1], text=[$2], ts=[TUMBLE_ROWTIME($0)]) - +- LogicalAggregate(group=[{0, 1}], text=[LISTAGG($2, $3)]) - +- LogicalProject($f0=[TUMBLE($1, 6000:INTERVAL SECOND)], id1=[$0], text=[$2], $f3=[_UTF-16LE'#']) - +- LogicalProject(id1=[$0], ts=[$1], text=[$2]) - +- LogicalFilter(condition=[AND(=($0, $3), >($1, -($4, 300000:INTERVAL MINUTE)), <($1, +($4, 180000:INTERVAL MINUTE)))]) - +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalWatermarkAssigner(fields=[id1, rowtime, text], rowtimeField=[rowtime], watermarkDelay=[0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]) - +- LogicalWatermarkAssigner(fields=[id2, rowtime, cnt, name, goods], rowtimeField=[rowtime], watermarkDelay=[0]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) - == Optimized Logical Plan == -Calc(select=[id1, rowtime AS ts, text], reuse_id=[1]) -+- WindowJoin(joinType=[InnerJoin], windowBounds=[isRowTime=true, leftLowerBound=-299999, leftUpperBound=179999, leftTimeIndex=1, rightTimeIndex=1], where=[AND(=(id1, id2), >(CAST(rowtime), -(CAST(rowtime0), 300000:INTERVAL MINUTE)), <(CAST(rowtime), +(CAST(rowtime0), 180000:INTERVAL MINUTE)))], select=[id1, rowtime, text, id2, rowtime0, cnt, name, goods]) - :- Exchange(distribution=[hash[id1]]) - : +- WatermarkAssigner(fields=[id1, rowtime, text], rowtimeField=[rowtime], watermarkDelay=[0], miniBatchInterval=[None]) - : +- DataStreamScan(table=[[default_catalog, default_database, T1]], fields=[id1, rowtime, text]) - +- Exchange(distribution=[hash[id2]]) - +- WatermarkAssigner(fields=[id2, rowtime, cnt, name, goods], rowtimeField=[rowtime], watermarkDelay=[0], miniBatchInterval=[None]) - +- DataStreamScan(table=[[default_catalog, default_database, T2]], fields=[id2, rowtime, cnt, name, goods]) - -GroupWindowAggregate(groupBy=[id1], window=[TumblingGroupWindow('w$, ts, 6000)], properties=[w$start, w$end, w$rowtime, w$proctime], select=[id1, LISTAGG(text, $f3) AS text, start('w$) AS w$start, end('w$) AS w$end, rowtime('w$) AS w$rowtime, proctime('w$) AS w$proctime], reuse_id=[2]) -+- Exchange(distribution=[hash[id1]]) - +- Calc(select=[ts, id1, text, _UTF-16LE'#' AS $f3]) - +- Reused(reference_id=[1]) +WindowJoin(joinType=[InnerJoin], windowBounds=[isRowTime=true, leftLowerBound=-299999, leftUpperBound=179999, leftTimeIndex=1, rightTimeIndex=1], where=[AND(=(id1, id2), >(CAST(rowtime), -(CAST(rowtime0), 300000:INTERVAL MINUTE)), <(CAST(rowtime), +(CAST(rowtime0), 180000:INTERVAL MINUTE)))], select=[id1, rowtime, text, id2, rowtime0, cnt, name, goods], reuse_id=[1]) +:- Exchange(distribution=[hash[id1]]) +: +- WatermarkAssigner(fields=[id1, rowtime, text], rowtimeField=[rowtime], watermarkDelay=[0], miniBatchInterval=[None]) +: +- DataStreamScan(table=[[default_catalog, default_database, T1]], fields=[id1, rowtime, text]) ++- Exchange(distribution=[hash[id2]]) + +- WatermarkAssigner(fields=[id2, rowtime, cnt, name, goods], rowtimeField=[rowtime], watermarkDelay=[0], miniBatchInterval=[None]) + +- DataStreamScan(table=[[default_catalog, default_database, T2]], fields=[id2, rowtime, cnt, name, goods]) + +Exchange(distribution=[hash[id1]], reuse_id=[2]) ++- Calc(select=[rowtime, id1, text, _UTF-16LE'#' AS $f3]) + +- Reused(reference_id=[1]) Sink(name=[`default_catalog`.`default_database`.`appendSink1`], fields=[a, b]) -+- GroupWindowAggregate(groupBy=[id1], window=[SlidingGroupWindow('w$, ts, 4000, 12000)], select=[id1, LISTAGG(text, $f3) AS EXPR$1]) ++- GroupWindowAggregate(groupBy=[id1], window=[SlidingGroupWindow('w$, $f0, 4000, 12000)], select=[id1, LISTAGG(text, $f3) AS EXPR$1]) +- Exchange(distribution=[hash[id1]]) - +- Calc(select=[w$rowtime AS ts, id1, text, _UTF-16LE'*' AS $f3]) - +- Reused(reference_id=[2]) + +- Calc(select=[w$rowtime AS $f0, id1, text, _UTF-16LE'*' AS $f3]) + +- GroupWindowAggregate(groupBy=[id1], window=[TumblingGroupWindow('w$, rowtime, 6000)], properties=[w$start, w$end, w$rowtime, w$proctime], select=[id1, LISTAGG(text, $f3) AS text, start('w$) AS w$start, end('w$) AS w$end, rowtime('w$) AS w$rowtime, proctime('w$) AS w$proctime]) + +- Reused(reference_id=[2]) Sink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a, b]) -+- GroupWindowAggregate(groupBy=[id1], window=[TumblingGroupWindow('w$, ts, 9000)], select=[id1, LISTAGG(text, $f3) AS EXPR$1]) ++- GroupWindowAggregate(groupBy=[id1], window=[TumblingGroupWindow('w$, rowtime, 9000)], select=[id1, LISTAGG(text, $f3) AS EXPR$1]) +- Exchange(distribution=[hash[id1]]) - +- Calc(select=[ts, id1, text, _UTF-16LE'-' AS $f3]) + +- Calc(select=[rowtime, id1, text, _UTF-16LE'-' AS $f3]) +- Reused(reference_id=[1]) Sink(name=[`default_catalog`.`default_database`.`appendSink3`], fields=[a, b]) +- GlobalGroupAggregate(groupBy=[id1], select=[id1, COUNT(count$0) AS EXPR$1]) +- Exchange(distribution=[hash[id1]]) +- LocalGroupAggregate(groupBy=[id1], select=[id1, COUNT(text) AS count$0]) - +- Calc(select=[id1, text]) + +- GroupWindowAggregate(groupBy=[id1], window=[TumblingGroupWindow('w$, rowtime, 6000)], select=[id1, LISTAGG(text, $f3) AS text]) +- Reused(reference_id=[2]) == Physical Execution Plan == @@ -321,59 +315,59 @@ Sink(name=[`default_catalog`.`default_database`.`appendSink3`], fields=[a, b]) ship_strategy : HASH : Operator - content : Calc(select=[id1, rowtime AS ts, text]) + content : Calc(select=[rowtime, id1, text, _UTF-16LE'#' AS $f3]) ship_strategy : FORWARD : Operator - content : Calc(select=[ts, id1, text, _UTF-16LE'#' AS $f3]) - ship_strategy : FORWARD + content : GroupWindowAggregate(groupBy=[id1], window=[TumblingGroupWindow('w$, rowtime, 6000)], properties=[w$start, w$end, w$rowtime, w$proctime], select=[id1, LISTAGG(text, $f3) AS text, start('w$) AS w$start, end('w$) AS w$end, rowtime('w$) AS w$rowtime, proctime('w$) AS w$proctime]) + ship_strategy : HASH : Operator - content : GroupWindowAggregate(groupBy=[id1], window=[TumblingGroupWindow('w$, ts, 6000)], properties=[w$start, w$end, w$rowtime, w$proctime], select=[id1, LISTAGG(text, $f3) AS text, start('w$) AS w$start, end('w$) AS w$end, rowtime('w$) AS w$rowtime, proctime('w$) AS w$proctime]) - ship_strategy : HASH + content : Calc(select=[w$rowtime AS $f0, id1, text, _UTF-16LE'*' AS $f3]) + ship_strategy : FORWARD : Operator - content : Calc(select=[w$rowtime AS ts, id1, text, _UTF-16LE'*' AS $f3]) - ship_strategy : FORWARD + content : GroupWindowAggregate(groupBy=[id1], window=[SlidingGroupWindow('w$, $f0, 4000, 12000)], select=[id1, LISTAGG(text, $f3) AS EXPR$1]) + ship_strategy : HASH : Operator - content : GroupWindowAggregate(groupBy=[id1], window=[SlidingGroupWindow('w$, ts, 4000, 12000)], select=[id1, LISTAGG(text, $f3) AS EXPR$1]) - ship_strategy : HASH + content : SinkConversionToRow + ship_strategy : FORWARD : Operator - content : SinkConversionToRow + content : Calc(select=[rowtime, id1, text, _UTF-16LE'-' AS $f3]) ship_strategy : FORWARD : Operator - content : Calc(select=[ts, id1, text, _UTF-16LE'-' AS $f3]) - ship_strategy : FORWARD + content : GroupWindowAggregate(groupBy=[id1], window=[TumblingGroupWindow('w$, rowtime, 9000)], select=[id1, LISTAGG(text, $f3) AS EXPR$1]) + ship_strategy : HASH : Operator - content : GroupWindowAggregate(groupBy=[id1], window=[TumblingGroupWindow('w$, ts, 9000)], select=[id1, LISTAGG(text, $f3) AS EXPR$1]) - ship_strategy : HASH + content : SinkConversionToRow + ship_strategy : FORWARD : Operator - content : SinkConversionToRow - ship_strategy : FORWARD + content : GroupWindowAggregate(groupBy=[id1], window=[TumblingGroupWindow('w$, rowtime, 6000)], select=[id1, LISTAGG(text, $f3) AS text]) + ship_strategy : HASH : Operator - content : Calc(select=[id1, text]) + content : LocalGroupAggregate(groupBy=[id1], select=[id1, COUNT(text) AS count$0]) ship_strategy : FORWARD : Operator - content : LocalGroupAggregate(groupBy=[id1], select=[id1, COUNT(text) AS count$0]) - ship_strategy : FORWARD + content : GlobalGroupAggregate(groupBy=[id1], select=[id1, COUNT(count$0) AS EXPR$1]) + ship_strategy : HASH : Operator - content : GlobalGroupAggregate(groupBy=[id1], select=[id1, COUNT(count$0) AS EXPR$1]) - ship_strategy : HASH + content : SinkConversionToTuple2 + ship_strategy : FORWARD : Operator - content : SinkConversionToTuple2 + content : Map ship_strategy : FORWARD - : Operator - content : Map + : Data Sink + content : Sink: TestingAppendTableSink ship_strategy : FORWARD : Data Sink @@ -381,13 +375,9 @@ Sink(name=[`default_catalog`.`default_database`.`appendSink3`], fields=[a, b]) ship_strategy : FORWARD : Data Sink - content : Sink: TestingAppendTableSink + content : Sink: TestingRetractTableSink ship_strategy : FORWARD - : Data Sink - content : Sink: TestingRetractTableSink - ship_strategy : FORWARD - ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala index f4ea542800fc..301e4975d337 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.api.scala.{StreamTableEnvironment, _} import org.apache.flink.table.planner.utils.TableTestUtil import org.apache.calcite.plan.RelOptUtil +import org.apache.calcite.sql.SqlExplainLevel import org.junit.Assert.assertEquals import org.junit.rules.ExpectedException import org.junit.{Rule, Test} @@ -70,9 +71,9 @@ class TableEnvironmentTest { tableEnv.registerTable("MyTable", table) val queryTable = tableEnv.sqlQuery("SELECT a, c, d FROM MyTable") val relNode = TableTestUtil.toRelNode(queryTable) - val actual = RelOptUtil.toString(relNode) - val expected = "LogicalProject(a=[$0], c=[$2], d=[$3])\n" + - " LogicalTableScan(table=[[default_catalog, default_database, MyTable]])\n" + val actual = RelOptUtil.toString(relNode, SqlExplainLevel.NO_ATTRIBUTES) + val expected = "LogicalProject\n" + + " LogicalTableScan\n" assertEquals(expected, actual) } } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala index bb3dadb79dc2..35b43e9b7790 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala @@ -22,15 +22,18 @@ import org.apache.flink.table.api.config.ExecutionConfigOptions import org.apache.flink.table.api.internal.TableEnvironmentImpl import org.apache.flink.table.api.{EnvironmentSettings, TableEnvironment, ValidationException} import org.apache.flink.table.catalog.{CatalogFunctionImpl, ObjectPath} +import org.apache.flink.table.planner.expressions.utils.Func0 import org.apache.flink.table.planner.factories.utils.TestCollectionTableFactory import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.JavaFunc0 import org.apache.flink.types.Row import org.junit.Assert.assertEquals +import org.junit.rules.ExpectedException import org.junit.runner.RunWith import org.junit.runners.Parameterized -import org.junit.{Before, Ignore, Test} +import org.junit.{Before, Ignore, Rule, Test} +import java.sql.Timestamp import java.util import scala.collection.JavaConversions._ @@ -38,6 +41,7 @@ import scala.collection.JavaConversions._ /** Test cases for catalog table. */ @RunWith(classOf[Parameterized]) class CatalogTableITCase(isStreamingMode: Boolean) { + //~ Instance fields -------------------------------------------------------- private val settings = if (isStreamingMode) { EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build() @@ -47,23 +51,10 @@ class CatalogTableITCase(isStreamingMode: Boolean) { private val tableEnv: TableEnvironment = TableEnvironmentImpl.create(settings) - private val SOURCE_DATA = List( - toRow(1, "a"), - toRow(2, "b"), - toRow(3, "c") - ) + var _expectedEx: ExpectedException = ExpectedException.none - private val DIM_DATA = List( - toRow(1, "aDim"), - toRow(2, "bDim"), - toRow(3, "cDim") - ) - - implicit def rowOrdering: Ordering[Row] = Ordering.by((r : Row) => { - val builder = new StringBuilder - 0 until r.getArity foreach(idx => builder.append(r.getField(idx))) - builder.toString() - }) + @Rule + def expectedEx: ExpectedException = _expectedEx @Before def before(): Unit = { @@ -82,6 +73,14 @@ class CatalogTableITCase(isStreamingMode: Boolean) { true) } + //~ Tools ------------------------------------------------------------------ + + implicit def rowOrdering: Ordering[Row] = Ordering.by((r : Row) => { + val builder = new StringBuilder + 0 until r.getArity foreach(idx => builder.append(r.getField(idx))) + builder.toString() + }) + def toRow(args: Any*):Row = { val row = new Row(args.length) 0 until args.length foreach { @@ -94,6 +93,8 @@ class CatalogTableITCase(isStreamingMode: Boolean) { tableEnv.execute(name) } + //~ Tests ------------------------------------------------------------------ + private def testUdf(funcPrefix: String): Unit = { val sinkDDL = """ @@ -176,40 +177,303 @@ class CatalogTableITCase(isStreamingMode: Boolean) { assertEquals(sourceData.sorted, TestCollectionTableFactory.RESULT.sorted) } - @Ignore // need to implement @Test - def testInsertTargetTableWithComputedColumn(): Unit = { - TestCollectionTableFactory.initData(SOURCE_DATA) + def testInsertSourceTableExpressionFields(): Unit = { + val sourceData = List( + toRow(1, "1000"), + toRow(2, "1"), + toRow(3, "2000"), + toRow(1, "2"), + toRow(2, "3000") + ) + val expected = List( + toRow(1, "1000", 2), + toRow(2, "1", 3), + toRow(3, "2000", 4), + toRow(1, "2", 2), + toRow(2, "3000", 3) + ) + TestCollectionTableFactory.initData(sourceData) val sourceDDL = """ |create table t1( | a int, | b varchar, + | c as a + 1 + |) with ( + | 'connector' = 'COLLECTION' + |) + """.stripMargin + val sinkDDL = + """ + |create table t2( + | a int, + | b varchar, | c int |) with ( | 'connector' = 'COLLECTION' |) """.stripMargin + val query = + """ + |insert into t2 + |select t1.a, t1.b, t1.c from t1 + """.stripMargin + tableEnv.sqlUpdate(sourceDDL) + tableEnv.sqlUpdate(sinkDDL) + tableEnv.sqlUpdate(query) + execJob("testJob") + assertEquals(expected.sorted, TestCollectionTableFactory.RESULT.sorted) + } + + // Test the computation expression in front of referenced columns. + @Test + def testInsertSourceTableExpressionFieldsBeforeReferences(): Unit = { + val sourceData = List( + toRow(1, "1000"), + toRow(2, "1"), + toRow(3, "2000"), + toRow(2, "2"), + toRow(2, "3000") + ) + val expected = List( + toRow(101, 1, "1000"), + toRow(102, 2, "1"), + toRow(103, 3, "2000"), + toRow(102, 2, "2"), + toRow(102, 2, "3000") + ) + TestCollectionTableFactory.initData(sourceData) + val sourceDDL = + """ + |create table t1( + | c as a + 100, + | a int, + | b varchar + |) with ( + | 'connector' = 'COLLECTION' + |) + """.stripMargin + val sinkDDL = + """ + |create table t2( + | c int, + | a int, + | b varchar + |) with ( + | 'connector' = 'COLLECTION' + |) + """.stripMargin + val query = + """ + |insert into t2 + |select t1.c, t1.a, t1.b from t1 + """.stripMargin + tableEnv.sqlUpdate(sourceDDL) + tableEnv.sqlUpdate(sinkDDL) + tableEnv.sqlUpdate(query) + execJob("testJob") + assertEquals(expected.sorted, TestCollectionTableFactory.RESULT.sorted) + } + + @Test + def testInsertSourceTableWithFuncField(): Unit = { + val sourceData = List( + toRow(1, "1990-02-10 12:34:56"), + toRow(2, "2019-09-10 9:23:41"), + toRow(3, "2019-09-10 9:23:42"), + toRow(1, "2019-09-10 9:23:43"), + toRow(2, "2019-09-10 9:23:44") + ) + val expected = List( + toRow(1, "1990-02-10 12:34:56", Timestamp.valueOf("1990-02-10 12:34:56")), + toRow(2, "2019-09-10 9:23:41", Timestamp.valueOf("2019-09-10 9:23:41")), + toRow(3, "2019-09-10 9:23:42", Timestamp.valueOf("2019-09-10 9:23:42")), + toRow(1, "2019-09-10 9:23:43", Timestamp.valueOf("2019-09-10 9:23:43")), + toRow(2, "2019-09-10 9:23:44", Timestamp.valueOf("2019-09-10 9:23:44")) + ) + TestCollectionTableFactory.initData(sourceData) + val sourceDDL = + """ + |create table t1( + | a int, + | b varchar, + | c as to_timestamp(b) + |) with ( + | 'connector' = 'COLLECTION' + |) + """.stripMargin + val sinkDDL = + """ + |create table t2( + | a int, + | b varchar, + | c timestamp + |) with ( + | 'connector' = 'COLLECTION' + |) + """.stripMargin + val query = + """ + |insert into t2 + |select t1.a, t1.b, t1.c from t1 + """.stripMargin + tableEnv.sqlUpdate(sourceDDL) + tableEnv.sqlUpdate(sinkDDL) + tableEnv.sqlUpdate(query) + execJob("testJob") + assertEquals(expected.sorted, TestCollectionTableFactory.RESULT.sorted) + } + + @Test + def testInsertSourceTableWithUserDefinedFuncField(): Unit = { + val sourceData = List( + toRow(1, "1990-02-10 12:34:56"), + toRow(2, "2019-09-10 9:23:41"), + toRow(3, "2019-09-10 9:23:42"), + toRow(1, "2019-09-10 9:23:43"), + toRow(2, "2019-09-10 9:23:44") + ) + val expected = List( + toRow(1, "1990-02-10 12:34:56", 1), + toRow(2, "2019-09-10 9:23:41", 2), + toRow(3, "2019-09-10 9:23:42", 3), + toRow(1, "2019-09-10 9:23:43", 1), + toRow(2, "2019-09-10 9:23:44", 2) + ) + TestCollectionTableFactory.initData(sourceData) + tableEnv.registerFunction("my_udf", Func0) + val sourceDDL = + """ + |create table t1( + | a int, + | b varchar, + | c as my_udf(a) + |) with ( + | 'connector' = 'COLLECTION' + |) + """.stripMargin val sinkDDL = """ |create table t2( | a int, | b varchar, + | c int not null + |) with ( + | 'connector' = 'COLLECTION' + |) + """.stripMargin + val query = + """ + |insert into t2 + |select t1.a, t1.b, t1.c from t1 + """.stripMargin + tableEnv.sqlUpdate(sourceDDL) + tableEnv.sqlUpdate(sinkDDL) + tableEnv.sqlUpdate(query) + execJob("testJob") + assertEquals(expected.sorted, TestCollectionTableFactory.RESULT.sorted) + } + + @Test + def testInsertSinkTableExpressionFields(): Unit = { + val sourceData = List( + toRow(1, "1000"), + toRow(2, "1"), + toRow(3, "2000"), + toRow(1, "2"), + toRow(2, "3000") + ) + val expected = List( + toRow(1, 2), + toRow(1, 2), + toRow(2, 3), + toRow(2, 3), + toRow(3, 4) + ) + TestCollectionTableFactory.initData(sourceData) + val sourceDDL = + """ + |create table t1( + | a int, + | b varchar, + | c as a + 1 + |) with ( + | 'connector' = 'COLLECTION' + |) + """.stripMargin + val sinkDDL = + """ + |create table t2( + | a int, + | b as c - 1, + | c int + |) with ( + | 'connector' = 'COLLECTION' + |) + """.stripMargin + val query = + """ + |insert into t2 + |select t1.a, t1.c from t1 + """.stripMargin + tableEnv.sqlUpdate(sourceDDL) + tableEnv.sqlUpdate(sinkDDL) + tableEnv.sqlUpdate(query) + execJob("testJob") + assertEquals(expected.sorted, TestCollectionTableFactory.RESULT.sorted) + } + + @Test + def testInsertSinkTableWithUnmatchedFields(): Unit = { + val sourceData = List( + toRow(1, "1000"), + toRow(2, "1"), + toRow(3, "2000"), + toRow(1, "2"), + toRow(2, "3000") + ) + val expected = List( + toRow(1, 2), + toRow(1, 2), + toRow(2, 3), + toRow(2, 3), + toRow(3, 4) + ) + TestCollectionTableFactory.initData(sourceData) + val sourceDDL = + """ + |create table t1( + | a int, + | b varchar, | c as a + 1 |) with ( | 'connector' = 'COLLECTION' |) """.stripMargin + val sinkDDL = + """ + |create table t2( + | a int, + | b as cast(a as varchar(20)) || cast(c as varchar(20)), + | c int + |) with ( + | 'connector' = 'COLLECTION' + |) + """.stripMargin val query = """ - |insert into t2(a, b) + |insert into t2 |select t1.a, t1.b from t1 """.stripMargin tableEnv.sqlUpdate(sourceDDL) tableEnv.sqlUpdate(sinkDDL) + expectedEx.expect(classOf[ValidationException]) + expectedEx.expectMessage("Field types of query result and registered TableSink " + + "`default_catalog`.`default_database`.`t2` do not match.") tableEnv.sqlUpdate(query) execJob("testJob") - assertEquals(SOURCE_DATA.sorted, TestCollectionTableFactory.RESULT.sorted) + assertEquals(expected.sorted, TestCollectionTableFactory.RESULT.sorted) } @Test diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/codegen/WatermarkGeneratorCodeGenTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/codegen/WatermarkGeneratorCodeGenTest.scala index dad1c7ddd321..48812041fa2c 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/codegen/WatermarkGeneratorCodeGenTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/codegen/WatermarkGeneratorCodeGenTest.scala @@ -30,8 +30,9 @@ import org.apache.flink.table.planner.delegation.PlannerContext import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.JavaFunc5 import org.apache.flink.table.runtime.generated.WatermarkGenerator import org.apache.flink.table.types.logical.{IntType, TimestampType} -import org.apache.calcite.jdbc.CalciteSchemaBuilder.asRootSchema +import org.apache.calcite.jdbc.CalciteSchemaBuilder.asRootSchema +import org.apache.calcite.plan.ConventionTraitDef import org.junit.Assert.{assertEquals, assertTrue} import org.junit.Test @@ -52,7 +53,7 @@ class WatermarkGeneratorCodeGenTest { functionCatalog, catalogManager, asRootSchema(new CatalogManagerCalciteSchema(catalogManager, false)), - Collections.emptyList()) + Collections.singletonList(ConventionTraitDef.INSTANCE)) val planner: FlinkPlannerImpl = plannerContext.createFlinkPlanner( catalogManager.getCurrentCatalog, catalogManager.getCurrentDatabase) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala index ad056cbf63da..7119c681341c 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala @@ -28,7 +28,7 @@ import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.datastream.{DataStream, DataStreamSink, DataStreamSource} import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import org.apache.flink.streaming.api.functions.sink.RichSinkFunction -import org.apache.flink.table.api.TableSchema +import org.apache.flink.table.api.{DataTypes, TableSchema} import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR import org.apache.flink.table.descriptors.{DescriptorProperties, Schema} import org.apache.flink.table.factories.{BatchTableSinkFactory, BatchTableSourceFactory, StreamTableSinkFactory, StreamTableSourceFactory} @@ -36,6 +36,7 @@ import org.apache.flink.table.functions.{AsyncTableFunction, TableFunction} import org.apache.flink.table.planner.factories.utils.TestCollectionTableFactory.{getCollectionSink, getCollectionSource} import org.apache.flink.table.sinks.{AppendStreamTableSink, BatchTableSink, StreamTableSink, TableSink} import org.apache.flink.table.sources.{BatchTableSource, LookupableTableSource, StreamTableSource, TableSource} +import org.apache.flink.table.types.utils.TypeConversions.fromDataTypeToLegacyInfo import org.apache.flink.types.Row import java.io.IOException @@ -116,14 +117,21 @@ object TestCollectionTableFactory { val properties = new DescriptorProperties() properties.putProperties(props) val schema = properties.getTableSchema(Schema.SCHEMA) - new CollectionTableSource(emitIntervalMS, schema, isStreaming) + new CollectionTableSource(emitIntervalMS, physicalSchema(schema), isStreaming) } def getCollectionSink(props: JMap[String, String]): CollectionTableSink = { val properties = new DescriptorProperties() properties.putProperties(props) val schema = properties.getTableSchema(Schema.SCHEMA) - new CollectionTableSink(schema.toRowType.asInstanceOf[RowTypeInfo]) + new CollectionTableSink(physicalSchema(schema).toRowType.asInstanceOf[RowTypeInfo]) + } + + def physicalSchema(schema: TableSchema): TableSchema = { + val builder = TableSchema.builder() + schema.getTableColumns.filter(c => !c.isGenerated) + .foreach(c => builder.field(c.getName, c.getType)) + builder.build() } /** diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/schema/ComputedColumnToRelTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/schema/ComputedColumnToRelTest.scala new file mode 100644 index 000000000000..8eac8267d5c4 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/schema/ComputedColumnToRelTest.scala @@ -0,0 +1,191 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.schema + +import org.apache.flink.table.api.TableConfig +import org.apache.flink.table.catalog.{Catalog, CatalogManager, FunctionCatalog, GenericInMemoryCatalog, ObjectIdentifier} +import org.apache.flink.table.functions.ScalarFunction +import org.apache.flink.table.module.ModuleManager +import org.apache.flink.table.operations.Operation +import org.apache.flink.table.operations.ddl.CreateTableOperation +import org.apache.flink.table.planner.calcite.{CalciteParser, FlinkPlannerImpl} +import org.apache.flink.table.planner.catalog.CatalogManagerCalciteSchema +import org.apache.flink.table.planner.delegation.PlannerContext +import org.apache.flink.table.planner.expressions.utils.Func0 +import org.apache.flink.table.planner.operations.{PlannerQueryOperation, SqlToOperationConverter} +import org.apache.flink.table.planner.plan.schema.ComputedColumnToRelTest.diffRepository +import org.apache.flink.table.planner.utils.DiffRepository + +import org.apache.calcite.jdbc.CalciteSchemaBuilder.asRootSchema +import org.apache.calcite.plan.ConventionTraitDef +import org.junit.rules.TestName +import org.junit.{Rule, Test} +import ComputedColumnToRelTest._ +import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil + +import org.apache.calcite.rel.RelNode + +import java.util.Collections + +/** + * Test cases to verify the plan for DDL tables with computed column. + */ +class ComputedColumnToRelTest { + // Used for get test case method name. + val testName: TestName = new TestName + + @Rule + def name: TestName = testName + + @Test + def testProjectSourceWithVirtualColumn(): Unit = { + // Create table with field as atom expression. + val ddl1 = + s""" + |create table t1( + | a int, + | b varchar, + | c as a + 1 + |) with ( + | 'connector' = 'COLLECTION' + |) + """.stripMargin + createTable(ddl1) + checkConvertTo("select * from t1") + } + + @Test + def testProjectSourceWithVirtualColumnAsBuiltinFunc(): Unit = { + // Create table with field as builtin function. + val ddl2 = + s""" + |create table t2( + | a int, + | b varchar, + | c as to_timestamp(b) + |) with ( + | 'connector' = 'COLLECTION' + |) + """.stripMargin + createTable(ddl2) + checkConvertTo("select * from t2") + } + + @Test + def testProjectSourceWithVirtualColumnAsUDF(): Unit = { + // Create table with field as user defined function. + createScalarFunc("my_udf", Func0) + val ddl3 = + s""" + |create table t3( + | a int, + | b varchar, + | c as my_udf(a) + |) with ( + | 'connector' = 'COLLECTION' + |) + """.stripMargin + createTable(ddl3) + checkConvertTo("select * from t3") + } + + @Test + def testProjectSourceWithVirtualColumnAsExternalUDF(): Unit = { + // Create table with field as user defined function. + createScalarFunc(ObjectIdentifier.of("my_catalog", "my_database", "my_udf"), Func0) + val ddl3 = + s""" + |create table t4( + | a int, + | b varchar, + | c as my_catalog.my_database.my_udf(a) + |) with ( + | 'connector' = 'COLLECTION' + |) + """.stripMargin + createTable(ddl3) + checkConvertTo("select * from t4") + } + + /** + * Check the relational expression plan right after FlinkPlannerImpl conversion. + * + * @param sql The SQL string + */ + def checkConvertTo(sql: String): Unit = { + val actual = FlinkRelOptUtil.toString(toRel(sql)) + diffRepository.assertEquals(name.getMethodName, "plan", s"$${plan}", actual) + } +} + +object ComputedColumnToRelTest { + val diffRepository: DiffRepository = DiffRepository.lookup(classOf[ComputedColumnToRelTest]) + + private val tableConfig: TableConfig = new TableConfig + private val catalog: Catalog = new GenericInMemoryCatalog("MockCatalog", "default_database") + private val catalogManager: CatalogManager = new CatalogManager("default_catalog", catalog) + private val moduleManager: ModuleManager = new ModuleManager + private val functionCatalog: FunctionCatalog = new FunctionCatalog(catalogManager, moduleManager) + private val plannerContext: PlannerContext = + new PlannerContext(tableConfig, + functionCatalog, + catalogManager, + asRootSchema(new CatalogManagerCalciteSchema(catalogManager, false)), + Collections.singletonList(ConventionTraitDef.INSTANCE)) + + /** Creates a FlinkPlannerImpl instance. */ + private def createPlanner(): FlinkPlannerImpl = { + plannerContext.createFlinkPlanner(catalogManager.getCurrentCatalog, + catalogManager.getCurrentDatabase) + } + + /** Creates a FlinkPlannerImpl instance. */ + private def createCalciteParser(): CalciteParser = { + plannerContext.createCalciteParser() + } + + private def parse(sql: String, planner: FlinkPlannerImpl, parser: CalciteParser): Operation = { + val node = parser.parse(sql) + SqlToOperationConverter.convert(planner, catalogManager, node).get + } + + def toRel(sql: String): RelNode = { + val operation: Operation = parse(sql, createPlanner(), createCalciteParser()) + assert(operation.isInstanceOf[PlannerQueryOperation]) + operation.asInstanceOf[PlannerQueryOperation].getCalciteTree + } + + def createTable(sql: String): Unit = { + val calciteParser = plannerContext.createCalciteParser() + val operation = parse(sql, createPlanner(), calciteParser) + assert(operation.isInstanceOf[CreateTableOperation]) + val createTableOperation: CreateTableOperation = operation.asInstanceOf[CreateTableOperation] + catalogManager.createTable(createTableOperation.getCatalogTable, + createTableOperation.getTableIdentifier, + createTableOperation.isIgnoreIfExists) + } + + def createScalarFunc(name: String, func: ScalarFunction): Unit = { + functionCatalog.registerTempSystemScalarFunction("my_udf", func) + } + + def createScalarFunc(objectIdentifier: ObjectIdentifier, func: ScalarFunction): Unit = { + functionCatalog.registerTempCatalogScalarFunction(objectIdentifier, func) + } +} diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtilTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtilTest.scala index 3b885ec7cf60..31a371cc5adc 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtilTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtilTest.scala @@ -47,18 +47,19 @@ class FlinkRelOptUtilTest { val result = tableEnv.sqlQuery(sqlQuery) val rel = TableTestUtil.toRelNode(result) + // Ignore the attributes because the data stream table name is random. val expected1 = """ - |LogicalProject(a=[$0], c=[$1], a0=[$2], c0=[$3]) - |+- LogicalJoin(condition=[=($0, $2)], joinType=[inner]) - | :- LogicalProject(a=[$0], c=[$2]) - | : +- LogicalFilter(condition=[>($1, 50)]) - | : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) - | +- LogicalProject(a=[*($0, 2)], c=[$2]) - | +- LogicalFilter(condition=[<($1, 50)]) - | +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + |LogicalProject + |+- LogicalJoin + | :- LogicalProject + | : +- LogicalFilter + | : +- LogicalTableScan + | +- LogicalProject + | +- LogicalFilter + | +- LogicalTableScan """.stripMargin - assertEquals(expected1.trim, FlinkRelOptUtil.toString(rel).trim) + assertEquals(expected1.trim, FlinkRelOptUtil.toString(rel, SqlExplainLevel.NO_ATTRIBUTES).trim) val expected2 = """ diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala index 86d3940b668a..2fc0b238bd59 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala @@ -226,7 +226,7 @@ object PartitionableSinkITCase { val fieldNames = Array("a", "b", "c") val dataType = Array(new IntType(), new BigIntType(), new VarCharType(VarCharType.MAX_LENGTH)) - val dataNullables = Array(false, false, false) + val dataNullables = Array(true, true, true) val testData = Seq( row(3, 2L, "Hello03"), diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateITCaseBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateITCaseBase.scala index b784a9a814ec..771c3cf08113 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateITCaseBase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateITCaseBase.scala @@ -231,13 +231,11 @@ abstract class AggregateITCaseBase(testName: String) extends BatchTestBase { Types.STRING, Types.STRING, Types.STRING) - val nullables4 = Array(false, false, false, false) registerCollection( "src", data, type4, - "a, b, c, d", - nullables4) + "a, b, c, d") val sql = s""" diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/DistinctAggregateITCaseBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/DistinctAggregateITCaseBase.scala index b202aa5b6bf3..07016a47af59 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/DistinctAggregateITCaseBase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/DistinctAggregateITCaseBase.scala @@ -54,7 +54,7 @@ abstract class DistinctAggregateITCaseBase extends BatchTestBase { row(r.getField(0), r.getField(1), v2) } - registerCollection("NullTable3", nullData3, type3, "a, b, c", Array(false, false, true)) + registerCollection("NullTable3", nullData3, type3, "a, b, c", Array(true, true, true)) prepareAggOp() } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/GroupingSetsITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/GroupingSetsITCase.scala index 2f03e77a4db2..584d0732704c 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/GroupingSetsITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/GroupingSetsITCase.scala @@ -40,7 +40,7 @@ class GroupingSetsITCase extends BatchTestBase { private val empsNames = "empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat" private val nullableOfEmps: Array[Boolean] = - Array(false, false, false, true, true, false, true, true, false, false) + Array(true, true, true, true, true, true, true, true, true, true) private lazy val empsData = Seq( row(100L, "Fred", 10, null, null, 40L, 25, true, false, localDate("1996-08-03")), row(110L, "Eric", 20, "M", "San Francisco", 3L, 80, null, false, localDate("2001-01-01")), @@ -52,7 +52,7 @@ class GroupingSetsITCase extends BatchTestBase { private val TABLE_NAME_EMP = "emp" private val empTypes = new RowTypeInfo(Types.STRING, Types.INT, Types.STRING) private val empNames = "ename, deptno, gender" - private val nullableOfEmp = Array(false, true, false) + private val nullableOfEmp = Array(true, true, true) private lazy val empData = Seq( row("Adam", 50, "M"), row("Alice", 30, "F"), @@ -68,7 +68,7 @@ class GroupingSetsITCase extends BatchTestBase { private val TABLE_NAME_DEPT = "dept" private val deptTypes = new RowTypeInfo(Types.INT, Types.STRING) private val deptNames = "deptno, dname" - private val nullableOfDept = Array(false, false) + private val nullableOfDept = Array(true, true) private lazy val deptData = Seq( row(10, "Sales"), row(20, "Marketing"), @@ -80,7 +80,7 @@ class GroupingSetsITCase extends BatchTestBase { private val scottEmpTypes = new RowTypeInfo(Types.INT, Types.STRING, Types.STRING, Types.INT, Types.LOCAL_DATE, Types.DOUBLE, Types.DOUBLE, Types.INT) private val scottEmpNames = "empno, ename, job, mgr, hiredate, sal, comm, deptno" - private val nullableOfScottEmp = Array(false, false, false, true, false, false, true, false) + private val nullableOfScottEmp = Array(true, true, true, true, true, true, true, true) private lazy val scottEmpData = Seq( row(7369, "SMITH", "CLERK", 7902, localDate("1980-12-17"), 800.00, null, 20), row(7499, "ALLEN", "SALESMAN", 7698, localDate("1981-02-20"), 1600.00, 300.00, 30), @@ -110,7 +110,7 @@ class GroupingSetsITCase extends BatchTestBase { } row(r.getField(0), r.getField(1), newField2) } - val nullablesOfNullsData3 = Array(false, false, true) + val nullablesOfNullsData3 = Array(true, true, true) registerCollection(TABLE_WITH_NULLS_NAME, nullableData3, type3, "f0, f1, f2", nullablesOfNullsData3) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/InnerJoinITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/InnerJoinITCase.scala index d2e6220392eb..4666cc28e73c 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/InnerJoinITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/InnerJoinITCase.scala @@ -79,10 +79,10 @@ class InnerJoinITCase(expectedJoinType: JoinType) extends BatchTestBase { @Before override def before(): Unit = { super.before() - registerCollection("myUpperCaseData", myUpperCaseData, INT_STRING, "N, L", Array(true, false)) - registerCollection("myLowerCaseData", myLowerCaseData, INT_STRING, "n, l", Array(true, false)) - registerCollection("myTestData1", myTestData1, INT_INT, "a, b", Array(false, false)) - registerCollection("myTestData2", myTestData2, INT_INT, "a, b", Array(false, false)) + registerCollection("myUpperCaseData", myUpperCaseData, INT_STRING, "N, L", Array(true, true)) + registerCollection("myLowerCaseData", myLowerCaseData, INT_STRING, "n, l", Array(true, true)) + registerCollection("myTestData1", myTestData1, INT_INT, "a, b", Array(true, true)) + registerCollection("myTestData2", myTestData2, INT_INT, "a, b", Array(true, true)) disableOtherJoinOpForJoin(tEnv, expectedJoinType) } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TestData.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TestData.scala index 09e8faea5711..958d83408a13 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TestData.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TestData.scala @@ -59,13 +59,13 @@ object TestData { val type3WithTimestamp = new RowTypeInfo(INT_TYPE_INFO, LONG_TYPE_INFO, STRING_TYPE_INFO, LOCAL_DATE_TIME) - val nullablesOfData1 = Array(false, false, false) + val nullablesOfData1 = Array(true, true, true) - val nullableOfSimpleData2 = Array(false, false) + val nullableOfSimpleData2 = Array(true, true) - val nullablesOfData2 = Array(false, false, false, false, false) + val nullablesOfData2 = Array(true, true, true, true, true) - val nullablesOfNullData2 = Array(true, false, false, false, false) + val nullablesOfNullData2 = Array(true, true, true, true, true) lazy val data1: Seq[Row] = Seq( row(2, "a", 6), @@ -129,14 +129,14 @@ object TestData { val allNullablesOfNullData3 = Array(true, true, true) - val nullablesOfNullData3 = Array(true, false, false) + val nullablesOfNullData3 = Array(true, true, true) lazy val nullData5: Seq[Row] = data5 ++ Seq( row(null, 999L, 999, "NullTuple", 999L), row(null, 999L, 999, "NullTuple", 999L) ) - val nullablesOfNullData5 = Array(true, false, false, false, false) + val nullablesOfNullData5 = Array(true, true, true, true, true) lazy val smallTupleData3: Seq[(Int, Long, String)] = { val data = new mutable.MutableList[(Int, Long, String)] @@ -148,7 +148,7 @@ object TestData { lazy val smallData3: Seq[Row] = smallTupleData3.map(d => row(d.productIterator.toList: _*)) - val nullablesOfSmallData3 = Array(false, false, false) + val nullablesOfSmallData3 = Array(true, true, true) lazy val smallTupleData5: Seq[(Int, Long, Int, String, Long)] = { val data = new mutable.MutableList[(Int, Long, Int, String, Long)] @@ -160,7 +160,7 @@ object TestData { lazy val smallData5: Seq[Row] = smallTupleData5.map(d => row(d.productIterator.toList: _*)) - val nullablesOfSmallData5 = Array(false, false, false, false, false) + val nullablesOfSmallData5 = Array(true, true, true, true, true) lazy val buildInData: Seq[Row] = Seq( row(false, 1.toByte, 2, 3L, 2.56, "abcd", "f%g", localDate("2017-12-12"), @@ -219,11 +219,11 @@ object TestData { lazy val data3: Seq[Row] = tupleData3.map(d => row(d.productIterator.toList: _*)) - val nullablesOfData3 = Array(false, false, false) + val nullablesOfData3 = Array(true, true, true) - val nullablesOfData4 = Array(false, false, false) + val nullablesOfData4 = Array(true, true, true) - val nullablesOfNullData4 = Array(false, false, true) + val nullablesOfNullData4 = Array(true, true, true) lazy val genericData3: Seq[Row] = Seq( row(new JTuple2("1", 1), new JTuple2(1, 1), 1), @@ -232,7 +232,7 @@ object TestData { row(new JTuple2("1", 1), new JTuple2(10, 1), 3) ) - val nullablesOfData3WithTimestamp = Array(true, false, false, false) + val nullablesOfData3WithTimestamp = Array(true, true, true, true) lazy val data3WithTimestamp: Seq[Row] = Seq( row(2, 2L, "Hello", unixTimestampToLocalDateTime(2000L)), @@ -288,7 +288,7 @@ object TestData { lazy val data5: Seq[Row] = tupleData5.map(d => row(d.productIterator.toList: _*)) - val nullablesOfData5 = Array(false, false, false, false, false) + val nullablesOfData5 = Array(true, true, true, true, true) lazy val data6: Seq[Row] = Seq( row(1, 1.1, "a", localDate("2017-04-08"), localTime("12:00:59"), @@ -323,7 +323,7 @@ object TestData { localDateTime("1937-07-07 08:08:08.888")) ) - val nullablesOfData6 = Array(false, false, false, false, false, false) + val nullablesOfData6 = Array(true, true, true, true, true, true) lazy val duplicateData5: Seq[Row] = Seq( row(1, 1L, 10, "Hallo", 1L), @@ -343,7 +343,7 @@ object TestData { row(5, 15L, 14, "EFG", 2L) ) - val nullablesOfDuplicateData5 = Array(false, false, false, false, false) + val nullablesOfDuplicateData5 = Array(true, true, true, true, true) lazy val numericData: Seq[Row] = Seq( row(1, 1L, 1.0f, 1.0d, JBigDecimal.valueOf(1)), @@ -351,7 +351,7 @@ object TestData { row(3, 3L, 3.0f, 3.0d, JBigDecimal.valueOf(3)) ) - val nullablesOfNumericData = Array(false, false, false, false, false) + val nullablesOfNumericData = Array(true, true, true, true, true) // person test data lazy val personData: Seq[Row] = Seq( @@ -368,7 +368,7 @@ object TestData { row(11, 20, "eva", 180, "f") ) - val nullablesOfPersonData = Array(false, false, false, false, false) + val nullablesOfPersonData = Array(true, true, true, true, true) val personType = new RowTypeInfo(INT_TYPE_INFO, INT_TYPE_INFO, STRING_TYPE_INFO, INT_TYPE_INFO, STRING_TYPE_INFO) @@ -405,13 +405,13 @@ object TestData { row(4, 1.0) ) - val nullablesOfData2_3 = Array(false, false) + val nullablesOfData2_3 = Array(true, true) lazy val intStringData: Seq[Row] = { (1 to 100).map(i => row(i, i.toString)) } - val nullablesOfIntStringData = Array(false, false) + val nullablesOfIntStringData = Array(true, true) lazy val bigIntStringData: Seq[Row] = { (1 to 10000).map(i => row(i, i.toString)) @@ -426,14 +426,14 @@ object TestData { row(3, 2) :: Nil } - val nullablesOfIntIntData2 = Array(false, false) + val nullablesOfIntIntData2 = Array(true, true) lazy val intIntData3: Seq[Row] = { row(1, null) :: row(2, 2) :: Nil } - val nullablesOfIntIntData3 = Array(false, false) + val nullablesOfIntIntData3 = Array(true, true) lazy val upperCaseData: Seq[Row] = Seq( row(1, "A"), @@ -443,7 +443,7 @@ object TestData { row(5, "E"), row(6, "F")) - val nullablesOfUpperCaseData = Array(false, false) + val nullablesOfUpperCaseData = Array(true, true) lazy val lowerCaseData: Seq[Row] = Seq( row(1, "a"), @@ -451,7 +451,7 @@ object TestData { row(3, "c"), row(4, "d")) - val nullablesOfLowerCaseData = Array(false, false) + val nullablesOfLowerCaseData = Array(true, true) lazy val allNulls: Seq[Row] = Seq( row(null), @@ -472,5 +472,5 @@ object TestData { STRING_TYPE_INFO, STRING_TYPE_INFO, STRING_TYPE_INFO, INT_TYPE_INFO, STRING_TYPE_INFO) - val nullablesOfProjectionTestData = Array(false, false, false, false, false, false, false, false) + val nullablesOfProjectionTestData = Array(true, true, true, true, true, true, true, true) } From 43ef98e339666334da04d4e0668ee5bfd7853b61 Mon Sep 17 00:00:00 2001 From: yanghua Date: Tue, 29 Oct 2019 16:36:49 +0800 Subject: [PATCH 542/746] [FLINK-14554] Correct the comment of ExistingSavepoint#readKeyedState to generate java doc --- .../java/org/apache/flink/state/api/ExistingSavepoint.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/ExistingSavepoint.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/ExistingSavepoint.java index de6205e6b0c0..f336ce1e7ea1 100644 --- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/ExistingSavepoint.java +++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/ExistingSavepoint.java @@ -214,13 +214,14 @@ public DataSet> readBroadcastState( return env.createInput(inputFormat, new TupleTypeInfo<>(keyTypeInfo, valueTypeInfo)); } - /* + /** * Read keyed state from an operator in a {@code Savepoint}. * @param uid The uid of the operator. * @param function The {@link KeyedStateReaderFunction} that is called for each key in state. * @param The type of the key in state. * @param The output type of the transform function. * @return A {@code DataSet} of objects read from keyed state. + * @throws IOException If the savepoint does not contain operator state with the given uid. */ public DataSet readKeyedState(String uid, KeyedStateReaderFunction function) throws IOException { @@ -265,6 +266,7 @@ public DataSet readKeyedState(String uid, KeyedStateReaderFunction * @param The type of the key in state. * @param The output type of the transform function. * @return A {@code DataSet} of objects read from keyed state. + * @throws IOException If the savepoint does not contain operator state with the given uid. */ public DataSet readKeyedState( String uid, From a43175c8b03e2dc3fcfc2a42a71be37cf05c92cd Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Wed, 6 Nov 2019 12:57:57 +0800 Subject: [PATCH 543/746] [FLINK-14601] [client] CLI documentation for list is missing '-a' --- .../java/org/apache/flink/client/cli/CliFrontendParser.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java index c0295c707312..d0b1a9bacf24 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java @@ -284,7 +284,9 @@ private static Options getInfoOptionsWithoutDeprecatedOptions(Options options) { private static Options getListOptionsWithoutDeprecatedOptions(Options options) { options.addOption(RUNNING_OPTION); - return options.addOption(SCHEDULED_OPTION); + options.addOption(ALL_OPTION); + options.addOption(SCHEDULED_OPTION); + return options; } private static Options getCancelOptionsWithoutDeprecatedOptions(Options options) { From 7a61c582c7213f123e10de4fd11a13d96425fd77 Mon Sep 17 00:00:00 2001 From: yanghua Date: Wed, 30 Oct 2019 15:44:30 +0800 Subject: [PATCH 544/746] [hotfix] Fix wrong Java doc comment of BroadcastStateBootstrapFunction.Context --- .../state/api/functions/BroadcastStateBootstrapFunction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/functions/BroadcastStateBootstrapFunction.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/functions/BroadcastStateBootstrapFunction.java index 5ac0d81e2b9d..147fa7a62378 100644 --- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/functions/BroadcastStateBootstrapFunction.java +++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/functions/BroadcastStateBootstrapFunction.java @@ -43,7 +43,7 @@ public abstract class BroadcastStateBootstrapFunction extends AbstractRichFu public abstract void processElement(IN value, Context ctx) throws Exception; /** - * Context that {@link StateBootstrapFunction}'s can use for getting additional data about an input + * Context that {@link BroadcastStateBootstrapFunction}'s can use for getting additional data about an input * record. * *

    The context is only valid for the duration of a {@link From 0f33630decd52045f93c93f9a3e9e156dc0c0bdc Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Fri, 8 Nov 2019 14:18:35 +0100 Subject: [PATCH 545/746] [FLINK-14374][runtime] Expose scheduler type constants in SchedulerNGFactoryFactory --- .../dispatcher/SchedulerNGFactoryFactory.java | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SchedulerNGFactoryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SchedulerNGFactoryFactory.java index a757bae702e0..d533bb9b4e57 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SchedulerNGFactoryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SchedulerNGFactoryFactory.java @@ -26,7 +26,14 @@ import org.apache.flink.runtime.scheduler.LegacySchedulerFactory; import org.apache.flink.runtime.scheduler.SchedulerNGFactory; -final class SchedulerNGFactoryFactory { +/** + * Factory for {@link SchedulerNGFactory}. + */ +public final class SchedulerNGFactoryFactory { + + public static final String SCHEDULER_TYPE_LEGACY = "legacy"; + + public static final String SCHEDULER_TYPE_NG = "ng"; private SchedulerNGFactoryFactory() {} @@ -36,10 +43,10 @@ static SchedulerNGFactory createSchedulerNGFactory( final String schedulerName = configuration.getString(JobManagerOptions.SCHEDULER); switch (schedulerName) { - case "legacy": + case SCHEDULER_TYPE_LEGACY: return new LegacySchedulerFactory(restartStrategyFactory); - case "ng": + case SCHEDULER_TYPE_NG: return new DefaultSchedulerFactory(); default: From 6aa0a6738391baf1fa6d5c486f80188d71fd104d Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Fri, 8 Nov 2019 14:19:48 +0100 Subject: [PATCH 546/746] [FLINK-14374][runtime,tests] Run RegionFailoverITCase also with DefaultScheduler enabled This closes #10134. --- .../checkpointing/RegionFailoverITCase.java | 30 +++++++++++++++++-- 1 file changed, 27 insertions(+), 3 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RegionFailoverITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RegionFailoverITCase.java index ac258992c08c..43f9fe99f0df 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RegionFailoverITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RegionFailoverITCase.java @@ -65,6 +65,8 @@ import org.junit.ClassRule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import java.io.IOException; import java.util.Collections; @@ -79,11 +81,15 @@ import java.util.stream.IntStream; import java.util.stream.StreamSupport; +import static org.apache.flink.runtime.dispatcher.SchedulerNGFactoryFactory.SCHEDULER_TYPE_LEGACY; +import static org.apache.flink.runtime.dispatcher.SchedulerNGFactoryFactory.SCHEDULER_TYPE_NG; +import static org.apache.flink.util.Preconditions.checkNotNull; import static org.junit.Assert.assertEquals; /** * Tests for region failover with multi regions. */ +@RunWith(Parameterized.class) public class RegionFailoverITCase extends TestLogger { private static final int FAIL_BASE = 1000; @@ -110,15 +116,33 @@ public class RegionFailoverITCase extends TestLogger { @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + private final String schedulerType; + + public RegionFailoverITCase(final String schedulerType) { + this.schedulerType = checkNotNull(schedulerType); + } + + @Parameterized.Parameters(name = "scheduler = {0}") + public static Object[] testParameters() { + return new Object[]{SCHEDULER_TYPE_NG, SCHEDULER_TYPE_LEGACY}; + } + @Before public void setup() throws Exception { Configuration configuration = new Configuration(); configuration.setString(JobManagerOptions.EXECUTION_FAILOVER_STRATEGY, "region"); configuration.setString(HighAvailabilityOptions.HA_MODE, TestingHAFactory.class.getName()); - // global failover times: 3, region failover times: NUM_OF_RESTARTS - configuration.setInteger(FailingRestartStrategy.NUM_FAILURES_CONFIG_OPTION, 3); - configuration.setString(RestartStrategyOptions.RESTART_STRATEGY, FailingRestartStrategy.class.getName()); + configuration.setString(JobManagerOptions.SCHEDULER, schedulerType); + + // If the LegacyScheduler is configured, we will use a custom RestartStrategy + // (FailingRestartStrategy). This is done to test FLINK-13452. DefaultScheduler takes a + // different code path, and also cannot be configured with custom RestartStrategies. + if (SCHEDULER_TYPE_LEGACY.equals(schedulerType)) { + // global failover times: 3, region failover times: NUM_OF_RESTARTS + configuration.setInteger(FailingRestartStrategy.NUM_FAILURES_CONFIG_OPTION, 3); + configuration.setString(RestartStrategyOptions.RESTART_STRATEGY, FailingRestartStrategy.class.getName()); + } cluster = new MiniClusterWithClientResource( new MiniClusterResourceConfiguration.Builder() From 7baf6875bdb4362b782b676bb33ef47ec329d049 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Fri, 8 Nov 2019 14:36:38 +0800 Subject: [PATCH 547/746] [hotfix][docs] Fix broken table that describes 'numberOfRestarts' metric --- docs/monitoring/metrics.md | 2 +- docs/monitoring/metrics.zh.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/monitoring/metrics.md b/docs/monitoring/metrics.md index 87fea3dbc873..f1501a421003 100644 --- a/docs/monitoring/metrics.md +++ b/docs/monitoring/metrics.md @@ -1253,7 +1253,7 @@ Metrics related to data exchange between task executors using netty network comm - Job (only available on JobManager) + Job (only available on JobManager) restartingTime The time it took to restart the job, or how long the current restart has been in progress (in milliseconds). Gauge diff --git a/docs/monitoring/metrics.zh.md b/docs/monitoring/metrics.zh.md index fa1db8e7b6e9..4490712e2353 100644 --- a/docs/monitoring/metrics.zh.md +++ b/docs/monitoring/metrics.zh.md @@ -1253,7 +1253,7 @@ Metrics related to data exchange between task executors using netty network comm - Job (only available on JobManager) + Job (only available on JobManager) restartingTime The time it took to restart the job, or how long the current restart has been in progress (in milliseconds). Gauge From 51c2421a0eff8d820f313a431bb33b6fd37cc151 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Mon, 11 Nov 2019 19:26:28 +0800 Subject: [PATCH 548/746] [hotfix][runtime] Rename metric 'numberOfRestarts' to 'numRestarts' --- docs/monitoring/metrics.md | 4 ++-- docs/monitoring/metrics.zh.md | 4 ++-- .../java/org/apache/flink/runtime/metrics/MetricNames.java | 2 +- .../org/apache/flink/runtime/scheduler/SchedulerBase.java | 4 ++-- .../test/checkpointing/ZooKeeperHighAvailabilityITCase.java | 4 ++-- 5 files changed, 9 insertions(+), 9 deletions(-) diff --git a/docs/monitoring/metrics.md b/docs/monitoring/metrics.md index f1501a421003..b2095c1601cf 100644 --- a/docs/monitoring/metrics.md +++ b/docs/monitoring/metrics.md @@ -1280,8 +1280,8 @@ Metrics related to data exchange between task executors using netty network comm Gauge - numberOfRestarts - The total number of restarts since this job was submitted, including full restarts and fine grained restarts. + numRestarts + The total number of restarts since this job was submitted, including full restarts and fine-grained restarts. Gauge diff --git a/docs/monitoring/metrics.zh.md b/docs/monitoring/metrics.zh.md index 4490712e2353..3c88228bc4f3 100644 --- a/docs/monitoring/metrics.zh.md +++ b/docs/monitoring/metrics.zh.md @@ -1280,8 +1280,8 @@ Metrics related to data exchange between task executors using netty network comm Gauge - numberOfRestarts - The total number of restarts since this job was submitted, including full restarts and fine grained restarts. + numRestarts + The total number of restarts since this job was submitted, including full restarts and fine-grained restarts. Gauge diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java index d6f539d2bb27..1d0bb6d18d8c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java @@ -51,7 +51,7 @@ private MetricNames() { public static final String TASK_SLOTS_TOTAL = "taskSlotsTotal"; public static final String NUM_REGISTERED_TASK_MANAGERS = "numRegisteredTaskManagers"; - public static final String NUMBER_OF_RESTARTS = "numberOfRestarts"; + public static final String NUM_RESTARTS = "numRestarts"; public static final String MEMORY_USED = "Used"; public static final String MEMORY_COMMITTED = "Committed"; 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 573ccea67730..cfb5a494df74 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 @@ -69,6 +69,7 @@ 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.MetricNames; import org.apache.flink.runtime.metrics.groups.JobManagerJobMetricGroup; import org.apache.flink.runtime.query.KvStateLocation; import org.apache.flink.runtime.query.KvStateLocationRegistry; @@ -99,7 +100,6 @@ import java.util.concurrent.Executor; import java.util.concurrent.ScheduledExecutorService; -import static org.apache.flink.runtime.metrics.MetricNames.NUMBER_OF_RESTARTS; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkState; @@ -196,7 +196,7 @@ public SchedulerBase( this.inputsLocationsRetriever = new ExecutionGraphToInputsLocationsRetrieverAdapter(executionGraph); - jobManagerJobMetricGroup.gauge(NUMBER_OF_RESTARTS, this::getNumberOfRestarts); + jobManagerJobMetricGroup.gauge(MetricNames.NUM_RESTARTS, this::getNumberOfRestarts); } private ExecutionGraph createAndRestoreExecutionGraph( diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java index 7d2e3b258b69..0c859f85641d 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java @@ -428,7 +428,7 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { } /** - * Reporter that exposes the {@code numberOfRestarts} metric. + * Reporter that exposes the {@code numRestarts} metric. */ public static class RestartReporter implements MetricReporter { static volatile Gauge numRestarts = null; @@ -443,7 +443,7 @@ public void close() { @Override public void notifyOfAddedMetric(Metric metric, String name, MetricGroup metricGroup) { - if (name.equals(MetricNames.NUMBER_OF_RESTARTS)) { + if (name.equals(MetricNames.NUM_RESTARTS)) { numRestarts = (Gauge) metric; } } From 644d658492a1bdada498b52d8cba8964be709699 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Mon, 11 Nov 2019 19:32:42 +0800 Subject: [PATCH 549/746] [FLINK-14164][runtime] Make metric 'fullRestarts' compatible with NG scheduler This closes #10150. --- .../executiongraph/ExecutionGraphBuilder.java | 2 - .../metrics/NumberOfFullRestartsGauge.java | 47 ------------------- .../flink/runtime/metrics/MetricNames.java | 1 + .../runtime/scheduler/SchedulerBase.java | 1 + 4 files changed, 2 insertions(+), 49 deletions(-) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/NumberOfFullRestartsGauge.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java index 6b0e389db579..96d7806da384 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java @@ -41,7 +41,6 @@ import org.apache.flink.runtime.executiongraph.failover.flip1.partitionrelease.PartitionReleaseStrategy; import org.apache.flink.runtime.executiongraph.failover.flip1.partitionrelease.PartitionReleaseStrategyFactoryLoader; import org.apache.flink.runtime.executiongraph.metrics.DownTimeGauge; -import org.apache.flink.runtime.executiongraph.metrics.NumberOfFullRestartsGauge; import org.apache.flink.runtime.executiongraph.metrics.RestartTimeGauge; import org.apache.flink.runtime.executiongraph.metrics.UpTimeGauge; import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; @@ -353,7 +352,6 @@ public static ExecutionGraph buildGraph( metrics.gauge(RestartTimeGauge.METRIC_NAME, new RestartTimeGauge(executionGraph)); metrics.gauge(DownTimeGauge.METRIC_NAME, new DownTimeGauge(executionGraph)); metrics.gauge(UpTimeGauge.METRIC_NAME, new UpTimeGauge(executionGraph)); - metrics.gauge(NumberOfFullRestartsGauge.METRIC_NAME, new NumberOfFullRestartsGauge(executionGraph)); executionGraph.getFailoverStrategy().registerMetrics(metrics); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/NumberOfFullRestartsGauge.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/NumberOfFullRestartsGauge.java deleted file mode 100644 index b1dcd4e7e7f2..000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/metrics/NumberOfFullRestartsGauge.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.executiongraph.metrics; - -import org.apache.flink.metrics.Gauge; -import org.apache.flink.runtime.executiongraph.ExecutionGraph; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * Gauge which returns the number of full restarts. - */ -public class NumberOfFullRestartsGauge implements Gauge { - - public static final String METRIC_NAME = "fullRestarts"; - - // ------------------------------------------------------------------------ - - private final ExecutionGraph eg; - - public NumberOfFullRestartsGauge(ExecutionGraph executionGraph) { - this.eg = checkNotNull(executionGraph); - } - - // ------------------------------------------------------------------------ - - @Override - public Long getValue() { - return eg.getNumberOfRestarts(); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java index 1d0bb6d18d8c..fbd33ff0ef59 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricNames.java @@ -52,6 +52,7 @@ private MetricNames() { public static final String NUM_REGISTERED_TASK_MANAGERS = "numRegisteredTaskManagers"; public static final String NUM_RESTARTS = "numRestarts"; + public static final String FULL_RESTARTS = "fullRestarts"; public static final String MEMORY_USED = "Used"; public static final String MEMORY_COMMITTED = "Committed"; 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 cfb5a494df74..95f378e9288a 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 @@ -197,6 +197,7 @@ public SchedulerBase( this.inputsLocationsRetriever = new ExecutionGraphToInputsLocationsRetrieverAdapter(executionGraph); jobManagerJobMetricGroup.gauge(MetricNames.NUM_RESTARTS, this::getNumberOfRestarts); + jobManagerJobMetricGroup.gauge(MetricNames.FULL_RESTARTS, this::getNumberOfRestarts); } private ExecutionGraph createAndRestoreExecutionGraph( From 658cea39b120bf39992c5216a03641d7e60ae581 Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Fri, 8 Nov 2019 15:22:05 -0800 Subject: [PATCH 550/746] [FLINK-14660][sql cli] add 'SHOW MODULES' to SQL CLI This closes #10140. --- .../flink/table/client/cli/CliClient.java | 20 +++++++++++++++++++ .../table/client/cli/SqlCommandParser.java | 4 ++++ .../flink/table/client/gateway/Executor.java | 5 +++++ .../client/gateway/local/LocalExecutor.java | 9 +++++++++ .../flink/table/client/cli/CliClientTest.java | 5 +++++ .../table/client/cli/CliResultViewTest.java | 5 +++++ 6 files changed, 48 insertions(+) diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java index e956039533fe..72a626b766c2 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java @@ -276,6 +276,9 @@ private void callCommand(SqlCommandCall cmdCall) { case SHOW_FUNCTIONS: callShowFunctions(); break; + case SHOW_MODULES: + callShowModules(); + break; case USE_CATALOG: callUseCatalog(cmdCall); break; @@ -421,6 +424,23 @@ private void callShowFunctions() { terminal.flush(); } + private void callShowModules() { + final List modules; + try { + modules = executor.listModules(context); + } catch (SqlExecutionException e) { + printExecutionException(e); + return; + } + if (modules.isEmpty()) { + terminal.writer().println(CliStrings.messageInfo(CliStrings.MESSAGE_EMPTY).toAnsi()); + } else { + // modules are already in the loaded order + modules.forEach((v) -> terminal.writer().println(v)); + } + terminal.flush(); + } + private void callUseCatalog(SqlCommandCall cmdCall) { try { executor.useCatalog(context, cmdCall.operands[0]); diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java index d4026d78b268..363f8f579d25 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java @@ -99,6 +99,10 @@ enum SqlCommand { "SHOW\\s+FUNCTIONS", NO_OPERANDS), + SHOW_MODULES( + "SHOW\\s+MODULES", + NO_OPERANDS), + USE_CATALOG( "USE\\s+CATALOG\\s+(.*)", SINGLE_OPERAND), diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/Executor.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/Executor.java index 1ad24e466fb4..d0c4437d1ab0 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/Executor.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/Executor.java @@ -65,6 +65,11 @@ public interface Executor { */ List listFunctions(SessionContext session) throws SqlExecutionException; + /** + * Lists all modules known to the executor in their loaded order. + */ + List listModules(SessionContext session) throws SqlExecutionException; + /** * Sets a catalog with given name as the current catalog. */ 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 f6166364e46e..9a90b5e91bd4 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 @@ -248,6 +248,15 @@ public List listFunctions(SessionContext session) throws SqlExecutionExc return context.wrapClassLoader(() -> Arrays.asList(tableEnv.listFunctions())); } + @Override + public List listModules(SessionContext session) throws SqlExecutionException { + final ExecutionContext context = getOrCreateExecutionContext(session); + final TableEnvironment tableEnv = context + .createEnvironmentInstance() + .getTableEnvironment(); + return context.wrapClassLoader(() -> Arrays.asList(tableEnv.listModules())); + } + @Override public void useCatalog(SessionContext session, String catalogName) throws SqlExecutionException { final ExecutionContext context = getOrCreateExecutionContext(session); diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientTest.java index ec466e3cb943..a3c74d3b39c0 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientTest.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientTest.java @@ -237,6 +237,11 @@ public List listFunctions(SessionContext session) throws SqlExecutionExc return null; } + @Override + public List listModules(SessionContext session) throws SqlExecutionException { + return null; + } + @Override public void useCatalog(SessionContext session, String catalogName) throws SqlExecutionException { diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliResultViewTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliResultViewTest.java index 8eff36aa776f..e49874a4dc08 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliResultViewTest.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliResultViewTest.java @@ -150,6 +150,11 @@ public List listFunctions(SessionContext session) throws SqlExecutionExc return null; } + @Override + public List listModules(SessionContext session) throws SqlExecutionException { + return null; + } + @Override public void useCatalog(SessionContext session, String catalogName) throws SqlExecutionException { From 767bf99adba9d220437056c74759310a25d9686c Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Thu, 7 Nov 2019 15:52:39 -0800 Subject: [PATCH 551/746] [FLINK-14420][doc] Add documentation for pluggable module this closes #10121. --- docs/dev/table/hive/hive_functions.md | 36 +++++++ docs/dev/table/hive/hive_functions.zh.md | 36 +++++++ docs/dev/table/modules.md | 126 +++++++++++++++++++++++ docs/dev/table/modules.zh.md | 126 +++++++++++++++++++++++ 4 files changed, 324 insertions(+) create mode 100644 docs/dev/table/modules.md create mode 100644 docs/dev/table/modules.zh.md diff --git a/docs/dev/table/hive/hive_functions.md b/docs/dev/table/hive/hive_functions.md index 8f1894f0a7b3..56394b508b2b 100644 --- a/docs/dev/table/hive/hive_functions.md +++ b/docs/dev/table/hive/hive_functions.md @@ -165,3 +165,39 @@ Support for Hive functions has only been tested for Flink batch in Blink planner Hive functions currently cannot be used across catalogs in Flink. Please reference to [Hive]({{ site.baseurl }}/dev/table/hive/index.html) for data type limitations. + + +## Use Hive Built-in Functions via HiveModule + +The `HiveModule` provides Hive built-in functions as Flink system (built-in) functions to Flink SQL and Table API users. + +

    +
    +{% highlight java %} + +String name = "myhive"; +String version = "2.3.4"; + +tableEnv.loadModue(name, new HiveModule(version)); +{% endhighlight %} +
    +
    +{% highlight scala %} + +val name = "myhive" +val version = "2.3.4" + +tableEnv.loadModue(name, new HiveModule(version)); +{% endhighlight %} +
    +
    +{% highlight yaml %} +modules: + - name: core + type: core + - name: myhive + type: hive + hive-version: 2.3.4 +{% endhighlight %} +
    +
    diff --git a/docs/dev/table/hive/hive_functions.zh.md b/docs/dev/table/hive/hive_functions.zh.md index 8f1894f0a7b3..56394b508b2b 100644 --- a/docs/dev/table/hive/hive_functions.zh.md +++ b/docs/dev/table/hive/hive_functions.zh.md @@ -165,3 +165,39 @@ Support for Hive functions has only been tested for Flink batch in Blink planner Hive functions currently cannot be used across catalogs in Flink. Please reference to [Hive]({{ site.baseurl }}/dev/table/hive/index.html) for data type limitations. + + +## Use Hive Built-in Functions via HiveModule + +The `HiveModule` provides Hive built-in functions as Flink system (built-in) functions to Flink SQL and Table API users. + +
    +
    +{% highlight java %} + +String name = "myhive"; +String version = "2.3.4"; + +tableEnv.loadModue(name, new HiveModule(version)); +{% endhighlight %} +
    +
    +{% highlight scala %} + +val name = "myhive" +val version = "2.3.4" + +tableEnv.loadModue(name, new HiveModule(version)); +{% endhighlight %} +
    +
    +{% highlight yaml %} +modules: + - name: core + type: core + - name: myhive + type: hive + hive-version: 2.3.4 +{% endhighlight %} +
    +
    diff --git a/docs/dev/table/modules.md b/docs/dev/table/modules.md new file mode 100644 index 000000000000..b73faa0e8dca --- /dev/null +++ b/docs/dev/table/modules.md @@ -0,0 +1,126 @@ +--- +title: "Modules" +is_beta: true +nav-parent_id: tableapi +nav-pos: 90 +--- + + +Modules allow users to extend Flink's built-in objects, such as defining functions that behave like Flink +built-in functions. They are pluggable, and while Flink provides a few pre-built modules, users can write +their own. + +For example, users can define their own geo functions and plug them into Flink as built-in functions to be used in +Flink SQL and Table APIs. Another example is users can load an out-of-shelf Hive module to use Hive built-in +functions as Flink built-in functions. + +* This will be replaced by the TOC +{:toc} + +## Module Types + +### CoreModule + +`CoreModule` contains all of Flink's system (built-in) functions and is loaded by default. + +### HiveModule + +The `HiveModule` provides Hive built-in functions as Flink's system functions to SQL and Table API users. +Flink's [Hive documentation]({{ site.baseurl }}/dev/table/hive/hive_functions.html) provides full details on setting up the module. + +### User-Defined Module + +Users can develop custom modules by implementing the `Module` interface. +To use custom modules in SQL CLI, users should develop both a module and its corresponding module factory by implementing +the `ModuleFactory` interface. + +A module factory defines a set of properties for configuring the module when the SQL CLI bootstraps. +Properties are passed to a discovery service where the service tries to match the properties to + a `ModuleFactory` and instantiate a corresponding module instance. + + +## Namespace and Resolution Order + +Objects provided by modules are considered part of Flink's system (built-in) objects; thus, they don't have any namespaces. + +When there are two objects of the same name residing in two modules, Flink always resolves the object reference to the one in the 1st loaded module. + +## Module API + +### Loading and unloading a Module + +Users can load and unload modules in an existing Flink session. + +
    +
    +{% highlight java %} +tableEnv.loadModule("myModule", new CustomModule()); +tableEnv.unloadModule("myModule"); +{% endhighlight %} +
    +
    + +All modules defined using YAML must provide a `type` property that specifies the type. +The following types are supported out of the box. + + + + + + + + + + + + + + + + + + +
    CatalogType Value
    CoreModulecore
    HiveModulehive
    + +{% highlight yaml %} +modules: + - name: core + type: core + - name: myhive + type: hive + hive-version: 1.2.1 +{% endhighlight %} +
    +
    + +### List Available Catalogs + +
    +
    +{% highlight java %} +tableEnv.listModules(); +{% endhighlight %} +
    +
    +{% highlight sql %} +Flink SQL> SHOW MODULES; +{% endhighlight %} +
    +
    diff --git a/docs/dev/table/modules.zh.md b/docs/dev/table/modules.zh.md new file mode 100644 index 000000000000..5aa7841bfd08 --- /dev/null +++ b/docs/dev/table/modules.zh.md @@ -0,0 +1,126 @@ +--- +title: "模块" +is_beta: true +nav-parent_id: tableapi +nav-pos: 90 +--- + + +Modules allow users to extend Flink's built-in objects, such as defining functions that behave like Flink +built-in functions. They are pluggable, and while Flink provides a few pre-built modules, users can write +their own. + +For example, users can define their own geo functions and plug them into Flink as built-in functions to be used in +Flink SQL and Table APIs. Another example is users can load an out-of-shelf Hive module to use Hive built-in +functions as Flink built-in functions. + +* This will be replaced by the TOC +{:toc} + +## Module Types + +### CoreModule + +`CoreModule` contains all of Flink's system (built-in) functions and is loaded by default. + +### HiveModule + +The `HiveModule` provides Hive built-in functions as Flink's system functions to SQL and Table API users. +Flink's [Hive documentation]({{ site.baseurl }}/dev/table/hive/hive_functions.html) provides full details on setting up the module. + +### User-Defined Module + +Users can develop custom modules by implementing the `Module` interface. +To use custom modules in SQL CLI, users should develop both a module and its corresponding module factory by implementing +the `ModuleFactory` interface. + +A module factory defines a set of properties for configuring the module when the SQL CLI bootstraps. +Properties are passed to a discovery service where the service tries to match the properties to + a `ModuleFactory` and instantiate a corresponding module instance. + + +## Namespace and Resolution Order + +Objects provided by modules are considered part of Flink's system (built-in) objects; thus, they don't have any namespaces. + +When there are two objects of the same name residing in two modules, Flink always resolves the object reference to the one in the 1st loaded module. + +## Module API + +### Loading and unloading a Module + +Users can load and unload modules in an existing Flink session. + +
    +
    +{% highlight java %} +tableEnv.loadModule("myModule", new CustomModule()); +tableEnv.unloadModule("myModule"); +{% endhighlight %} +
    +
    + +All modules defined using YAML must provide a `type` property that specifies the type. +The following types are supported out of the box. + + + + + + + + + + + + + + + + + + +
    CatalogType Value
    CoreModulecore
    HiveModulehive
    + +{% highlight yaml %} +modules: + - name: core + type: core + - name: myhive + type: hive + hive-version: 1.2.1 +{% endhighlight %} +
    +
    + +### List Available Catalogs + +
    +
    +{% highlight java %} +tableEnv.listModules(); +{% endhighlight %} +
    +
    +{% highlight sql %} +Flink SQL> SHOW MODULES; +{% endhighlight %} +
    +
    From 55686bcf0eb679f9729fc6f63e3bb19e6c6b961d Mon Sep 17 00:00:00 2001 From: godfrey he Date: Tue, 12 Nov 2019 15:04:01 +0800 Subject: [PATCH 552/746] [FLINK-14656][table-planner-blink] blink planner should also fetch catalog statistics for permanent table (#10119) --- .../catalog/DatabaseCalciteSchema.java | 18 +-- .../catalog/CatalogStatisticsTest.java | 105 ++++++++++++++---- .../explain/testGetStatsFromCatalog.out | 32 ------ 3 files changed, 96 insertions(+), 59 deletions(-) delete mode 100644 flink-table/flink-table-planner-blink/src/test/resources/explain/testGetStatsFromCatalog.out diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/DatabaseCalciteSchema.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/DatabaseCalciteSchema.java index 9ccfbf1464fa..7cc80acaaac7 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/DatabaseCalciteSchema.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/DatabaseCalciteSchema.java @@ -104,7 +104,6 @@ private Table convertPermanentTable( ObjectPath tablePath, CatalogBaseTable table, @Nullable TableFactory tableFactory) { - // TODO supports GenericCatalogView if (table instanceof QueryOperationCatalogView) { return convertQueryOperationView(tablePath, (QueryOperationCatalogView) table); } else if (table instanceof ConnectorCatalogTable) { @@ -116,7 +115,9 @@ private Table convertPermanentTable( return convertSinkTable(connectorTable); } } else if (table instanceof CatalogTable) { - return convertCatalogTable(tablePath, (CatalogTable) table, tableFactory); + CatalogTable catalogTable = (CatalogTable) table; + TableStats tableStats = extractTableStats(catalogTable, tablePath); + return convertCatalogTable(tablePath, catalogTable, tableFactory, tableStats); } else { throw new TableException("Unsupported table type: " + table); } @@ -125,7 +126,6 @@ private Table convertPermanentTable( private Table convertTemporaryTable( ObjectPath tablePath, CatalogBaseTable table) { - // TODO supports GenericCatalogView if (table instanceof QueryOperationCatalogView) { return convertQueryOperationView(tablePath, (QueryOperationCatalogView) table); } else if (table instanceof ConnectorCatalogTable) { @@ -136,7 +136,7 @@ private Table convertTemporaryTable( return convertSinkTable(connectorTable); } } else if (table instanceof CatalogTable) { - return convertCatalogTable(tablePath, (CatalogTable) table, null); + return convertCatalogTable(tablePath, (CatalogTable) table, null, TableStats.UNKNOWN); } else { throw new TableException("Unsupported table type: " + table); } @@ -188,7 +188,7 @@ private Table convertSourceTable( table); } - private TableStats extractTableStats(ConnectorCatalogTable table, ObjectPath tablePath) { + private TableStats extractTableStats(CatalogTable table, ObjectPath tablePath) { TableStats tableStats = TableStats.UNKNOWN; try { // TODO supports stats for partitionable table @@ -208,7 +208,11 @@ private TableStats extractTableStats(ConnectorCatalogTable table, ObjectPa } } - private Table convertCatalogTable(ObjectPath tablePath, CatalogTable table, @Nullable TableFactory tableFactory) { + private Table convertCatalogTable( + ObjectPath tablePath, + CatalogTable table, + @Nullable TableFactory tableFactory, + TableStats tableStats) { TableSource tableSource; if (tableFactory != null) { if (tableFactory instanceof TableSourceFactory) { @@ -228,7 +232,7 @@ private Table convertCatalogTable(ObjectPath tablePath, CatalogTable table, @Nul return new TableSourceTable<>( tableSource, !((StreamTableSource) tableSource).isBounded(), - FlinkStatistic.UNKNOWN(), + FlinkStatistic.builder().tableStats(tableStats).build(), table ); } diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/catalog/CatalogStatisticsTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/catalog/CatalogStatisticsTest.java index c3d2317972f8..08e4cd027fd4 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/catalog/CatalogStatisticsTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/catalog/CatalogStatisticsTest.java @@ -20,12 +20,14 @@ import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.Table; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.catalog.CatalogTableImpl; import org.apache.flink.table.catalog.ConnectorCatalogTable; import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.catalog.exceptions.TablePartitionedException; import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataBase; import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataBoolean; @@ -35,17 +37,24 @@ import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataString; import org.apache.flink.table.catalog.stats.CatalogTableStatistics; import org.apache.flink.table.catalog.stats.Date; +import org.apache.flink.table.descriptors.DescriptorProperties; +import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery; +import org.apache.flink.table.planner.plan.stats.ValueInterval$; import org.apache.flink.table.planner.utils.TableTestUtil; import org.apache.flink.table.planner.utils.TestTableSource; import org.apache.flink.table.types.DataType; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.util.ImmutableBitSet; import org.junit.Test; +import java.util.Arrays; import java.util.HashMap; import java.util.Map; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; /** * Test for Catalog Statistics. @@ -59,42 +68,66 @@ public class CatalogStatisticsTest { ).build(); @Test - public void testGetStatsFromCatalog() throws Exception { + public void testGetStatsFromCatalogForConnectorCatalogTable() throws Exception { EnvironmentSettings settings = EnvironmentSettings.newInstance().useBlinkPlanner().inBatchMode().build(); TableEnvironment tEnv = TableEnvironment.create(settings); Catalog catalog = tEnv.getCatalog(tEnv.getCurrentCatalog()).orElse(null); assertNotNull(catalog); catalog.createTable( - ObjectPath.fromString("default_database.T1"), - ConnectorCatalogTable.source(new TestTableSource(true, tableSchema), true), - false); + ObjectPath.fromString("default_database.T1"), + ConnectorCatalogTable.source(new TestTableSource(true, tableSchema), true), + false); + + alterTableStatistics(catalog); + + assertStatistics(tEnv); + } + + @Test + public void testGetStatsFromCatalogForCatalogTableImpl() throws Exception { + EnvironmentSettings settings = EnvironmentSettings.newInstance().useBlinkPlanner().inBatchMode().build(); + TableEnvironment tEnv = TableEnvironment.create(settings); + Catalog catalog = tEnv.getCatalog(tEnv.getCurrentCatalog()).orElse(null); + assertNotNull(catalog); + + Map properties = new HashMap<>(); + properties.put("connector.type", "filesystem"); + properties.put("connector.property-version", "1"); + properties.put("connector.path", "/path/to/csv"); + + properties.put("format.type", "csv"); + properties.put("format.property-version", "1"); + properties.put("format.field-delimiter", ";"); + + // schema + DescriptorProperties descriptorProperties = new DescriptorProperties(true); + descriptorProperties.putTableSchema("format.fields", tableSchema); + properties.putAll(descriptorProperties.asMap()); + catalog.createTable( - ObjectPath.fromString("default_database.T2"), - ConnectorCatalogTable.source(new TestTableSource(true, tableSchema), true), - false); + ObjectPath.fromString("default_database.T1"), + new CatalogTableImpl(tableSchema, properties, ""), + false); + + alterTableStatistics(catalog); + + assertStatistics(tEnv); + } + private void alterTableStatistics(Catalog catalog) throws TableNotExistException, TablePartitionedException { catalog.alterTableStatistics(ObjectPath.fromString("default_database.T1"), new CatalogTableStatistics(100, 10, 1000L, 2000L), true); - catalog.alterTableStatistics(ObjectPath.fromString("default_database.T2"), - new CatalogTableStatistics(100000000, 1000, 1000000000L, 2000000000L), true); catalog.alterTableColumnStatistics(ObjectPath.fromString("default_database.T1"), createColumnStats(), true); - catalog.alterTableColumnStatistics(ObjectPath.fromString("default_database.T2"), createColumnStats(), true); - - Table table = tEnv.sqlQuery("select * from T1, T2 where T1.s3 = T2.s3"); - String result = tEnv.explain(table); - // T1 is broadcast side - String expected = TableTestUtil.readFromResource("/explain/testGetStatsFromCatalog.out"); - assertEquals(expected, TableTestUtil.replaceStageId(result)); } private CatalogColumnStatistics createColumnStats() { CatalogColumnStatisticsDataBoolean booleanColStats = new CatalogColumnStatisticsDataBoolean(55L, 45L, 5L); - CatalogColumnStatisticsDataLong longColStats = new CatalogColumnStatisticsDataLong(-123L, 763322L, 23L, 79L); + CatalogColumnStatisticsDataLong longColStats = new CatalogColumnStatisticsDataLong(-123L, 763322L, 23L, 77L); CatalogColumnStatisticsDataString stringColStats = new CatalogColumnStatisticsDataString(152L, 43.5D, 20L, 0L); CatalogColumnStatisticsDataDate dateColStats = - new CatalogColumnStatisticsDataDate(new Date(71L), new Date(17923L), 1321, 0L); + new CatalogColumnStatisticsDataDate(new Date(71L), new Date(17923L), 100, 0L); CatalogColumnStatisticsDataDouble doubleColStats = - new CatalogColumnStatisticsDataDouble(-123.35D, 7633.22D, 23L, 79L); + new CatalogColumnStatisticsDataDouble(-123.35D, 7633.22D, 73L, 27L); Map colStatsMap = new HashMap<>(6); colStatsMap.put("b1", booleanColStats); colStatsMap.put("l2", longColStats); @@ -104,4 +137,36 @@ private CatalogColumnStatistics createColumnStats() { return new CatalogColumnStatistics(colStatsMap); } + private void assertStatistics(TableEnvironment tEnv) { + RelNode t1 = TableTestUtil.toRelNode(tEnv.sqlQuery("select * from T1")); + FlinkRelMetadataQuery mq = FlinkRelMetadataQuery.reuseOrCreate(t1.getCluster().getMetadataQuery()); + assertEquals(100.0, mq.getRowCount(t1), 0.0); + assertEquals(Arrays.asList(1.0, 8.0, 43.5, 12.0, 8.0), mq.getAverageColumnSizes(t1)); + + // boolean type + assertEquals(2.0, mq.getDistinctRowCount(t1, ImmutableBitSet.of(0), null), 0.0); + assertEquals(5.0, mq.getColumnNullCount(t1, 0), 0.0); + assertNull(mq.getColumnInterval(t1, 0)); + + // long type + assertEquals(23.0, mq.getDistinctRowCount(t1, ImmutableBitSet.of(1), null), 0.0); + assertEquals(77.0, mq.getColumnNullCount(t1, 1), 0.0); + assertEquals(ValueInterval$.MODULE$.apply(-123L, 763322L, true, true), mq.getColumnInterval(t1, 1)); + + // string type + assertEquals(20.0, mq.getDistinctRowCount(t1, ImmutableBitSet.of(2), null), 0.0); + assertEquals(0.0, mq.getColumnNullCount(t1, 2), 0.0); + assertNull(mq.getColumnInterval(t1, 2)); + + // date type + assertEquals(100.0, mq.getDistinctRowCount(t1, ImmutableBitSet.of(3), null), 0.0); + assertEquals(0.0, mq.getColumnNullCount(t1, 3), 0.0); + assertNull(mq.getColumnInterval(t1, 3)); + + // double type + assertEquals(73.0, mq.getDistinctRowCount(t1, ImmutableBitSet.of(4), null), 0.0); + assertEquals(27.0, mq.getColumnNullCount(t1, 4), 0.0); + assertEquals(ValueInterval$.MODULE$.apply(-123.35, 7633.22, true, true), mq.getColumnInterval(t1, 4)); + } + } diff --git a/flink-table/flink-table-planner-blink/src/test/resources/explain/testGetStatsFromCatalog.out b/flink-table/flink-table-planner-blink/src/test/resources/explain/testGetStatsFromCatalog.out deleted file mode 100644 index 56d2ca86114a..000000000000 --- a/flink-table/flink-table-planner-blink/src/test/resources/explain/testGetStatsFromCatalog.out +++ /dev/null @@ -1,32 +0,0 @@ -== Abstract Syntax Tree == -LogicalProject(b1=[$0], l2=[$1], s3=[$2], d4=[$3], dd5=[$4], b10=[$5], l20=[$6], s30=[$7], d40=[$8], dd50=[$9]) -+- LogicalFilter(condition=[=($2, $7)]) - +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(b1, l2, s3, d4, dd5)]]]) - +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(b1, l2, s3, d4, dd5)]]]) - -== Optimized Logical Plan == -HashJoin(joinType=[InnerJoin], where=[=(s3, s30)], select=[b1, l2, s3, d4, dd5, b10, l20, s30, d40, dd50], isBroadcast=[true], build=[left]) -:- Exchange(distribution=[broadcast]) -: +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(b1, l2, s3, d4, dd5)]]], fields=[b1, l2, s3, d4, dd5]) -+- TableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(b1, l2, s3, d4, dd5)]]], fields=[b1, l2, s3, d4, dd5]) - -== Physical Execution Plan == - : Data Source - content : collect elements with CollectionInputFormat - - : Operator - content : SourceConversion(table=[default_catalog.default_database.T1, source: [TestTableSource(b1, l2, s3, d4, dd5)]], fields=[b1, l2, s3, d4, dd5]) - ship_strategy : FORWARD - - : Data Source - content : collect elements with CollectionInputFormat - - : Operator - content : SourceConversion(table=[default_catalog.default_database.T2, source: [TestTableSource(b1, l2, s3, d4, dd5)]], fields=[b1, l2, s3, d4, dd5]) - ship_strategy : FORWARD - - : Operator - content : HashJoin(joinType=[InnerJoin], where=[(s3 = s30)], select=[b1, l2, s3, d4, dd5, b10, l20, s30, d40, dd50], isBroadcast=[true], build=[left]) - ship_strategy : BROADCAST - From 5d8081f65095de092e08150adf5d301c74a38530 Mon Sep 17 00:00:00 2001 From: Maximilian Bode Date: Sun, 20 Oct 2019 15:26:28 +0200 Subject: [PATCH 553/746] [FLINK-14468][docs] Update Kubernetes docs - use Deployment from apps/v1 instead of extensions/v1beta1 - run as unprivileged user --- docs/ops/deployment/kubernetes.md | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/docs/ops/deployment/kubernetes.md b/docs/ops/deployment/kubernetes.md index 25ffcd3a0237..0099eb5c38ac 100644 --- a/docs/ops/deployment/kubernetes.md +++ b/docs/ops/deployment/kubernetes.md @@ -145,12 +145,16 @@ data: `jobmanager-deployment.yaml` {% highlight yaml %} -apiVersion: extensions/v1beta1 +apiVersion: apps/v1 kind: Deployment metadata: name: flink-jobmanager spec: replicas: 1 + selector: + matchLabels: + app: flink + component: jobmanager template: metadata: labels: @@ -183,6 +187,8 @@ spec: volumeMounts: - name: flink-config-volume mountPath: /opt/flink/conf + securityContext: + runAsUser: 9999 # refers to user _flink_ from official flink image, change if necessary volumes: - name: flink-config-volume configMap: @@ -196,12 +202,16 @@ spec: `taskmanager-deployment.yaml` {% highlight yaml %} -apiVersion: extensions/v1beta1 +apiVersion: apps/v1 kind: Deployment metadata: name: flink-taskmanager spec: replicas: 2 + selector: + matchLabels: + app: flink + component: taskmanager template: metadata: labels: @@ -230,6 +240,8 @@ spec: volumeMounts: - name: flink-config-volume mountPath: /opt/flink/conf/ + securityContext: + runAsUser: 9999 # refers to user _flink_ from official flink image, change if necessary volumes: - name: flink-config-volume configMap: From 056e1fc39abae9f55910749e581aec966c1ef65c Mon Sep 17 00:00:00 2001 From: Jingsong Lee Date: Tue, 12 Nov 2019 20:34:50 +0800 Subject: [PATCH 554/746] [FLINK-14714][table-runtime-blink] BytesHashMap should not log the whole stack when start spilling (#10156) --- .../operators/aggregate/BytesHashMap.java | 46 +++++++++++-------- 1 file changed, 26 insertions(+), 20 deletions(-) diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/BytesHashMap.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/BytesHashMap.java index f10a896c6f17..e06ae7168603 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/BytesHashMap.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/BytesHashMap.java @@ -18,6 +18,7 @@ package org.apache.flink.table.runtime.operators.aggregate; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.core.memory.MemorySegmentFactory; import org.apache.flink.core.memory.MemorySegmentSource; @@ -238,7 +239,8 @@ private int calcNumBucketSegments(LogicalType[] keyTypes, LogicalType[] valueTyp * @return true when BytesHashMap's valueTypeInfos.length == 0. * Any appended value will be ignored and replaced with a reusedValue as a present tag. */ - public boolean isHashSetMode() { + @VisibleForTesting + boolean isHashSetMode() { return hashSetMode; } @@ -333,7 +335,6 @@ public BinaryRow append(LookupInfo info, BinaryRow value) throws IOException { spillInBytes += recordArea.segments.size() * ((long) segmentSize); throw e; } - } public long getNumSpillFiles() { @@ -383,7 +384,7 @@ private void resetBucketSegments(List resetBucketSegs) { private void growAndRehash() throws EOFException { // allocate the new data structures int required = 2 * bucketSegments.size(); - if (required * numBucketsPerSegment > Integer.MAX_VALUE) { + if (required * (long) numBucketsPerSegment > Integer.MAX_VALUE) { LOG.warn("We can't handle more than Integer.MAX_VALUE buckets (eg. because hash functions return int)"); throw new EOFException(); } @@ -400,14 +401,10 @@ private void growAndRehash() throws EOFException { newBucketSegments.add(freeMemorySegments.remove(freeMemorySegments.size() - 1)); } - int numBuckets = newBucketSegments.size() * numBucketsPerSegment; - this.log2NumBuckets = MathUtils.log2strict(numBuckets); - this.numBucketsMask = (1 << MathUtils.log2strict(numBuckets)) - 1; - this.numBucketsMask2 = (1 << MathUtils.log2strict(numBuckets >> 1)) - 1; - this.growthThreshold = (int) (numBuckets * LOAD_FACTOR); + setBucketVariables(newBucketSegments); } catch (MemoryAllocationException e) { LOG.warn("BytesHashMap can't allocate {} pages, and now used {} pages", - required, reservedNumBuffers, e); + required, reservedNumBuffers); throw new EOFException(); } long reHashStartTime = System.currentTimeMillis(); @@ -428,7 +425,9 @@ private void growAndRehash() throws EOFException { hashCode2 = calcSecondHashCode(hashCode1); } newPos = (int) ((hashCode1 + step * hashCode2) & numBucketsMask); + // which segment contains the bucket bucketSegmentIndex = newPos >>> numBucketsPerSegmentBits; + // offset of the bucket in the segment bucketOffset = (newPos & numBucketsPerSegmentMask) << BUCKET_SIZE_BITS; step += STEP_INCREMENT; } @@ -442,12 +441,21 @@ private void growAndRehash() throws EOFException { this.bucketSegments = newBucketSegments; } + private void setBucketVariables(List bucketSegments) { + int numBuckets = bucketSegments.size() * numBucketsPerSegment; + this.log2NumBuckets = MathUtils.log2strict(numBuckets); + this.numBucketsMask = (1 << MathUtils.log2strict(numBuckets)) - 1; + this.numBucketsMask2 = (1 << MathUtils.log2strict(numBuckets >> 1)) - 1; + this.growthThreshold = (int) (numBuckets * LOAD_FACTOR); + } + /** * Returns a destructive iterator for iterating over the entries of this map. It frees each page * as it moves onto next one. Notice: it is illegal to call any method on the map after * `destructiveIterator()` has been called. * @return an entry iterator for iterating the value appended in the hash map. */ + @SuppressWarnings("WeakerAccess") public MutableObjectIterator getEntryIterator() { if (destructiveIterator != null) { throw new IllegalArgumentException("DestructiveIterator is not null, so this method can't be invoke!"); @@ -458,10 +466,12 @@ public MutableObjectIterator getEntryIterator() { /** * @return the underlying memory segments of the hash map's record area */ + @SuppressWarnings("WeakerAccess") public ArrayList getRecordAreaMemorySegments() { return recordArea.segments; } + @SuppressWarnings("WeakerAccess") public List getBucketAreaMemorySegments() { return bucketSegments; } @@ -487,16 +497,11 @@ public void free(boolean reservedRecordMemory) { destructiveIterator = null; } - /** * reset the map's record and bucket area's memory segments for reusing. */ public void reset() { - int numBuckets = bucketSegments.size() * numBucketsPerSegment; - this.log2NumBuckets = MathUtils.log2strict(numBuckets); - this.numBucketsMask = (1 << MathUtils.log2strict(numBuckets)) - 1; - this.numBucketsMask2 = (1 << MathUtils.log2strict(numBuckets >> 1)) - 1; - this.growthThreshold = (int) (numBuckets * LOAD_FACTOR); + setBucketVariables(bucketSegments); //reset the record segments. recordArea.reset(); resetBucketSegments(bucketSegments); @@ -586,16 +591,17 @@ BinaryRow readValue(BinaryRow reuse) throws IOException { // ----------------------- Iterator ----------------------- - MutableObjectIterator destructiveEntryIterator() { + private MutableObjectIterator destructiveEntryIterator() { return new RecordArea.DestructiveEntryIterator(); } - final class DestructiveEntryIterator extends AbstractPagedInputView implements MutableObjectIterator { + private final class DestructiveEntryIterator extends AbstractPagedInputView + implements MutableObjectIterator { private int count = 0; private int currentSegmentIndex = 0; - public DestructiveEntryIterator() { + private DestructiveEntryIterator() { super(segments.get(0), segmentSize, 0); destructiveIterator = this; } @@ -618,7 +624,7 @@ public Entry next(Entry reuse) throws IOException { } @Override - public Entry next() throws IOException { + public Entry next() { throw new UnsupportedOperationException(""); } @@ -628,7 +634,7 @@ protected int getLimitForSegment(MemorySegment segment) { } @Override - protected MemorySegment nextSegment(MemorySegment current) throws EOFException, IOException { + protected MemorySegment nextSegment(MemorySegment current) { return segments.get(++currentSegmentIndex); } } From 7128fdcf5084aa7d380b81d4b1af724658b2666e Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Mon, 11 Nov 2019 20:07:52 +0800 Subject: [PATCH 555/746] [FLINK-14693][python] Fix Python tox checks failure on travis Previously, we use conda to install tox. However, tox 3.14.0 depends on both 0.19 and 0.23 of importlib_metadata at the same time and conda will try to install both these two versions and it will cause problems occasionlly. In this commit, use pip to install tox to avoid this problem. This closes #10148. --- flink-python/dev/lint-python.sh | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/flink-python/dev/lint-python.sh b/flink-python/dev/lint-python.sh index 6ff52296bef9..f6961ed66363 100755 --- a/flink-python/dev/lint-python.sh +++ b/flink-python/dev/lint-python.sh @@ -204,20 +204,21 @@ function install_py_env() { # In some situations,you need to run the script with "sudo". e.g. sudo ./lint-python.sh function install_tox() { if [ -f "$TOX_PATH" ]; then - $CONDA_PATH remove -p $CONDA_HOME tox -y -q 2>&1 >/dev/null + $PIP_PATH uninstall tox -y -q 2>&1 >/dev/null if [ $? -ne 0 ]; then - echo "conda remove tox failed \ + echo "pip uninstall tox failed \ please try to exec the script again.\ if failed many times, you can try to exec in the form of sudo ./lint-python.sh -f" exit 1 fi fi - # virtualenv 16.6.2 released in 2019-07-14 is incompatible with py27 and py34, - # force install an older version(16.0.0) to avoid this problem. - $CONDA_PATH install -p $CONDA_HOME -c conda-forge virtualenv=16.0.0 tox -y -q 2>&1 >/dev/null + # tox 3.14.0 depends on both 0.19 and 0.23 of importlib_metadata at the same time and + # conda will try to install both these two versions and it will cause problems occasionally. + # Using pip as the package manager could avoid this problem. + $PIP_PATH install -q virtualenv==16.0.0 tox==3.14.0 2>&1 >/dev/null if [ $? -ne 0 ]; then - echo "conda install tox failed \ + echo "pip install tox failed \ please try to exec the script again.\ if failed many times, you can try to exec in the form of sudo ./lint-python.sh -f" exit 1 @@ -537,6 +538,9 @@ CONDA_HOME=$CURRENT_DIR/.conda # conda path CONDA_PATH=$CONDA_HOME/bin/conda +# pip path +PIP_PATH=$CONDA_HOME/bin/pip + # tox path TOX_PATH=$CONDA_HOME/bin/tox From b872f198ffa6eb8cd57af9cd9c579e8ac3c6b19d Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Tue, 8 Oct 2019 15:03:46 +0800 Subject: [PATCH 556/746] [hotfix] Introduce constants MemorySize#ZERO and MemorySize#MAX_VALUE. --- .../java/org/apache/flink/configuration/MemorySize.java | 4 ++++ .../org/apache/flink/configuration/MemorySizeTest.java | 2 +- .../clusterframework/TaskExecutorResourceUtils.java | 4 ++-- .../clusterframework/TaskExecutorResourceUtilsTest.java | 8 ++++---- 4 files changed, 11 insertions(+), 7 deletions(-) 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 5d9e22d2a854..7ca2e8b8b5ea 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 @@ -46,6 +46,10 @@ public class MemorySize implements java.io.Serializable { private static final long serialVersionUID = 1L; + public static final MemorySize ZERO = new MemorySize(0L); + + public static final MemorySize MAX_VALUE = new MemorySize(Long.MAX_VALUE); + // ------------------------------------------------------------------------ /** The memory size, in bytes. */ diff --git a/flink-core/src/test/java/org/apache/flink/configuration/MemorySizeTest.java b/flink-core/src/test/java/org/apache/flink/configuration/MemorySizeTest.java index 9cc1fe313754..676198ad7d16 100644 --- a/flink-core/src/test/java/org/apache/flink/configuration/MemorySizeTest.java +++ b/flink-core/src/test/java/org/apache/flink/configuration/MemorySizeTest.java @@ -36,7 +36,7 @@ public class MemorySizeTest { @Test public void testUnitConversion() { - final MemorySize zero = new MemorySize(0); + final MemorySize zero = MemorySize.ZERO; assertEquals(0, zero.getBytes()); assertEquals(0, zero.getKibiBytes()); assertEquals(0, zero.getMebiBytes()); 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 ee13943a78a1..082d4b7d7356 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 @@ -346,8 +346,8 @@ private static MemorySize getManagedMemorySize(final Configuration config) { } private static RangeFraction getManagedMemoryRangeFraction(final Configuration config) { - final MemorySize minSize = new MemorySize(0); - final MemorySize maxSize = new MemorySize(Long.MAX_VALUE); + final MemorySize minSize = MemorySize.ZERO; + final MemorySize maxSize = MemorySize.MAX_VALUE; final double fraction = config.getFloat(TaskManagerOptions.MANAGED_MEMORY_FRACTION); if (fraction >= 1 || fraction < 0) { throw new IllegalConfigurationException("Configured Managed Memory fraction (" 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 8577d8f1c834..29abfc2687a5 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 @@ -170,7 +170,7 @@ public void testConfigShuffleMemoryRangeFailure() { @Test public void testConfigShuffleMemoryFraction() { - final MemorySize shuffleMin = MemorySize.parse("0m"); + final MemorySize shuffleMin = MemorySize.ZERO; final MemorySize shuffleMax = MemorySize.parse("1t"); final float fraction = 0.2f; @@ -364,7 +364,7 @@ public void testConfigOffHeapManagedMemoryLegacyOffHeap() { conf.setBoolean(legacyOption, true); validateInAllConfigurations(conf, taskExecutorResourceSpec -> { assertThat(taskExecutorResourceSpec.getOffHeapManagedMemorySize(), is(taskExecutorResourceSpec.getManagedMemorySize())); - assertThat(taskExecutorResourceSpec.getOnHeapManagedMemorySize(), is(new MemorySize(0L))); + assertThat(taskExecutorResourceSpec.getOnHeapManagedMemorySize(), is(MemorySize.ZERO)); }); } @@ -381,7 +381,7 @@ public void testConfigOffHeapManagedMemoryLegacyOnHeap() { conf.setBoolean(legacyOption, false); validateInAllConfigurations(conf, taskExecutorResourceSpec -> { assertThat(taskExecutorResourceSpec.getOnHeapManagedMemorySize(), is(taskExecutorResourceSpec.getManagedMemorySize())); - assertThat(taskExecutorResourceSpec.getOffHeapManagedMemorySize(), is(new MemorySize(0L))); + assertThat(taskExecutorResourceSpec.getOffHeapManagedMemorySize(), is(MemorySize.ZERO)); }); } @@ -425,7 +425,7 @@ public void testConfigJvmOverheadRangeFailure() { @Test public void testConfigJvmOverheadFraction() { - final MemorySize minSize = MemorySize.parse("0m"); + final MemorySize minSize = MemorySize.ZERO; final MemorySize maxSize = MemorySize.parse("1t"); final float fraction = 0.2f; From 95fe8869ec8410084a24a3900829d20b25847ce8 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Tue, 8 Oct 2019 18:03:10 +0800 Subject: [PATCH 557/746] [hotfix] Move SimpleSlotContext to test scope. This class is for the legacy code, and there's no usage of this class in production codes. --- .../java/org/apache/flink/runtime/instance/SimpleSlotContext.java | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename flink-runtime/src/{main => test}/java/org/apache/flink/runtime/instance/SimpleSlotContext.java (100%) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlotContext.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotContext.java similarity index 100% rename from flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlotContext.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotContext.java From 740adc1598baec2bccada6862ed3ce240c896606 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Wed, 9 Oct 2019 10:55:44 +0800 Subject: [PATCH 558/746] [hotfix] Fix misusage of UNKNOWN and ANY ResourceProfiles - UNKNOWN should be used for describing: - A resource requirement (SlotRequest, SlotProfile, ExecutionJobVertex, etc.) that the exact amount of resource needed is not specified. - Remaining resource of a multi task slot (SlotInfoAndResources) that contains tasks with unknown resource profiles. - UNKNOWN should not be used for describing total resource of a slot / task executor (TaskSlot, SlotOffer, SlotReport, etc.), which should always be specific with FLIP-49. - ANY should be used for describing a set of resource (SlotOffer, SlotReport, TaskSlot, etc.) that is large enough to match any request. It's for testability purpose only. With FLIP-49, task executors / slots should always have specific resource profiles in production. This commit fix misusages of the two constants. --- .../types/ResourceProfile.java | 27 +++++++++++-- .../utils/SimpleSlotProvider.java | 4 +- .../runtime/instance/SimpleSlotContext.java | 2 +- .../runtime/jobmaster/JobMasterTest.java | 6 +-- .../slotpool/AllocatedSlotsTest.java | 2 +- .../slotpool/SingleLogicalSlotTest.java | 2 +- .../slotpool/SlotPoolCoLocationTest.java | 4 +- .../jobmaster/slotpool/SlotPoolImplTest.java | 16 ++++---- .../SlotPoolRequestCompletionTest.java | 2 +- .../slotpool/SlotPoolSlotSharingTest.java | 8 ++-- .../ResourceManagerTaskExecutorTest.java | 4 +- .../slotmanager/SlotManagerImplTest.java | 20 +++++----- .../TestingResourceActionsBuilder.java | 4 +- .../TaskExecutorPartitionLifecycleTest.java | 2 +- .../taskexecutor/TaskExecutorTest.java | 38 +++++++++---------- .../TaskSubmissionTestEnvironment.java | 2 +- .../taskexecutor/slot/TaskSlotTableTest.java | 2 +- 17 files changed, 84 insertions(+), 61 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java index c373fbda931e..edd588db1ab4 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.clusterframework.types; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.operators.ResourceSpec; import org.apache.flink.api.common.resources.Resource; import org.apache.flink.util.Preconditions; @@ -51,10 +52,18 @@ public class ResourceProfile implements Serializable, Comparable getExtendedResources() { */ public boolean isMatching(ResourceProfile required) { - if (required == UNKNOWN) { + if (this.equals(ANY)) { + return true; + } + + if (this.equals(required)) { + return true; + } + + if (this.equals(UNKNOWN)) { + return false; + } + + if (required.equals(UNKNOWN)) { return true; } 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 e6257016dc9e..09dc90e6de55 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 @@ -77,7 +77,7 @@ public SimpleSlotProvider(JobID jobId, int numSlots, TaskManagerGateway taskMana new TaskManagerLocation(ResourceID.generate(), InetAddress.getLoopbackAddress(), 10000 + i), 0, taskManagerGateway, - ResourceProfile.UNKNOWN); + ResourceProfile.ANY); slots.add(as); } @@ -137,7 +137,7 @@ public void returnLogicalSlot(LogicalSlot logicalSlot) { logicalSlot.getTaskManagerLocation(), logicalSlot.getPhysicalSlotNumber(), logicalSlot.getTaskManagerGateway(), - ResourceProfile.UNKNOWN); + ResourceProfile.ANY); slots.add(as); allocatedSlots.remove(logicalSlot.getSlotRequestId()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotContext.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotContext.java index 60719e72ce02..5a1887db2f1e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotContext.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SimpleSlotContext.java @@ -45,7 +45,7 @@ public SimpleSlotContext( TaskManagerLocation taskManagerLocation, int physicalSlotNumber, TaskManagerGateway taskManagerGateway) { - this(allocationId, taskManagerLocation, physicalSlotNumber, taskManagerGateway, ResourceProfile.UNKNOWN); + this(allocationId, taskManagerLocation, physicalSlotNumber, taskManagerGateway, ResourceProfile.ANY); } public SimpleSlotContext( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java index 601e42e8e4c1..39685f2a3b8e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java @@ -459,7 +459,7 @@ public void testAllocatedSlotReportDoesNotContainStaleInformation() throws Excep // wait for the completion of the registration registrationResponse.get(); - final SlotOffer slotOffer = new SlotOffer(new AllocationID(), 0, ResourceProfile.UNKNOWN); + final SlotOffer slotOffer = new SlotOffer(new AllocationID(), 0, ResourceProfile.ANY); final CompletableFuture> slotOfferFuture = jobMasterGateway.offerSlots(taskManagerLocation.getResourceID(), Collections.singleton(slotOffer), testingTimeout); @@ -888,7 +888,7 @@ public void testSlotRequestTimeoutWhenNoSlotOffering() throws Exception { assertThat(submittedTaskFuture.isDone(), is(false)); - final SlotOffer slotOffer = new SlotOffer(slotRequest.getAllocationId(), 0, ResourceProfile.UNKNOWN); + final SlotOffer slotOffer = new SlotOffer(slotRequest.getAllocationId(), 0, ResourceProfile.ANY); final CompletableFuture> acceptedSlotsFuture = jobMasterGateway.offerSlots(taskManagerLocation.getResourceID(), Collections.singleton(slotOffer), testingTimeout); @@ -1960,7 +1960,7 @@ private Collection registerSlotsAtJobMaster( .mapToObj( index -> { final AllocationID allocationId = allocationIdsResourceManagerGateway.takeAllocationId(); - return new SlotOffer(allocationId, index, ResourceProfile.UNKNOWN); + return new SlotOffer(allocationId, index, ResourceProfile.ANY); }) .collect(Collectors.toList()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/AllocatedSlotsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/AllocatedSlotsTest.java index 3a6e0ed7a282..aebf02fa7627 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/AllocatedSlotsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/AllocatedSlotsTest.java @@ -142,7 +142,7 @@ private AllocatedSlot createSlot(final AllocationID allocationId, final TaskMana allocationId, taskManagerLocation, 0, - ResourceProfile.UNKNOWN, + ResourceProfile.ANY, new SimpleAckingTaskManagerGateway()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SingleLogicalSlotTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SingleLogicalSlotTest.java index 4570269b003c..b751b65cf941 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SingleLogicalSlotTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SingleLogicalSlotTest.java @@ -86,7 +86,7 @@ private static SlotContext createSlotContext() { new LocalTaskManagerLocation(), 0, new SimpleAckingTaskManagerGateway(), - ResourceProfile.UNKNOWN); + ResourceProfile.ANY); } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolCoLocationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolCoLocationTest.java index 9e821020dfe6..125648647eda 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolCoLocationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolCoLocationTest.java @@ -134,7 +134,7 @@ public void testSimpleCoLocatedSlotScheduling() throws ExecutionException, Inter Collections.singletonList(new SlotOffer( allocationId1, 0, - ResourceProfile.UNKNOWN))); + ResourceProfile.ANY))); Collection slotOfferFuture2 = slotPoolGateway.offerSlots( taskManagerLocation, @@ -142,7 +142,7 @@ public void testSimpleCoLocatedSlotScheduling() throws ExecutionException, Inter Collections.singletonList(new SlotOffer( allocationId2, 0, - ResourceProfile.UNKNOWN))); + ResourceProfile.ANY))); assertFalse(slotOfferFuture1.isEmpty()); assertFalse(slotOfferFuture2.isEmpty()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolImplTest.java index d41b76b2c56a..cb001d23e603 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolImplTest.java @@ -441,7 +441,7 @@ public void testFulfillingSlotRequestsWithUnusedOfferedSlots() throws Exception assertEquals(allocationId1, canceledAllocations.take()); - final SlotOffer slotOffer = new SlotOffer(allocationId1, 0, ResourceProfile.UNKNOWN); + final SlotOffer slotOffer = new SlotOffer(allocationId1, 0, ResourceProfile.ANY); slotPool.registerTaskManager(taskManagerLocation.getResourceID()); @@ -475,7 +475,7 @@ public void testShutdownReleasesAllSlots() throws Exception { new SlotOffer( new AllocationID(), i, - ResourceProfile.UNKNOWN)); + ResourceProfile.ANY)); } final ArrayBlockingQueue freedSlotQueue = new ArrayBlockingQueue<>(numSlotOffers); @@ -528,8 +528,8 @@ public void testCheckIdleSlot() throws Exception { final AllocationID expiredSlotID = new AllocationID(); final AllocationID freshSlotID = new AllocationID(); - final SlotOffer slotToExpire = new SlotOffer(expiredSlotID, 0, ResourceProfile.UNKNOWN); - final SlotOffer slotToNotExpire = new SlotOffer(freshSlotID, 1, ResourceProfile.UNKNOWN); + final SlotOffer slotToExpire = new SlotOffer(expiredSlotID, 0, ResourceProfile.ANY); + final SlotOffer slotToNotExpire = new SlotOffer(freshSlotID, 1, ResourceProfile.ANY); assertThat(slotPool.registerTaskManager(taskManagerLocation.getResourceID()), Matchers.is(true)); @@ -577,7 +577,7 @@ public void testDiscardIdleSlotIfReleasingFailed() throws Exception { Scheduler scheduler = setupScheduler(slotPool, mainThreadExecutor); final AllocationID expiredAllocationId = new AllocationID(); - final SlotOffer slotToExpire = new SlotOffer(expiredAllocationId, 0, ResourceProfile.UNKNOWN); + final SlotOffer slotToExpire = new SlotOffer(expiredAllocationId, 0, ResourceProfile.ANY); OneShotLatch freeSlotLatch = new OneShotLatch(); taskManagerGateway.setFreeSlotFunction((AllocationID allocationId, Throwable cause) -> { @@ -633,7 +633,7 @@ public void testFreeFailedSlots() throws Exception { final List slotOffers = new ArrayList<>(parallelism); for (int i = 0; i < parallelism; i++) { - slotOffers.add(new SlotOffer(allocationIds.take(), i, ResourceProfile.UNKNOWN)); + slotOffers.add(new SlotOffer(allocationIds.take(), i, ResourceProfile.ANY)); } slotPool.registerTaskManager(taskManagerLocation.getResourceID()); @@ -691,11 +691,11 @@ public void testCreateAllocatedSlotReport() throws Exception { final List slotOffers = new ArrayList<>(2); final AllocationID allocatedId = allocationIds.take(); - slotOffers.add(new SlotOffer(allocatedId, 0, ResourceProfile.UNKNOWN)); + slotOffers.add(new SlotOffer(allocatedId, 0, ResourceProfile.ANY)); allocatedSlotInfos.add(new AllocatedSlotInfo(0, allocatedId)); final AllocationID availableId = new AllocationID(); - slotOffers.add(new SlotOffer(availableId, 1, ResourceProfile.UNKNOWN)); + slotOffers.add(new SlotOffer(availableId, 1, ResourceProfile.ANY)); allocatedSlotInfos.add(new AllocatedSlotInfo(1, availableId)); slotPool.registerTaskManager(taskManagerLocation.getResourceID()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolRequestCompletionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolRequestCompletionTest.java index a1765db182ba..604572b513a8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolRequestCompletionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolRequestCompletionTest.java @@ -93,7 +93,7 @@ private void runSlotRequestCompletionTest( final LocalTaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); slotPool.registerTaskManager(taskManagerLocation.getResourceID()); - final SlotOffer slotOffer = new SlotOffer(new AllocationID(), 0, ResourceProfile.UNKNOWN); + final SlotOffer slotOffer = new SlotOffer(new AllocationID(), 0, ResourceProfile.ANY); final Collection acceptedSlots = slotPool.offerSlots(taskManagerLocation, new SimpleAckingTaskManagerGateway(), Collections.singleton(slotOffer)); assertThat(acceptedSlots, containsInAnyOrder(slotOffer)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolSlotSharingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolSlotSharingTest.java index ef1fe31cbd63..793c79bec8c7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolSlotSharingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolSlotSharingTest.java @@ -89,7 +89,7 @@ public void testSingleQueuedSharedSlotScheduling() throws Exception { new SlotOffer( allocationId, 0, - ResourceProfile.UNKNOWN)); + ResourceProfile.ANY)); assertTrue(booleanCompletableFuture); @@ -200,7 +200,7 @@ public void testQueuedSharedSlotScheduling() throws Exception { new SlotOffer( allocationId1, 0, - ResourceProfile.UNKNOWN)); + ResourceProfile.ANY)); assertTrue(offerFuture); @@ -296,7 +296,7 @@ public void testQueuedMultipleSlotSharingGroups() throws Exception { new SlotOffer( allocationId1, 0, - ResourceProfile.UNKNOWN)); + ResourceProfile.ANY)); boolean offerFuture2 = slotPool.offerSlot( taskManagerLocation, @@ -304,7 +304,7 @@ public void testQueuedMultipleSlotSharingGroups() throws Exception { new SlotOffer( allocationId2, 0, - ResourceProfile.UNKNOWN)); + ResourceProfile.ANY)); assertTrue(offerFuture1); assertTrue(offerFuture2); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java index 394f63d6dad5..c2555e27e2af 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java @@ -254,7 +254,7 @@ public void testDelayedRegisterTaskExecutor() throws Exception { assertTrue(response instanceof TaskExecutorRegistrationSuccess); // on success, send slot report for taskmanager registration - final SlotReport slotReport = new SlotReport(new SlotStatus(new SlotID(taskExecutorResourceID, 0), ResourceProfile.UNKNOWN)); + final SlotReport slotReport = new SlotReport(new SlotStatus(new SlotID(taskExecutorResourceID, 0), ResourceProfile.ANY)); rmGateway.sendSlotReport(taskExecutorResourceID, ((TaskExecutorRegistrationSuccess) response).getRegistrationId(), slotReport, TIMEOUT).get(); @@ -301,7 +301,7 @@ public void testDisconnectTaskExecutor() throws Exception { private Collection createSlots(int numberSlots) { return IntStream.range(0, numberSlots) .mapToObj(index -> - new SlotStatus(new SlotID(taskExecutorResourceID, index), ResourceProfile.UNKNOWN)) + new SlotStatus(new SlotID(taskExecutorResourceID, index), ResourceProfile.ANY)) .collect(Collectors.toList()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImplTest.java index ffcd98404050..4b568fa3eba6 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerImplTest.java @@ -1012,7 +1012,7 @@ public void testReportAllocatedSlot() throws Exception { final SlotID slotId = new SlotID(taskManagerId, 0); final SlotStatus initialSlotStatus = new SlotStatus( slotId, - ResourceProfile.UNKNOWN); + ResourceProfile.ANY); final SlotReport initialSlotReport = new SlotReport(initialSlotStatus); slotManager.registerTaskManager(taskExecutorConnection, initialSlotReport); @@ -1022,7 +1022,7 @@ public void testReportAllocatedSlot() throws Exception { // Now report this slot as allocated final SlotStatus slotStatus = new SlotStatus( slotId, - ResourceProfile.UNKNOWN, + ResourceProfile.ANY, new JobID(), new AllocationID()); final SlotReport slotReport = new SlotReport( @@ -1075,7 +1075,7 @@ public void testSlotRequestFailure() throws Exception { final ResourceID taskExecutorResourceId = ResourceID.generate(); final TaskExecutorConnection taskExecutionConnection = new TaskExecutorConnection(taskExecutorResourceId, testingTaskExecutorGateway); - final SlotReport slotReport = new SlotReport(createEmptySlotStatus(new SlotID(taskExecutorResourceId, 0), ResourceProfile.UNKNOWN)); + final SlotReport slotReport = new SlotReport(createEmptySlotStatus(new SlotID(taskExecutorResourceId, 0), ResourceProfile.ANY)); final CompletableFuture firstManualSlotRequestResponse = new CompletableFuture<>(); responseQueue.offer(firstManualSlotRequestResponse); @@ -1131,7 +1131,7 @@ public void testSlotRequestRemovedIfTMReportAllocation() throws Exception { final ResourceID taskExecutorResourceId = ResourceID.generate(); final TaskExecutorConnection taskExecutionConnection = new TaskExecutorConnection(taskExecutorResourceId, testingTaskExecutorGateway); - final SlotReport slotReport = new SlotReport(createEmptySlotStatus(new SlotID(taskExecutorResourceId, 0), ResourceProfile.UNKNOWN)); + final SlotReport slotReport = new SlotReport(createEmptySlotStatus(new SlotID(taskExecutorResourceId, 0), ResourceProfile.ANY)); final CompletableFuture firstManualSlotRequestResponse = new CompletableFuture<>(); responseQueue.offer(firstManualSlotRequestResponse); @@ -1261,7 +1261,7 @@ private Set extractFailedAllocationsForJob(JobID jobId2, Map numberSlots)) .build(); + final ResourceProfile resourceProfile = new ResourceProfile(1.0, 100); try (final SlotManagerImpl slotManager = createSlotManager(ResourceManagerId.generate(), resourceActions)) { final JobID jobId = new JobID(); - assertThat(slotManager.registerSlotRequest(createSlotRequest(jobId)), is(true)); + assertThat(slotManager.registerSlotRequest(createSlotRequest(jobId, resourceProfile)), is(true)); assertThat(slotManager.getNumberPendingTaskManagerSlots(), is(numberSlots)); assertThat(slotManager.getNumberAssignedPendingTaskManagerSlots(), is(1)); assertThat(slotManager.getNumberRegisteredSlots(), is(0)); final TaskExecutorConnection taskExecutorConnection = createTaskExecutorConnection(); - final SlotReport slotReport = createSlotReport(taskExecutorConnection.getResourceID(), numberSlots - 1); + final SlotReport slotReport = + createSlotReport(taskExecutorConnection.getResourceID(), numberSlots - 1, resourceProfile, SlotManagerImplTest::createEmptySlotStatus); slotManager.registerTaskManager(taskExecutorConnection, slotReport); @@ -1449,7 +1451,7 @@ public void testOnlyFreeSlotsCanFulfillPendingTaskManagerSlot() throws Exception final TaskExecutorConnection taskExecutorConnection = createTaskExecutorConnection(); final SlotID slotId = new SlotID(taskExecutorConnection.getResourceID(), 0); - final SlotStatus slotStatus = new SlotStatus(slotId, ResourceProfile.UNKNOWN, jobId, new AllocationID()); + final SlotStatus slotStatus = new SlotStatus(slotId, ResourceProfile.ANY, jobId, new AllocationID()); final SlotReport slotReport = new SlotReport(slotStatus); slotManager.registerTaskManager(taskExecutorConnection, slotReport); @@ -1499,7 +1501,7 @@ private SlotReport createSingleAllocatedSlotReport(ResourceID resourceID, JobID return createSlotReport( resourceID, 1, - ResourceProfile.UNKNOWN, + ResourceProfile.ANY, (slotId, resourceProfile) -> new SlotStatus(slotId, resourceProfile, jobId, new AllocationID())); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/TestingResourceActionsBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/TestingResourceActionsBuilder.java index ac7afd4283ef..7f0866cba66f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/TestingResourceActionsBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/TestingResourceActionsBuilder.java @@ -36,7 +36,7 @@ */ public class TestingResourceActionsBuilder { private BiConsumer releaseResourceConsumer = (ignoredA, ignoredB) -> {}; - private FunctionWithException, ResourceManagerException> allocateResourceFunction = (ignored) -> Collections.singleton(ResourceProfile.UNKNOWN); + private FunctionWithException, ResourceManagerException> allocateResourceFunction = (ignored) -> Collections.singleton(ResourceProfile.ANY); private Consumer> notifyAllocationFailureConsumer = (ignored) -> {}; public TestingResourceActionsBuilder setReleaseResourceConsumer(BiConsumer releaseResourceConsumer) { @@ -52,7 +52,7 @@ public TestingResourceActionsBuilder setAllocateResourceFunction(FunctionWithExc public TestingResourceActionsBuilder setAllocateResourceConsumer(Consumer allocateResourceConsumer) { this.allocateResourceFunction = (ResourceProfile resourceProfile) -> { allocateResourceConsumer.accept(resourceProfile); - return Collections.singleton(ResourceProfile.UNKNOWN); + return Collections.singleton(ResourceProfile.ANY); }; return this; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java index 6646fb0ceb55..0e92c86727ae 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java @@ -448,7 +448,7 @@ private TestingTaskExecutor createTestingTaskExecutor(TaskManagerServices taskMa private static TaskSlotTable createTaskSlotTable() { return new TaskSlotTable( - Collections.singletonList(ResourceProfile.UNKNOWN), + Collections.singletonList(ResourceProfile.ANY), new TimerService<>(TestingUtils.defaultExecutor(), timeout.toMilliseconds())); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index a68eb35b3555..3fb355175da1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -259,7 +259,7 @@ public void teardown() throws Exception { @Test public void testShouldShutDownTaskManagerServicesInPostStop() throws Exception { - final TaskSlotTable taskSlotTable = new TaskSlotTable(Collections.singleton(ResourceProfile.UNKNOWN), timerService); + final TaskSlotTable taskSlotTable = new TaskSlotTable(Collections.singleton(ResourceProfile.ANY), timerService); final JobLeaderService jobLeaderService = new JobLeaderService(taskManagerLocation, RetryingRegistrationConfiguration.defaultConfiguration()); @@ -307,7 +307,7 @@ public void testShouldShutDownTaskManagerServicesInPostStop() throws Exception { @Test public void testHeartbeatTimeoutWithJobManager() throws Exception { - final TaskSlotTable taskSlotTable = new TaskSlotTable(Arrays.asList(ResourceProfile.UNKNOWN), timerService); + final TaskSlotTable taskSlotTable = new TaskSlotTable(Arrays.asList(ResourceProfile.ANY), timerService); final JobLeaderService jobLeaderService = new JobLeaderService(taskManagerLocation, RetryingRegistrationConfiguration.defaultConfiguration()); @@ -549,7 +549,7 @@ public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception { rpc.registerGateway(resourceManagerAddress, testingResourceManagerGateway); - final TaskSlotTable taskSlotTable = new TaskSlotTable(Collections.singleton(ResourceProfile.UNKNOWN), timerService); + final TaskSlotTable taskSlotTable = new TaskSlotTable(Collections.singleton(ResourceProfile.ANY), timerService); final TaskExecutorLocalStateStoresManager localStateStoresManager = createTaskExecutorLocalStateStoresManager(); final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() .setTaskManagerLocation(taskManagerLocation) @@ -812,7 +812,7 @@ private ResourceManagerId createAndRegisterResourceManager( private TaskExecutor createTaskExecutorWithJobManagerTable(JobManagerTable jobManagerTable) throws IOException { final TaskExecutorLocalStateStoresManager localStateStoresManager = createTaskExecutorLocalStateStoresManager(); return createTaskExecutor(new TaskManagerServicesBuilder() - .setTaskSlotTable(new TaskSlotTable(Collections.singletonList(ResourceProfile.UNKNOWN), timerService)) + .setTaskSlotTable(new TaskSlotTable(Collections.singletonList(ResourceProfile.ANY), timerService)) .setJobManagerTable(jobManagerTable) .setTaskStateManager(localStateStoresManager) .build()); @@ -847,7 +847,7 @@ private JobManagerTable createJobManagerTableWithOneJob(JobMasterId jobMasterId) */ @Test public void testJobLeaderDetection() throws Exception { - final TaskSlotTable taskSlotTable = new TaskSlotTable(Collections.singleton(ResourceProfile.UNKNOWN), timerService); + final TaskSlotTable taskSlotTable = new TaskSlotTable(Collections.singleton(ResourceProfile.ANY), timerService); final JobManagerTable jobManagerTable = new JobManagerTable(); final JobLeaderService jobLeaderService = new JobLeaderService(taskManagerLocation, RetryingRegistrationConfiguration.defaultConfiguration()); @@ -956,7 +956,7 @@ public void testSlotAcceptance() throws Exception { final AllocationID allocationId1 = new AllocationID(); final AllocationID allocationId2 = new AllocationID(); - final SlotOffer offer1 = new SlotOffer(allocationId1, 0, ResourceProfile.UNKNOWN); + final SlotOffer offer1 = new SlotOffer(allocationId1, 0, ResourceProfile.ANY); final JobMasterGateway jobMasterGateway = mock(JobMasterGateway.class); @@ -1030,7 +1030,7 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { final AllocationID allocationId1 = new AllocationID(); final AllocationID allocationId2 = new AllocationID(); - final SlotOffer offer1 = new SlotOffer(allocationId1, 0, ResourceProfile.UNKNOWN); + final SlotOffer offer1 = new SlotOffer(allocationId1, 0, ResourceProfile.ANY); final OneShotLatch offerSlotsLatch = new OneShotLatch(); final OneShotLatch taskInTerminalState = new OneShotLatch(); @@ -1171,7 +1171,7 @@ public void testRMHeartbeatStopWhenLeadershipRevoked() throws Exception { final RecordingHeartbeatServices heartbeatServices = new RecordingHeartbeatServices(heartbeatInterval, heartbeatTimeout); final ResourceID rmResourceID = ResourceID.generate(); - final TaskSlotTable taskSlotTable = new TaskSlotTable(Collections.singleton(ResourceProfile.UNKNOWN), timerService); + final TaskSlotTable taskSlotTable = new TaskSlotTable(Collections.singleton(ResourceProfile.ANY), timerService); final String rmAddress = "rm"; final TestingResourceManagerGateway rmGateway = new TestingResourceManagerGateway( @@ -1224,7 +1224,7 @@ public void testRMHeartbeatStopWhenLeadershipRevoked() throws Exception { @Test public void testRemoveJobFromJobLeaderService() throws Exception { final TaskSlotTable taskSlotTable = new TaskSlotTable( - Collections.singleton(ResourceProfile.UNKNOWN), + Collections.singleton(ResourceProfile.ANY), timerService); final TaskExecutorLocalStateStoresManager localStateStoresManager = createTaskExecutorLocalStateStoresManager(); @@ -1316,7 +1316,7 @@ public void testMaximumRegistrationDuration() throws Exception { @Test public void testMaximumRegistrationDurationAfterConnectionLoss() throws Exception { configuration.setString(TaskManagerOptions.REGISTRATION_TIMEOUT, "100 ms"); - final TaskSlotTable taskSlotTable = new TaskSlotTable(Collections.singleton(ResourceProfile.UNKNOWN), timerService); + final TaskSlotTable taskSlotTable = new TaskSlotTable(Collections.singleton(ResourceProfile.ANY), timerService); final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder().setTaskSlotTable(taskSlotTable).build(); final TaskExecutor taskExecutor = createTaskExecutor(taskManagerServices, new HeartbeatServices(10L, 10L)); @@ -1366,7 +1366,7 @@ public void testMaximumRegistrationDurationAfterConnectionLoss() throws Exceptio */ @Test public void testIgnoringSlotRequestsIfNotRegistered() throws Exception { - final TaskSlotTable taskSlotTable = new TaskSlotTable(Collections.singleton(ResourceProfile.UNKNOWN), timerService); + final TaskSlotTable taskSlotTable = new TaskSlotTable(Collections.singleton(ResourceProfile.ANY), timerService); final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder().setTaskSlotTable(taskSlotTable).build(); final TaskExecutor taskExecutor = createTaskExecutor(taskManagerServices); @@ -1411,7 +1411,7 @@ public void testIgnoringSlotRequestsIfNotRegistered() throws Exception { */ @Test public void testReconnectionAttemptIfExplicitlyDisconnected() throws Exception { - final TaskSlotTable taskSlotTable = new TaskSlotTable(Collections.singleton(ResourceProfile.UNKNOWN), timerService); + final TaskSlotTable taskSlotTable = new TaskSlotTable(Collections.singleton(ResourceProfile.ANY), timerService); final TaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); final TaskExecutor taskExecutor = createTaskExecutor(new TaskManagerServicesBuilder() .setTaskSlotTable(taskSlotTable) @@ -1459,7 +1459,7 @@ public void testReconnectionAttemptIfExplicitlyDisconnected() throws Exception { */ @Test public void testInitialSlotReport() throws Exception { - final TaskSlotTable taskSlotTable = new TaskSlotTable(Collections.singleton(ResourceProfile.UNKNOWN), timerService); + final TaskSlotTable taskSlotTable = new TaskSlotTable(Collections.singleton(ResourceProfile.ANY), timerService); final TaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() .setTaskSlotTable(taskSlotTable) @@ -1494,7 +1494,7 @@ public void testInitialSlotReport() throws Exception { */ @Test public void testInitialSlotReportFailure() throws Exception { - final TaskSlotTable taskSlotTable = new TaskSlotTable(Collections.singleton(ResourceProfile.UNKNOWN), timerService); + final TaskSlotTable taskSlotTable = new TaskSlotTable(Collections.singleton(ResourceProfile.ANY), timerService); final TaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation(); final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() .setTaskSlotTable(taskSlotTable) @@ -1552,7 +1552,7 @@ public CompletableFuture apply(Tuple4 slotReports; private TestingTaskSlotTable(Queue slotReports) { - super(Collections.singleton(ResourceProfile.UNKNOWN), new TimerService<>(TestingUtils.defaultExecutor(), 10000L)); + super(Collections.singleton(ResourceProfile.ANY), new TimerService<>(TestingUtils.defaultExecutor(), 10000L)); this.slotReports = slotReports; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java index de8a78a07daa..c1bf805132eb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java @@ -190,7 +190,7 @@ public TestingFatalErrorHandler getTestingFatalErrorHandler() { private TaskSlotTable generateTaskSlotTable(int numSlot) { Collection resourceProfiles = new ArrayList<>(); for (int i = 0; i < numSlot; i++) { - resourceProfiles.add(ResourceProfile.UNKNOWN); + resourceProfiles.add(ResourceProfile.ANY); } return new TaskSlotTable(resourceProfiles, timerService); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableTest.java index 46ba55c30c2a..17f46d34544b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableTest.java @@ -54,7 +54,7 @@ public class TaskSlotTableTest extends TestLogger { */ @Test public void testTryMarkSlotActive() throws SlotNotFoundException { - final TaskSlotTable taskSlotTable = createTaskSlotTable(Collections.nCopies(3, ResourceProfile.UNKNOWN)); + final TaskSlotTable taskSlotTable = createTaskSlotTable(Collections.nCopies(3, ResourceProfile.ANY)); try { taskSlotTable.start(new TestingSlotActionsBuilder().build()); From 4a5e8c083969cad215819ad6a3a0d2919e8c2d5e Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Thu, 17 Oct 2019 18:21:26 +0800 Subject: [PATCH 559/746] [hotfix] Deduplicate setting operator resource / parallelism argument checks. --- .../util/OperatorValidationUtils.java | 75 +++++++++++++++++++ .../apache/flink/api/dag/Transformation.java | 11 +-- .../flink/api/java/operators/DataSink.java | 13 +--- .../api/java/operators/DeltaIteration.java | 13 +--- .../flink/api/java/operators/Operator.java | 14 +--- .../api/datastream/DataStreamSink.java | 10 +-- .../api/datastream/DataStreamSource.java | 10 +-- .../SingleOutputStreamOperator.java | 23 ++---- 8 files changed, 102 insertions(+), 67 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/operators/util/OperatorValidationUtils.java diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/util/OperatorValidationUtils.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/util/OperatorValidationUtils.java new file mode 100644 index 000000000000..2e95c70003c9 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/util/OperatorValidationUtils.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.api.common.operators.util; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.operators.ResourceSpec; +import org.apache.flink.util.Preconditions; + +/** + * Utils for checking operators' resource and parallelism settings. + */ +public class OperatorValidationUtils { + + private OperatorValidationUtils() {} + + public static void validateParallelism(int parallelism) { + validateParallelism(parallelism, true); + } + + public static void validateParallelism(int parallelism, boolean canBeParallel) { + Preconditions.checkArgument(canBeParallel || parallelism == 1, + "The parallelism of non parallel operator must be 1."); + Preconditions.checkArgument(parallelism > 0 || parallelism == ExecutionConfig.PARALLELISM_DEFAULT, + "The parallelism of an operator must be at least 1, or ExecutionConfig.PARALLELISM_DEFAULT (use system default)."); + } + + public static void validateMaxParallelism(int maxParallelism) { + validateMaxParallelism(maxParallelism, Integer.MAX_VALUE, true); + } + + public static void validateMaxParallelism(int maxParallelism, int upperBound) { + validateMaxParallelism(maxParallelism, upperBound, true); + } + + public static void validateMaxParallelism(int maxParallelism, boolean canBeParallel) { + validateMaxParallelism(maxParallelism, Integer.MAX_VALUE, canBeParallel); + } + + public static void validateMaxParallelism(int maxParallelism, int upperBound, boolean canBeParallel) { + Preconditions.checkArgument(maxParallelism > 0, + "The maximum parallelism must be greater than 0."); + Preconditions.checkArgument(canBeParallel || maxParallelism == 1, + "The maximum parallelism of non parallel operator must be 1."); + Preconditions.checkArgument(maxParallelism > 0 && maxParallelism <= upperBound, + "Maximum parallelism must be between 1 and " + upperBound + ". Found: " + maxParallelism); + } + + public static void validateResources(ResourceSpec resources) { + Preconditions.checkNotNull(resources, "The resources must be not null."); + Preconditions.checkArgument(resources.isValid(), "The values in resources must be not less than 0."); + } + + public static void validateMinAndPreferredResources(ResourceSpec minResources, ResourceSpec preferredResources) { + Preconditions.checkNotNull(minResources, "The min resources must be not null."); + Preconditions.checkNotNull(preferredResources, "The preferred resources must be not null."); + Preconditions.checkArgument(minResources.isValid() && preferredResources.isValid() && minResources.lessThanOrEqual(preferredResources), + "The values in resources must be not less than 0 and the preferred resources must be greater than the min resources."); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/dag/Transformation.java b/flink-core/src/main/java/org/apache/flink/api/dag/Transformation.java index a4e239973f4f..39d34d721215 100644 --- a/flink-core/src/main/java/org/apache/flink/api/dag/Transformation.java +++ b/flink-core/src/main/java/org/apache/flink/api/dag/Transformation.java @@ -19,9 +19,9 @@ package org.apache.flink.api.dag; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.InvalidTypesException; import org.apache.flink.api.common.operators.ResourceSpec; +import org.apache.flink.api.common.operators.util.OperatorValidationUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.MissingTypeInfo; import org.apache.flink.util.Preconditions; @@ -202,9 +202,7 @@ public int getParallelism() { * @param parallelism The new parallelism to set on this {@code Transformation}. */ public void setParallelism(int parallelism) { - Preconditions.checkArgument( - parallelism > 0 || parallelism == ExecutionConfig.PARALLELISM_DEFAULT, - "The parallelism must be at least one, or ExecutionConfig.PARALLELISM_DEFAULT (use system default)."); + OperatorValidationUtils.validateParallelism(parallelism); this.parallelism = parallelism; } @@ -223,10 +221,7 @@ public int getMaxParallelism() { * @param maxParallelism Maximum parallelism for this stream transformation. */ public void setMaxParallelism(int maxParallelism) { - Preconditions.checkArgument(maxParallelism > 0 - && maxParallelism <= UPPER_BOUND_MAX_PARALLELISM, - "Maximum parallelism must be between 1 and " + UPPER_BOUND_MAX_PARALLELISM - + ". Found: " + maxParallelism); + OperatorValidationUtils.validateMaxParallelism(maxParallelism, UPPER_BOUND_MAX_PARALLELISM); this.maxParallelism = maxParallelism; } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java index bac85ee92f8c..cd25aa452c5c 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java @@ -31,11 +31,11 @@ import org.apache.flink.api.common.operators.Ordering; import org.apache.flink.api.common.operators.ResourceSpec; import org.apache.flink.api.common.operators.UnaryOperatorInformation; +import org.apache.flink.api.common.operators.util.OperatorValidationUtils; import org.apache.flink.api.common.typeinfo.NothingTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; import org.apache.flink.configuration.Configuration; -import org.apache.flink.util.Preconditions; import java.util.Arrays; @@ -286,8 +286,7 @@ public int getParallelism() { * @return This data sink with set parallelism. */ public DataSink setParallelism(int parallelism) { - Preconditions.checkArgument(parallelism > 0 || parallelism == ExecutionConfig.PARALLELISM_DEFAULT, - "The parallelism of an operator must be at least 1."); + OperatorValidationUtils.validateParallelism(parallelism); this.parallelism = parallelism; @@ -330,10 +329,7 @@ public ResourceSpec getPreferredResources() { * @return The data sink with set minimum and preferred resources. */ private DataSink setResources(ResourceSpec minResources, ResourceSpec preferredResources) { - Preconditions.checkNotNull(minResources, "The min resources must be not null."); - Preconditions.checkNotNull(preferredResources, "The preferred resources must be not null."); - Preconditions.checkArgument(minResources.isValid() && preferredResources.isValid() && minResources.lessThanOrEqual(preferredResources), - "The values in resources must be not less than 0 and the preferred resources must be greater than the min resources."); + OperatorValidationUtils.validateMinAndPreferredResources(minResources, preferredResources); this.minResources = minResources; this.preferredResources = preferredResources; @@ -348,8 +344,7 @@ private DataSink setResources(ResourceSpec minResources, ResourceSpec preferr * @return The data sink with set minimum and preferred resources. */ private DataSink setResources(ResourceSpec resources) { - Preconditions.checkNotNull(resources, "The resources must be not null."); - Preconditions.checkArgument(resources.isValid(), "The values in resources must be not less than 0."); + OperatorValidationUtils.validateResources(resources); this.minResources = resources; this.preferredResources = resources; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java index dc80e70e2e24..be52a27ea5f2 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java @@ -27,11 +27,11 @@ import org.apache.flink.api.common.aggregators.ConvergenceCriterion; import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.operators.ResourceSpec; +import org.apache.flink.api.common.operators.util.OperatorValidationUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.types.Value; -import org.apache.flink.util.Preconditions; import java.util.Arrays; @@ -182,8 +182,7 @@ public String getName() { * @return The iteration object, for function call chaining. */ public DeltaIteration parallelism(int parallelism) { - Preconditions.checkArgument(parallelism > 0 || parallelism == ExecutionConfig.PARALLELISM_DEFAULT, - "The parallelism must be positive, or ExecutionConfig.PARALLELISM_DEFAULT (use default)."); + OperatorValidationUtils.validateParallelism(parallelism); this.parallelism = parallelism; return this; } @@ -211,10 +210,7 @@ public int getParallelism() { * @return The iteration with set minimum and preferred resources. */ private DeltaIteration setResources(ResourceSpec minResources, ResourceSpec preferredResources) { - Preconditions.checkNotNull(minResources, "The min resources must be not null."); - Preconditions.checkNotNull(preferredResources, "The preferred resources must be not null."); - Preconditions.checkArgument(minResources.isValid() && preferredResources.isValid() && minResources.lessThanOrEqual(preferredResources), - "The values in resources must be not less than 0 and the preferred resources must be greater than the min resources."); + OperatorValidationUtils.validateMinAndPreferredResources(minResources, preferredResources); this.minResources = minResources; this.preferredResources = preferredResources; @@ -230,8 +226,7 @@ private DeltaIteration setResources(ResourceSpec minResources, ResourceS * @return The iteration with set minimum and preferred resources. */ private DeltaIteration setResources(ResourceSpec resources) { - Preconditions.checkNotNull(resources, "The resources must be not null."); - Preconditions.checkArgument(resources.isValid(), "The values in resources must be not less than 0."); + OperatorValidationUtils.validateResources(resources); this.minResources = resources; this.preferredResources = resources; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java index 142c1453e906..40999b0c93eb 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/Operator.java @@ -21,10 +21,10 @@ import org.apache.flink.annotation.Public; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.operators.ResourceSpec; +import org.apache.flink.api.common.operators.util.OperatorValidationUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.util.Preconditions; /** * Base class of all operators in the Java API. @@ -119,8 +119,7 @@ public O name(String newName) { * @return The operator with set parallelism. */ public O setParallelism(int parallelism) { - Preconditions.checkArgument(parallelism > 0 || parallelism == ExecutionConfig.PARALLELISM_DEFAULT, - "The parallelism must be at least one, or ExecutionConfig.PARALLELISM_DEFAULT (use system default)."); + OperatorValidationUtils.validateParallelism(parallelism); this.parallelism = parallelism; @@ -143,11 +142,7 @@ public O setParallelism(int parallelism) { * @return The operator with set minimum and preferred resources. */ private O setResources(ResourceSpec minResources, ResourceSpec preferredResources) { - Preconditions.checkNotNull(minResources, "The min resources must be not null."); - Preconditions.checkNotNull(preferredResources, "The preferred resources must be not null."); - - Preconditions.checkArgument(minResources.isValid() && preferredResources.isValid() && minResources.lessThanOrEqual(preferredResources), - "The values in resources must be not less than 0 and the preferred resources must be greater than the min resources."); + OperatorValidationUtils.validateMinAndPreferredResources(minResources, preferredResources); this.minResources = minResources; this.preferredResources = preferredResources; @@ -164,8 +159,7 @@ private O setResources(ResourceSpec minResources, ResourceSpec preferredResource * @return The operator with set minimum and preferred resources. */ private O setResources(ResourceSpec resources) { - Preconditions.checkNotNull(resources, "The resources must be not null."); - Preconditions.checkArgument(resources.isValid(), "The values in resources must be not less than 0."); + OperatorValidationUtils.validateResources(resources); this.minResources = resources; this.preferredResources = resources; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java index 074d481da28a..6f7291f442ac 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java @@ -21,10 +21,10 @@ import org.apache.flink.annotation.Public; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.operators.ResourceSpec; +import org.apache.flink.api.common.operators.util.OperatorValidationUtils; import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.StreamSink; import org.apache.flink.streaming.api.transformations.SinkTransformation; -import org.apache.flink.util.Preconditions; /** * A Stream Sink. This is used for emitting elements from a streaming topology. @@ -127,10 +127,7 @@ public DataStreamSink setParallelism(int parallelism) { * @return The sink with set minimum and preferred resources. */ private DataStreamSink setResources(ResourceSpec minResources, ResourceSpec preferredResources) { - Preconditions.checkNotNull(minResources, "The min resources must be not null."); - Preconditions.checkNotNull(preferredResources, "The preferred resources must be not null."); - Preconditions.checkArgument(minResources.isValid() && preferredResources.isValid() && minResources.lessThanOrEqual(preferredResources), - "The values in resources must be not less than 0 and the preferred resources must be greater than the min resources."); + OperatorValidationUtils.validateMinAndPreferredResources(minResources, preferredResources); transformation.setResources(minResources, preferredResources); @@ -144,8 +141,7 @@ private DataStreamSink setResources(ResourceSpec minResources, ResourceSpec p * @return The sink with set minimum and preferred resources. */ private DataStreamSink setResources(ResourceSpec resources) { - Preconditions.checkNotNull(resources, "The resources must be not null."); - Preconditions.checkArgument(resources.isValid(), "The values in resources must be not less than 0."); + OperatorValidationUtils.validateResources(resources); transformation.setResources(resources, resources); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java index d08514f847c4..f5fb05fb5aad 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSource.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.api.datastream; import org.apache.flink.annotation.Public; +import org.apache.flink.api.common.operators.util.OperatorValidationUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.operators.StreamSource; @@ -51,11 +52,8 @@ public DataStreamSource(SingleOutputStreamOperator operator) { @Override public DataStreamSource setParallelism(int parallelism) { - if (parallelism != 1 && !isParallel) { - throw new IllegalArgumentException("Source: " + transformation.getId() + " is not a parallel source"); - } else { - super.setParallelism(parallelism); - return this; - } + OperatorValidationUtils.validateMaxParallelism(parallelism, isParallel); + super.setParallelism(parallelism); + return this; } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java index 5ed69f2a5adc..88cdce18d165 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.functions.InvalidTypesException; import org.apache.flink.api.common.operators.ResourceSpec; +import org.apache.flink.api.common.operators.util.OperatorValidationUtils; import org.apache.flink.api.common.typeinfo.TypeHint; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.dag.Transformation; @@ -30,7 +31,6 @@ import org.apache.flink.streaming.api.transformations.PhysicalTransformation; import org.apache.flink.streaming.api.transformations.SideOutputTransformation; import org.apache.flink.util.OutputTag; -import org.apache.flink.util.Preconditions; import java.util.HashMap; import java.util.Map; @@ -139,9 +139,7 @@ public SingleOutputStreamOperator setUidHash(String uidHash) { * @return The operator with set parallelism. */ public SingleOutputStreamOperator setParallelism(int parallelism) { - Preconditions.checkArgument(canBeParallel() || parallelism == 1, - "The parallelism of non parallel operator must be 1."); - + OperatorValidationUtils.validateParallelism(parallelism, canBeParallel()); transformation.setParallelism(parallelism); return this; @@ -158,12 +156,7 @@ public SingleOutputStreamOperator setParallelism(int parallelism) { */ @PublicEvolving public SingleOutputStreamOperator setMaxParallelism(int maxParallelism) { - Preconditions.checkArgument(maxParallelism > 0, - "The maximum parallelism must be greater than 0."); - - Preconditions.checkArgument(canBeParallel() || maxParallelism == 1, - "The maximum parallelism of non parallel operator must be 1."); - + OperatorValidationUtils.validateMaxParallelism(maxParallelism, canBeParallel()); transformation.setMaxParallelism(maxParallelism); return this; @@ -183,11 +176,7 @@ public SingleOutputStreamOperator setMaxParallelism(int maxParallelism) { * @return The operator with set minimum and preferred resources. */ private SingleOutputStreamOperator setResources(ResourceSpec minResources, ResourceSpec preferredResources) { - Preconditions.checkNotNull(minResources, "The min resources must be not null."); - Preconditions.checkNotNull(preferredResources, "The preferred resources must be not null."); - Preconditions.checkArgument(minResources.isValid() && preferredResources.isValid() && minResources.lessThanOrEqual(preferredResources), - "The values in resources must be not less than 0 and the preferred resources must be greater than the min resources."); - + OperatorValidationUtils.validateMinAndPreferredResources(minResources, preferredResources); transformation.setResources(minResources, preferredResources); return this; @@ -200,9 +189,7 @@ private SingleOutputStreamOperator setResources(ResourceSpec minResources, Re * @return The operator with set minimum and preferred resources. */ private SingleOutputStreamOperator setResources(ResourceSpec resources) { - Preconditions.checkNotNull(resources, "The resources must be not null."); - Preconditions.checkArgument(resources.isValid(), "The values in resources must be not less than 0."); - + OperatorValidationUtils.validateResources(resources); transformation.setResources(resources, resources); return this; From 9d224102829be248749f45985dad357c42d521fc Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Wed, 9 Oct 2019 15:03:46 +0800 Subject: [PATCH 560/746] [hotfix] UNKNOWN ResourceSpec should not be numerically compared. --- .../apache/flink/api/common/operators/ResourceSpec.java | 3 +++ .../common/operators/util/OperatorValidationUtils.java | 8 ++++++-- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/ResourceSpec.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/ResourceSpec.java index 6b450fb92bad..3b8b21240ee3 100755 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/ResourceSpec.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/ResourceSpec.java @@ -228,6 +228,9 @@ public boolean isValid() { * @return True if current resource is less than or equal with the other resource, otherwise return false. */ public boolean lessThanOrEqual(@Nonnull ResourceSpec other) { + if (this.equals(UNKNOWN) || other.equals(UNKNOWN)) { + throw new IllegalArgumentException("UNKNOWN ResourceSpecs cannot be numerically compared."); + } int cmp1 = Double.compare(this.cpuCores, other.cpuCores); int cmp2 = Integer.compare(this.heapMemoryInMB, other.heapMemoryInMB); int cmp3 = Integer.compare(this.directMemoryInMB, other.directMemoryInMB); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/util/OperatorValidationUtils.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/util/OperatorValidationUtils.java index 2e95c70003c9..6b8e3d7629f2 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/util/OperatorValidationUtils.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/util/OperatorValidationUtils.java @@ -69,7 +69,11 @@ public static void validateResources(ResourceSpec resources) { public static void validateMinAndPreferredResources(ResourceSpec minResources, ResourceSpec preferredResources) { Preconditions.checkNotNull(minResources, "The min resources must be not null."); Preconditions.checkNotNull(preferredResources, "The preferred resources must be not null."); - Preconditions.checkArgument(minResources.isValid() && preferredResources.isValid() && minResources.lessThanOrEqual(preferredResources), - "The values in resources must be not less than 0 and the preferred resources must be greater than the min resources."); + Preconditions.checkArgument(minResources.isValid() && preferredResources.isValid(), + "The resources must either be UNKNOWN or all the fields are no less than 0."); + Preconditions.checkArgument((minResources == ResourceSpec.UNKNOWN && preferredResources == ResourceSpec.UNKNOWN) || + (minResources != ResourceSpec.UNKNOWN && preferredResources != ResourceSpec.UNKNOWN && minResources.lessThanOrEqual(preferredResources)), + "The resources must be either both UNKNOWN or both not UNKNOWN. If not UNKNOWN," + + " the preferred resources must be greater than or equal to the min resources."); } } From 1cbee5e3ac1b6429da043fca04637e970a2a80a4 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Thu, 17 Oct 2019 19:50:19 +0800 Subject: [PATCH 561/746] [hotfix] Make MemorySize comparable. --- .../java/org/apache/flink/configuration/MemorySize.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) 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 7ca2e8b8b5ea..40c143b66e9a 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 @@ -42,7 +42,7 @@ * */ @PublicEvolving -public class MemorySize implements java.io.Serializable { +public class MemorySize implements java.io.Serializable, Comparable { private static final long serialVersionUID = 1L; @@ -120,6 +120,11 @@ public String toString() { return bytes + " bytes"; } + @Override + public int compareTo(MemorySize that) { + return Long.compare(this.bytes, that.bytes); + } + // ------------------------------------------------------------------------ // Calculations // ------------------------------------------------------------------------ From 2c81c3f58cb27b4a4358a57b8a3c15329bd34008 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Mon, 11 Nov 2019 15:29:00 +0100 Subject: [PATCH 562/746] [FLINK-14405][runtime] Update ResourceSpec to align with FLIP-49 resource types --- .../api/common/operators/ResourceSpec.java | 275 ++++++++++-------- .../common/operators/ResourceSpecTest.java | 46 +-- .../flink/api/java/operator/OperatorTest.java | 4 +- .../plantranslate/JobGraphGeneratorTest.java | 26 +- .../types/ResourceProfileTest.java | 34 +-- .../runtime/dispatcher/DispatcherTest.java | 2 +- .../flink/streaming/api/DataStreamTest.java | 28 +- .../graph/StreamingJobGraphGeneratorTest.java | 20 +- .../plan/nodes/resource/NodeResourceUtil.java | 5 +- 9 files changed, 241 insertions(+), 199 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/ResourceSpec.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/ResourceSpec.java index 3b8b21240ee3..3d2b4fa95339 100755 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/ResourceSpec.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/ResourceSpec.java @@ -21,8 +21,10 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.resources.GPUResource; import org.apache.flink.api.common.resources.Resource; +import org.apache.flink.configuration.MemorySize; import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.io.Serializable; import java.util.HashMap; @@ -30,22 +32,20 @@ import java.util.Objects; import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; /** * Describe the different resource factors of the operator with UDF. * - *

    The state backend provides the method to estimate memory usages based on state size in the resource. - * *

    Resource provides {@link #merge(ResourceSpec)} method for chained operators when generating job graph. * *

    Resource provides {@link #lessThanOrEqual(ResourceSpec)} method to compare these fields in sequence: *

      *
    1. CPU cores
    2. - *
    3. Heap Memory Size
    4. - *
    5. Direct Memory Size
    6. - *
    7. Native Memory Size
    8. - *
    9. State Size
    10. - *
    11. Managed Memory Size
    12. + *
    13. Task Heap Memory
    14. + *
    15. Task Off-Heap Memory
    16. + *
    17. On-Heap Managed Memory
    18. + *
    19. Off-Heap Managed Memory
    20. *
    21. Extended resources
    22. *
    */ @@ -70,20 +70,21 @@ public final class ResourceSpec implements Serializable { /** How many cpu cores are needed, use double so we can specify cpu like 0.1. */ private final double cpuCores; - /** How many java heap memory in mb are needed. */ - private final int heapMemoryInMB; - - /** How many nio direct memory in mb are needed. */ - private final int directMemoryInMB; + /** How much task heap memory is needed. */ + @Nullable // can be null only for UNKNOWN + private final MemorySize taskHeapMemory; - /** How many native memory in mb are needed. */ - private final int nativeMemoryInMB; + /** How much task off-heap memory is needed. */ + @Nullable // can be null only for UNKNOWN + private final MemorySize taskOffHeapMemory; - /** How many state size in mb are used. */ - private final int stateSizeInMB; + /** How much on-heap managed memory is needed. */ + @Nullable // can be null only for UNKNOWN + private final MemorySize onHeapManagedMemory; - /** The required amount of managed memory (in MB). */ - private final int managedMemoryInMB; + /** How much off-heap managed memory is needed. */ + @Nullable // can be null only for UNKNOWN + private final MemorySize offHeapManagedMemory; private final Map extendedResources = new HashMap<>(1); @@ -91,34 +92,27 @@ public final class ResourceSpec implements Serializable { * Creates a new ResourceSpec with full resources. * * @param cpuCores The number of CPU cores (possibly fractional, i.e., 0.2 cores) - * @param heapMemoryInMB The size of the java heap memory, in megabytes. - * @param directMemoryInMB The size of the java nio direct memory, in megabytes. - * @param nativeMemoryInMB The size of the native memory, in megabytes. - * @param stateSizeInMB The state size for storing in checkpoint. - * @param managedMemoryInMB The size of managed memory, in megabytes. + * @param taskHeapMemory The size of the task heap memory. + * @param taskOffHeapMemory The size of the task off-heap memory. + * @param onHeapManagedMemory The size of the on-heap managed memory. + * @param offHeapManagedMemory The size of the off-heap managed memory. * @param extendedResources The extended resources, associated with the resource manager used */ private ResourceSpec( - double cpuCores, - int heapMemoryInMB, - int directMemoryInMB, - int nativeMemoryInMB, - int stateSizeInMB, - int managedMemoryInMB, - Resource... extendedResources) { + double cpuCores, + MemorySize taskHeapMemory, + MemorySize taskOffHeapMemory, + MemorySize onHeapManagedMemory, + MemorySize offHeapManagedMemory, + Resource... extendedResources) { + checkArgument(cpuCores >= 0, "The cpu cores of the resource spec should not be negative."); - checkArgument(heapMemoryInMB >= 0, "The heap memory of the resource spec should not be negative"); - checkArgument(directMemoryInMB >= 0, "The direct memory of the resource spec should not be negative"); - checkArgument(nativeMemoryInMB >= 0, "The native memory of the resource spec should not be negative"); - checkArgument(stateSizeInMB >= 0, "The state size of the resource spec should not be negative"); - checkArgument(managedMemoryInMB >= 0, "The managed memory of the resource spec should not be negative"); this.cpuCores = cpuCores; - this.heapMemoryInMB = heapMemoryInMB; - this.directMemoryInMB = directMemoryInMB; - this.nativeMemoryInMB = nativeMemoryInMB; - this.stateSizeInMB = stateSizeInMB; - this.managedMemoryInMB = managedMemoryInMB; + this.taskHeapMemory = checkNotNull(taskHeapMemory); + this.taskOffHeapMemory = checkNotNull(taskOffHeapMemory); + this.onHeapManagedMemory = checkNotNull(onHeapManagedMemory); + this.offHeapManagedMemory = checkNotNull(offHeapManagedMemory); for (Resource resource : extendedResources) { if (resource != null) { this.extendedResources.put(resource.getName(), resource); @@ -126,21 +120,46 @@ private ResourceSpec( } } + /** + * Creates a new ResourceSpec with full resources. + * + * @param cpuCores The number of CPU cores (possibly fractional, i.e., 0.2 cores) + * @param taskHeapMemoryMB The size of the task heap memory, in megabytes. + * @param taskOffHeapMemoryMB The size of the task off-heap memory, in megabytes. + * @param onHeapManagedMemoryMB The size of the on-heap managed memory, in megabytes. + * @param offHeapManagedMemoryMB The size of the off-heap managed memory. + * @param extendedResources The extended resources, associated with the resource manager used + */ + private ResourceSpec( + double cpuCores, + int taskHeapMemoryMB, + int taskOffHeapMemoryMB, + int onHeapManagedMemoryMB, + int offHeapManagedMemoryMB, + Resource... extendedResources) { + this( + cpuCores, + MemorySize.parse(taskHeapMemoryMB + "m"), + MemorySize.parse(taskOffHeapMemoryMB + "m"), + MemorySize.parse(onHeapManagedMemoryMB + "m"), + MemorySize.parse(offHeapManagedMemoryMB + "m"), + extendedResources); + } + /** * Creates a new ResourceSpec with all fields unknown. */ private ResourceSpec() { this.cpuCores = -1; - this.heapMemoryInMB = -1; - this.directMemoryInMB = -1; - this.nativeMemoryInMB = -1; - this.stateSizeInMB = -1; - this.managedMemoryInMB = -1; + this.taskHeapMemory = null; + this.taskOffHeapMemory = null; + this.onHeapManagedMemory = null; + this.offHeapManagedMemory = null; } /** * Used by system internally to merge the other resources of chained operators - * when generating the job graph or merge the resource consumed by state backend. + * when generating the job graph. * * @param other Reference to resource to merge in. * @return The new resource with merged values. @@ -151,12 +170,11 @@ public ResourceSpec merge(ResourceSpec other) { } ResourceSpec target = new ResourceSpec( - this.cpuCores + other.cpuCores, - this.heapMemoryInMB + other.heapMemoryInMB, - this.directMemoryInMB + other.directMemoryInMB, - this.nativeMemoryInMB + other.nativeMemoryInMB, - this.stateSizeInMB + other.stateSizeInMB, - this.managedMemoryInMB + other.managedMemoryInMB); + this.cpuCores + other.cpuCores, + this.taskHeapMemory.add(other.taskHeapMemory), + this.taskOffHeapMemory.add(other.taskOffHeapMemory), + this.onHeapManagedMemory.add(other.onHeapManagedMemory), + this.offHeapManagedMemory.add(other.offHeapManagedMemory)); target.extendedResources.putAll(extendedResources); for (Resource resource : other.extendedResources.values()) { target.extendedResources.merge(resource.getName(), resource, (v1, v2) -> v1.merge(v2)); @@ -165,30 +183,32 @@ public ResourceSpec merge(ResourceSpec other) { } public double getCpuCores() { + throwUnsupportedOperationExceptionIfUnknown(); return this.cpuCores; } - public int getHeapMemory() { - return this.heapMemoryInMB; + public MemorySize getTaskHeapMemory() { + throwUnsupportedOperationExceptionIfUnknown(); + return this.taskHeapMemory; } - public int getDirectMemory() { - return this.directMemoryInMB; + public MemorySize getTaskOffHeapMemory() { + throwUnsupportedOperationExceptionIfUnknown(); + return taskOffHeapMemory; } - public int getNativeMemory() { - return this.nativeMemoryInMB; + public MemorySize getOnHeapManagedMemory() { + throwUnsupportedOperationExceptionIfUnknown(); + return onHeapManagedMemory; } - public int getStateSize() { - return this.stateSizeInMB; - } - - public int getManagedMemory() { - return this.managedMemoryInMB; + public MemorySize getOffHeapManagedMemory() { + throwUnsupportedOperationExceptionIfUnknown(); + return offHeapManagedMemory; } public double getGPUResource() { + throwUnsupportedOperationExceptionIfUnknown(); Resource gpuResource = extendedResources.get(GPU_NAME); if (gpuResource != null) { return gpuResource.getValue(); @@ -198,26 +218,34 @@ public double getGPUResource() { } public Map getExtendedResources() { + throwUnsupportedOperationExceptionIfUnknown(); return extendedResources; } + private void throwUnsupportedOperationExceptionIfUnknown() { + if (this.equals(UNKNOWN)) { + throw new UnsupportedOperationException(); + } + } + /** * Check whether all the field values are valid. * * @return True if all the values are equal or greater than 0, otherwise false. */ public boolean isValid() { - if (this.cpuCores >= 0 && this.heapMemoryInMB >= 0 && this.directMemoryInMB >= 0 && - this.nativeMemoryInMB >= 0 && this.stateSizeInMB >= 0 && managedMemoryInMB >= 0) { - for (Resource resource : extendedResources.values()) { - if (resource.getValue() < 0) { - return false; - } - } + if (this.equals(UNKNOWN)) { return true; - } else { + } + if (this.cpuCores < 0) { return false; } + for (Resource resource : extendedResources.values()) { + if (resource.getValue() < 0) { + return false; + } + } + return true; } /** @@ -232,12 +260,11 @@ public boolean lessThanOrEqual(@Nonnull ResourceSpec other) { throw new IllegalArgumentException("UNKNOWN ResourceSpecs cannot be numerically compared."); } int cmp1 = Double.compare(this.cpuCores, other.cpuCores); - int cmp2 = Integer.compare(this.heapMemoryInMB, other.heapMemoryInMB); - int cmp3 = Integer.compare(this.directMemoryInMB, other.directMemoryInMB); - int cmp4 = Integer.compare(this.nativeMemoryInMB, other.nativeMemoryInMB); - int cmp5 = Integer.compare(this.stateSizeInMB, other.stateSizeInMB); - int cmp6 = Integer.compare(this.managedMemoryInMB, other.managedMemoryInMB); - if (cmp1 <= 0 && cmp2 <= 0 && cmp3 <= 0 && cmp4 <= 0 && cmp5 <= 0 && cmp6 <= 0) { + int cmp2 = this.taskHeapMemory.compareTo(other.taskHeapMemory); + int cmp3 = this.taskOffHeapMemory.compareTo(other.taskOffHeapMemory); + int cmp4 = this.onHeapManagedMemory.compareTo(other.onHeapManagedMemory); + int cmp5 = this.offHeapManagedMemory.compareTo(other.offHeapManagedMemory); + if (cmp1 <= 0 && cmp2 <= 0 && cmp3 <= 0 && cmp4 <= 0 && cmp5 <= 0) { for (Resource resource : extendedResources.values()) { if (!other.extendedResources.containsKey(resource.getName()) || other.extendedResources.get(resource.getName()).getResourceAggregateType() != resource.getResourceAggregateType() || @@ -257,12 +284,11 @@ public boolean equals(Object obj) { } else if (obj != null && obj.getClass() == ResourceSpec.class) { ResourceSpec that = (ResourceSpec) obj; return this.cpuCores == that.cpuCores && - this.heapMemoryInMB == that.heapMemoryInMB && - this.directMemoryInMB == that.directMemoryInMB && - this.nativeMemoryInMB == that.nativeMemoryInMB && - this.stateSizeInMB == that.stateSizeInMB && - this.managedMemoryInMB == that.managedMemoryInMB && - Objects.equals(this.extendedResources, that.extendedResources); + Objects.equals(this.taskHeapMemory, that.taskHeapMemory) && + Objects.equals(this.taskOffHeapMemory, that.taskOffHeapMemory) && + Objects.equals(this.onHeapManagedMemory, that.onHeapManagedMemory) && + Objects.equals(this.offHeapManagedMemory, that.offHeapManagedMemory) && + Objects.equals(extendedResources, that.extendedResources); } else { return false; } @@ -272,29 +298,31 @@ public boolean equals(Object obj) { public int hashCode() { final long cpuBits = Double.doubleToLongBits(cpuCores); int result = (int) (cpuBits ^ (cpuBits >>> 32)); - result = 31 * result + heapMemoryInMB; - result = 31 * result + directMemoryInMB; - result = 31 * result + nativeMemoryInMB; - result = 31 * result + stateSizeInMB; - result = 31 * result + managedMemoryInMB; + result = 31 * result + Objects.hashCode(taskHeapMemory); + result = 31 * result + Objects.hashCode(taskOffHeapMemory); + result = 31 * result + Objects.hashCode(onHeapManagedMemory); + result = 31 * result + Objects.hashCode(offHeapManagedMemory); result = 31 * result + extendedResources.hashCode(); return result; } @Override public String toString() { + if (this.equals(UNKNOWN)) { + return "ResourceSpec{UNKNOWN}"; + } + StringBuilder extend = new StringBuilder(); for (Resource resource : extendedResources.values()) { extend.append(", ").append(resource.getName()).append("=").append(resource.getValue()); } return "ResourceSpec{" + - "cpuCores=" + cpuCores + - ", heapMemoryInMB=" + heapMemoryInMB + - ", directMemoryInMB=" + directMemoryInMB + - ", nativeMemoryInMB=" + nativeMemoryInMB + - ", stateSizeInMB=" + stateSizeInMB + - ", managedMemoryInMB=" + managedMemoryInMB + extend + - '}'; + "cpuCores=" + cpuCores + + ", taskHeapMemory=" + taskHeapMemory + + ", taskOffHeapMemory=" + taskOffHeapMemory + + ", onHeapManagedMemory=" + onHeapManagedMemory + + ", offHeapManagedMemory=" + offHeapManagedMemory + + '}'; } // ------------------------------------------------------------------------ @@ -319,12 +347,11 @@ public static Builder newBuilder() { */ public static class Builder { - private double cpuCores; - private int heapMemoryInMB; - private int directMemoryInMB; - private int nativeMemoryInMB; - private int stateSizeInMB; - private int managedMemoryInMB; + private double cpuCores = 0.0; + private MemorySize taskHeapMemory = MemorySize.ZERO; + private MemorySize taskOffHeapMemory = MemorySize.ZERO; + private MemorySize onHeapManagedMemory = MemorySize.ZERO; + private MemorySize offHeapManagedMemory = MemorySize.ZERO; private GPUResource gpuResource; public Builder setCpuCores(double cpuCores) { @@ -332,28 +359,43 @@ public Builder setCpuCores(double cpuCores) { return this; } - public Builder setHeapMemoryInMB(int heapMemory) { - this.heapMemoryInMB = heapMemory; + public Builder setTaskHeapMemory(MemorySize taskHeapMemory) { + this.taskHeapMemory = taskHeapMemory; + return this; + } + + public Builder setTaskHeapMemoryMB(int taskHeapMemoryMB) { + this.taskHeapMemory = MemorySize.parse(taskHeapMemoryMB + "m"); + return this; + } + + public Builder setTaskOffHeapMemory(MemorySize taskOffHeapMemory) { + this.taskOffHeapMemory = taskOffHeapMemory; + return this; + } + + public Builder setOffTaskHeapMemoryMB(int taskOffHeapMemoryMB) { + this.taskOffHeapMemory = MemorySize.parse(taskOffHeapMemoryMB + "m"); return this; } - public Builder setDirectMemoryInMB(int directMemory) { - this.directMemoryInMB = directMemory; + public Builder setOnHeapManagedMemory(MemorySize onHeapManagedMemory) { + this.onHeapManagedMemory = onHeapManagedMemory; return this; } - public Builder setNativeMemoryInMB(int nativeMemory) { - this.nativeMemoryInMB = nativeMemory; + public Builder setOnHeapManagedMemoryMB(int onHeapManagedMemoryMB) { + this.onHeapManagedMemory = MemorySize.parse(onHeapManagedMemoryMB + "m"); return this; } - public Builder setStateSizeInMB(int stateSize) { - this.stateSizeInMB = stateSize; + public Builder setOffHeapManagedMemory(MemorySize offHeapManagedMemory) { + this.offHeapManagedMemory = offHeapManagedMemory; return this; } - public Builder setManagedMemoryInMB(int managedMemory) { - this.managedMemoryInMB = managedMemory; + public Builder setOffHeapManagedMemoryMB(int offHeapManagedMemoryMB) { + this.offHeapManagedMemory = MemorySize.parse(offHeapManagedMemoryMB + "m"); return this; } @@ -365,11 +407,10 @@ public Builder setGPUResource(double gpus) { public ResourceSpec build() { return new ResourceSpec( cpuCores, - heapMemoryInMB, - directMemoryInMB, - nativeMemoryInMB, - stateSizeInMB, - managedMemoryInMB, + taskHeapMemory, + taskOffHeapMemory, + onHeapManagedMemory, + offHeapManagedMemory, gpuResource); } } diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/ResourceSpecTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/ResourceSpecTest.java index e232f42486d3..1dc25ead571e 100755 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/ResourceSpecTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/ResourceSpecTest.java @@ -36,19 +36,19 @@ public class ResourceSpecTest extends TestLogger { @Test public void testIsValid() throws Exception { - ResourceSpec rs = ResourceSpec.newBuilder().setCpuCores(1.0).setHeapMemoryInMB(100).build(); + ResourceSpec rs = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); assertTrue(rs.isValid()); rs = ResourceSpec.newBuilder(). setCpuCores(1.0). - setHeapMemoryInMB(100). + setTaskHeapMemoryMB(100). setGPUResource(1). build(); assertTrue(rs.isValid()); rs = ResourceSpec.newBuilder(). setCpuCores(1.0). - setHeapMemoryInMB(100). + setTaskHeapMemoryMB(100). setGPUResource(-1). build(); assertFalse(rs.isValid()); @@ -56,14 +56,14 @@ public void testIsValid() throws Exception { @Test public void testLessThanOrEqual() throws Exception { - ResourceSpec rs1 = ResourceSpec.newBuilder().setCpuCores(1.0).setHeapMemoryInMB(100).build(); - ResourceSpec rs2 = ResourceSpec.newBuilder().setCpuCores(1.0).setHeapMemoryInMB(100).build(); + ResourceSpec rs1 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); + ResourceSpec rs2 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); assertTrue(rs1.lessThanOrEqual(rs2)); assertTrue(rs2.lessThanOrEqual(rs1)); ResourceSpec rs3 = ResourceSpec.newBuilder(). setCpuCores(1.0). - setHeapMemoryInMB(100). + setTaskHeapMemoryMB(100). setGPUResource(1.1). build(); assertTrue(rs1.lessThanOrEqual(rs3)); @@ -71,7 +71,7 @@ public void testLessThanOrEqual() throws Exception { ResourceSpec rs4 = ResourceSpec.newBuilder(). setCpuCores(1.0). - setHeapMemoryInMB(100). + setTaskHeapMemoryMB(100). setGPUResource(2.2). build(); assertFalse(rs4.lessThanOrEqual(rs3)); @@ -80,26 +80,26 @@ public void testLessThanOrEqual() throws Exception { @Test public void testEquals() throws Exception { - ResourceSpec rs1 = ResourceSpec.newBuilder().setCpuCores(1.0).setHeapMemoryInMB(100).build(); - ResourceSpec rs2 = ResourceSpec.newBuilder().setCpuCores(1.0).setHeapMemoryInMB(100).build(); + ResourceSpec rs1 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); + ResourceSpec rs2 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); assertEquals(rs1, rs2); assertEquals(rs2, rs1); ResourceSpec rs3 = ResourceSpec.newBuilder(). setCpuCores(1.0). - setHeapMemoryInMB(100). + setTaskHeapMemoryMB(100). setGPUResource(2.2). build(); ResourceSpec rs4 = ResourceSpec.newBuilder(). setCpuCores(1.0). - setHeapMemoryInMB(100). + setTaskHeapMemoryMB(100). setGPUResource(1). build(); assertNotEquals(rs3, rs4); ResourceSpec rs5 = ResourceSpec.newBuilder(). setCpuCores(1.0). - setHeapMemoryInMB(100). + setTaskHeapMemoryMB(100). setGPUResource(2.2). build(); assertEquals(rs3, rs5); @@ -107,25 +107,25 @@ public void testEquals() throws Exception { @Test public void testHashCode() throws Exception { - ResourceSpec rs1 = ResourceSpec.newBuilder().setCpuCores(1.0).setHeapMemoryInMB(100).build(); - ResourceSpec rs2 = ResourceSpec.newBuilder().setCpuCores(1.0).setHeapMemoryInMB(100).build(); + ResourceSpec rs1 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); + ResourceSpec rs2 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); assertEquals(rs1.hashCode(), rs2.hashCode()); ResourceSpec rs3 = ResourceSpec.newBuilder(). setCpuCores(1.0). - setHeapMemoryInMB(100). + setTaskHeapMemoryMB(100). setGPUResource(2.2). build(); ResourceSpec rs4 = ResourceSpec.newBuilder(). setCpuCores(1.0). - setHeapMemoryInMB(100). + setTaskHeapMemoryMB(100). setGPUResource(1). build(); assertNotEquals(rs3.hashCode(), rs4.hashCode()); ResourceSpec rs5 = ResourceSpec.newBuilder(). setCpuCores(1.0). - setHeapMemoryInMB(100). + setTaskHeapMemoryMB(100). setGPUResource(2.2). build(); assertEquals(rs3.hashCode(), rs5.hashCode()); @@ -135,14 +135,14 @@ public void testHashCode() throws Exception { public void testMerge() throws Exception { ResourceSpec rs1 = ResourceSpec.newBuilder(). setCpuCores(1.0). - setHeapMemoryInMB(100). + setTaskHeapMemoryMB(100). setGPUResource(1.1). build(); - ResourceSpec rs2 = ResourceSpec.newBuilder().setCpuCores(1.0).setHeapMemoryInMB(100).build(); + ResourceSpec rs2 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); ResourceSpec rs3 = rs1.merge(rs2); assertEquals(2.0, rs3.getCpuCores(), 0.000001); - assertEquals(200, rs3.getHeapMemory()); + assertEquals(200, rs3.getTaskHeapMemory().getMebiBytes()); assertEquals(1.1, rs3.getGPUResource(), 0.000001); ResourceSpec rs4 = rs1.merge(rs3); @@ -153,7 +153,7 @@ public void testMerge() throws Exception { public void testSerializable() throws Exception { ResourceSpec rs1 = ResourceSpec.newBuilder(). setCpuCores(1.0). - setHeapMemoryInMB(100). + setTaskHeapMemoryMB(100). setGPUResource(1.1). build(); @@ -166,7 +166,7 @@ public void testMergeThisUnknown() throws Exception { final ResourceSpec spec1 = ResourceSpec.UNKNOWN; final ResourceSpec spec2 = ResourceSpec.newBuilder() .setCpuCores(1.0) - .setHeapMemoryInMB(100) + .setTaskHeapMemoryMB(100) .setGPUResource(1.1) .build(); @@ -179,7 +179,7 @@ public void testMergeThisUnknown() throws Exception { public void testMergeOtherUnknown() throws Exception { final ResourceSpec spec1 = ResourceSpec.newBuilder() .setCpuCores(1.0) - .setHeapMemoryInMB(100) + .setTaskHeapMemoryMB(100) .setGPUResource(1.1) .build(); final ResourceSpec spec2 = ResourceSpec.UNKNOWN; diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/OperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/OperatorTest.java index d231a8cdcfc6..f01f1babec7c 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/operator/OperatorTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/OperatorTest.java @@ -60,8 +60,8 @@ public void testConfigurationOfResource() throws Exception{ opMethod.setAccessible(true); // verify explicit change in resources - ResourceSpec minResources = ResourceSpec.newBuilder().setCpuCores(1.0).setHeapMemoryInMB(100).build(); - ResourceSpec preferredResources = ResourceSpec.newBuilder().setCpuCores(2.0).setHeapMemoryInMB(200).build(); + ResourceSpec minResources = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); + ResourceSpec preferredResources = ResourceSpec.newBuilder().setCpuCores(2.0).setTaskHeapMemoryMB(200).build(); opMethod.invoke(operator, minResources, preferredResources); assertEquals(minResources, operator.getMinResources()); diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/plantranslate/JobGraphGeneratorTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/plantranslate/JobGraphGeneratorTest.java index 9e4134e70ce7..5b2c6617e142 100644 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/plantranslate/JobGraphGeneratorTest.java +++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/plantranslate/JobGraphGeneratorTest.java @@ -73,13 +73,13 @@ public class JobGraphGeneratorTest { */ @Test public void testResourcesForChainedOperators() throws Exception { - ResourceSpec resource1 = ResourceSpec.newBuilder().setCpuCores(0.1).setHeapMemoryInMB(100).build(); - ResourceSpec resource2 = ResourceSpec.newBuilder().setCpuCores(0.2).setHeapMemoryInMB(200).build(); - ResourceSpec resource3 = ResourceSpec.newBuilder().setCpuCores(0.3).setHeapMemoryInMB(300).build(); - ResourceSpec resource4 = ResourceSpec.newBuilder().setCpuCores(0.4).setHeapMemoryInMB(400).build(); - ResourceSpec resource5 = ResourceSpec.newBuilder().setCpuCores(0.5).setHeapMemoryInMB(500).build(); - ResourceSpec resource6 = ResourceSpec.newBuilder().setCpuCores(0.6).setHeapMemoryInMB(600).build(); - ResourceSpec resource7 = ResourceSpec.newBuilder().setCpuCores(0.7).setHeapMemoryInMB(700).build(); + ResourceSpec resource1 = ResourceSpec.newBuilder().setCpuCores(0.1).setTaskHeapMemoryMB(100).build(); + ResourceSpec resource2 = ResourceSpec.newBuilder().setCpuCores(0.2).setTaskHeapMemoryMB(200).build(); + ResourceSpec resource3 = ResourceSpec.newBuilder().setCpuCores(0.3).setTaskHeapMemoryMB(300).build(); + ResourceSpec resource4 = ResourceSpec.newBuilder().setCpuCores(0.4).setTaskHeapMemoryMB(400).build(); + ResourceSpec resource5 = ResourceSpec.newBuilder().setCpuCores(0.5).setTaskHeapMemoryMB(500).build(); + ResourceSpec resource6 = ResourceSpec.newBuilder().setCpuCores(0.6).setTaskHeapMemoryMB(600).build(); + ResourceSpec resource7 = ResourceSpec.newBuilder().setCpuCores(0.7).setTaskHeapMemoryMB(700).build(); Method opMethod = Operator.class.getDeclaredMethod("setResources", ResourceSpec.class); opMethod.setAccessible(true); @@ -147,12 +147,12 @@ public boolean filter(Long value) throws Exception { */ @Test public void testResourcesForDeltaIteration() throws Exception{ - ResourceSpec resource1 = ResourceSpec.newBuilder().setCpuCores(0.1).setHeapMemoryInMB(100).build(); - ResourceSpec resource2 = ResourceSpec.newBuilder().setCpuCores(0.2).setHeapMemoryInMB(200).build(); - ResourceSpec resource3 = ResourceSpec.newBuilder().setCpuCores(0.3).setHeapMemoryInMB(300).build(); - ResourceSpec resource4 = ResourceSpec.newBuilder().setCpuCores(0.4).setHeapMemoryInMB(400).build(); - ResourceSpec resource5 = ResourceSpec.newBuilder().setCpuCores(0.5).setHeapMemoryInMB(500).build(); - ResourceSpec resource6 = ResourceSpec.newBuilder().setCpuCores(0.6).setHeapMemoryInMB(600).build(); + ResourceSpec resource1 = ResourceSpec.newBuilder().setCpuCores(0.1).setTaskHeapMemoryMB(100).build(); + ResourceSpec resource2 = ResourceSpec.newBuilder().setCpuCores(0.2).setTaskHeapMemoryMB(200).build(); + ResourceSpec resource3 = ResourceSpec.newBuilder().setCpuCores(0.3).setTaskHeapMemoryMB(300).build(); + ResourceSpec resource4 = ResourceSpec.newBuilder().setCpuCores(0.4).setTaskHeapMemoryMB(400).build(); + ResourceSpec resource5 = ResourceSpec.newBuilder().setCpuCores(0.5).setTaskHeapMemoryMB(500).build(); + ResourceSpec resource6 = ResourceSpec.newBuilder().setCpuCores(0.6).setTaskHeapMemoryMB(600).build(); Method opMethod = Operator.class.getDeclaredMethod("setResources", ResourceSpec.class); opMethod.setAccessible(true); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java index 5264cebe3897..b58f7a521bfc 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java @@ -39,7 +39,7 @@ public class ResourceProfileTest { @Test - public void testMatchRequirement() throws Exception { + public void testMatchRequirement() { ResourceProfile rp1 = new ResourceProfile(1.0, 100, 100, 100, 0, 0, Collections.emptyMap()); ResourceProfile rp2 = new ResourceProfile(1.0, 200, 200, 200, 0, 0, Collections.emptyMap()); ResourceProfile rp3 = new ResourceProfile(2.0, 100, 100, 100, 0, 0, Collections.emptyMap()); @@ -64,12 +64,12 @@ public void testMatchRequirement() throws Exception { ResourceSpec rs1 = ResourceSpec.newBuilder(). setCpuCores(1.0). - setHeapMemoryInMB(100). + setTaskHeapMemoryMB(100). setGPUResource(2.2). build(); ResourceSpec rs2 = ResourceSpec.newBuilder(). setCpuCores(1.0). - setHeapMemoryInMB(100). + setTaskHeapMemoryMB(100). setGPUResource(1.1). build(); @@ -84,26 +84,26 @@ public void testUnknownMatchesUnknown() { } @Test - public void testEquals() throws Exception { - ResourceSpec rs1 = ResourceSpec.newBuilder().setCpuCores(1.0).setHeapMemoryInMB(100).build(); - ResourceSpec rs2 = ResourceSpec.newBuilder().setCpuCores(1.0).setHeapMemoryInMB(100).build(); + public void testEquals() { + ResourceSpec rs1 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); + ResourceSpec rs2 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); assertEquals(ResourceProfile.fromResourceSpec(rs1, 0), ResourceProfile.fromResourceSpec(rs2, 0)); ResourceSpec rs3 = ResourceSpec.newBuilder(). setCpuCores(1.0). - setHeapMemoryInMB(100). + setTaskHeapMemoryMB(100). setGPUResource(2.2). build(); ResourceSpec rs4 = ResourceSpec.newBuilder(). setCpuCores(1.0). - setHeapMemoryInMB(100). + setTaskHeapMemoryMB(100). setGPUResource(1.1). build(); assertNotEquals(ResourceProfile.fromResourceSpec(rs3, 0), ResourceProfile.fromResourceSpec(rs4, 0)); ResourceSpec rs5 = ResourceSpec.newBuilder(). setCpuCores(1.0). - setHeapMemoryInMB(100). + setTaskHeapMemoryMB(100). setGPUResource(2.2). build(); assertEquals(ResourceProfile.fromResourceSpec(rs3, 100), ResourceProfile.fromResourceSpec(rs5, 100)); @@ -127,14 +127,14 @@ public void testEquals() throws Exception { } @Test - public void testCompareTo() throws Exception { - ResourceSpec rs1 = ResourceSpec.newBuilder().setCpuCores(1.0).setHeapMemoryInMB(100).build(); - ResourceSpec rs2 = ResourceSpec.newBuilder().setCpuCores(1.0).setHeapMemoryInMB(100).build(); + public void testCompareTo() { + ResourceSpec rs1 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); + ResourceSpec rs2 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); assertEquals(0, ResourceProfile.fromResourceSpec(rs1, 0).compareTo(ResourceProfile.fromResourceSpec(rs2, 0))); ResourceSpec rs3 = ResourceSpec.newBuilder(). setCpuCores(1.0). - setHeapMemoryInMB(100). + setTaskHeapMemoryMB(100). setGPUResource(2.2). build(); assertEquals(-1, ResourceProfile.fromResourceSpec(rs1, 0).compareTo(ResourceProfile.fromResourceSpec(rs3, 0))); @@ -142,7 +142,7 @@ public void testCompareTo() throws Exception { ResourceSpec rs4 = ResourceSpec.newBuilder(). setCpuCores(1.0). - setHeapMemoryInMB(100). + setTaskHeapMemoryMB(100). setGPUResource(1.1). build(); assertEquals(1, ResourceProfile.fromResourceSpec(rs3, 0).compareTo(ResourceProfile.fromResourceSpec(rs4, 0))); @@ -150,17 +150,17 @@ public void testCompareTo() throws Exception { ResourceSpec rs5 = ResourceSpec.newBuilder(). setCpuCores(1.0). - setHeapMemoryInMB(100). + setTaskHeapMemoryMB(100). setGPUResource(2.2). build(); assertEquals(0, ResourceProfile.fromResourceSpec(rs3, 0).compareTo(ResourceProfile.fromResourceSpec(rs5, 0))); } @Test - public void testGet() throws Exception { + public void testGet() { ResourceSpec rs = ResourceSpec.newBuilder(). setCpuCores(1.0). - setHeapMemoryInMB(100). + setTaskHeapMemoryMB(100). setGPUResource(1.6). build(); ResourceProfile rp = ResourceProfile.fromResourceSpec(rs, 50); 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 ba952ba96cec..795c31e30436 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 @@ -311,7 +311,7 @@ public void testJobSubmissionWithPartialResourceConfigured() throws Exception { DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); - ResourceSpec resourceSpec = ResourceSpec.newBuilder().setCpuCores(2).build(); + ResourceSpec resourceSpec = ResourceSpec.newBuilder().setCpuCores(2).setTaskHeapMemoryMB(0).build(); final JobVertex firstVertex = new JobVertex("firstVertex"); firstVertex.setInvokableClass(NoOpInvokable.class); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java index 215e5485648f..f73e933388b6 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java @@ -547,26 +547,26 @@ public void invoke(Long value) throws Exception { public void testResources() throws Exception{ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - ResourceSpec minResource1 = ResourceSpec.newBuilder().setCpuCores(1.0).setHeapMemoryInMB(100).build(); - ResourceSpec preferredResource1 = ResourceSpec.newBuilder().setCpuCores(2.0).setHeapMemoryInMB(200).build(); + ResourceSpec minResource1 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); + ResourceSpec preferredResource1 = ResourceSpec.newBuilder().setCpuCores(2.0).setTaskHeapMemoryMB(200).build(); - ResourceSpec minResource2 = ResourceSpec.newBuilder().setCpuCores(1.0).setHeapMemoryInMB(200).build(); - ResourceSpec preferredResource2 = ResourceSpec.newBuilder().setCpuCores(2.0).setHeapMemoryInMB(300).build(); + ResourceSpec minResource2 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(200).build(); + ResourceSpec preferredResource2 = ResourceSpec.newBuilder().setCpuCores(2.0).setTaskHeapMemoryMB(300).build(); - ResourceSpec minResource3 = ResourceSpec.newBuilder().setCpuCores(1.0).setHeapMemoryInMB(300).build(); - ResourceSpec preferredResource3 = ResourceSpec.newBuilder().setCpuCores(2.0).setHeapMemoryInMB(400).build(); + ResourceSpec minResource3 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(300).build(); + ResourceSpec preferredResource3 = ResourceSpec.newBuilder().setCpuCores(2.0).setTaskHeapMemoryMB(400).build(); - ResourceSpec minResource4 = ResourceSpec.newBuilder().setCpuCores(1.0).setHeapMemoryInMB(400).build(); - ResourceSpec preferredResource4 = ResourceSpec.newBuilder().setCpuCores(2.0).setHeapMemoryInMB(500).build(); + ResourceSpec minResource4 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(400).build(); + ResourceSpec preferredResource4 = ResourceSpec.newBuilder().setCpuCores(2.0).setTaskHeapMemoryMB(500).build(); - ResourceSpec minResource5 = ResourceSpec.newBuilder().setCpuCores(1.0).setHeapMemoryInMB(500).build(); - ResourceSpec preferredResource5 = ResourceSpec.newBuilder().setCpuCores(2.0).setHeapMemoryInMB(600).build(); + ResourceSpec minResource5 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(500).build(); + ResourceSpec preferredResource5 = ResourceSpec.newBuilder().setCpuCores(2.0).setTaskHeapMemoryMB(600).build(); - ResourceSpec minResource6 = ResourceSpec.newBuilder().setCpuCores(1.0).setHeapMemoryInMB(600).build(); - ResourceSpec preferredResource6 = ResourceSpec.newBuilder().setCpuCores(2.0).setHeapMemoryInMB(700).build(); + ResourceSpec minResource6 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(600).build(); + ResourceSpec preferredResource6 = ResourceSpec.newBuilder().setCpuCores(2.0).setTaskHeapMemoryMB(700).build(); - ResourceSpec minResource7 = ResourceSpec.newBuilder().setCpuCores(1.0).setHeapMemoryInMB(700).build(); - ResourceSpec preferredResource7 = ResourceSpec.newBuilder().setCpuCores(2.0).setHeapMemoryInMB(800).build(); + ResourceSpec minResource7 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(700).build(); + ResourceSpec preferredResource7 = ResourceSpec.newBuilder().setCpuCores(2.0).setTaskHeapMemoryMB(800).build(); Method opMethod = SingleOutputStreamOperator.class.getDeclaredMethod("setResources", ResourceSpec.class, ResourceSpec.class); opMethod.setAccessible(true); 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 64a2e1d599d0..408410cc0d60 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 @@ -213,11 +213,11 @@ public Integer map(Integer value) throws Exception { */ @Test public void testResourcesForChainedSourceSink() throws Exception { - ResourceSpec resource1 = ResourceSpec.newBuilder().setCpuCores(0.1).setHeapMemoryInMB(100).build(); - ResourceSpec resource2 = ResourceSpec.newBuilder().setCpuCores(0.2).setHeapMemoryInMB(200).build(); - ResourceSpec resource3 = ResourceSpec.newBuilder().setCpuCores(0.3).setHeapMemoryInMB(300).build(); - ResourceSpec resource4 = ResourceSpec.newBuilder().setCpuCores(0.4).setHeapMemoryInMB(400).build(); - ResourceSpec resource5 = ResourceSpec.newBuilder().setCpuCores(0.5).setHeapMemoryInMB(500).build(); + ResourceSpec resource1 = ResourceSpec.newBuilder().setCpuCores(0.1).setTaskHeapMemoryMB(100).build(); + ResourceSpec resource2 = ResourceSpec.newBuilder().setCpuCores(0.2).setTaskHeapMemoryMB(200).build(); + ResourceSpec resource3 = ResourceSpec.newBuilder().setCpuCores(0.3).setTaskHeapMemoryMB(300).build(); + ResourceSpec resource4 = ResourceSpec.newBuilder().setCpuCores(0.4).setTaskHeapMemoryMB(400).build(); + ResourceSpec resource5 = ResourceSpec.newBuilder().setCpuCores(0.5).setTaskHeapMemoryMB(500).build(); Method opMethod = SingleOutputStreamOperator.class.getDeclaredMethod("setResources", ResourceSpec.class); opMethod.setAccessible(true); @@ -285,11 +285,11 @@ public void invoke(Tuple2 value) throws Exception { */ @Test public void testResourcesForIteration() throws Exception { - ResourceSpec resource1 = ResourceSpec.newBuilder().setCpuCores(0.1).setHeapMemoryInMB(100).build(); - ResourceSpec resource2 = ResourceSpec.newBuilder().setCpuCores(0.2).setHeapMemoryInMB(200).build(); - ResourceSpec resource3 = ResourceSpec.newBuilder().setCpuCores(0.3).setHeapMemoryInMB(300).build(); - ResourceSpec resource4 = ResourceSpec.newBuilder().setCpuCores(0.4).setHeapMemoryInMB(400).build(); - ResourceSpec resource5 = ResourceSpec.newBuilder().setCpuCores(0.5).setHeapMemoryInMB(500).build(); + ResourceSpec resource1 = ResourceSpec.newBuilder().setCpuCores(0.1).setTaskHeapMemoryMB(100).build(); + ResourceSpec resource2 = ResourceSpec.newBuilder().setCpuCores(0.2).setTaskHeapMemoryMB(200).build(); + ResourceSpec resource3 = ResourceSpec.newBuilder().setCpuCores(0.3).setTaskHeapMemoryMB(300).build(); + ResourceSpec resource4 = ResourceSpec.newBuilder().setCpuCores(0.4).setTaskHeapMemoryMB(400).build(); + ResourceSpec resource5 = ResourceSpec.newBuilder().setCpuCores(0.5).setTaskHeapMemoryMB(500).build(); Method opMethod = SingleOutputStreamOperator.class.getDeclaredMethod("setResources", ResourceSpec.class); opMethod.setAccessible(true); diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/resource/NodeResourceUtil.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/resource/NodeResourceUtil.java index b5e1925c2883..7b2111fc1c9d 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/resource/NodeResourceUtil.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/resource/NodeResourceUtil.java @@ -34,8 +34,9 @@ public class NodeResourceUtil { * Build resourceSpec from managedMem. */ public static ResourceSpec fromManagedMem(int managedMem) { - ResourceSpec.Builder builder = ResourceSpec.newBuilder(); - builder.setManagedMemoryInMB(managedMem); + ResourceSpec.Builder builder = ResourceSpec.newBuilder().setCpuCores(0.0).setTaskHeapMemoryMB(0); + // TODO: before operators separate on-heap/off-heap managed memory, we use on-heap managed memory to denote total managed memory + builder.setOnHeapManagedMemoryMB(managedMem); return builder.build(); } } From b6c2943d39d3a5101f02de128b0328d5a3b97b31 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Mon, 11 Nov 2019 15:29:55 +0100 Subject: [PATCH 563/746] [FLINK-14405][runtime] Update ResourceProfile to align with FLIP-49 resource types --- .../types/ResourceProfile.java | 335 +++++++++++------- .../executiongraph/ExecutionJobVertex.java | 3 +- .../taskexecutor/TaskManagerServices.java | 11 +- ...onPreferenceSlotSelectionStrategyTest.java | 2 +- .../types/ResourceProfileTest.java | 89 +++-- 5 files changed, 262 insertions(+), 178 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java index edd588db1ab4..afc7ee516c73 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java @@ -21,9 +21,10 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.operators.ResourceSpec; import org.apache.flink.api.common.resources.Resource; -import org.apache.flink.util.Preconditions; +import org.apache.flink.configuration.MemorySize; import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.io.Serializable; import java.util.Collections; @@ -64,30 +65,35 @@ public class ResourceProfile implements Serializable, Comparable extendedResources = new HashMap<>(1); @@ -98,46 +104,80 @@ public class ResourceProfile implements Serializable, Comparable extendedResources) { - Preconditions.checkArgument(cpuCores >= 0); - Preconditions.checkArgument(heapMemoryInMB >= 0); - Preconditions.checkArgument(directMemoryInMB >= 0); - Preconditions.checkArgument(nativeMemoryInMB >= 0); - Preconditions.checkArgument(networkMemoryInMB >= 0); - Preconditions.checkArgument(managedMemoryInMB >= 0); this.cpuCores = cpuCores; - this.heapMemoryInMB = heapMemoryInMB; - this.directMemoryInMB = directMemoryInMB; - this.nativeMemoryInMB = nativeMemoryInMB; - this.networkMemoryInMB = networkMemoryInMB; - this.managedMemoryInMB = managedMemoryInMB; + this.taskHeapMemory = taskHeapMemory; + this.taskOffHeapMemory = taskOffHeapMemory; + this.onHeapManagedMemory = onHeapManagedMemory; + this.offHeapManagedMemory = offHeapManagedMemory; + this.shuffleMemory = shuffleMemory; if (extendedResources != null) { this.extendedResources.putAll(extendedResources); } } + /** + * Creates a new ResourceProfile. + * + * @param cpuCores The number of CPU cores (possibly fractional, i.e., 0.2 cores) + * @param taskHeapMemoryMB The size of the task heap memory, in megabytes. + * @param taskOffHeapMemoryMB The size of the task off-heap memory, in megabytes. + * @param onHeapManagedMemoryMB The size of the on-heap managed memory, in megabytes. + * @param offHeapManagedMemoryMB The size of the off-heap managed memory, in megabytes. + * @param shuffleMemoryMB The size of the shuffle memory, in megabytes. + * @param extendedResources The extended resources such as GPU and FPGA + */ + public ResourceProfile( + double cpuCores, + int taskHeapMemoryMB, + int taskOffHeapMemoryMB, + int onHeapManagedMemoryMB, + int offHeapManagedMemoryMB, + int shuffleMemoryMB, + Map extendedResources) { + + this( + cpuCores, + MemorySize.parse(taskHeapMemoryMB + "m"), + MemorySize.parse(taskOffHeapMemoryMB + "m"), + MemorySize.parse(onHeapManagedMemoryMB + "m"), + MemorySize.parse(offHeapManagedMemoryMB + "m"), + MemorySize.parse(shuffleMemoryMB + "m"), + extendedResources); + } + /** * Creates a new simple ResourceProfile used for testing. * * @param cpuCores The number of CPU cores (possibly fractional, i.e., 0.2 cores) - * @param heapMemoryInMB The size of the heap memory, in megabytes. + * @param taskHeapMemory The size of the task heap memory. */ - public ResourceProfile(double cpuCores, int heapMemoryInMB) { - this(cpuCores, heapMemoryInMB, 0, 0, 0, 0, Collections.emptyMap()); + public ResourceProfile(double cpuCores, MemorySize taskHeapMemory) { + this(cpuCores, taskHeapMemory, MemorySize.ZERO, MemorySize.ZERO, MemorySize.ZERO, MemorySize.ZERO, Collections.emptyMap()); + } + + /** + * Creates a new simple ResourceProfile used for testing. + * + * @param cpuCores The number of CPU cores (possibly fractional, i.e., 0.2 cores) + * @param taskHeapMemoryMB The size of the task heap memory, in megabytes. + */ + public ResourceProfile(double cpuCores, int taskHeapMemoryMB) { + this(cpuCores, MemorySize.parse(taskHeapMemoryMB + "m"), MemorySize.ZERO, MemorySize.ZERO, MemorySize.ZERO, MemorySize.ZERO, Collections.emptyMap()); } /** @@ -145,11 +185,11 @@ public ResourceProfile(double cpuCores, int heapMemoryInMB) { */ private ResourceProfile() { this.cpuCores = -1.0; - this.heapMemoryInMB = -1; - this.directMemoryInMB = -1; - this.nativeMemoryInMB = -1; - this.networkMemoryInMB = -1; - this.managedMemoryInMB = -1; + this.taskHeapMemory = null; + this.taskOffHeapMemory = null; + this.onHeapManagedMemory = null; + this.offHeapManagedMemory = null; + this.shuffleMemory = null; } /** @@ -159,12 +199,12 @@ private ResourceProfile() { */ public ResourceProfile(ResourceProfile other) { this(other.cpuCores, - other.heapMemoryInMB, - other.directMemoryInMB, - other.nativeMemoryInMB, - other.networkMemoryInMB, - other.managedMemoryInMB, - other.extendedResources); + other.taskHeapMemory, + other.taskOffHeapMemory, + other.onHeapManagedMemory, + other.offHeapManagedMemory, + other.shuffleMemory, + other.extendedResources); } // ------------------------------------------------------------------------ @@ -175,68 +215,78 @@ public ResourceProfile(ResourceProfile other) { * @return The cpu cores, 1.0 means a full cpu thread */ public double getCpuCores() { + throwUnsupportedOperationExecptionIfUnknown(); return cpuCores; } /** - * Get the heap memory needed in MB. + * Get the task heap memory needed. * - * @return The heap memory in MB + * @return The task heap memory */ - public int getHeapMemoryInMB() { - return heapMemoryInMB; + public MemorySize getTaskHeapMemory() { + throwUnsupportedOperationExecptionIfUnknown(); + return taskHeapMemory; } /** - * Get the direct memory needed in MB. + * Get the task off-heap memory needed. * - * @return The direct memory in MB + * @return The task off-heap memory */ - public int getDirectMemoryInMB() { - return directMemoryInMB; + public MemorySize getTaskOffHeapMemory() { + throwUnsupportedOperationExecptionIfUnknown(); + return taskOffHeapMemory; } /** - * Get the native memory needed in MB. + * Get the on-heap managed memory needed. * - * @return The native memory in MB + * @return The on-heap managed memory */ - public int getNativeMemoryInMB() { - return nativeMemoryInMB; + public MemorySize getOnHeapManagedMemory() { + throwUnsupportedOperationExecptionIfUnknown(); + return onHeapManagedMemory; } - /** - * Get the memory needed for task to communicate with its upstreams and downstreams in MB. - * @return The network memory in MB + /**off + * Get the off-heap managed memory needed. + * + * @return The off-heap managed memory */ - public int getNetworkMemoryInMB() { - return networkMemoryInMB; + public MemorySize getOffHeapManagedMemory() { + throwUnsupportedOperationExecptionIfUnknown(); + return offHeapManagedMemory; } /** - * Get the managed memory needed in MB. - * @return The managed memory in MB. + * Get the shuffle memory needed. + * + * @return The shuffle memory */ - public int getManagedMemoryInMB() { - return managedMemoryInMB; + public MemorySize getShuffleMemory() { + throwUnsupportedOperationExecptionIfUnknown(); + return shuffleMemory; } /** - * Get the total memory needed in MB. + * Get the total memory needed. * - * @return The total memory in MB + * @return The total memory */ - public int getMemoryInMB() { - return heapMemoryInMB + directMemoryInMB + nativeMemoryInMB + networkMemoryInMB + managedMemoryInMB; + public MemorySize getTotalMemory() { + throwUnsupportedOperationExecptionIfUnknown(); + return getOperatorsMemory().add(shuffleMemory); } /** - * Get the memory the operators needed in MB. + * Get the memory the operators needed. * - * @return The operator memory in MB + * @return The operator memory */ - public int getOperatorsMemoryInMB() { - return heapMemoryInMB + directMemoryInMB + nativeMemoryInMB + managedMemoryInMB; + public MemorySize getOperatorsMemory() { + throwUnsupportedOperationExecptionIfUnknown(); + return taskHeapMemory.add(taskOffHeapMemory).add(onHeapManagedMemory).add(offHeapManagedMemory); } /** @@ -245,9 +295,16 @@ public int getOperatorsMemoryInMB() { * @return The extended resources */ public Map getExtendedResources() { + throwUnsupportedOperationExecptionIfUnknown(); return Collections.unmodifiableMap(extendedResources); } + private void throwUnsupportedOperationExecptionIfUnknown() { + if (this.equals(UNKNOWN)) { + throw new UnsupportedOperationException(); + } + } + /** * Check whether required resource profile can be matched. * @@ -273,11 +330,11 @@ public boolean isMatching(ResourceProfile required) { } if (cpuCores >= required.getCpuCores() && - heapMemoryInMB >= required.getHeapMemoryInMB() && - directMemoryInMB >= required.getDirectMemoryInMB() && - nativeMemoryInMB >= required.getNativeMemoryInMB() && - networkMemoryInMB >= required.getNetworkMemoryInMB() && - managedMemoryInMB >= required.getManagedMemoryInMB()) { + taskHeapMemory.getBytes() >= required.taskHeapMemory.getBytes() && + taskOffHeapMemory.getBytes() >= required.taskOffHeapMemory.getBytes() && + onHeapManagedMemory.getBytes() >= required.onHeapManagedMemory.getBytes() && + offHeapManagedMemory.getBytes() >= required.offHeapManagedMemory.getBytes() && + shuffleMemory.getBytes() >= required.shuffleMemory.getBytes()) { for (Map.Entry resource : required.extendedResources.entrySet()) { if (!extendedResources.containsKey(resource.getKey()) || !extendedResources.get(resource.getKey()).getResourceAggregateType().equals(resource.getValue().getResourceAggregateType()) || @@ -292,7 +349,15 @@ public boolean isMatching(ResourceProfile required) { @Override public int compareTo(@Nonnull ResourceProfile other) { - int cmp = Integer.compare(this.getMemoryInMB(), other.getMemoryInMB()); + if (this == other) { + return 0; + } else if (this.equals(UNKNOWN)) { + return -1; + } else if (other.equals(UNKNOWN)) { + return 1; + } + + int cmp = this.getTotalMemory().compareTo(other.getTotalMemory()); if (cmp == 0) { cmp = Double.compare(this.cpuCores, other.cpuCores); } @@ -328,11 +393,11 @@ public int compareTo(@Nonnull ResourceProfile other) { public int hashCode() { final long cpuBits = Double.doubleToLongBits(cpuCores); int result = (int) (cpuBits ^ (cpuBits >>> 32)); - result = 31 * result + heapMemoryInMB; - result = 31 * result + directMemoryInMB; - result = 31 * result + nativeMemoryInMB; - result = 31 * result + networkMemoryInMB; - result = 31 * result + managedMemoryInMB; + result = 31 * result + Objects.hashCode(taskHeapMemory); + result = 31 * result + Objects.hashCode(taskOffHeapMemory); + result = 31 * result + Objects.hashCode(onHeapManagedMemory); + result = 31 * result + Objects.hashCode(offHeapManagedMemory); + result = 31 * result + Objects.hashCode(shuffleMemory); result = 31 * result + extendedResources.hashCode(); return result; } @@ -341,16 +406,15 @@ public int hashCode() { public boolean equals(Object obj) { if (obj == this) { return true; - } - else if (obj != null && obj.getClass() == ResourceProfile.class) { + } else if (obj != null && obj.getClass() == ResourceProfile.class) { ResourceProfile that = (ResourceProfile) obj; return this.cpuCores == that.cpuCores && - this.heapMemoryInMB == that.heapMemoryInMB && - this.directMemoryInMB == that.directMemoryInMB && - this.nativeMemoryInMB == that.nativeMemoryInMB && - this.networkMemoryInMB == that.networkMemoryInMB && - this.managedMemoryInMB == that.managedMemoryInMB && - Objects.equals(extendedResources, that.extendedResources); + Objects.equals(taskHeapMemory, that.taskHeapMemory) && + Objects.equals(taskOffHeapMemory, that.taskOffHeapMemory) && + Objects.equals(onHeapManagedMemory, that.onHeapManagedMemory) && + Objects.equals(offHeapManagedMemory, that.offHeapManagedMemory) && + Objects.equals(shuffleMemory, that.shuffleMemory) && + Objects.equals(extendedResources, that.extendedResources); } return false; } @@ -380,11 +444,11 @@ public ResourceProfile merge(@Nonnull ResourceProfile other) { return new ResourceProfile( addNonNegativeDoublesConsideringOverflow(cpuCores, other.cpuCores), - addNonNegativeIntegersConsideringOverflow(heapMemoryInMB, other.heapMemoryInMB), - addNonNegativeIntegersConsideringOverflow(directMemoryInMB, other.directMemoryInMB), - addNonNegativeIntegersConsideringOverflow(nativeMemoryInMB, other.nativeMemoryInMB), - addNonNegativeIntegersConsideringOverflow(networkMemoryInMB, other.networkMemoryInMB), - addNonNegativeIntegersConsideringOverflow(managedMemoryInMB, other.managedMemoryInMB), + taskHeapMemory.add(other.taskHeapMemory), + taskOffHeapMemory.add(other.taskOffHeapMemory), + onHeapManagedMemory.add(other.onHeapManagedMemory), + offHeapManagedMemory.add(other.offHeapManagedMemory), + shuffleMemory.add(other.shuffleMemory), resultExtendedResource); } @@ -416,11 +480,11 @@ public ResourceProfile subtract(ResourceProfile other) { return new ResourceProfile( subtractDoublesConsideringInf(cpuCores, other.cpuCores), - subtractIntegersConsideringInf(heapMemoryInMB, other.heapMemoryInMB), - subtractIntegersConsideringInf(directMemoryInMB, other.directMemoryInMB), - subtractIntegersConsideringInf(nativeMemoryInMB, other.nativeMemoryInMB), - subtractIntegersConsideringInf(networkMemoryInMB, other.networkMemoryInMB), - subtractIntegersConsideringInf(managedMemoryInMB, other.managedMemoryInMB), + taskHeapMemory.subtract(other.taskHeapMemory), + taskOffHeapMemory.subtract(other.taskOffHeapMemory), + onHeapManagedMemory.subtract(other.onHeapManagedMemory), + offHeapManagedMemory.subtract(other.offHeapManagedMemory), + shuffleMemory.subtract(other.shuffleMemory), resultExtendedResource ); } @@ -428,44 +492,44 @@ public ResourceProfile subtract(ResourceProfile other) { private double addNonNegativeDoublesConsideringOverflow(double first, double second) { double result = first + second; - if (result == Double.POSITIVE_INFINITY) { - return Double.MAX_VALUE; + if (Double.isInfinite(result)) { + throw new ArithmeticException("double overflow"); } return result; } - private int addNonNegativeIntegersConsideringOverflow(int first, int second) { - int result = first + second; + private double subtractDoublesConsideringInf(double first, double second) { + double result = first - second; - if (result < 0) { - return Integer.MAX_VALUE; + if (Double.isInfinite(result)) { + throw new ArithmeticException("double overflow"); } return result; } - private double subtractDoublesConsideringInf(double first, double second) { - return first == Double.MAX_VALUE ? Double.MAX_VALUE : first - second; - } - - private int subtractIntegersConsideringInf(int first, int second) { - return first == Integer.MAX_VALUE ? Integer.MAX_VALUE : first - second; - } - @Override public String toString() { + if (this.equals(UNKNOWN)) { + return "ResourceProfile{UNKNOWN}"; + } + + if (this.equals(ANY)) { + return "ResourceProfile{ANY}"; + } + final StringBuilder resources = new StringBuilder(extendedResources.size() * 10); for (Map.Entry resource : extendedResources.entrySet()) { resources.append(", ").append(resource.getKey()).append('=').append(resource.getValue()); } return "ResourceProfile{" + "cpuCores=" + cpuCores + - ", heapMemoryInMB=" + heapMemoryInMB + - ", directMemoryInMB=" + directMemoryInMB + - ", nativeMemoryInMB=" + nativeMemoryInMB + - ", networkMemoryInMB=" + networkMemoryInMB + - ", managedMemoryInMB=" + managedMemoryInMB + resources + + ", taskHeapMemory=" + taskHeapMemory + + ", taskOffHeapMemory=" + taskOffHeapMemory + + ", onHeapManagedMemory=" + onHeapManagedMemory + + ", offHeapManagedMemory=" + offHeapManagedMemory + + ", shuffleMemory=" + shuffleMemory + resources + '}'; } @@ -482,7 +546,12 @@ private Object readResolve() { // factories // ------------------------------------------------------------------------ - public static ResourceProfile fromResourceSpec(ResourceSpec resourceSpec, int networkMemory) { + @VisibleForTesting + static ResourceProfile fromResourceSpec(ResourceSpec resourceSpec) { + return fromResourceSpec(resourceSpec, MemorySize.ZERO); + } + + public static ResourceProfile fromResourceSpec(ResourceSpec resourceSpec, MemorySize networkMemory) { if (ResourceSpec.UNKNOWN.equals(resourceSpec)) { return UNKNOWN; } @@ -490,12 +559,12 @@ public static ResourceProfile fromResourceSpec(ResourceSpec resourceSpec, int ne Map copiedExtendedResources = new HashMap<>(resourceSpec.getExtendedResources()); return new ResourceProfile( - resourceSpec.getCpuCores(), - resourceSpec.getHeapMemory(), - resourceSpec.getDirectMemory(), - resourceSpec.getNativeMemory(), - networkMemory, - resourceSpec.getManagedMemory(), - copiedExtendedResources); + resourceSpec.getCpuCores(), + resourceSpec.getTaskHeapMemory(), + resourceSpec.getTaskOffHeapMemory(), + resourceSpec.getOnHeapManagedMemory(), + resourceSpec.getOffHeapManagedMemory(), + networkMemory, + copiedExtendedResources); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java index 1a0d389e1f6a..058c1667d198 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java @@ -27,6 +27,7 @@ import org.apache.flink.api.common.accumulators.AccumulatorHelper; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.MemorySize; import org.apache.flink.core.io.InputSplit; import org.apache.flink.core.io.InputSplitAssigner; import org.apache.flink.core.io.InputSplitSource; @@ -192,7 +193,7 @@ public ExecutionJobVertex( } this.parallelism = numTaskVertices; - this.resourceProfile = ResourceProfile.fromResourceSpec(jobVertex.getMinResources(), 0); + this.resourceProfile = ResourceProfile.fromResourceSpec(jobVertex.getMinResources(), MemorySize.ZERO); this.taskVertices = new ExecutionVertex[numTaskVertices]; this.operatorIDs = Collections.unmodifiableList(jobVertex.getOperatorIDs()); 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 d91431181921..119146b48a34 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 @@ -493,11 +493,12 @@ public static ResourceProfile computeSlotResourceProfile(int numOfSlots, long ma int managedMemoryPerSlotMB = (int) bytesToMegabytes(managedMemorySize / numOfSlots); return new ResourceProfile( Double.MAX_VALUE, - Integer.MAX_VALUE, - Integer.MAX_VALUE, - Integer.MAX_VALUE, - Integer.MAX_VALUE, - managedMemoryPerSlotMB, + MemorySize.MAX_VALUE, + MemorySize.MAX_VALUE, + // TODO: before operators separate on-heap/off-heap managed memory, we use on-heap managed memory to denote total managed memory + MemorySize.parse(managedMemoryPerSlotMB + "m"), + MemorySize.MAX_VALUE, + MemorySize.MAX_VALUE, Collections.emptyMap()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/LocationPreferenceSlotSelectionStrategyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/LocationPreferenceSlotSelectionStrategyTest.java index 148d81842678..05c5960d6558 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/LocationPreferenceSlotSelectionStrategyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/LocationPreferenceSlotSelectionStrategyTest.java @@ -49,7 +49,7 @@ public void testResourceProfileRespected() { Assert.assertTrue(match.get().getSlotInfo().getResourceProfile().isMatching(slotProfile.getResourceProfile())); ResourceProfile evenBiggerResourceProfile = - new ResourceProfile(biggerResourceProfile.getCpuCores() + 1, resourceProfile.getHeapMemoryInMB()); + new ResourceProfile(biggerResourceProfile.getCpuCores() + 1, resourceProfile.getTaskHeapMemory()); slotProfile = new SlotProfile(evenBiggerResourceProfile, Collections.emptyList(), Collections.emptySet()); match = runMatching(slotProfile); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java index b58f7a521bfc..78e948b2b087 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java @@ -20,11 +20,14 @@ import org.apache.flink.api.common.operators.ResourceSpec; import org.apache.flink.api.common.resources.GPUResource; +import org.apache.flink.configuration.MemorySize; import org.apache.flink.core.testutils.CommonTestUtils; import org.junit.Test; +import java.util.ArrayList; import java.util.Collections; +import java.util.List; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -73,9 +76,9 @@ public void testMatchRequirement() { setGPUResource(1.1). build(); - assertFalse(rp1.isMatching(ResourceProfile.fromResourceSpec(rs1, 0))); - assertTrue(ResourceProfile.fromResourceSpec(rs1, 0).isMatching(ResourceProfile.fromResourceSpec(rs2, 0))); - assertFalse(ResourceProfile.fromResourceSpec(rs2, 0).isMatching(ResourceProfile.fromResourceSpec(rs1, 0))); + assertFalse(rp1.isMatching(ResourceProfile.fromResourceSpec(rs1))); + assertTrue(ResourceProfile.fromResourceSpec(rs1).isMatching(ResourceProfile.fromResourceSpec(rs2))); + assertFalse(ResourceProfile.fromResourceSpec(rs2).isMatching(ResourceProfile.fromResourceSpec(rs1))); } @Test @@ -87,7 +90,7 @@ public void testUnknownMatchesUnknown() { public void testEquals() { ResourceSpec rs1 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); ResourceSpec rs2 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); - assertEquals(ResourceProfile.fromResourceSpec(rs1, 0), ResourceProfile.fromResourceSpec(rs2, 0)); + assertEquals(ResourceProfile.fromResourceSpec(rs1), ResourceProfile.fromResourceSpec(rs2)); ResourceSpec rs3 = ResourceSpec.newBuilder(). setCpuCores(1.0). @@ -99,14 +102,15 @@ public void testEquals() { setTaskHeapMemoryMB(100). setGPUResource(1.1). build(); - assertNotEquals(ResourceProfile.fromResourceSpec(rs3, 0), ResourceProfile.fromResourceSpec(rs4, 0)); + assertNotEquals(ResourceProfile.fromResourceSpec(rs3), ResourceProfile.fromResourceSpec(rs4)); ResourceSpec rs5 = ResourceSpec.newBuilder(). setCpuCores(1.0). setTaskHeapMemoryMB(100). setGPUResource(2.2). build(); - assertEquals(ResourceProfile.fromResourceSpec(rs3, 100), ResourceProfile.fromResourceSpec(rs5, 100)); + MemorySize networkMemory = MemorySize.parse(100 + "m"); + assertEquals(ResourceProfile.fromResourceSpec(rs3, networkMemory), ResourceProfile.fromResourceSpec(rs5, networkMemory)); ResourceProfile rp1 = new ResourceProfile(1.0, 100, 100, 100, 100, 100, Collections.emptyMap()); ResourceProfile rp2 = new ResourceProfile(1.1, 100, 100, 100, 100, 100, Collections.emptyMap()); @@ -130,30 +134,30 @@ public void testEquals() { public void testCompareTo() { ResourceSpec rs1 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); ResourceSpec rs2 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); - assertEquals(0, ResourceProfile.fromResourceSpec(rs1, 0).compareTo(ResourceProfile.fromResourceSpec(rs2, 0))); + assertEquals(0, ResourceProfile.fromResourceSpec(rs1).compareTo(ResourceProfile.fromResourceSpec(rs2))); ResourceSpec rs3 = ResourceSpec.newBuilder(). setCpuCores(1.0). setTaskHeapMemoryMB(100). setGPUResource(2.2). build(); - assertEquals(-1, ResourceProfile.fromResourceSpec(rs1, 0).compareTo(ResourceProfile.fromResourceSpec(rs3, 0))); - assertEquals(1, ResourceProfile.fromResourceSpec(rs3, 0).compareTo(ResourceProfile.fromResourceSpec(rs1, 0))); + assertEquals(-1, ResourceProfile.fromResourceSpec(rs1).compareTo(ResourceProfile.fromResourceSpec(rs3))); + assertEquals(1, ResourceProfile.fromResourceSpec(rs3).compareTo(ResourceProfile.fromResourceSpec(rs1))); ResourceSpec rs4 = ResourceSpec.newBuilder(). setCpuCores(1.0). setTaskHeapMemoryMB(100). setGPUResource(1.1). build(); - assertEquals(1, ResourceProfile.fromResourceSpec(rs3, 0).compareTo(ResourceProfile.fromResourceSpec(rs4, 0))); - assertEquals(-1, ResourceProfile.fromResourceSpec(rs4, 0).compareTo(ResourceProfile.fromResourceSpec(rs3, 0))); + assertEquals(1, ResourceProfile.fromResourceSpec(rs3).compareTo(ResourceProfile.fromResourceSpec(rs4))); + assertEquals(-1, ResourceProfile.fromResourceSpec(rs4).compareTo(ResourceProfile.fromResourceSpec(rs3))); ResourceSpec rs5 = ResourceSpec.newBuilder(). setCpuCores(1.0). setTaskHeapMemoryMB(100). setGPUResource(2.2). build(); - assertEquals(0, ResourceProfile.fromResourceSpec(rs3, 0).compareTo(ResourceProfile.fromResourceSpec(rs5, 0))); + assertEquals(0, ResourceProfile.fromResourceSpec(rs3).compareTo(ResourceProfile.fromResourceSpec(rs5))); } @Test @@ -163,26 +167,26 @@ public void testGet() { setTaskHeapMemoryMB(100). setGPUResource(1.6). build(); - ResourceProfile rp = ResourceProfile.fromResourceSpec(rs, 50); + ResourceProfile rp = ResourceProfile.fromResourceSpec(rs, MemorySize.parse(50 + "m")); assertEquals(1.0, rp.getCpuCores(), 0.000001); - assertEquals(150, rp.getMemoryInMB()); - assertEquals(100, rp.getOperatorsMemoryInMB()); + assertEquals(150, rp.getTotalMemory().getMebiBytes()); + assertEquals(100, rp.getOperatorsMemory().getMebiBytes()); assertEquals(1.6, rp.getExtendedResources().get(ResourceSpec.GPU_NAME).getValue(), 0.000001); } @Test - public void testMerge() throws Exception { + public void testMerge() { ResourceProfile rp1 = new ResourceProfile(1.0, 100, 100, 100, 100, 100, Collections.emptyMap()); ResourceProfile rp2 = new ResourceProfile(2.0, 200, 200, 200, 200, 200, - Collections.singletonMap("gpu", new GPUResource(2.0))); + Collections.singletonMap("gpu", new GPUResource(2.0))); ResourceProfile rp1MergeRp1 = new ResourceProfile(2.0, 200, 200, 200, 200, 200, - Collections.emptyMap()); + Collections.emptyMap()); ResourceProfile rp1MergeRp2 = new ResourceProfile(3.0, 300, 300, 300, 300, 300, - Collections.singletonMap("gpu", new GPUResource(2.0))); + Collections.singletonMap("gpu", new GPUResource(2.0))); ResourceProfile rp2MergeRp2 = new ResourceProfile(4.0, 400, 400, 400, 400, 400, - Collections.singletonMap("gpu", new GPUResource(4.0))); + Collections.singletonMap("gpu", new GPUResource(4.0))); assertEquals(rp1MergeRp1, rp1.merge(rp1)); assertEquals(rp1MergeRp2, rp1.merge(rp2)); @@ -198,20 +202,34 @@ public void testMerge() throws Exception { } @Test - public void testMergeWithOverflow() throws Exception { + public void testMergeWithOverflow() { final double largeDouble = Double.MAX_VALUE - 1.0; - final int largeInteger = Integer.MAX_VALUE - 100; + final MemorySize largeMemory = MemorySize.MAX_VALUE.subtract(MemorySize.parse("100m")); ResourceProfile rp1 = new ResourceProfile(3.0, 300, 300, 300, 300, 300, Collections.emptyMap()); - ResourceProfile rp2 = new ResourceProfile(largeDouble, largeInteger, largeInteger, largeInteger, largeInteger, largeInteger, Collections.emptyMap()); + ResourceProfile rp2 = new ResourceProfile(largeDouble, largeMemory, largeMemory, largeMemory, largeMemory, largeMemory, Collections.emptyMap()); - assertEquals(ResourceProfile.ANY, rp2.merge(rp2)); - assertEquals(ResourceProfile.ANY, rp2.merge(rp1)); - assertEquals(ResourceProfile.ANY, rp1.merge(rp2)); + List exceptions = new ArrayList<>(); + try { + rp2.merge(rp2); + } catch (ArithmeticException e) { + exceptions.add(e); + } + try { + rp2.merge(rp1); + } catch (ArithmeticException e) { + exceptions.add(e); + } + try { + rp1.merge(rp2); + } catch (ArithmeticException e) { + exceptions.add(e); + } + assertEquals(3, exceptions.size()); } @Test - public void testSubtract() throws Exception { + public void testSubtract() { ResourceProfile rp1 = new ResourceProfile(1.0, 100, 100, 100, 100, 100, Collections.emptyMap()); ResourceProfile rp2 = new ResourceProfile(2.0, 200, 200, 200, 200, 200, Collections.emptyMap()); ResourceProfile rp3 = new ResourceProfile(3.0, 300, 300, 300, 300, 300, Collections.emptyMap()); @@ -219,11 +237,6 @@ public void testSubtract() throws Exception { assertEquals(rp1, rp3.subtract(rp2)); assertEquals(rp1, rp2.subtract(rp1)); - ResourceProfile rp4 = new ResourceProfile(Double.MAX_VALUE, 100, Integer.MAX_VALUE, Integer.MAX_VALUE, Integer.MAX_VALUE, Integer.MAX_VALUE, Collections.emptyMap()); - ResourceProfile rp5 = new ResourceProfile(Double.MAX_VALUE, 0, Integer.MAX_VALUE, Integer.MAX_VALUE, Integer.MAX_VALUE, Integer.MAX_VALUE, Collections.emptyMap()); - - assertEquals(rp5, rp4.subtract(rp1)); - try { rp1.subtract(rp2); fail("The subtract should failed due to trying to subtract a larger resource"); @@ -240,23 +253,23 @@ public void testSubtract() throws Exception { assertEquals(ResourceProfile.UNKNOWN, ResourceProfile.UNKNOWN.subtract(ResourceProfile.UNKNOWN)); } - @Test + @Test(expected = IllegalArgumentException.class) public void testSubtractWithInfValues() { // Does not equals to ANY since it has extended resources. ResourceProfile rp1 = new ResourceProfile(Double.MAX_VALUE, Integer.MAX_VALUE, Integer.MAX_VALUE, Integer.MAX_VALUE, Integer.MAX_VALUE, - Integer.MAX_VALUE, Collections.singletonMap("gpu", new GPUResource(4.0))); + Integer.MAX_VALUE, Collections.singletonMap("gpu", new GPUResource(4.0))); ResourceProfile rp2 = new ResourceProfile(2.0, 200, 200, 200, 200, 200, - Collections.emptyMap()); + Collections.emptyMap()); - assertEquals(rp1, rp1.subtract(rp2)); + rp2.subtract(rp1); } @Test public void testFromSpecWithSerializationCopy() throws Exception { final ResourceSpec copiedSpec = CommonTestUtils.createCopySerializable(ResourceSpec.UNKNOWN); - final ResourceProfile profile = ResourceProfile.fromResourceSpec(copiedSpec, 0); + final ResourceProfile profile = ResourceProfile.fromResourceSpec(copiedSpec); - assertEquals(ResourceProfile.fromResourceSpec(ResourceSpec.UNKNOWN, 0), profile); + assertEquals(ResourceProfile.fromResourceSpec(ResourceSpec.UNKNOWN), profile); } @Test From 001733b411a1e8cad8c61cb6bc90f9afcfed42d1 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Fri, 18 Oct 2019 12:24:41 +0800 Subject: [PATCH 564/746] [FLINK-14495][core] Limit ResourceSpec to always specify cpu cores and task heap memory size, unless it UNKNOWN. --- .../api/common/operators/ResourceSpec.java | 17 +++-- .../common/operators/ResourceSpecTest.java | 72 ++++++------------- .../flink/api/java/operator/OperatorTest.java | 4 +- .../plantranslate/JobGraphGeneratorTest.java | 26 +++---- .../types/ResourceProfileTest.java | 44 ++++-------- .../runtime/dispatcher/DispatcherTest.java | 2 +- .../flink/streaming/api/DataStreamTest.java | 28 ++++---- .../graph/StreamingJobGraphGeneratorTest.java | 20 +++--- .../plan/nodes/resource/NodeResourceUtil.java | 2 +- 9 files changed, 89 insertions(+), 126 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/ResourceSpec.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/ResourceSpec.java index 3d2b4fa95339..bca130190e81 100755 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/ResourceSpec.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/ResourceSpec.java @@ -338,8 +338,12 @@ private Object readResolve() { // builder // ------------------------------------------------------------------------ - public static Builder newBuilder() { - return new Builder(); + public static Builder newBuilder(double cpuCores, MemorySize taskHeapMemory) { + return new Builder(cpuCores, taskHeapMemory); + } + + public static Builder newBuilder(double cpuCores, int taskHeapMemoryMB) { + return newBuilder(cpuCores, MemorySize.parse(taskHeapMemoryMB + "m")); } /** @@ -347,13 +351,18 @@ public static Builder newBuilder() { */ public static class Builder { - private double cpuCores = 0.0; - private MemorySize taskHeapMemory = MemorySize.ZERO; + private double cpuCores; + private MemorySize taskHeapMemory; private MemorySize taskOffHeapMemory = MemorySize.ZERO; private MemorySize onHeapManagedMemory = MemorySize.ZERO; private MemorySize offHeapManagedMemory = MemorySize.ZERO; private GPUResource gpuResource; + private Builder(double cpuCores, MemorySize taskHeapMemory) { + this.cpuCores = cpuCores; + this.taskHeapMemory = taskHeapMemory; + } + public Builder setCpuCores(double cpuCores) { this.cpuCores = cpuCores; return this; diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/ResourceSpecTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/ResourceSpecTest.java index 1dc25ead571e..afa0663b0874 100755 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/ResourceSpecTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/ResourceSpecTest.java @@ -36,19 +36,15 @@ public class ResourceSpecTest extends TestLogger { @Test public void testIsValid() throws Exception { - ResourceSpec rs = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); + ResourceSpec rs = ResourceSpec.newBuilder(1.0, 100).build(); assertTrue(rs.isValid()); - rs = ResourceSpec.newBuilder(). - setCpuCores(1.0). - setTaskHeapMemoryMB(100). + rs = ResourceSpec.newBuilder(1.0, 100). setGPUResource(1). build(); assertTrue(rs.isValid()); - rs = ResourceSpec.newBuilder(). - setCpuCores(1.0). - setTaskHeapMemoryMB(100). + rs = ResourceSpec.newBuilder(1.0, 100). setGPUResource(-1). build(); assertFalse(rs.isValid()); @@ -56,22 +52,18 @@ public void testIsValid() throws Exception { @Test public void testLessThanOrEqual() throws Exception { - ResourceSpec rs1 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); - ResourceSpec rs2 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); + ResourceSpec rs1 = ResourceSpec.newBuilder(1.0, 100).build(); + ResourceSpec rs2 = ResourceSpec.newBuilder(1.0, 100).build(); assertTrue(rs1.lessThanOrEqual(rs2)); assertTrue(rs2.lessThanOrEqual(rs1)); - ResourceSpec rs3 = ResourceSpec.newBuilder(). - setCpuCores(1.0). - setTaskHeapMemoryMB(100). + ResourceSpec rs3 = ResourceSpec.newBuilder(1.0, 100). setGPUResource(1.1). build(); assertTrue(rs1.lessThanOrEqual(rs3)); assertFalse(rs3.lessThanOrEqual(rs1)); - ResourceSpec rs4 = ResourceSpec.newBuilder(). - setCpuCores(1.0). - setTaskHeapMemoryMB(100). + ResourceSpec rs4 = ResourceSpec.newBuilder(1.0, 100). setGPUResource(2.2). build(); assertFalse(rs4.lessThanOrEqual(rs3)); @@ -80,26 +72,20 @@ public void testLessThanOrEqual() throws Exception { @Test public void testEquals() throws Exception { - ResourceSpec rs1 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); - ResourceSpec rs2 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); + ResourceSpec rs1 = ResourceSpec.newBuilder(1.0, 100).build(); + ResourceSpec rs2 = ResourceSpec.newBuilder(1.0, 100).build(); assertEquals(rs1, rs2); assertEquals(rs2, rs1); - ResourceSpec rs3 = ResourceSpec.newBuilder(). - setCpuCores(1.0). - setTaskHeapMemoryMB(100). + ResourceSpec rs3 = ResourceSpec.newBuilder(1.0, 100). setGPUResource(2.2). build(); - ResourceSpec rs4 = ResourceSpec.newBuilder(). - setCpuCores(1.0). - setTaskHeapMemoryMB(100). + ResourceSpec rs4 = ResourceSpec.newBuilder(1.0, 100). setGPUResource(1). build(); assertNotEquals(rs3, rs4); - ResourceSpec rs5 = ResourceSpec.newBuilder(). - setCpuCores(1.0). - setTaskHeapMemoryMB(100). + ResourceSpec rs5 = ResourceSpec.newBuilder(1.0, 100). setGPUResource(2.2). build(); assertEquals(rs3, rs5); @@ -107,25 +93,19 @@ public void testEquals() throws Exception { @Test public void testHashCode() throws Exception { - ResourceSpec rs1 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); - ResourceSpec rs2 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); + ResourceSpec rs1 = ResourceSpec.newBuilder(1.0, 100).build(); + ResourceSpec rs2 = ResourceSpec.newBuilder(1.0, 100).build(); assertEquals(rs1.hashCode(), rs2.hashCode()); - ResourceSpec rs3 = ResourceSpec.newBuilder(). - setCpuCores(1.0). - setTaskHeapMemoryMB(100). + ResourceSpec rs3 = ResourceSpec.newBuilder(1.0, 100). setGPUResource(2.2). build(); - ResourceSpec rs4 = ResourceSpec.newBuilder(). - setCpuCores(1.0). - setTaskHeapMemoryMB(100). + ResourceSpec rs4 = ResourceSpec.newBuilder(1.0, 100). setGPUResource(1). build(); assertNotEquals(rs3.hashCode(), rs4.hashCode()); - ResourceSpec rs5 = ResourceSpec.newBuilder(). - setCpuCores(1.0). - setTaskHeapMemoryMB(100). + ResourceSpec rs5 = ResourceSpec.newBuilder(1.0, 100). setGPUResource(2.2). build(); assertEquals(rs3.hashCode(), rs5.hashCode()); @@ -133,12 +113,10 @@ public void testHashCode() throws Exception { @Test public void testMerge() throws Exception { - ResourceSpec rs1 = ResourceSpec.newBuilder(). - setCpuCores(1.0). - setTaskHeapMemoryMB(100). + ResourceSpec rs1 = ResourceSpec.newBuilder(1.0, 100). setGPUResource(1.1). build(); - ResourceSpec rs2 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); + ResourceSpec rs2 = ResourceSpec.newBuilder(1.0, 100).build(); ResourceSpec rs3 = rs1.merge(rs2); assertEquals(2.0, rs3.getCpuCores(), 0.000001); @@ -151,9 +129,7 @@ public void testMerge() throws Exception { @Test public void testSerializable() throws Exception { - ResourceSpec rs1 = ResourceSpec.newBuilder(). - setCpuCores(1.0). - setTaskHeapMemoryMB(100). + ResourceSpec rs1 = ResourceSpec.newBuilder(1.0, 100). setGPUResource(1.1). build(); @@ -164,9 +140,7 @@ public void testSerializable() throws Exception { @Test public void testMergeThisUnknown() throws Exception { final ResourceSpec spec1 = ResourceSpec.UNKNOWN; - final ResourceSpec spec2 = ResourceSpec.newBuilder() - .setCpuCores(1.0) - .setTaskHeapMemoryMB(100) + final ResourceSpec spec2 = ResourceSpec.newBuilder(1.0, 100) .setGPUResource(1.1) .build(); @@ -177,9 +151,7 @@ public void testMergeThisUnknown() throws Exception { @Test public void testMergeOtherUnknown() throws Exception { - final ResourceSpec spec1 = ResourceSpec.newBuilder() - .setCpuCores(1.0) - .setTaskHeapMemoryMB(100) + final ResourceSpec spec1 = ResourceSpec.newBuilder(1.0, 100) .setGPUResource(1.1) .build(); final ResourceSpec spec2 = ResourceSpec.UNKNOWN; diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operator/OperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operator/OperatorTest.java index f01f1babec7c..45c7e979f357 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/operator/OperatorTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/operator/OperatorTest.java @@ -60,8 +60,8 @@ public void testConfigurationOfResource() throws Exception{ opMethod.setAccessible(true); // verify explicit change in resources - ResourceSpec minResources = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); - ResourceSpec preferredResources = ResourceSpec.newBuilder().setCpuCores(2.0).setTaskHeapMemoryMB(200).build(); + ResourceSpec minResources = ResourceSpec.newBuilder(1.0, 100).build(); + ResourceSpec preferredResources = ResourceSpec.newBuilder(2.0, 200).build(); opMethod.invoke(operator, minResources, preferredResources); assertEquals(minResources, operator.getMinResources()); diff --git a/flink-optimizer/src/test/java/org/apache/flink/optimizer/plantranslate/JobGraphGeneratorTest.java b/flink-optimizer/src/test/java/org/apache/flink/optimizer/plantranslate/JobGraphGeneratorTest.java index 5b2c6617e142..bb483c37f05f 100644 --- a/flink-optimizer/src/test/java/org/apache/flink/optimizer/plantranslate/JobGraphGeneratorTest.java +++ b/flink-optimizer/src/test/java/org/apache/flink/optimizer/plantranslate/JobGraphGeneratorTest.java @@ -73,13 +73,13 @@ public class JobGraphGeneratorTest { */ @Test public void testResourcesForChainedOperators() throws Exception { - ResourceSpec resource1 = ResourceSpec.newBuilder().setCpuCores(0.1).setTaskHeapMemoryMB(100).build(); - ResourceSpec resource2 = ResourceSpec.newBuilder().setCpuCores(0.2).setTaskHeapMemoryMB(200).build(); - ResourceSpec resource3 = ResourceSpec.newBuilder().setCpuCores(0.3).setTaskHeapMemoryMB(300).build(); - ResourceSpec resource4 = ResourceSpec.newBuilder().setCpuCores(0.4).setTaskHeapMemoryMB(400).build(); - ResourceSpec resource5 = ResourceSpec.newBuilder().setCpuCores(0.5).setTaskHeapMemoryMB(500).build(); - ResourceSpec resource6 = ResourceSpec.newBuilder().setCpuCores(0.6).setTaskHeapMemoryMB(600).build(); - ResourceSpec resource7 = ResourceSpec.newBuilder().setCpuCores(0.7).setTaskHeapMemoryMB(700).build(); + ResourceSpec resource1 = ResourceSpec.newBuilder(0.1, 100).build(); + ResourceSpec resource2 = ResourceSpec.newBuilder(0.2, 200).build(); + ResourceSpec resource3 = ResourceSpec.newBuilder(0.3, 300).build(); + ResourceSpec resource4 = ResourceSpec.newBuilder(0.4, 400).build(); + ResourceSpec resource5 = ResourceSpec.newBuilder(0.5, 500).build(); + ResourceSpec resource6 = ResourceSpec.newBuilder(0.6, 600).build(); + ResourceSpec resource7 = ResourceSpec.newBuilder(0.7, 700).build(); Method opMethod = Operator.class.getDeclaredMethod("setResources", ResourceSpec.class); opMethod.setAccessible(true); @@ -147,12 +147,12 @@ public boolean filter(Long value) throws Exception { */ @Test public void testResourcesForDeltaIteration() throws Exception{ - ResourceSpec resource1 = ResourceSpec.newBuilder().setCpuCores(0.1).setTaskHeapMemoryMB(100).build(); - ResourceSpec resource2 = ResourceSpec.newBuilder().setCpuCores(0.2).setTaskHeapMemoryMB(200).build(); - ResourceSpec resource3 = ResourceSpec.newBuilder().setCpuCores(0.3).setTaskHeapMemoryMB(300).build(); - ResourceSpec resource4 = ResourceSpec.newBuilder().setCpuCores(0.4).setTaskHeapMemoryMB(400).build(); - ResourceSpec resource5 = ResourceSpec.newBuilder().setCpuCores(0.5).setTaskHeapMemoryMB(500).build(); - ResourceSpec resource6 = ResourceSpec.newBuilder().setCpuCores(0.6).setTaskHeapMemoryMB(600).build(); + ResourceSpec resource1 = ResourceSpec.newBuilder(0.1, 100).build(); + ResourceSpec resource2 = ResourceSpec.newBuilder(0.2, 200).build(); + ResourceSpec resource3 = ResourceSpec.newBuilder(0.3, 300).build(); + ResourceSpec resource4 = ResourceSpec.newBuilder(0.4, 400).build(); + ResourceSpec resource5 = ResourceSpec.newBuilder(0.5, 500).build(); + ResourceSpec resource6 = ResourceSpec.newBuilder(0.6, 600).build(); Method opMethod = Operator.class.getDeclaredMethod("setResources", ResourceSpec.class); opMethod.setAccessible(true); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java index 78e948b2b087..f889ccefbea9 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java @@ -65,14 +65,10 @@ public void testMatchRequirement() { ResourceProfile rp5 = new ResourceProfile(2.0, 100, 100, 100, 100, 100, null); assertFalse(rp4.isMatching(rp5)); - ResourceSpec rs1 = ResourceSpec.newBuilder(). - setCpuCores(1.0). - setTaskHeapMemoryMB(100). + ResourceSpec rs1 = ResourceSpec.newBuilder(1.0, 100). setGPUResource(2.2). build(); - ResourceSpec rs2 = ResourceSpec.newBuilder(). - setCpuCores(1.0). - setTaskHeapMemoryMB(100). + ResourceSpec rs2 = ResourceSpec.newBuilder(1.0, 100). setGPUResource(1.1). build(); @@ -88,25 +84,19 @@ public void testUnknownMatchesUnknown() { @Test public void testEquals() { - ResourceSpec rs1 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); - ResourceSpec rs2 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); + ResourceSpec rs1 = ResourceSpec.newBuilder(1.0, 100).build(); + ResourceSpec rs2 = ResourceSpec.newBuilder(1.0, 100).build(); assertEquals(ResourceProfile.fromResourceSpec(rs1), ResourceProfile.fromResourceSpec(rs2)); - ResourceSpec rs3 = ResourceSpec.newBuilder(). - setCpuCores(1.0). - setTaskHeapMemoryMB(100). + ResourceSpec rs3 = ResourceSpec.newBuilder(1.0, 100). setGPUResource(2.2). build(); - ResourceSpec rs4 = ResourceSpec.newBuilder(). - setCpuCores(1.0). - setTaskHeapMemoryMB(100). + ResourceSpec rs4 = ResourceSpec.newBuilder(1.0, 100). setGPUResource(1.1). build(); assertNotEquals(ResourceProfile.fromResourceSpec(rs3), ResourceProfile.fromResourceSpec(rs4)); - ResourceSpec rs5 = ResourceSpec.newBuilder(). - setCpuCores(1.0). - setTaskHeapMemoryMB(100). + ResourceSpec rs5 = ResourceSpec.newBuilder(1.0, 100). setGPUResource(2.2). build(); MemorySize networkMemory = MemorySize.parse(100 + "m"); @@ -132,29 +122,23 @@ public void testEquals() { @Test public void testCompareTo() { - ResourceSpec rs1 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); - ResourceSpec rs2 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); + ResourceSpec rs1 = ResourceSpec.newBuilder(1.0, 100).build(); + ResourceSpec rs2 = ResourceSpec.newBuilder(1.0, 100).build(); assertEquals(0, ResourceProfile.fromResourceSpec(rs1).compareTo(ResourceProfile.fromResourceSpec(rs2))); - ResourceSpec rs3 = ResourceSpec.newBuilder(). - setCpuCores(1.0). - setTaskHeapMemoryMB(100). + ResourceSpec rs3 = ResourceSpec.newBuilder(1.0, 100). setGPUResource(2.2). build(); assertEquals(-1, ResourceProfile.fromResourceSpec(rs1).compareTo(ResourceProfile.fromResourceSpec(rs3))); assertEquals(1, ResourceProfile.fromResourceSpec(rs3).compareTo(ResourceProfile.fromResourceSpec(rs1))); - ResourceSpec rs4 = ResourceSpec.newBuilder(). - setCpuCores(1.0). - setTaskHeapMemoryMB(100). + ResourceSpec rs4 = ResourceSpec.newBuilder(1.0, 100). setGPUResource(1.1). build(); assertEquals(1, ResourceProfile.fromResourceSpec(rs3).compareTo(ResourceProfile.fromResourceSpec(rs4))); assertEquals(-1, ResourceProfile.fromResourceSpec(rs4).compareTo(ResourceProfile.fromResourceSpec(rs3))); - ResourceSpec rs5 = ResourceSpec.newBuilder(). - setCpuCores(1.0). - setTaskHeapMemoryMB(100). + ResourceSpec rs5 = ResourceSpec.newBuilder(1.0, 100). setGPUResource(2.2). build(); assertEquals(0, ResourceProfile.fromResourceSpec(rs3).compareTo(ResourceProfile.fromResourceSpec(rs5))); @@ -162,9 +146,7 @@ public void testCompareTo() { @Test public void testGet() { - ResourceSpec rs = ResourceSpec.newBuilder(). - setCpuCores(1.0). - setTaskHeapMemoryMB(100). + ResourceSpec rs = ResourceSpec.newBuilder(1.0, 100). setGPUResource(1.6). build(); ResourceProfile rp = ResourceProfile.fromResourceSpec(rs, MemorySize.parse(50 + "m")); 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 795c31e30436..681e013e5e4f 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 @@ -311,7 +311,7 @@ public void testJobSubmissionWithPartialResourceConfigured() throws Exception { DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); - ResourceSpec resourceSpec = ResourceSpec.newBuilder().setCpuCores(2).setTaskHeapMemoryMB(0).build(); + ResourceSpec resourceSpec = ResourceSpec.newBuilder(2.0, 0).build(); final JobVertex firstVertex = new JobVertex("firstVertex"); firstVertex.setInvokableClass(NoOpInvokable.class); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java index f73e933388b6..a8b7be1fe416 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java @@ -547,26 +547,26 @@ public void invoke(Long value) throws Exception { public void testResources() throws Exception{ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - ResourceSpec minResource1 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(100).build(); - ResourceSpec preferredResource1 = ResourceSpec.newBuilder().setCpuCores(2.0).setTaskHeapMemoryMB(200).build(); + ResourceSpec minResource1 = ResourceSpec.newBuilder(1.0, 100).build(); + ResourceSpec preferredResource1 = ResourceSpec.newBuilder(2.0, 200).build(); - ResourceSpec minResource2 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(200).build(); - ResourceSpec preferredResource2 = ResourceSpec.newBuilder().setCpuCores(2.0).setTaskHeapMemoryMB(300).build(); + ResourceSpec minResource2 = ResourceSpec.newBuilder(1.0, 200).build(); + ResourceSpec preferredResource2 = ResourceSpec.newBuilder(2.0, 300).build(); - ResourceSpec minResource3 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(300).build(); - ResourceSpec preferredResource3 = ResourceSpec.newBuilder().setCpuCores(2.0).setTaskHeapMemoryMB(400).build(); + ResourceSpec minResource3 = ResourceSpec.newBuilder(1.0, 300).build(); + ResourceSpec preferredResource3 = ResourceSpec.newBuilder(2.0, 400).build(); - ResourceSpec minResource4 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(400).build(); - ResourceSpec preferredResource4 = ResourceSpec.newBuilder().setCpuCores(2.0).setTaskHeapMemoryMB(500).build(); + ResourceSpec minResource4 = ResourceSpec.newBuilder(1.0, 400).build(); + ResourceSpec preferredResource4 = ResourceSpec.newBuilder(2.0, 500).build(); - ResourceSpec minResource5 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(500).build(); - ResourceSpec preferredResource5 = ResourceSpec.newBuilder().setCpuCores(2.0).setTaskHeapMemoryMB(600).build(); + ResourceSpec minResource5 = ResourceSpec.newBuilder(1.0, 500).build(); + ResourceSpec preferredResource5 = ResourceSpec.newBuilder(2.0, 600).build(); - ResourceSpec minResource6 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(600).build(); - ResourceSpec preferredResource6 = ResourceSpec.newBuilder().setCpuCores(2.0).setTaskHeapMemoryMB(700).build(); + ResourceSpec minResource6 = ResourceSpec.newBuilder(1.0, 600).build(); + ResourceSpec preferredResource6 = ResourceSpec.newBuilder(2.0, 700).build(); - ResourceSpec minResource7 = ResourceSpec.newBuilder().setCpuCores(1.0).setTaskHeapMemoryMB(700).build(); - ResourceSpec preferredResource7 = ResourceSpec.newBuilder().setCpuCores(2.0).setTaskHeapMemoryMB(800).build(); + ResourceSpec minResource7 = ResourceSpec.newBuilder(1.0, 700).build(); + ResourceSpec preferredResource7 = ResourceSpec.newBuilder(2.0, 800).build(); Method opMethod = SingleOutputStreamOperator.class.getDeclaredMethod("setResources", ResourceSpec.class, ResourceSpec.class); opMethod.setAccessible(true); 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 408410cc0d60..283383c1655f 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 @@ -213,11 +213,11 @@ public Integer map(Integer value) throws Exception { */ @Test public void testResourcesForChainedSourceSink() throws Exception { - ResourceSpec resource1 = ResourceSpec.newBuilder().setCpuCores(0.1).setTaskHeapMemoryMB(100).build(); - ResourceSpec resource2 = ResourceSpec.newBuilder().setCpuCores(0.2).setTaskHeapMemoryMB(200).build(); - ResourceSpec resource3 = ResourceSpec.newBuilder().setCpuCores(0.3).setTaskHeapMemoryMB(300).build(); - ResourceSpec resource4 = ResourceSpec.newBuilder().setCpuCores(0.4).setTaskHeapMemoryMB(400).build(); - ResourceSpec resource5 = ResourceSpec.newBuilder().setCpuCores(0.5).setTaskHeapMemoryMB(500).build(); + ResourceSpec resource1 = ResourceSpec.newBuilder(0.1, 100).build(); + ResourceSpec resource2 = ResourceSpec.newBuilder(0.2, 200).build(); + ResourceSpec resource3 = ResourceSpec.newBuilder(0.3, 300).build(); + ResourceSpec resource4 = ResourceSpec.newBuilder(0.4, 400).build(); + ResourceSpec resource5 = ResourceSpec.newBuilder(0.5, 500).build(); Method opMethod = SingleOutputStreamOperator.class.getDeclaredMethod("setResources", ResourceSpec.class); opMethod.setAccessible(true); @@ -285,11 +285,11 @@ public void invoke(Tuple2 value) throws Exception { */ @Test public void testResourcesForIteration() throws Exception { - ResourceSpec resource1 = ResourceSpec.newBuilder().setCpuCores(0.1).setTaskHeapMemoryMB(100).build(); - ResourceSpec resource2 = ResourceSpec.newBuilder().setCpuCores(0.2).setTaskHeapMemoryMB(200).build(); - ResourceSpec resource3 = ResourceSpec.newBuilder().setCpuCores(0.3).setTaskHeapMemoryMB(300).build(); - ResourceSpec resource4 = ResourceSpec.newBuilder().setCpuCores(0.4).setTaskHeapMemoryMB(400).build(); - ResourceSpec resource5 = ResourceSpec.newBuilder().setCpuCores(0.5).setTaskHeapMemoryMB(500).build(); + ResourceSpec resource1 = ResourceSpec.newBuilder(0.1, 100).build(); + ResourceSpec resource2 = ResourceSpec.newBuilder(0.2, 200).build(); + ResourceSpec resource3 = ResourceSpec.newBuilder(0.3, 300).build(); + ResourceSpec resource4 = ResourceSpec.newBuilder(0.4, 400).build(); + ResourceSpec resource5 = ResourceSpec.newBuilder(0.5, 500).build(); Method opMethod = SingleOutputStreamOperator.class.getDeclaredMethod("setResources", ResourceSpec.class); opMethod.setAccessible(true); diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/resource/NodeResourceUtil.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/resource/NodeResourceUtil.java index 7b2111fc1c9d..93ee2b19ad45 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/resource/NodeResourceUtil.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/resource/NodeResourceUtil.java @@ -34,7 +34,7 @@ public class NodeResourceUtil { * Build resourceSpec from managedMem. */ public static ResourceSpec fromManagedMem(int managedMem) { - ResourceSpec.Builder builder = ResourceSpec.newBuilder().setCpuCores(0.0).setTaskHeapMemoryMB(0); + ResourceSpec.Builder builder = ResourceSpec.newBuilder(0.0, 0); // TODO: before operators separate on-heap/off-heap managed memory, we use on-heap managed memory to denote total managed memory builder.setOnHeapManagedMemoryMB(managedMem); return builder.build(); From 099e9657517ddde4dc2bc2cefd4855cd57d88feb Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Thu, 7 Nov 2019 11:16:40 +0800 Subject: [PATCH 565/746] [hotfix] Remove unused constructors and unnecessary javadocs for ResourceSpec. --- .../api/common/operators/ResourceSpec.java | 36 ------------------- 1 file changed, 36 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/ResourceSpec.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/ResourceSpec.java index bca130190e81..fb0e73a22106 100755 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/ResourceSpec.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/ResourceSpec.java @@ -88,16 +88,6 @@ public final class ResourceSpec implements Serializable { private final Map extendedResources = new HashMap<>(1); - /** - * Creates a new ResourceSpec with full resources. - * - * @param cpuCores The number of CPU cores (possibly fractional, i.e., 0.2 cores) - * @param taskHeapMemory The size of the task heap memory. - * @param taskOffHeapMemory The size of the task off-heap memory. - * @param onHeapManagedMemory The size of the on-heap managed memory. - * @param offHeapManagedMemory The size of the off-heap managed memory. - * @param extendedResources The extended resources, associated with the resource manager used - */ private ResourceSpec( double cpuCores, MemorySize taskHeapMemory, @@ -120,32 +110,6 @@ private ResourceSpec( } } - /** - * Creates a new ResourceSpec with full resources. - * - * @param cpuCores The number of CPU cores (possibly fractional, i.e., 0.2 cores) - * @param taskHeapMemoryMB The size of the task heap memory, in megabytes. - * @param taskOffHeapMemoryMB The size of the task off-heap memory, in megabytes. - * @param onHeapManagedMemoryMB The size of the on-heap managed memory, in megabytes. - * @param offHeapManagedMemoryMB The size of the off-heap managed memory. - * @param extendedResources The extended resources, associated with the resource manager used - */ - private ResourceSpec( - double cpuCores, - int taskHeapMemoryMB, - int taskOffHeapMemoryMB, - int onHeapManagedMemoryMB, - int offHeapManagedMemoryMB, - Resource... extendedResources) { - this( - cpuCores, - MemorySize.parse(taskHeapMemoryMB + "m"), - MemorySize.parse(taskOffHeapMemoryMB + "m"), - MemorySize.parse(onHeapManagedMemoryMB + "m"), - MemorySize.parse(offHeapManagedMemoryMB + "m"), - extendedResources); - } - /** * Creates a new ResourceSpec with all fields unknown. */ From bfd6aa503e9cecec9e2d977a4dfb44d91a4a52eb Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Thu, 7 Nov 2019 11:20:37 +0800 Subject: [PATCH 566/746] [hotfix] Remove unused constructors and unnecessary javadoces, and annotate constructors used only in testing codes as VisibleForTestting for ResourceProfile. --- .../types/ResourceProfile.java | 41 ++----------------- 1 file changed, 3 insertions(+), 38 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java index afc7ee516c73..dd0202449da3 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java @@ -130,17 +130,7 @@ public ResourceProfile( } } - /** - * Creates a new ResourceProfile. - * - * @param cpuCores The number of CPU cores (possibly fractional, i.e., 0.2 cores) - * @param taskHeapMemoryMB The size of the task heap memory, in megabytes. - * @param taskOffHeapMemoryMB The size of the task off-heap memory, in megabytes. - * @param onHeapManagedMemoryMB The size of the on-heap managed memory, in megabytes. - * @param offHeapManagedMemoryMB The size of the off-heap managed memory, in megabytes. - * @param shuffleMemoryMB The size of the shuffle memory, in megabytes. - * @param extendedResources The extended resources such as GPU and FPGA - */ + @VisibleForTesting public ResourceProfile( double cpuCores, int taskHeapMemoryMB, @@ -160,22 +150,12 @@ public ResourceProfile( extendedResources); } - /** - * Creates a new simple ResourceProfile used for testing. - * - * @param cpuCores The number of CPU cores (possibly fractional, i.e., 0.2 cores) - * @param taskHeapMemory The size of the task heap memory. - */ + @VisibleForTesting public ResourceProfile(double cpuCores, MemorySize taskHeapMemory) { this(cpuCores, taskHeapMemory, MemorySize.ZERO, MemorySize.ZERO, MemorySize.ZERO, MemorySize.ZERO, Collections.emptyMap()); } - /** - * Creates a new simple ResourceProfile used for testing. - * - * @param cpuCores The number of CPU cores (possibly fractional, i.e., 0.2 cores) - * @param taskHeapMemoryMB The size of the task heap memory, in megabytes. - */ + @VisibleForTesting public ResourceProfile(double cpuCores, int taskHeapMemoryMB) { this(cpuCores, MemorySize.parse(taskHeapMemoryMB + "m"), MemorySize.ZERO, MemorySize.ZERO, MemorySize.ZERO, MemorySize.ZERO, Collections.emptyMap()); } @@ -192,21 +172,6 @@ private ResourceProfile() { this.shuffleMemory = null; } - /** - * Creates a copy of the given ResourceProfile. - * - * @param other The ResourceProfile to copy. - */ - public ResourceProfile(ResourceProfile other) { - this(other.cpuCores, - other.taskHeapMemory, - other.taskOffHeapMemory, - other.onHeapManagedMemory, - other.offHeapManagedMemory, - other.shuffleMemory, - other.extendedResources); - } - // ------------------------------------------------------------------------ /** From 36d1b622dfdd8630f1e3f874de520c130db14148 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Thu, 7 Nov 2019 13:22:24 +0800 Subject: [PATCH 567/746] [hotfix] Preserve the singleton property for ResourceProfile#ANY. --- .../clusterframework/types/ResourceProfile.java | 13 +++++++++++-- .../clusterframework/types/ResourceProfileTest.java | 7 +++++++ 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java index dd0202449da3..53fe4effe490 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java @@ -503,8 +503,17 @@ public String toString() { // ------------------------------------------------------------------------ private Object readResolve() { - // try to preserve the singleton property for UNKNOWN - return this.equals(UNKNOWN) ? UNKNOWN : this; + // try to preserve the singleton property for UNKNOWN and ANY + + if (this.equals(UNKNOWN)) { + return UNKNOWN; + } + + if (this.equals(ANY)) { + return ANY; + } + + return this; } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java index f889ccefbea9..99f36c784bec 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java @@ -260,4 +260,11 @@ public void testSingletonPropertyOfUnknown() throws Exception { assertSame(ResourceProfile.UNKNOWN, copiedProfile); } + + @Test + public void testSingletonPropertyOfAny() throws Exception { + final ResourceProfile copiedProfile = CommonTestUtils.createCopySerializable(ResourceProfile.ANY); + + assertSame(ResourceProfile.ANY, copiedProfile); + } } From e2b654893d63d1fd55e6a4475a45b984e01b4165 Mon Sep 17 00:00:00 2001 From: ifndef-SleePy Date: Tue, 8 Oct 2019 20:57:34 +0800 Subject: [PATCH 568/746] [FLINK-14344][checkpointing] PendingCheckpoint supports acknowledging master state MasterState acknowledgements are now independent of task states acknowledgments. This is done as a preparation for asynchronous MasterHooks firing, where we will first asynchronously fire MasterHooks, wait for their acknowledgements and only then start the checkpoint. --- .../checkpoint/CheckpointCoordinator.java | 9 +- .../runtime/checkpoint/PendingCheckpoint.java | 43 ++++-- .../runtime/checkpoint/hooks/MasterHooks.java | 13 +- .../CheckpointCoordinatorMasterHooksTest.java | 30 +--- .../checkpoint/CheckpointCoordinatorTest.java | 26 ++-- .../CheckpointCoordinatorTestingUtils.java | 31 +++- .../checkpoint/PendingCheckpointTest.java | 145 +++++++++++++++++- .../checkpoint/hooks/TestMasterHook.java | 4 +- 8 files changed, 227 insertions(+), 74 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index 052534d66d8d..5c07472c1827 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -573,6 +573,7 @@ public CompletableFuture triggerCheckpoint( checkpointID, timestamp, ackTasks, + masterHooks.keySet(), props, checkpointStorageLocation, executor); @@ -622,10 +623,10 @@ public CompletableFuture triggerCheckpoint( } // trigger the master hooks for the checkpoint - final List masterStates = MasterHooks.triggerMasterHooks(masterHooks.values(), + final Map masterStates = MasterHooks.triggerMasterHooks(masterHooks.values(), checkpointID, timestamp, executor, Time.milliseconds(checkpointTimeout)); - for (MasterState s : masterStates) { - checkpoint.addMasterState(s); + for (Map.Entry entry : masterStates.entrySet()) { + checkpoint.acknowledgeMasterState(entry.getKey(), entry.getValue()); } } // end of lock scope @@ -776,7 +777,7 @@ public boolean receiveAcknowledgeMessage(AcknowledgeCheckpoint message, String t LOG.debug("Received acknowledge message for checkpoint {} from task {} of job {} at {}.", checkpointId, message.getTaskExecutionId(), message.getJob(), taskManagerLocationInfo); - if (checkpoint.isFullyAcknowledged()) { + if (checkpoint.isTasksFullyAcknowledged()) { completePendingCheckpoint(checkpoint); } break; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java index ac086ec1407d..557f3b887664 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java @@ -38,6 +38,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -88,7 +89,9 @@ public enum TaskAcknowledgeResult { private final Map notYetAcknowledgedTasks; - private final List masterState; + private final List masterStates; + + private final Set notYetAcknowledgedMasterStates; /** Set of acknowledged tasks. */ private final Set acknowledgedTasks; @@ -122,6 +125,7 @@ public PendingCheckpoint( long checkpointId, long checkpointTimestamp, Map verticesToConfirm, + Collection masterStateIdentifiers, CheckpointProperties props, CheckpointStorageLocation targetLocation, Executor executor) { @@ -138,7 +142,8 @@ public PendingCheckpoint( this.executor = Preconditions.checkNotNull(executor); this.operatorStates = new HashMap<>(); - this.masterState = new ArrayList<>(); + this.masterStates = new ArrayList<>(masterStateIdentifiers.size()); + this.notYetAcknowledgedMasterStates = new HashSet<>(masterStateIdentifiers); this.acknowledgedTasks = new HashSet<>(verticesToConfirm.size()); this.onCompletionPromise = new CompletableFuture<>(); } @@ -173,8 +178,16 @@ public Map getOperatorStates() { return operatorStates; } - public boolean isFullyAcknowledged() { - return this.notYetAcknowledgedTasks.isEmpty() && !discarded; + public List getMasterStates() { + return masterStates; + } + + public boolean isMasterStatesFullyAcknowledged() { + return notYetAcknowledgedMasterStates.isEmpty() && !discarded; + } + + public boolean isTasksFullyAcknowledged() { + return notYetAcknowledgedTasks.isEmpty() && !discarded; } public boolean isAcknowledgedBy(ExecutionAttemptID executionAttemptId) { @@ -247,12 +260,15 @@ public CompletableFuture getCompletionFuture() { public CompletedCheckpoint finalizeCheckpoint() throws IOException { synchronized (lock) { - checkState(isFullyAcknowledged(), "Pending checkpoint has not been fully acknowledged yet."); + checkState(isMasterStatesFullyAcknowledged(), + "Pending checkpoint has not been fully acknowledged by master states yet."); + checkState(isTasksFullyAcknowledged(), + "Pending checkpoint has not been fully acknowledged by tasks yet."); // make sure we fulfill the promise with an exception if something fails try { // write out the metadata - final Savepoint savepoint = new SavepointV2(checkpointId, operatorStates.values(), masterState); + final Savepoint savepoint = new SavepointV2(checkpointId, operatorStates.values(), masterStates); final CompletedCheckpointStorageLocation finalizedLocation; try (CheckpointMetadataOutputStream out = targetLocation.createMetadataOutputStream()) { @@ -266,7 +282,7 @@ public CompletedCheckpoint finalizeCheckpoint() throws IOException { checkpointTimestamp, System.currentTimeMillis(), operatorStates, - masterState, + masterStates, props, finalizedLocation); @@ -383,22 +399,23 @@ public TaskAcknowledgeResult acknowledgeTask( } /** - * Adds a master state (state generated on the checkpoint coordinator) to + * Acknowledges a master state (state generated on the checkpoint coordinator) to * the pending checkpoint. * - * @param state The state to add + * @param identifier The identifier of the master state + * @param state The state to acknowledge */ - public void addMasterState(MasterState state) { - checkNotNull(state); + public void acknowledgeMasterState(String identifier, @Nullable MasterState state) { synchronized (lock) { if (!discarded) { - masterState.add(state); + if (notYetAcknowledgedMasterStates.remove(identifier) && state != null) { + masterStates.add(state); + } } } } - // ------------------------------------------------------------------------ // Cancellation // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/hooks/MasterHooks.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/hooks/MasterHooks.java index 0521676fd047..de8490cb8169 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/hooks/MasterHooks.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/hooks/MasterHooks.java @@ -34,8 +34,8 @@ import java.util.ArrayList; import java.util.Collection; +import java.util.HashMap; import java.util.LinkedHashMap; -import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -114,23 +114,20 @@ public static void close( * @throws FlinkException Thrown, if the hooks throw an exception, or the state+ * deserialization fails. */ - public static List triggerMasterHooks( - Collection> hooks, + public static Map triggerMasterHooks( + Collection> hooks, long checkpointId, long timestamp, Executor executor, Time timeout) throws FlinkException { - final ArrayList states = new ArrayList<>(hooks.size()); + final Map states = new HashMap<>(hooks.size()); for (MasterTriggerRestoreHook hook : hooks) { MasterState state = triggerHook(hook, checkpointId, timestamp, executor, timeout); - if (state != null) { - states.add(state); - } + states.put(hook.getIdentifier(), state); } - states.trimToSize(); return states; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java index 953ec0dba4ca..cb6091e6188f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java @@ -40,7 +40,6 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -48,6 +47,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; +import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.StringSerializer; import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.mockExecutionVertex; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; @@ -479,34 +479,6 @@ private static T mockGeneric(Class clazz) { // ------------------------------------------------------------------------ - /** - * A test implementation of {@link SimpleVersionedSerializer} for String type. - */ - public static final class StringSerializer implements SimpleVersionedSerializer { - - static final int VERSION = 77; - - @Override - public int getVersion() { - return VERSION; - } - - @Override - public byte[] serialize(String checkpointData) throws IOException { - return checkpointData.getBytes(StandardCharsets.UTF_8); - } - - @Override - public String deserialize(int version, byte[] serialized) throws IOException { - if (version != VERSION) { - throw new IOException("version mismatch"); - } - return new String(serialized, StandardCharsets.UTF_8); - } - } - - // ------------------------------------------------------------------------ - private static final class LongSerializer implements SimpleVersionedSerializer { static final int VERSION = 5; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java index e458437a90d0..4914693a6d05 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java @@ -362,7 +362,7 @@ public void failJobDueToTaskFailure(Throwable cause, ExecutionAttemptID failingT // acknowledge from one of the tasks coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID2, checkpointId), TASK_MANAGER_LOCATION_INFO); assertFalse(checkpoint.isDiscarded()); - assertFalse(checkpoint.isFullyAcknowledged()); + assertFalse(checkpoint.isTasksFullyAcknowledged()); // decline checkpoint from the other task coord.receiveDeclineMessage(new DeclineCheckpoint(jid, attemptID1, checkpointId), TASK_MANAGER_LOCATION_INFO); @@ -430,7 +430,7 @@ public void testTriggerAndDeclineCheckpointSimple() { assertEquals(0, checkpoint.getNumberOfAcknowledgedTasks()); assertEquals(0, checkpoint.getOperatorStates().size()); assertFalse(checkpoint.isDiscarded()); - assertFalse(checkpoint.isFullyAcknowledged()); + assertFalse(checkpoint.isTasksFullyAcknowledged()); // check that the vertices received the trigger checkpoint message verify(vertex1.getCurrentExecutionAttempt()).triggerCheckpoint(checkpointId, timestamp, CheckpointOptions.forCheckpointWithDefaultLocation()); @@ -441,12 +441,12 @@ public void testTriggerAndDeclineCheckpointSimple() { assertEquals(1, checkpoint.getNumberOfAcknowledgedTasks()); assertEquals(1, checkpoint.getNumberOfNonAcknowledgedTasks()); assertFalse(checkpoint.isDiscarded()); - assertFalse(checkpoint.isFullyAcknowledged()); + assertFalse(checkpoint.isTasksFullyAcknowledged()); // acknowledge the same task again (should not matter) coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID2, checkpointId), "Unknown location"); assertFalse(checkpoint.isDiscarded()); - assertFalse(checkpoint.isFullyAcknowledged()); + assertFalse(checkpoint.isTasksFullyAcknowledged()); // decline checkpoint from the other task, this should cancel the checkpoint // and trigger a new one @@ -528,7 +528,7 @@ public void testTriggerAndDeclineCheckpointComplex() { assertEquals(0, checkpoint1.getNumberOfAcknowledgedTasks()); assertEquals(0, checkpoint1.getOperatorStates().size()); assertFalse(checkpoint1.isDiscarded()); - assertFalse(checkpoint1.isFullyAcknowledged()); + assertFalse(checkpoint1.isTasksFullyAcknowledged()); assertNotNull(checkpoint2); assertEquals(checkpoint2Id, checkpoint2.getCheckpointId()); @@ -538,7 +538,7 @@ public void testTriggerAndDeclineCheckpointComplex() { assertEquals(0, checkpoint2.getNumberOfAcknowledgedTasks()); assertEquals(0, checkpoint2.getOperatorStates().size()); assertFalse(checkpoint2.isDiscarded()); - assertFalse(checkpoint2.isFullyAcknowledged()); + assertFalse(checkpoint2.isTasksFullyAcknowledged()); // check that the vertices received the trigger checkpoint message { @@ -573,7 +573,7 @@ public void testTriggerAndDeclineCheckpointComplex() { assertEquals(0, checkpointNew.getNumberOfAcknowledgedTasks()); assertEquals(0, checkpointNew.getOperatorStates().size()); assertFalse(checkpointNew.isDiscarded()); - assertFalse(checkpointNew.isFullyAcknowledged()); + assertFalse(checkpointNew.isTasksFullyAcknowledged()); assertNotEquals(checkpoint1.getCheckpointId(), checkpointNew.getCheckpointId()); // decline again, nothing should happen @@ -631,7 +631,7 @@ public void testTriggerAndConfirmSimpleCheckpoint() { assertEquals(0, checkpoint.getNumberOfAcknowledgedTasks()); assertEquals(0, checkpoint.getOperatorStates().size()); assertFalse(checkpoint.isDiscarded()); - assertFalse(checkpoint.isFullyAcknowledged()); + assertFalse(checkpoint.isTasksFullyAcknowledged()); // check that the vertices received the trigger checkpoint message { @@ -654,13 +654,13 @@ public void testTriggerAndConfirmSimpleCheckpoint() { assertEquals(1, checkpoint.getNumberOfAcknowledgedTasks()); assertEquals(1, checkpoint.getNumberOfNonAcknowledgedTasks()); assertFalse(checkpoint.isDiscarded()); - assertFalse(checkpoint.isFullyAcknowledged()); + assertFalse(checkpoint.isTasksFullyAcknowledged()); verify(taskOperatorSubtaskStates2, never()).registerSharedStates(any(SharedStateRegistry.class)); // acknowledge the same task again (should not matter) coord.receiveAcknowledgeMessage(acknowledgeCheckpoint1, TASK_MANAGER_LOCATION_INFO); assertFalse(checkpoint.isDiscarded()); - assertFalse(checkpoint.isFullyAcknowledged()); + assertFalse(checkpoint.isTasksFullyAcknowledged()); verify(subtaskState2, never()).registerSharedStates(any(SharedStateRegistry.class)); // acknowledge the other task. @@ -1389,7 +1389,7 @@ public void testTriggerAndConfirmSimpleSavepoint() throws Exception { assertEquals(0, pending.getNumberOfAcknowledgedTasks()); assertEquals(0, pending.getOperatorStates().size()); assertFalse(pending.isDiscarded()); - assertFalse(pending.isFullyAcknowledged()); + assertFalse(pending.isTasksFullyAcknowledged()); assertFalse(pending.canBeSubsumed()); OperatorID opID1 = OperatorID.fromJobVertexID(vertex1.getJobvertexId()); @@ -1407,13 +1407,13 @@ public void testTriggerAndConfirmSimpleSavepoint() throws Exception { assertEquals(1, pending.getNumberOfAcknowledgedTasks()); assertEquals(1, pending.getNumberOfNonAcknowledgedTasks()); assertFalse(pending.isDiscarded()); - assertFalse(pending.isFullyAcknowledged()); + assertFalse(pending.isTasksFullyAcknowledged()); assertFalse(savepointFuture.isDone()); // acknowledge the same task again (should not matter) coord.receiveAcknowledgeMessage(acknowledgeCheckpoint2, TASK_MANAGER_LOCATION_INFO); assertFalse(pending.isDiscarded()); - assertFalse(pending.isFullyAcknowledged()); + assertFalse(pending.isTasksFullyAcknowledged()); assertFalse(savepointFuture.isDone()); // acknowledge the other task. diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java index 89af1531cc70..2bb5625dd4b3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java @@ -22,8 +22,8 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.mock.Whitebox; -import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.Execution; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; @@ -55,6 +55,7 @@ import java.io.IOException; import java.io.Serializable; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -65,10 +66,8 @@ import java.util.Random; import java.util.UUID; import java.util.concurrent.Executor; -import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; -import static org.apache.flink.util.Preconditions.checkNotNull; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.mockito.ArgumentMatchers.any; @@ -566,4 +565,30 @@ static ExecutionJobVertex mockExecutionJobVertex(JobVertexID id, ExecutionVertex } return vertex; } + + /** + * A test implementation of {@link SimpleVersionedSerializer} for String type. + */ + public static final class StringSerializer implements SimpleVersionedSerializer { + + static final int VERSION = 77; + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(String checkpointData) throws IOException { + return checkpointData.getBytes(StandardCharsets.UTF_8); + } + + @Override + public String deserialize(int version, byte[] serialized) throws IOException { + if (version != VERSION) { + throw new IOException("version mismatch"); + } + return new String(serialized, StandardCharsets.UTF_8); + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java index af1afb7f8252..37ea6264b3ca 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java @@ -19,8 +19,12 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.time.Time; import org.apache.flink.core.fs.Path; import org.apache.flink.core.fs.local.LocalFileSystem; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.StringSerializer; +import org.apache.flink.runtime.checkpoint.hooks.MasterHooks; import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; @@ -37,17 +41,25 @@ import org.junit.rules.TemporaryFolder; import org.mockito.Mockito; +import javax.annotation.Nullable; + import java.io.IOException; import java.lang.reflect.Field; import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Queue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; import java.util.concurrent.ScheduledFuture; +import java.util.stream.Collectors; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -158,7 +170,7 @@ public void testCompletionFuture() throws Exception { assertFalse(future.isDone()); pending.acknowledgeTask(ATTEMPT_ID, null, new CheckpointMetrics()); - assertTrue(pending.isFullyAcknowledged()); + assertTrue(pending.isTasksFullyAcknowledged()); pending.finalizeCheckpoint(); assertTrue(future.isDone()); @@ -339,13 +351,105 @@ public void testSetCanceller() throws Exception { verify(canceller).cancel(false); } + @Test + public void testMasterState() throws Exception { + final TestingMasterTriggerRestoreHook masterHook = + new TestingMasterTriggerRestoreHook("master hook"); + masterHook.addStateContent("state"); + + final PendingCheckpoint pending = createPendingCheckpoint( + CheckpointProperties.forCheckpoint( + CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION), + Collections.singletonList(masterHook.getIdentifier())); + + final Map masterStates = MasterHooks.triggerMasterHooks( + Collections.singletonList(masterHook), + 0, + System.currentTimeMillis(), + Executors.directExecutor(), + Time.milliseconds(1024)); + assertEquals(1, masterStates.size()); + + pending.acknowledgeMasterState( + masterHook.getIdentifier(), masterStates.get(masterHook.getIdentifier())); + assertTrue(pending.isMasterStatesFullyAcknowledged()); + assertFalse(pending.isTasksFullyAcknowledged()); + + pending.acknowledgeTask(ATTEMPT_ID, null, new CheckpointMetrics()); + assertTrue(pending.isTasksFullyAcknowledged()); + + final List resultMasterStates = pending.getMasterStates(); + assertEquals(1, resultMasterStates.size()); + final String deserializedState = masterHook. + createCheckpointDataSerializer(). + deserialize(StringSerializer.VERSION, resultMasterStates.get(0).bytes()); + assertEquals("state", deserializedState); + } + + @Test + public void testMasterStateWithNullState() throws Exception { + final TestingMasterTriggerRestoreHook masterHook = + new TestingMasterTriggerRestoreHook("master hook"); + masterHook.addStateContent("state"); + + final TestingMasterTriggerRestoreHook nullableMasterHook = + new TestingMasterTriggerRestoreHook("nullable master hook"); + + final List masterHooks = new ArrayList<>(); + masterHooks.add(masterHook); + masterHooks.add(nullableMasterHook); + + final PendingCheckpoint pending = createPendingCheckpoint( + CheckpointProperties.forCheckpoint( + CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION), + masterHooks + .stream() + .map(TestingMasterTriggerRestoreHook::getIdentifier) + .collect(Collectors.toList())); + + final Map masterStates = MasterHooks.triggerMasterHooks( + masterHooks, + 0, + System.currentTimeMillis(), + Executors.directExecutor(), + Time.milliseconds(1024)); + assertEquals(2, masterStates.size()); + + pending.acknowledgeMasterState( + masterHook.getIdentifier(), masterStates.get(masterHook.getIdentifier())); + assertFalse(pending.isMasterStatesFullyAcknowledged()); + + pending.acknowledgeMasterState( + nullableMasterHook.getIdentifier(), masterStates.get(nullableMasterHook.getIdentifier())); + assertTrue(pending.isMasterStatesFullyAcknowledged()); + assertFalse(pending.isTasksFullyAcknowledged()); + + pending.acknowledgeTask(ATTEMPT_ID, null, new CheckpointMetrics()); + assertTrue(pending.isTasksFullyAcknowledged()); + + final List resultMasterStates = pending.getMasterStates(); + assertEquals(1, resultMasterStates.size()); + final String deserializedState = masterHook. + createCheckpointDataSerializer(). + deserialize(StringSerializer.VERSION, resultMasterStates.get(0).bytes()); + assertEquals("state", deserializedState); + } + // ------------------------------------------------------------------------ private PendingCheckpoint createPendingCheckpoint(CheckpointProperties props) throws IOException { - return createPendingCheckpoint(props, Executors.directExecutor()); + return createPendingCheckpoint(props, Collections.emptyList(), Executors.directExecutor()); } private PendingCheckpoint createPendingCheckpoint(CheckpointProperties props, Executor executor) throws IOException { + return createPendingCheckpoint(props, Collections.emptyList(), executor); + } + + private PendingCheckpoint createPendingCheckpoint(CheckpointProperties props, Collection masterStateIdentifiers) throws IOException { + return createPendingCheckpoint(props, masterStateIdentifiers, Executors.directExecutor()); + } + + private PendingCheckpoint createPendingCheckpoint(CheckpointProperties props, Collection masterStateIdentifiers, Executor executor) throws IOException { final Path checkpointDir = new Path(tmpFolder.newFolder().toURI()); final FsCheckpointStorageLocation location = new FsCheckpointStorageLocation( @@ -362,6 +466,7 @@ private PendingCheckpoint createPendingCheckpoint(CheckpointProperties props, Ex 0, 1, ackTasks, + masterStateIdentifiers, props, location, executor); @@ -391,4 +496,40 @@ public void runQueuedCommands() { } } } + + private static final class TestingMasterTriggerRestoreHook implements MasterTriggerRestoreHook { + + private final String identifier; + private final ArrayDeque stateContents; + + public TestingMasterTriggerRestoreHook(String identifier) { + this.identifier = checkNotNull(identifier); + stateContents = new ArrayDeque<>(); + } + + public void addStateContent(String stateContent) { + stateContents.add(stateContent); + } + + @Override + public String getIdentifier() { + return identifier; + } + + @Nullable + @Override + public CompletableFuture triggerCheckpoint(long checkpointId, long timestamp, Executor executor) throws Exception { + return CompletableFuture.completedFuture(stateContents.poll()); + } + + @Override + public void restoreCheckpoint(long checkpointId, @Nullable String checkpointData) throws Exception { + + } + + @Override + public SimpleVersionedSerializer createCheckpointDataSerializer() { + return new StringSerializer(); + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/hooks/TestMasterHook.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/hooks/TestMasterHook.java index bc651e45d39a..0cf1c5ea5d54 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/hooks/TestMasterHook.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/hooks/TestMasterHook.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.checkpoint.hooks; import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorMasterHooksTest; +import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils; import org.apache.flink.runtime.checkpoint.MasterTriggerRestoreHook; import javax.annotation.Nullable; @@ -74,7 +74,7 @@ public void restoreCheckpoint(final long checkpointId, @Nullable final String ch @Override public SimpleVersionedSerializer createCheckpointDataSerializer() { - return new CheckpointCoordinatorMasterHooksTest.StringSerializer(); + return new CheckpointCoordinatorTestingUtils.StringSerializer(); } public int getRestoreCount() { From 552246161cafeae8af450e245c5ba6e6104e8e73 Mon Sep 17 00:00:00 2001 From: ifndef-SleePy Date: Wed, 9 Oct 2019 19:07:37 +0800 Subject: [PATCH 569/746] [hotfix] Correct code style of MasterTriggerRestoreHook --- .../checkpoint/MasterTriggerRestoreHook.java | 46 +++++++++---------- 1 file changed, 23 insertions(+), 23 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/MasterTriggerRestoreHook.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/MasterTriggerRestoreHook.java index 4476d066c3a7..09a960a34daa 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/MasterTriggerRestoreHook.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/MasterTriggerRestoreHook.java @@ -29,7 +29,7 @@ * The interface for hooks that can be called by the checkpoint coordinator when triggering or * restoring a checkpoint. Such a hook is useful for example when preparing external systems for * taking or restoring checkpoints. - * + * *

    The {@link #triggerCheckpoint(long, long, Executor)} method (called when triggering a checkpoint) * can return a result (via a future) that will be stored as part of the checkpoint metadata. * When restoring a checkpoint, that stored result will be given to the {@link #restoreCheckpoint(long, Object)} @@ -44,7 +44,7 @@ *

    The MasterTriggerRestoreHook is defined when creating the streaming dataflow graph. It is attached * to the job graph, which gets sent to the cluster for execution. To avoid having to make the hook * itself serializable, these hooks are attached to the job graph via a {@link MasterTriggerRestoreHook.Factory}. - * + * * @param The type of the data produced by the hook and stored as part of the checkpoint metadata. * If the hook never stores any data, this can be typed to {@code Void}. */ @@ -53,17 +53,17 @@ public interface MasterTriggerRestoreHook { /** * Gets the identifier of this hook. The identifier is used to identify a specific hook in the * presence of multiple hooks and to give it the correct checkpointed data upon checkpoint restoration. - * + * *

    The identifier should be unique between different hooks of a job, but deterministic/constant * so that upon resuming a savepoint, the hook will get the correct data. * For example, if the hook calls into another storage system and persists namespace/schema specific * information, then the name of the storage system, together with the namespace/schema name could * be an appropriate identifier. - * + * *

    When multiple hooks of the same name are created and attached to a job graph, only the first * one is actually used. This can be exploited to deduplicate hooks that would do the same thing. - * - * @return The identifier of the hook. + * + * @return The identifier of the hook. */ String getIdentifier(); @@ -89,28 +89,28 @@ default void close() throws Exception { /** * This method is called by the checkpoint coordinator prior when triggering a checkpoint, prior * to sending the "trigger checkpoint" messages to the source tasks. - * + * *

    If the hook implementation wants to store data as part of the checkpoint, it may return * that data via a future, otherwise it should return null. The data is stored as part of * the checkpoint metadata under the hooks identifier (see {@link #getIdentifier()}). - * + * *

    If the action by this hook needs to be executed synchronously, then this method should * directly execute the action synchronously and block until it is complete. The returned future * (if any) would typically be a completed future. - * + * *

    If the action should be executed asynchronously and only needs to complete before the * checkpoint is considered completed, then the method may use the given executor to execute the * actual action and would signal its completion by completing the future. For hooks that do not * need to store data, the future would be completed with null. - * + * * @param checkpointId The ID (logical timestamp, monotonously increasing) of the checkpoint * @param timestamp The wall clock timestamp when the checkpoint was triggered, for - * info/logging purposes. + * info/logging purposes. * @param executor The executor for asynchronous actions - * + * * @return Optionally, a future that signals when the hook has completed and that contains * data to be stored with the checkpoint. - * + * * @throws Exception Exceptions encountered when calling the hook will cause the checkpoint to abort. */ @Nullable @@ -118,13 +118,13 @@ default void close() throws Exception { /** * This method is called by the checkpoint coordinator prior to restoring the state of a checkpoint. - * If the checkpoint did store data from this hook, that data will be passed to this method. - * + * If the checkpoint did store data from this hook, that data will be passed to this method. + * * @param checkpointId The ID (logical timestamp) of the restored checkpoint - * @param checkpointData The data originally stored in the checkpoint by this hook, possibly null. - * + * @param checkpointData The data originally stored in the checkpoint by this hook, possibly null. + * * @throws Exception Exceptions thrown while restoring the checkpoint will cause the restore - * operation to fail and to possibly fall back to another checkpoint. + * operation to fail and to possibly fall back to another checkpoint. */ void restoreCheckpoint(long checkpointId, @Nullable T checkpointData) throws Exception; @@ -132,11 +132,11 @@ default void close() throws Exception { * Creates a the serializer to (de)serializes the data stored by this hook. The serializer * serializes the result of the Future returned by the {@link #triggerCheckpoint(long, long, Executor)} * method, and deserializes the data stored in the checkpoint into the object passed to the - * {@link #restoreCheckpoint(long, Object)} method. - * + * {@link #restoreCheckpoint(long, Object)} method. + * *

    If the hook never returns any data to be stored, then this method may return null as the * serializer. - * + * * @return The serializer to (de)serializes the data stored by this hook */ @Nullable @@ -148,8 +148,8 @@ default void close() throws Exception { /** * A factory to instantiate a {@code MasterTriggerRestoreHook}. - * - * The hooks are defined when creating the streaming dataflow graph and are attached + * + *

    The hooks are defined when creating the streaming dataflow graph and are attached * to the job graph, which gets sent to the cluster for execution. To avoid having to make * the hook implementation serializable, a serializable hook factory is actually attached to the * job graph instead of the hook implementation itself. From d282c987867cf9a0ac54f0e62749563830793f30 Mon Sep 17 00:00:00 2001 From: ifndef-SleePy Date: Thu, 10 Oct 2019 15:24:20 +0800 Subject: [PATCH 570/746] [FLINK-14344][checkpointing] Emphasize that MasterTriggerRestoreHook#triggerCheckpoint should be non-blocking in java doc --- .../flink/runtime/checkpoint/MasterTriggerRestoreHook.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/MasterTriggerRestoreHook.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/MasterTriggerRestoreHook.java index 09a960a34daa..98eeb708e842 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/MasterTriggerRestoreHook.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/MasterTriggerRestoreHook.java @@ -95,14 +95,17 @@ default void close() throws Exception { * the checkpoint metadata under the hooks identifier (see {@link #getIdentifier()}). * *

    If the action by this hook needs to be executed synchronously, then this method should - * directly execute the action synchronously and block until it is complete. The returned future - * (if any) would typically be a completed future. + * directly execute the action synchronously. The returned future (if any) would typically be a + * completed future. * *

    If the action should be executed asynchronously and only needs to complete before the * checkpoint is considered completed, then the method may use the given executor to execute the * actual action and would signal its completion by completing the future. For hooks that do not * need to store data, the future would be completed with null. * + *

    Please note that this method should be non-blocking. Any heavy operation like IO operation + * should be executed asynchronously with given executor. + * * @param checkpointId The ID (logical timestamp, monotonously increasing) of the checkpoint * @param timestamp The wall clock timestamp when the checkpoint was triggered, for * info/logging purposes. From 82c2057c27d77ea2a2e7407a120a5478ae1a7701 Mon Sep 17 00:00:00 2001 From: ifndef-SleePy Date: Wed, 9 Oct 2019 18:00:30 +0800 Subject: [PATCH 571/746] [FLINK-14344][checkpointing] MasterHooks class supports asynchronous triggering This commits add supports for asynchronous triggering `MasterHooks`. As for now, they are still being fired synchronously by the `CheckpointCoordinator`. --- .../checkpoint/CheckpointCoordinator.java | 13 +- .../runtime/checkpoint/hooks/MasterHooks.java | 137 +++++++----------- .../checkpoint/PendingCheckpointTest.java | 43 +++--- 3 files changed, 74 insertions(+), 119 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index 5c07472c1827..9924a7abaab4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -20,7 +20,6 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.checkpoint.hooks.MasterHooks; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.concurrent.ScheduledExecutor; @@ -622,12 +621,14 @@ public CompletableFuture triggerCheckpoint( cancellerHandle.cancel(false); } - // trigger the master hooks for the checkpoint - final Map masterStates = MasterHooks.triggerMasterHooks(masterHooks.values(), - checkpointID, timestamp, executor, Time.milliseconds(checkpointTimeout)); - for (Map.Entry entry : masterStates.entrySet()) { - checkpoint.acknowledgeMasterState(entry.getKey(), entry.getValue()); + // TODO, asynchronously snapshots master hook without waiting here + for (MasterTriggerRestoreHook masterHook : masterHooks.values()) { + final MasterState masterState = + MasterHooks.triggerHook(masterHook, checkpointID, timestamp, executor) + .get(checkpointTimeout, TimeUnit.MILLISECONDS); + checkpoint.acknowledgeMasterState(masterHook.getIdentifier(), masterState); } + Preconditions.checkState(checkpoint.isMasterStatesFullyAcknowledged()); } // end of lock scope diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/hooks/MasterHooks.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/hooks/MasterHooks.java index de8490cb8169..3a16900c4f25 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/hooks/MasterHooks.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/hooks/MasterHooks.java @@ -18,11 +18,11 @@ package org.apache.flink.runtime.checkpoint.hooks; -import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.runtime.checkpoint.MasterState; import org.apache.flink.runtime.checkpoint.MasterTriggerRestoreHook; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.LambdaUtil; @@ -34,13 +34,11 @@ import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; import java.util.LinkedHashMap; import java.util.Map; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; +import java.util.concurrent.CompletionException; import java.util.concurrent.Executor; -import java.util.concurrent.TimeoutException; /** * Collection of methods to deal with checkpoint master hooks. @@ -100,102 +98,65 @@ public static void close( // ------------------------------------------------------------------------ /** - * Triggers all given master hooks and returns state objects for each hook that - * produced a state. - * - * @param hooks The hooks to trigger + * Trigger master hook and return a completable future with state. + * @param hook The master hook given * @param checkpointId The checkpoint ID of the triggering checkpoint * @param timestamp The (informational) timestamp for the triggering checkpoint * @param executor An executor that can be used for asynchronous I/O calls - * @param timeout The maximum time that a hook may take to complete - * - * @return A list containing all states produced by the hooks - * - * @throws FlinkException Thrown, if the hooks throw an exception, or the state+ - * deserialization fails. + * @param The type of data produced by the hook + * @return the completable future with state */ - public static Map triggerMasterHooks( - Collection> hooks, - long checkpointId, - long timestamp, - Executor executor, - Time timeout) throws FlinkException { - - final Map states = new HashMap<>(hooks.size()); - - for (MasterTriggerRestoreHook hook : hooks) { - MasterState state = triggerHook(hook, checkpointId, timestamp, executor, timeout); - states.put(hook.getIdentifier(), state); - } - - return states; - } - - private static MasterState triggerHook( - MasterTriggerRestoreHook hook, + public static CompletableFuture triggerHook( + MasterTriggerRestoreHook hook, long checkpointId, long timestamp, - Executor executor, - Time timeout) throws FlinkException { - - @SuppressWarnings("unchecked") - final MasterTriggerRestoreHook typedHook = (MasterTriggerRestoreHook) hook; + Executor executor) { - final String id = typedHook.getIdentifier(); - final SimpleVersionedSerializer serializer = typedHook.createCheckpointDataSerializer(); + final String id = hook.getIdentifier(); + final SimpleVersionedSerializer serializer = hook.createCheckpointDataSerializer(); - // call the hook! - final CompletableFuture resultFuture; try { - resultFuture = typedHook.triggerCheckpoint(checkpointId, timestamp, executor); - } - catch (Throwable t) { - ExceptionUtils.rethrowIfFatalErrorOrOOM(t); - throw new FlinkException("Error while triggering checkpoint master hook '" + id + '\'', t); - } + // call the hook! + final CompletableFuture resultFuture = + hook.triggerCheckpoint(checkpointId, timestamp, executor); - // is there is a result future, wait for its completion - // in the future we want to make this asynchronous with futures (no pun intended) - if (resultFuture == null) { - return null; - } - else { - final T result; - try { - result = resultFuture.get(timeout.getSize(), timeout.getUnit()); - } - catch (InterruptedException e) { - // cannot continue here - restore interrupt status and leave - Thread.currentThread().interrupt(); - throw new FlinkException("Checkpoint master hook was interrupted"); - } - catch (ExecutionException e) { - throw new FlinkException("Checkpoint master hook '" + id + "' produced an exception", e.getCause()); - } - catch (TimeoutException e) { - throw new FlinkException("Checkpoint master hook '" + id + - "' did not complete in time (" + timeout + ')'); + if (resultFuture == null) { + return CompletableFuture.completedFuture(null); } - // if the result of the future is not null, return it as state - if (result == null) { - return null; - } - else if (serializer != null) { - try { - final int version = serializer.getVersion(); - final byte[] bytes = serializer.serialize(result); - - return new MasterState(id, bytes, version); - } - catch (Throwable t) { - ExceptionUtils.rethrowIfFatalErrorOrOOM(t); - throw new FlinkException("Failed to serialize state of master hook '" + id + '\'', t); - } - } - else { - throw new FlinkException("Checkpoint hook '" + id + " is stateful but creates no serializer"); - } + return resultFuture + .thenApply(result -> { + // if the result of the future is not null, return it as state + if (result == null) { + return null; + } + else if (serializer != null) { + try { + final int version = serializer.getVersion(); + final byte[] bytes = serializer.serialize(result); + + return new MasterState(id, bytes, version); + } + catch (Throwable t) { + ExceptionUtils.rethrowIfFatalErrorOrOOM(t); + throw new CompletionException(new FlinkException( + "Failed to serialize state of master hook '" + id + '\'', t)); + } + } + else { + throw new CompletionException(new FlinkException( + "Checkpoint hook '" + id + " is stateful but creates no serializer")); + } + }) + .exceptionally((throwable) -> { + throw new CompletionException(new FlinkException( + "Checkpoint master hook '" + id + "' produced an exception", + throwable.getCause())); + }); + } + catch (Throwable t) { + return FutureUtils.completedExceptionally(new FlinkException( + "Error while triggering checkpoint master hook '" + id + '\'', t)); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java index 37ea6264b3ca..6c3ad80f338e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.time.Time; import org.apache.flink.core.fs.Path; import org.apache.flink.core.fs.local.LocalFileSystem; import org.apache.flink.core.io.SimpleVersionedSerializer; @@ -56,7 +55,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; import java.util.concurrent.ScheduledFuture; -import java.util.stream.Collectors; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.junit.Assert.assertEquals; @@ -362,16 +360,13 @@ public void testMasterState() throws Exception { CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION), Collections.singletonList(masterHook.getIdentifier())); - final Map masterStates = MasterHooks.triggerMasterHooks( - Collections.singletonList(masterHook), + final MasterState masterState = MasterHooks.triggerHook( + masterHook, 0, System.currentTimeMillis(), - Executors.directExecutor(), - Time.milliseconds(1024)); - assertEquals(1, masterStates.size()); + Executors.directExecutor()).get(); - pending.acknowledgeMasterState( - masterHook.getIdentifier(), masterStates.get(masterHook.getIdentifier())); + pending.acknowledgeMasterState(masterHook.getIdentifier(), masterState); assertTrue(pending.isMasterStatesFullyAcknowledged()); assertFalse(pending.isTasksFullyAcknowledged()); @@ -395,32 +390,30 @@ public void testMasterStateWithNullState() throws Exception { final TestingMasterTriggerRestoreHook nullableMasterHook = new TestingMasterTriggerRestoreHook("nullable master hook"); - final List masterHooks = new ArrayList<>(); - masterHooks.add(masterHook); - masterHooks.add(nullableMasterHook); + final List masterIdentifiers = new ArrayList<>(2); + masterIdentifiers.add(masterHook.getIdentifier()); + masterIdentifiers.add(nullableMasterHook.getIdentifier()); final PendingCheckpoint pending = createPendingCheckpoint( CheckpointProperties.forCheckpoint( CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION), - masterHooks - .stream() - .map(TestingMasterTriggerRestoreHook::getIdentifier) - .collect(Collectors.toList())); + masterIdentifiers); - final Map masterStates = MasterHooks.triggerMasterHooks( - masterHooks, + final MasterState masterStateNormal = MasterHooks.triggerHook( + masterHook, 0, System.currentTimeMillis(), - Executors.directExecutor(), - Time.milliseconds(1024)); - assertEquals(2, masterStates.size()); + Executors.directExecutor()).get(); - pending.acknowledgeMasterState( - masterHook.getIdentifier(), masterStates.get(masterHook.getIdentifier())); + pending.acknowledgeMasterState(masterHook.getIdentifier(), masterStateNormal); assertFalse(pending.isMasterStatesFullyAcknowledged()); - pending.acknowledgeMasterState( - nullableMasterHook.getIdentifier(), masterStates.get(nullableMasterHook.getIdentifier())); + final MasterState masterStateNull = MasterHooks.triggerHook( + nullableMasterHook, + 0, + System.currentTimeMillis(), + Executors.directExecutor()).get(); + pending.acknowledgeMasterState(nullableMasterHook.getIdentifier(), masterStateNull); assertTrue(pending.isMasterStatesFullyAcknowledged()); assertFalse(pending.isTasksFullyAcknowledged()); From b5da46170733313abed0f6172691a67a681af280 Mon Sep 17 00:00:00 2001 From: ifndef-SleePy Date: Thu, 7 Nov 2019 00:30:54 +0800 Subject: [PATCH 572/746] [hotfix] Correct the acknowledging method name of PendingCheckpoint --- .../checkpoint/CheckpointCoordinator.java | 4 +-- .../runtime/checkpoint/PendingCheckpoint.java | 8 +++--- .../checkpoint/CheckpointCoordinatorTest.java | 26 +++++++++---------- .../checkpoint/PendingCheckpointTest.java | 16 ++++++------ 4 files changed, 27 insertions(+), 27 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index 9924a7abaab4..9c92ac0efcf0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -628,7 +628,7 @@ public CompletableFuture triggerCheckpoint( .get(checkpointTimeout, TimeUnit.MILLISECONDS); checkpoint.acknowledgeMasterState(masterHook.getIdentifier(), masterState); } - Preconditions.checkState(checkpoint.isMasterStatesFullyAcknowledged()); + Preconditions.checkState(checkpoint.areMasterStatesFullyAcknowledged()); } // end of lock scope @@ -778,7 +778,7 @@ public boolean receiveAcknowledgeMessage(AcknowledgeCheckpoint message, String t LOG.debug("Received acknowledge message for checkpoint {} from task {} of job {} at {}.", checkpointId, message.getTaskExecutionId(), message.getJob(), taskManagerLocationInfo); - if (checkpoint.isTasksFullyAcknowledged()) { + if (checkpoint.areTasksFullyAcknowledged()) { completePendingCheckpoint(checkpoint); } break; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java index 557f3b887664..29803970c69f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java @@ -182,11 +182,11 @@ public List getMasterStates() { return masterStates; } - public boolean isMasterStatesFullyAcknowledged() { + public boolean areMasterStatesFullyAcknowledged() { return notYetAcknowledgedMasterStates.isEmpty() && !discarded; } - public boolean isTasksFullyAcknowledged() { + public boolean areTasksFullyAcknowledged() { return notYetAcknowledgedTasks.isEmpty() && !discarded; } @@ -260,9 +260,9 @@ public CompletableFuture getCompletionFuture() { public CompletedCheckpoint finalizeCheckpoint() throws IOException { synchronized (lock) { - checkState(isMasterStatesFullyAcknowledged(), + checkState(areMasterStatesFullyAcknowledged(), "Pending checkpoint has not been fully acknowledged by master states yet."); - checkState(isTasksFullyAcknowledged(), + checkState(areTasksFullyAcknowledged(), "Pending checkpoint has not been fully acknowledged by tasks yet."); // make sure we fulfill the promise with an exception if something fails diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java index 4914693a6d05..13b3d1c3f511 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java @@ -362,7 +362,7 @@ public void failJobDueToTaskFailure(Throwable cause, ExecutionAttemptID failingT // acknowledge from one of the tasks coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID2, checkpointId), TASK_MANAGER_LOCATION_INFO); assertFalse(checkpoint.isDiscarded()); - assertFalse(checkpoint.isTasksFullyAcknowledged()); + assertFalse(checkpoint.areTasksFullyAcknowledged()); // decline checkpoint from the other task coord.receiveDeclineMessage(new DeclineCheckpoint(jid, attemptID1, checkpointId), TASK_MANAGER_LOCATION_INFO); @@ -430,7 +430,7 @@ public void testTriggerAndDeclineCheckpointSimple() { assertEquals(0, checkpoint.getNumberOfAcknowledgedTasks()); assertEquals(0, checkpoint.getOperatorStates().size()); assertFalse(checkpoint.isDiscarded()); - assertFalse(checkpoint.isTasksFullyAcknowledged()); + assertFalse(checkpoint.areTasksFullyAcknowledged()); // check that the vertices received the trigger checkpoint message verify(vertex1.getCurrentExecutionAttempt()).triggerCheckpoint(checkpointId, timestamp, CheckpointOptions.forCheckpointWithDefaultLocation()); @@ -441,12 +441,12 @@ public void testTriggerAndDeclineCheckpointSimple() { assertEquals(1, checkpoint.getNumberOfAcknowledgedTasks()); assertEquals(1, checkpoint.getNumberOfNonAcknowledgedTasks()); assertFalse(checkpoint.isDiscarded()); - assertFalse(checkpoint.isTasksFullyAcknowledged()); + assertFalse(checkpoint.areTasksFullyAcknowledged()); // acknowledge the same task again (should not matter) coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID2, checkpointId), "Unknown location"); assertFalse(checkpoint.isDiscarded()); - assertFalse(checkpoint.isTasksFullyAcknowledged()); + assertFalse(checkpoint.areTasksFullyAcknowledged()); // decline checkpoint from the other task, this should cancel the checkpoint // and trigger a new one @@ -528,7 +528,7 @@ public void testTriggerAndDeclineCheckpointComplex() { assertEquals(0, checkpoint1.getNumberOfAcknowledgedTasks()); assertEquals(0, checkpoint1.getOperatorStates().size()); assertFalse(checkpoint1.isDiscarded()); - assertFalse(checkpoint1.isTasksFullyAcknowledged()); + assertFalse(checkpoint1.areTasksFullyAcknowledged()); assertNotNull(checkpoint2); assertEquals(checkpoint2Id, checkpoint2.getCheckpointId()); @@ -538,7 +538,7 @@ public void testTriggerAndDeclineCheckpointComplex() { assertEquals(0, checkpoint2.getNumberOfAcknowledgedTasks()); assertEquals(0, checkpoint2.getOperatorStates().size()); assertFalse(checkpoint2.isDiscarded()); - assertFalse(checkpoint2.isTasksFullyAcknowledged()); + assertFalse(checkpoint2.areTasksFullyAcknowledged()); // check that the vertices received the trigger checkpoint message { @@ -573,7 +573,7 @@ public void testTriggerAndDeclineCheckpointComplex() { assertEquals(0, checkpointNew.getNumberOfAcknowledgedTasks()); assertEquals(0, checkpointNew.getOperatorStates().size()); assertFalse(checkpointNew.isDiscarded()); - assertFalse(checkpointNew.isTasksFullyAcknowledged()); + assertFalse(checkpointNew.areTasksFullyAcknowledged()); assertNotEquals(checkpoint1.getCheckpointId(), checkpointNew.getCheckpointId()); // decline again, nothing should happen @@ -631,7 +631,7 @@ public void testTriggerAndConfirmSimpleCheckpoint() { assertEquals(0, checkpoint.getNumberOfAcknowledgedTasks()); assertEquals(0, checkpoint.getOperatorStates().size()); assertFalse(checkpoint.isDiscarded()); - assertFalse(checkpoint.isTasksFullyAcknowledged()); + assertFalse(checkpoint.areTasksFullyAcknowledged()); // check that the vertices received the trigger checkpoint message { @@ -654,13 +654,13 @@ public void testTriggerAndConfirmSimpleCheckpoint() { assertEquals(1, checkpoint.getNumberOfAcknowledgedTasks()); assertEquals(1, checkpoint.getNumberOfNonAcknowledgedTasks()); assertFalse(checkpoint.isDiscarded()); - assertFalse(checkpoint.isTasksFullyAcknowledged()); + assertFalse(checkpoint.areTasksFullyAcknowledged()); verify(taskOperatorSubtaskStates2, never()).registerSharedStates(any(SharedStateRegistry.class)); // acknowledge the same task again (should not matter) coord.receiveAcknowledgeMessage(acknowledgeCheckpoint1, TASK_MANAGER_LOCATION_INFO); assertFalse(checkpoint.isDiscarded()); - assertFalse(checkpoint.isTasksFullyAcknowledged()); + assertFalse(checkpoint.areTasksFullyAcknowledged()); verify(subtaskState2, never()).registerSharedStates(any(SharedStateRegistry.class)); // acknowledge the other task. @@ -1389,7 +1389,7 @@ public void testTriggerAndConfirmSimpleSavepoint() throws Exception { assertEquals(0, pending.getNumberOfAcknowledgedTasks()); assertEquals(0, pending.getOperatorStates().size()); assertFalse(pending.isDiscarded()); - assertFalse(pending.isTasksFullyAcknowledged()); + assertFalse(pending.areTasksFullyAcknowledged()); assertFalse(pending.canBeSubsumed()); OperatorID opID1 = OperatorID.fromJobVertexID(vertex1.getJobvertexId()); @@ -1407,13 +1407,13 @@ public void testTriggerAndConfirmSimpleSavepoint() throws Exception { assertEquals(1, pending.getNumberOfAcknowledgedTasks()); assertEquals(1, pending.getNumberOfNonAcknowledgedTasks()); assertFalse(pending.isDiscarded()); - assertFalse(pending.isTasksFullyAcknowledged()); + assertFalse(pending.areTasksFullyAcknowledged()); assertFalse(savepointFuture.isDone()); // acknowledge the same task again (should not matter) coord.receiveAcknowledgeMessage(acknowledgeCheckpoint2, TASK_MANAGER_LOCATION_INFO); assertFalse(pending.isDiscarded()); - assertFalse(pending.isTasksFullyAcknowledged()); + assertFalse(pending.areTasksFullyAcknowledged()); assertFalse(savepointFuture.isDone()); // acknowledge the other task. diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java index 6c3ad80f338e..cdda728bf786 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java @@ -168,7 +168,7 @@ public void testCompletionFuture() throws Exception { assertFalse(future.isDone()); pending.acknowledgeTask(ATTEMPT_ID, null, new CheckpointMetrics()); - assertTrue(pending.isTasksFullyAcknowledged()); + assertTrue(pending.areTasksFullyAcknowledged()); pending.finalizeCheckpoint(); assertTrue(future.isDone()); @@ -367,11 +367,11 @@ public void testMasterState() throws Exception { Executors.directExecutor()).get(); pending.acknowledgeMasterState(masterHook.getIdentifier(), masterState); - assertTrue(pending.isMasterStatesFullyAcknowledged()); - assertFalse(pending.isTasksFullyAcknowledged()); + assertTrue(pending.areMasterStatesFullyAcknowledged()); + assertFalse(pending.areTasksFullyAcknowledged()); pending.acknowledgeTask(ATTEMPT_ID, null, new CheckpointMetrics()); - assertTrue(pending.isTasksFullyAcknowledged()); + assertTrue(pending.areTasksFullyAcknowledged()); final List resultMasterStates = pending.getMasterStates(); assertEquals(1, resultMasterStates.size()); @@ -406,7 +406,7 @@ public void testMasterStateWithNullState() throws Exception { Executors.directExecutor()).get(); pending.acknowledgeMasterState(masterHook.getIdentifier(), masterStateNormal); - assertFalse(pending.isMasterStatesFullyAcknowledged()); + assertFalse(pending.areMasterStatesFullyAcknowledged()); final MasterState masterStateNull = MasterHooks.triggerHook( nullableMasterHook, @@ -414,11 +414,11 @@ public void testMasterStateWithNullState() throws Exception { System.currentTimeMillis(), Executors.directExecutor()).get(); pending.acknowledgeMasterState(nullableMasterHook.getIdentifier(), masterStateNull); - assertTrue(pending.isMasterStatesFullyAcknowledged()); - assertFalse(pending.isTasksFullyAcknowledged()); + assertTrue(pending.areMasterStatesFullyAcknowledged()); + assertFalse(pending.areTasksFullyAcknowledged()); pending.acknowledgeTask(ATTEMPT_ID, null, new CheckpointMetrics()); - assertTrue(pending.isTasksFullyAcknowledged()); + assertTrue(pending.areTasksFullyAcknowledged()); final List resultMasterStates = pending.getMasterStates(); assertEquals(1, resultMasterStates.size()); From 0cf9c2846ad1b293b91059d172dd68dd0ffbf803 Mon Sep 17 00:00:00 2001 From: joe <15692118798@163.com> Date: Mon, 4 Nov 2019 19:23:10 +0800 Subject: [PATCH 573/746] [FLINK-14499][metric] MetricRegistry#getMetricQueryServiceGatewayRpcAddress is Nonnull This closes #10074 . --- .../flink/runtime/dispatcher/Dispatcher.java | 2 - .../dispatcher/DispatcherServices.java | 7 ++- .../dispatcher/PartialDispatcherServices.java | 7 ++- ...alDispatcherServicesWithJobGraphStore.java | 3 +- .../runtime/taskexecutor/TaskExecutor.java | 5 +-- .../DispatcherResourceCleanupTest.java | 17 +++++++- .../TaskExecutorPartitionLifecycleTest.java | 17 ++++++-- .../TaskExecutorSubmissionTest.java | 43 +++++++++++++++++++ .../taskexecutor/TaskExecutorTest.java | 37 +++++++++++----- .../TaskSubmissionTestEnvironment.java | 14 ++++-- .../taskexecutor/TestingTaskExecutor.java | 3 +- 11 files changed, 119 insertions(+), 36 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 6fa9e7f35070..6a35b2dc4b10 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 @@ -71,7 +71,6 @@ import org.apache.flink.util.function.FunctionWithException; import javax.annotation.Nonnull; -import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; @@ -123,7 +122,6 @@ public abstract class Dispatcher extends PermanentlyFencedRpcEndpoint> jobManagerTerminationFutures; 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 89cef28d869f..9e544250232f 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 @@ -29,7 +29,6 @@ import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import javax.annotation.Nonnull; -import javax.annotation.Nullable; /** * {@link Dispatcher} services container. @@ -63,7 +62,7 @@ public class DispatcherServices { @Nonnull private final HistoryServerArchivist historyServerArchivist; - @Nullable + @Nonnull private final String metricQueryServiceAddress; @Nonnull @@ -81,7 +80,7 @@ public DispatcherServices( @Nonnull ArchivedExecutionGraphStore archivedExecutionGraphStore, @Nonnull FatalErrorHandler fatalErrorHandler, @Nonnull HistoryServerArchivist historyServerArchivist, - @Nullable String metricQueryServiceAddress, + @Nonnull String metricQueryServiceAddress, @Nonnull JobManagerMetricGroup jobManagerMetricGroup, @Nonnull JobGraphWriter jobGraphWriter, @Nonnull JobManagerRunnerFactory jobManagerRunnerFactory) { @@ -144,7 +143,7 @@ public HistoryServerArchivist getHistoryServerArchivist() { return historyServerArchivist; } - @Nullable + @Nonnull public String getMetricQueryServiceAddress() { return metricQueryServiceAddress; } 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 index 89049a09b4cc..044494a915c3 100644 --- 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 @@ -27,7 +27,6 @@ import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import javax.annotation.Nonnull; -import javax.annotation.Nullable; /** * Partial {@link DispatcherServices} services container which needs to @@ -62,7 +61,7 @@ public class PartialDispatcherServices { @Nonnull private final HistoryServerArchivist historyServerArchivist; - @Nullable + @Nonnull private final String metricQueryServiceAddress; public PartialDispatcherServices( @@ -75,7 +74,7 @@ public PartialDispatcherServices( @Nonnull ArchivedExecutionGraphStore archivedExecutionGraphStore, @Nonnull FatalErrorHandler fatalErrorHandler, @Nonnull HistoryServerArchivist historyServerArchivist, - @Nullable String metricQueryServiceAddress) { + @Nonnull String metricQueryServiceAddress) { this.configuration = configuration; this.highAvailabilityServices = highAvailabilityServices; this.resourceManagerGatewayRetriever = resourceManagerGatewayRetriever; @@ -133,7 +132,7 @@ public HistoryServerArchivist getHistoryServerArchivist() { return historyServerArchivist; } - @Nullable + @Nonnull public String getMetricQueryServiceAddress() { return metricQueryServiceAddress; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobGraphStore.java index 0eec923c710c..bb807742cc48 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobGraphStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobGraphStore.java @@ -28,7 +28,6 @@ import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import javax.annotation.Nonnull; -import javax.annotation.Nullable; /** * {@link DispatcherFactory} services container. @@ -48,7 +47,7 @@ public PartialDispatcherServicesWithJobGraphStore( @Nonnull ArchivedExecutionGraphStore archivedExecutionGraphStore, @Nonnull FatalErrorHandler fatalErrorHandler, @Nonnull HistoryServerArchivist historyServerArchivist, - @Nullable String metricQueryServiceAddress, + @Nonnull String metricQueryServiceAddress, @Nonnull JobGraphWriter jobGraphWriter) { super( configuration, 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 4b15ebc2499e..0f2d7562caa0 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 @@ -166,7 +166,6 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { private final BlobCacheService blobCacheService; /** The address to metric query service on this Task Manager. */ - @Nullable private final String metricQueryServiceAddress; // --------- TaskManager services -------- @@ -240,7 +239,7 @@ public TaskExecutor( TaskManagerServices taskExecutorServices, HeartbeatServices heartbeatServices, TaskManagerMetricGroup taskManagerMetricGroup, - @Nullable String metricQueryServiceAddress, + String metricQueryServiceAddress, BlobCacheService blobCacheService, FatalErrorHandler fatalErrorHandler, TaskExecutorPartitionTracker partitionTracker) { @@ -256,7 +255,7 @@ public TaskExecutor( this.partitionTracker = partitionTracker; this.taskManagerMetricGroup = checkNotNull(taskManagerMetricGroup); this.blobCacheService = checkNotNull(blobCacheService); - this.metricQueryServiceAddress = metricQueryServiceAddress; + this.metricQueryServiceAddress = checkNotNull(metricQueryServiceAddress); this.taskSlotTable = taskExecutorServices.getTaskSlotTable(); this.jobManagerTable = taskExecutorServices.getJobManagerTable(); 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 7e614e6e9f62..45ff469bb922 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 @@ -31,6 +31,7 @@ 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.clusterframework.types.ResourceID; import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; @@ -46,6 +47,8 @@ import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; import org.apache.flink.runtime.jobmaster.factories.JobManagerJobMetricGroupFactory; import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; 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; @@ -101,6 +104,8 @@ public class DispatcherResourceCleanupTest extends TestLogger { private static TestingRpcService rpcService; + private static MetricRegistryImpl metricRegistry; + private JobID jobId; private JobGraph jobGraph; @@ -174,6 +179,8 @@ public void setup() throws Exception { assertThat(storedHABlobFuture.get(), equalTo(permanentBlobKey)); fatalErrorHandler = new TestingFatalErrorHandler(); + + metricRegistry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); } private TestingJobManagerRunnerFactory startDispatcherAndSubmitJob() throws Exception { @@ -192,6 +199,10 @@ private void startDispatcher(JobManagerRunnerFactory jobManagerRunnerFactory) th TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway(); final HeartbeatServices heartbeatServices = new HeartbeatServices(1000L, 1000L); final MemoryArchivedExecutionGraphStore archivedExecutionGraphStore = new MemoryArchivedExecutionGraphStore(); + + metricRegistry.startQueryService(rpcService, new ResourceID("mqs")); + final String metricQueryServiceAddress = metricRegistry.getMetricQueryServiceGatewayRpcAddress(); + dispatcher = new TestingDispatcher( rpcService, Dispatcher.DISPATCHER_NAME + UUID.randomUUID(), @@ -206,7 +217,7 @@ private void startDispatcher(JobManagerRunnerFactory jobManagerRunnerFactory) th archivedExecutionGraphStore, fatalErrorHandler, VoidHistoryServerArchivist.INSTANCE, - null, + metricQueryServiceAddress, UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), jobGraphWriter, jobManagerRunnerFactory)); @@ -232,6 +243,10 @@ public static void teardownClass() throws ExecutionException, InterruptedExcepti if (rpcService != null) { rpcService.stopService().get(); } + + if (metricRegistry != null) { + metricRegistry.shutdown().get(); + } } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java index 0e92c86727ae..ae485b4ce9b4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java @@ -56,6 +56,8 @@ import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder; import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService; import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway; import org.apache.flink.runtime.rpc.RpcUtils; @@ -104,6 +106,10 @@ public class TaskExecutorPartitionLifecycleTest extends TestLogger { private static final TestingRpcService RPC = new TestingRpcService(); + private static final MetricRegistryImpl metricRegistry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + + private String metricQueryServiceAddress; + private final TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices(); private final SettableLeaderRetrievalService jobManagerLeaderRetriever = new SettableLeaderRetrievalService(); private final SettableLeaderRetrievalService resourceManagerLeaderRetriever = new SettableLeaderRetrievalService(); @@ -114,6 +120,8 @@ public class TaskExecutorPartitionLifecycleTest extends TestLogger { @Before public void setup() { + metricRegistry.startQueryService(RPC, new ResourceID("mqs")); + metricQueryServiceAddress = metricRegistry.getMetricQueryServiceGatewayRpcAddress(); haServices.setResourceManagerLeaderRetriever(resourceManagerLeaderRetriever); haServices.setJobMasterLeaderRetriever(jobId, jobManagerLeaderRetriever); } @@ -126,6 +134,7 @@ public void shutdown() { @AfterClass public static void shutdownClass() throws ExecutionException, InterruptedException { RPC.stopService().get(); + metricRegistry.shutdown().get(); } @Test @@ -155,7 +164,7 @@ public void testConnectionTerminationAfterExternalRelease() throws Exception { final ResultPartitionDeploymentDescriptor resultPartitionDeploymentDescriptor = PartitionTestUtils.createPartitionDeploymentDescriptor(ResultPartitionType.BLOCKING); final ResultPartitionID resultPartitionId = resultPartitionDeploymentDescriptor.getShuffleDescriptor().getResultPartitionID(); - final TestingTaskExecutor taskExecutor = createTestingTaskExecutor(taskManagerServices, partitionTracker); + final TestingTaskExecutor taskExecutor = createTestingTaskExecutor(taskManagerServices, partitionTracker, metricQueryServiceAddress); try { taskExecutor.start(); @@ -309,7 +318,7 @@ private void testPartitionRelease( final TaskExecutorPartitionTracker partitionTracker = new TaskExecutorPartitionTrackerImpl(); - final TestingTaskExecutor taskExecutor = createTestingTaskExecutor(taskManagerServices, partitionTracker); + final TestingTaskExecutor taskExecutor = createTestingTaskExecutor(taskManagerServices, partitionTracker, metricQueryServiceAddress); final CompletableFuture initialSlotReportFuture = new CompletableFuture<>(); @@ -429,7 +438,7 @@ public void invoke() throws Exception { } } - private TestingTaskExecutor createTestingTaskExecutor(TaskManagerServices taskManagerServices, TaskExecutorPartitionTracker partitionTracker) throws IOException { + private TestingTaskExecutor createTestingTaskExecutor(TaskManagerServices taskManagerServices, TaskExecutorPartitionTracker partitionTracker, String metricQueryServiceAddress) throws IOException { return new TestingTaskExecutor( RPC, TaskManagerConfiguration.fromConfiguration(new Configuration()), @@ -437,7 +446,7 @@ private TestingTaskExecutor createTestingTaskExecutor(TaskManagerServices taskMa taskManagerServices, new HeartbeatServices(10_000L, 30_000L), UnregisteredMetricGroups.createUnregisteredTaskManagerMetricGroup(), - null, + metricQueryServiceAddress, new BlobCacheService( new Configuration(), new VoidBlobStore(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java index be0e295b5309..c0f8632c835b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java @@ -37,6 +37,9 @@ import org.apache.flink.runtime.executiongraph.JobInformation; import org.apache.flink.runtime.executiongraph.PartitionInfo; import org.apache.flink.runtime.executiongraph.TaskInformation; +import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; +import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; @@ -63,6 +66,8 @@ import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TestLogger; +import org.junit.After; +import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestName; @@ -75,6 +80,7 @@ import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import static org.apache.flink.runtime.util.NettyShuffleDescriptorBuilder.createRemoteWithIdAndLocation; import static org.hamcrest.CoreMatchers.is; @@ -99,11 +105,37 @@ public class TaskExecutorSubmissionTest extends TestLogger { private JobID jobId = new JobID(); + private MetricRegistryImpl metricRegistry; + + private TestingRpcService rpcService; + + private String metricQueryServiceAddress; + + @Before + public void setup() { + rpcService = new TestingRpcService(); + metricRegistry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + metricRegistry.startQueryService(rpcService, new ResourceID("mqs")); + metricQueryServiceAddress = metricRegistry.getMetricQueryServiceGatewayRpcAddress(); + } + + @After + public void teardown() throws ExecutionException, InterruptedException { + if (rpcService != null) { + rpcService.stopService().get(); + } + + if (metricRegistry != null) { + metricRegistry.shutdown().get(); + } + } + /** * Tests that we can submit a task to the TaskManager given that we've allocated a slot there. */ @Test(timeout = 10000L) public void testTaskSubmission() throws Exception { + final ExecutionAttemptID eid = new ExecutionAttemptID(); final TaskDeploymentDescriptor tdd = createTestTaskDeploymentDescriptor("test task", eid, TaskExecutorTest.TestInvokable.class); @@ -114,6 +146,7 @@ public void testTaskSubmission() throws Exception { new TaskSubmissionTestEnvironment.Builder(jobId) .setSlotSize(1) .addTaskManagerActionListener(eid, ExecutionState.RUNNING, taskRunningFuture) + .setMetricQueryServiceAddress(metricQueryServiceAddress) .build()) { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); @@ -141,6 +174,7 @@ public void testSubmitTaskFailure() throws Exception { try (TaskSubmissionTestEnvironment env = new TaskSubmissionTestEnvironment.Builder(jobId) + .setMetricQueryServiceAddress(metricQueryServiceAddress) .build()) { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); @@ -173,6 +207,7 @@ public void testTaskSubmissionAndCancelling() throws Exception { .addTaskManagerActionListener(eid1, ExecutionState.RUNNING, task1RunningFuture) .addTaskManagerActionListener(eid2, ExecutionState.RUNNING, task2RunningFuture) .addTaskManagerActionListener(eid1, ExecutionState.CANCELED, task1CanceledFuture) + .setMetricQueryServiceAddress(metricQueryServiceAddress) .build()) { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); @@ -221,6 +256,7 @@ public void testGateChannelEdgeMismatch() throws Exception { .addTaskManagerActionListener(eid2, ExecutionState.RUNNING, task2RunningFuture) .addTaskManagerActionListener(eid1, ExecutionState.FAILED, task1FailedFuture) .addTaskManagerActionListener(eid2, ExecutionState.FAILED, task2FailedFuture) + .setMetricQueryServiceAddress(metricQueryServiceAddress) .setSlotSize(2) .build()) { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); @@ -274,6 +310,7 @@ public void testRunJobWithForwardChannel() throws Exception { .addTaskManagerActionListener(eid2, ExecutionState.RUNNING, task2RunningFuture) .addTaskManagerActionListener(eid1, ExecutionState.FINISHED, task1FinishedFuture) .addTaskManagerActionListener(eid2, ExecutionState.FINISHED, task2FinishedFuture) + .setMetricQueryServiceAddress(metricQueryServiceAddress) .setJobMasterId(jobMasterId) .setJobMasterGateway(testingJobMasterGateway) .useRealNonMockShuffleEnvironment() @@ -340,6 +377,7 @@ public void testCancellingDependentAndStateUpdateFails() throws Exception { .addTaskManagerActionListener(eid2, ExecutionState.RUNNING, task2RunningFuture) .addTaskManagerActionListener(eid1, ExecutionState.FAILED, task1FailedFuture) .addTaskManagerActionListener(eid2, ExecutionState.CANCELED, task2CanceledFuture) + .setMetricQueryServiceAddress(metricQueryServiceAddress) .setJobMasterId(jobMasterId) .setJobMasterGateway(testingJobMasterGateway) .useRealNonMockShuffleEnvironment() @@ -390,6 +428,7 @@ public void testRemotePartitionNotFound() throws Exception { .setSlotSize(2) .addTaskManagerActionListener(eid, ExecutionState.RUNNING, taskRunningFuture) .addTaskManagerActionListener(eid, ExecutionState.FAILED, taskFailedFuture) + .setMetricQueryServiceAddress(metricQueryServiceAddress) .setConfiguration(config) .setLocalCommunication(false) .useRealNonMockShuffleEnvironment() @@ -423,6 +462,7 @@ public void testUpdateTaskInputPartitionsFailure() throws Exception { new TaskSubmissionTestEnvironment.Builder(jobId) .setShuffleEnvironment(shuffleEnvironment) .setSlotSize(1) + .setMetricQueryServiceAddress(metricQueryServiceAddress) .addTaskManagerActionListener(eid, ExecutionState.RUNNING, taskRunningFuture) .addTaskManagerActionListener(eid, ExecutionState.FAILED, taskFailedFuture) .build()) { @@ -474,6 +514,7 @@ public void testLocalPartitionNotFound() throws Exception { new TaskSubmissionTestEnvironment.Builder(jobId) .setResourceID(producerLocation) .setSlotSize(1) + .setMetricQueryServiceAddress(metricQueryServiceAddress) .addTaskManagerActionListener(eid, ExecutionState.RUNNING, taskRunningFuture) .addTaskManagerActionListener(eid, ExecutionState.FAILED, taskFailedFuture) .setConfiguration(config) @@ -533,6 +574,7 @@ public void testFailingScheduleOrUpdateConsumers() throws Exception { new TaskSubmissionTestEnvironment.Builder(jobId) .setSlotSize(1) .setConfiguration(configuration) + .setMetricQueryServiceAddress(metricQueryServiceAddress) .addTaskManagerActionListener(eid, ExecutionState.RUNNING, taskRunningFuture) .setJobMasterId(jobMasterId) .setJobMasterGateway(testingJobMasterGateway) @@ -578,6 +620,7 @@ public void testRequestStackTraceSample() throws Exception { try (TaskSubmissionTestEnvironment env = new TaskSubmissionTestEnvironment.Builder(jobId) .setSlotSize(1) + .setMetricQueryServiceAddress(metricQueryServiceAddress) .addTaskManagerActionListener(eid, ExecutionState.RUNNING, taskRunningFuture) .addTaskManagerActionListener(eid, ExecutionState.CANCELED, taskCanceledFuture) .build()) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index 3fb355175da1..de128a581055 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -73,6 +73,8 @@ import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.memory.MemoryManagerBuilder; import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; +import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.query.KvStateRegistry; import org.apache.flink.runtime.registration.RegistrationResponse; @@ -185,6 +187,8 @@ public class TaskExecutorTest extends TestLogger { private static final Time timeout = Time.milliseconds(10000L); + private MetricRegistryImpl metricRegistry = new MetricRegistryImpl(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()); + private TestingRpcService rpc; private BlobCacheService dummyBlobCacheService; @@ -207,6 +211,8 @@ public class TaskExecutorTest extends TestLogger { private NettyShuffleEnvironment nettyShuffleEnvironment; + private String metricQueryServiceAddress; + @Before public void setup() throws IOException { rpc = new TestingRpcService(); @@ -231,6 +237,9 @@ public void setup() throws IOException { haServices.setJobMasterLeaderRetriever(jobId, jobManagerLeaderRetriever); nettyShuffleEnvironment = new NettyShuffleEnvironmentBuilder().build(); + + metricRegistry.startQueryService(rpc, new ResourceID("mqs")); + metricQueryServiceAddress = metricRegistry.getMetricQueryServiceGatewayRpcAddress(); } @After @@ -254,6 +263,10 @@ public void teardown() throws Exception { nettyShuffleEnvironment.close(); } + if (metricRegistry != null) { + metricRegistry.shutdown(); + } + testingFatalErrorHandler.rethrowError(); } @@ -346,7 +359,7 @@ public void testHeartbeatTimeoutWithJobManager() throws Exception { .setTaskStateManager(localStateStoresManager) .build(); - final TestingTaskExecutor taskManager = createTestingTaskExecutor(taskManagerServices, heartbeatServices); + final TestingTaskExecutor taskManager = createTestingTaskExecutor(taskManagerServices, heartbeatServices, metricQueryServiceAddress); try { taskManager.start(); @@ -427,7 +440,7 @@ public void testHeartbeatTimeoutWithResourceManager() throws Exception { .setTaskStateManager(localStateStoresManager) .build(); - final TaskExecutor taskManager = createTaskExecutor(taskManagerServices, heartbeatServices); + final TaskExecutor taskManager = createTaskExecutor(taskManagerServices, heartbeatServices, metricQueryServiceAddress); try { taskManager.start(); @@ -1190,7 +1203,7 @@ public void testRMHeartbeatStopWhenLeadershipRevoked() throws Exception { .setTaskStateManager(localStateStoresManager) .build(); - final TaskExecutor taskExecutor = createTaskExecutor(taskManagerServices, heartbeatServices); + final TaskExecutor taskExecutor = createTaskExecutor(taskManagerServices, heartbeatServices, metricQueryServiceAddress); try { taskExecutor.start(); @@ -1319,7 +1332,8 @@ public void testMaximumRegistrationDurationAfterConnectionLoss() throws Exceptio final TaskSlotTable taskSlotTable = new TaskSlotTable(Collections.singleton(ResourceProfile.ANY), timerService); final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder().setTaskSlotTable(taskSlotTable).build(); - final TaskExecutor taskExecutor = createTaskExecutor(taskManagerServices, new HeartbeatServices(10L, 10L)); + + final TaskExecutor taskExecutor = createTaskExecutor(taskManagerServices, new HeartbeatServices(10L, 10L), metricQueryServiceAddress); taskExecutor.start(); @@ -1697,6 +1711,7 @@ public void testLogNotFoundHandling() throws Throwable { new Builder(jobId) .setConfiguration(config) .setLocalCommunication(false) + .setMetricQueryServiceAddress(metricQueryServiceAddress) .build()) { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); try { @@ -1711,7 +1726,7 @@ public void testLogNotFoundHandling() throws Throwable { @Test(timeout = 10000L) public void testTerminationOnFatalError() throws Throwable { - try (TaskSubmissionTestEnvironment env = new Builder(jobId).build()) { + try (TaskSubmissionTestEnvironment env = new Builder(jobId).setMetricQueryServiceAddress(metricQueryServiceAddress).build()) { String testExceptionMsg = "Test exception of fatal error."; env.getTaskExecutor().onFatalError(new Exception(testExceptionMsg)); @@ -1905,10 +1920,10 @@ private TaskExecutorLocalStateStoresManager createTaskExecutorLocalStateStoresMa @Nonnull private TaskExecutor createTaskExecutor(TaskManagerServices taskManagerServices) { - return createTaskExecutor(taskManagerServices, HEARTBEAT_SERVICES); + return createTaskExecutor(taskManagerServices, HEARTBEAT_SERVICES, metricQueryServiceAddress); } - private TaskExecutor createTaskExecutor(TaskManagerServices taskManagerServices, HeartbeatServices heartbeatServices) { + private TaskExecutor createTaskExecutor(TaskManagerServices taskManagerServices, HeartbeatServices heartbeatServices, String metricQueryServiceAddress) { return new TaskExecutor( rpc, TaskManagerConfiguration.fromConfiguration(configuration), @@ -1916,17 +1931,17 @@ private TaskExecutor createTaskExecutor(TaskManagerServices taskManagerServices, taskManagerServices, heartbeatServices, UnregisteredMetricGroups.createUnregisteredTaskManagerMetricGroup(), - null, + metricQueryServiceAddress, dummyBlobCacheService, testingFatalErrorHandler, new TaskExecutorPartitionTrackerImpl()); } private TestingTaskExecutor createTestingTaskExecutor(TaskManagerServices taskManagerServices) { - return createTestingTaskExecutor(taskManagerServices, HEARTBEAT_SERVICES); + return createTestingTaskExecutor(taskManagerServices, HEARTBEAT_SERVICES, metricQueryServiceAddress); } - private TestingTaskExecutor createTestingTaskExecutor(TaskManagerServices taskManagerServices, HeartbeatServices heartbeatServices) { + private TestingTaskExecutor createTestingTaskExecutor(TaskManagerServices taskManagerServices, HeartbeatServices heartbeatServices, String metricQueryServiceAddress) { return new TestingTaskExecutor( rpc, TaskManagerConfiguration.fromConfiguration(configuration), @@ -1934,7 +1949,7 @@ private TestingTaskExecutor createTestingTaskExecutor(TaskManagerServices taskMa taskManagerServices, heartbeatServices, UnregisteredMetricGroups.createUnregisteredTaskManagerMetricGroup(), - null, + metricQueryServiceAddress, dummyBlobCacheService, testingFatalErrorHandler, new TaskExecutorPartitionTrackerImpl()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java index c1bf805132eb..efc23c4ca470 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java @@ -104,6 +104,7 @@ private TaskSubmissionTestEnvironment( TestingJobMasterGateway testingJobMasterGateway, Configuration configuration, List>> taskManagerActionListeners, + String metricQueryServiceAddress, TestingRpcService testingRpcService, ShuffleEnvironment shuffleEnvironment) throws Exception { @@ -161,7 +162,7 @@ private TaskSubmissionTestEnvironment( .setTaskStateManager(localStateStoresManager) .build(); - taskExecutor = createTaskExecutor(taskManagerServices, configuration); + taskExecutor = createTaskExecutor(taskManagerServices, metricQueryServiceAddress, configuration); taskExecutor.start(); taskExecutor.waitUntilStarted(); @@ -196,7 +197,7 @@ private TaskSlotTable generateTaskSlotTable(int numSlot) { } @Nonnull - private TestingTaskExecutor createTaskExecutor(TaskManagerServices taskManagerServices, Configuration configuration) { + private TestingTaskExecutor createTaskExecutor(TaskManagerServices taskManagerServices, String metricQueryServiceAddress, Configuration configuration) { return new TestingTaskExecutor( testingRpcService, TaskManagerConfiguration.fromConfiguration(configuration), @@ -204,7 +205,7 @@ private TestingTaskExecutor createTaskExecutor(TaskManagerServices taskManagerSe taskManagerServices, heartbeatServices, UnregisteredMetricGroups.createUnregisteredTaskManagerMetricGroup(), - null, + metricQueryServiceAddress, blobCacheService, testingFatalErrorHandler, new TaskExecutorPartitionTrackerImpl() @@ -285,6 +286,7 @@ public static final class Builder { @SuppressWarnings("OptionalUsedAsFieldOrParameterType") private Optional> optionalShuffleEnvironment = Optional.empty(); private ResourceID resourceID = ResourceID.generate(); + private String metricQueryServiceAddress; private List>> taskManagerActionListeners = new ArrayList<>(); @@ -292,6 +294,11 @@ public Builder(JobID jobId) { this.jobId = jobId; } + public Builder setMetricQueryServiceAddress(String metricQueryServiceAddress) { + this.metricQueryServiceAddress = metricQueryServiceAddress; + return this; + } + public Builder useRealNonMockShuffleEnvironment() { this.optionalShuffleEnvironment = Optional.empty(); this.mockShuffleEnvironment = false; @@ -359,6 +366,7 @@ public TaskSubmissionTestEnvironment build() throws Exception { jobMasterGateway, configuration, taskManagerActionListeners, + metricQueryServiceAddress, testingRpcService, network); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutor.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutor.java index cf8daa4fe475..354a1304ae4f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutor.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutor.java @@ -26,7 +26,6 @@ import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; -import javax.annotation.Nullable; import java.util.concurrent.CompletableFuture; @@ -43,7 +42,7 @@ public TestingTaskExecutor( TaskManagerServices taskExecutorServices, HeartbeatServices heartbeatServices, TaskManagerMetricGroup taskManagerMetricGroup, - @Nullable String metricQueryServiceAddress, + String metricQueryServiceAddress, BlobCacheService blobCacheService, FatalErrorHandler fatalErrorHandler, TaskExecutorPartitionTracker partitionTracker) { From 8d8c088f7df957e86586e85b11690fd1efafd174 Mon Sep 17 00:00:00 2001 From: tison Date: Tue, 12 Nov 2019 23:01:31 +0800 Subject: [PATCH 574/746] [hotfix][metric] MetricRegistryImpl#LOG is private --- .../org/apache/flink/runtime/metrics/MetricRegistryImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java index 33025bb47c91..662a95825890 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryImpl.java @@ -60,7 +60,7 @@ * connection between {@link MetricGroup MetricGroups} and {@link MetricReporter MetricReporters}. */ public class MetricRegistryImpl implements MetricRegistry { - static final Logger LOG = LoggerFactory.getLogger(MetricRegistryImpl.class); + private static final Logger LOG = LoggerFactory.getLogger(MetricRegistryImpl.class); private final Object lock = new Object(); From 57160ba271218a255bd1a5e376aeffd813cd904a Mon Sep 17 00:00:00 2001 From: Yun Tang Date: Tue, 12 Nov 2019 23:26:18 +0800 Subject: [PATCH 575/746] [FLINK-14704] Remove unused SynchronousSavepointLatch After FLINK-12482, SynchronousSavepointLatch has been useless and should be removed. This closes #10154 . --- .../tasks/SynchronousSavepointLatch.java | 121 ------------ .../SynchronousSavepointSyncLatchTest.java | 182 ------------------ 2 files changed, 303 deletions(-) delete mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SynchronousSavepointLatch.java delete mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousSavepointSyncLatchTest.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SynchronousSavepointLatch.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SynchronousSavepointLatch.java deleted file mode 100644 index a6a583d05193..000000000000 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SynchronousSavepointLatch.java +++ /dev/null @@ -1,121 +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.annotation.VisibleForTesting; -import org.apache.flink.runtime.checkpoint.CheckpointType; -import org.apache.flink.util.function.RunnableWithException; - -import javax.annotation.Nullable; -import javax.annotation.concurrent.GuardedBy; - -/** - * A synchronization primitive used by the {@link StreamTask} to wait - * for the completion of a {@link CheckpointType#SYNC_SAVEPOINT}. - */ -class SynchronousSavepointLatch { - - private static final long NOT_SET_CHECKPOINT_ID = -1L; - - enum CompletionResult { - COMPLETED, - CANCELED, - } - - @GuardedBy("synchronizationPoint") - private volatile boolean waiting; - - @GuardedBy("synchronizationPoint") - @Nullable - private volatile CompletionResult completionResult; - - private final Object synchronizationPoint; - - private volatile long checkpointId; - - SynchronousSavepointLatch() { - this.synchronizationPoint = new Object(); - - this.waiting = false; - this.checkpointId = NOT_SET_CHECKPOINT_ID; - } - - long getCheckpointId() { - return checkpointId; - } - - void setCheckpointId(final long checkpointId) { - if (this.checkpointId == NOT_SET_CHECKPOINT_ID) { - this.checkpointId = checkpointId; - } - } - - void blockUntilCheckpointIsAcknowledged() throws InterruptedException { - synchronized (synchronizationPoint) { - if (isSet()) { - while (completionResult == null) { - waiting = true; - synchronizationPoint.wait(); - } - waiting = false; - } - } - } - - void acknowledgeCheckpointAndTrigger(final long checkpointId, RunnableWithException runnable) throws Exception { - synchronized (synchronizationPoint) { - if (completionResult == null && this.checkpointId == checkpointId) { - completionResult = CompletionResult.COMPLETED; - try { - runnable.run(); - } finally { - synchronizationPoint.notifyAll(); - } - } - } - } - - void cancelCheckpointLatch() { - synchronized (synchronizationPoint) { - if (completionResult == null) { - completionResult = CompletionResult.CANCELED; - synchronizationPoint.notifyAll(); - } - } - } - - @VisibleForTesting - boolean isWaiting() { - return waiting; - } - - @VisibleForTesting - boolean isCompleted() { - return completionResult == CompletionResult.COMPLETED; - } - - @VisibleForTesting - boolean isCanceled() { - return completionResult == CompletionResult.CANCELED; - } - - boolean isSet() { - return checkpointId != NOT_SET_CHECKPOINT_ID; - } -} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousSavepointSyncLatchTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousSavepointSyncLatchTest.java deleted file mode 100644 index 59f5a8195871..000000000000 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousSavepointSyncLatchTest.java +++ /dev/null @@ -1,182 +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.util.function.RunnableWithException; - -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -/** - * Tests for the {@link SynchronousSavepointLatch}. - */ -public class SynchronousSavepointSyncLatchTest { - - private ExecutorService executors; - - @Before - public void startExecutorService() { - executors = Executors.newCachedThreadPool(); - } - - @After - public void terminateExecutors() throws InterruptedException { - while (!executors.isTerminated()) { - executors.shutdownNow(); - executors.awaitTermination(10, TimeUnit.SECONDS); - } - } - - @Test - public void triggerUnblocksWait() throws Exception { - final SynchronousSavepointLatch latchUnderTest = new SynchronousSavepointLatch(); - - latchUnderTest.setCheckpointId(1L); - assertFalse(latchUnderTest.isWaiting()); - - Future future = runThreadWaitingForCheckpointAck(latchUnderTest); - while (!latchUnderTest.isWaiting()) { - Thread.sleep(5L); - } - - final AtomicBoolean triggered = new AtomicBoolean(); - - // wrong checkpoint id. - latchUnderTest.acknowledgeCheckpointAndTrigger(2L, () -> triggered.set(true)); - assertFalse(triggered.get()); - assertFalse(latchUnderTest.isCompleted()); - assertTrue(latchUnderTest.isWaiting()); - - latchUnderTest.acknowledgeCheckpointAndTrigger(1L, () -> triggered.set(true)); - assertTrue(triggered.get()); - assertTrue(latchUnderTest.isCompleted()); - - future.get(); - assertFalse(latchUnderTest.isWaiting()); - } - - @Test - public void cancelUnblocksWait() throws Exception { - final SynchronousSavepointLatch latchUnderTest = new SynchronousSavepointLatch(); - - latchUnderTest.setCheckpointId(1L); - assertFalse(latchUnderTest.isWaiting()); - - Future future = runThreadWaitingForCheckpointAck(latchUnderTest); - while (!latchUnderTest.isWaiting()) { - Thread.sleep(5L); - } - - latchUnderTest.cancelCheckpointLatch(); - assertTrue(latchUnderTest.isCanceled()); - - future.get(); - assertFalse(latchUnderTest.isWaiting()); - } - - @Test - public void waitAfterTriggerIsNotBlocking() throws Exception { - final SynchronousSavepointLatch latchUnderTest = new SynchronousSavepointLatch(); - - latchUnderTest.setCheckpointId(1L); - latchUnderTest.acknowledgeCheckpointAndTrigger(1L, () -> {}); - - latchUnderTest.blockUntilCheckpointIsAcknowledged(); - } - - @Test - public void waitAfterCancelIsNotBlocking() throws Exception { - final SynchronousSavepointLatch latchUnderTest = new SynchronousSavepointLatch(); - - latchUnderTest.setCheckpointId(1L); - latchUnderTest.cancelCheckpointLatch(); - assertTrue(latchUnderTest.isCanceled()); - - latchUnderTest.blockUntilCheckpointIsAcknowledged(); - } - - @Test - public void triggeringInvokesCallbackAtMostOnce() throws Exception { - final SynchronousSavepointLatch latchUnderTest = new SynchronousSavepointLatch(); - - latchUnderTest.setCheckpointId(1L); - - AtomicInteger counter = new AtomicInteger(); - Future future1 = runThreadTriggeringCheckpoint(latchUnderTest, 1L, counter::incrementAndGet); - Future future2 = runThreadTriggeringCheckpoint(latchUnderTest, 1L, counter::incrementAndGet); - Future future3 = runThreadTriggeringCheckpoint(latchUnderTest, 1L, counter::incrementAndGet); - future1.get(); - future2.get(); - future3.get(); - - assertEquals(1, counter.get()); - } - - @Test - public void triggeringAfterCancelDoesNotInvokeCallback() throws Exception { - final SynchronousSavepointLatch latchUnderTest = new SynchronousSavepointLatch(); - - latchUnderTest.setCheckpointId(1L); - latchUnderTest.cancelCheckpointLatch(); - assertTrue(latchUnderTest.isCanceled()); - - final AtomicBoolean triggered = new AtomicBoolean(); - latchUnderTest.acknowledgeCheckpointAndTrigger(1L, () -> triggered.set(true)); - assertFalse(triggered.get()); - } - - @Test - public void checkpointIdIsSetOnlyOnce() { - final SynchronousSavepointLatch latchUnderTest = new SynchronousSavepointLatch(); - - latchUnderTest.setCheckpointId(1L); - assertTrue(latchUnderTest.isSet()); - assertEquals(1L, latchUnderTest.getCheckpointId()); - - latchUnderTest.setCheckpointId(2L); - assertTrue(latchUnderTest.isSet()); - assertEquals(1L, latchUnderTest.getCheckpointId()); - } - - private Future runThreadWaitingForCheckpointAck(SynchronousSavepointLatch latch) { - return executors.submit(() -> { - latch.blockUntilCheckpointIsAcknowledged(); - return null; - }); - } - - private Future runThreadTriggeringCheckpoint(SynchronousSavepointLatch latch, long checkpointId, RunnableWithException runnable) { - return executors.submit(() -> { - latch.acknowledgeCheckpointAndTrigger(checkpointId, runnable); - return null; - }); - } -} From 563a4adecc0e9273c09dda83da2875a4d6a30ee0 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Mon, 28 Oct 2019 15:48:46 +0100 Subject: [PATCH 576/746] [FLINK-14486][table-api, docs] Update documentation regarding Temporary Objects This closes #10078 --- docs/dev/table/catalogs.md | 8 +- docs/dev/table/catalogs.zh.md | 8 +- docs/dev/table/common.md | 412 ++++++++++------- docs/dev/table/common.zh.md | 426 +++++++++++------- docs/dev/table/connect.md | 10 +- docs/dev/table/connect.zh.md | 14 +- docs/dev/table/sourceSinks.md | 6 +- docs/dev/table/sourceSinks.zh.md | 24 +- docs/dev/table/sql.md | 62 ++- docs/dev/table/sql.zh.md | 72 +-- docs/dev/table/streaming/temporal_tables.md | 4 +- docs/dev/table/streaming/time_attributes.md | 8 +- .../dev/table/streaming/time_attributes.zh.md | 8 +- docs/dev/table/tableApi.md | 182 ++++---- docs/dev/table/tableApi.zh.md | 180 ++++---- docs/ops/scala_shell.md | 4 +- docs/ops/scala_shell.zh.md | 4 +- docs/tutorials/python_table_api.md | 12 +- 18 files changed, 820 insertions(+), 624 deletions(-) diff --git a/docs/dev/table/catalogs.md b/docs/dev/table/catalogs.md index 52503e6af7e0..cfd34b547e83 100644 --- a/docs/dev/table/catalogs.md +++ b/docs/dev/table/catalogs.md @@ -36,8 +36,7 @@ Or permanent metadata, like that in a Hive Metastore. Catalogs provide a unified ### GenericInMemoryCatalog -Flink sessions always have a built-in `GenericInMemoryCatalog` named `default_catalog`, which has a built-in default database named `default_database`. -All temporary metadata, such tables defined using `TableEnvironment#registerTable` is registered to this catalog. +The `GenericInMemoryCatalog` is an in-memory implementation of a catalog. All objects will be available only for the lifetime of the session. ### HiveCatalog @@ -59,7 +58,8 @@ The set of properties will be passed to a discovery service where the service tr ### Registering a Catalog -Users can register additional catalogs into an existing Flink session. +Users have access to a default in-memory catalog named `default_catalog`, that is always created by default. This catalog by default has a single database called `default_database`. +Users can also register additional catalogs into an existing Flink session.

    @@ -124,7 +124,7 @@ Metadata from catalogs that are not the current catalog are accessible by provid
    {% highlight java %} -tableEnv.scan("not_the_current_catalog", "not_the_current_db", "my_table"); +tableEnv.from("not_the_current_catalog.not_the_current_db.my_table"); {% endhighlight %}
    diff --git a/docs/dev/table/catalogs.zh.md b/docs/dev/table/catalogs.zh.md index 52503e6af7e0..cfd34b547e83 100644 --- a/docs/dev/table/catalogs.zh.md +++ b/docs/dev/table/catalogs.zh.md @@ -36,8 +36,7 @@ Or permanent metadata, like that in a Hive Metastore. Catalogs provide a unified ### GenericInMemoryCatalog -Flink sessions always have a built-in `GenericInMemoryCatalog` named `default_catalog`, which has a built-in default database named `default_database`. -All temporary metadata, such tables defined using `TableEnvironment#registerTable` is registered to this catalog. +The `GenericInMemoryCatalog` is an in-memory implementation of a catalog. All objects will be available only for the lifetime of the session. ### HiveCatalog @@ -59,7 +58,8 @@ The set of properties will be passed to a discovery service where the service tr ### Registering a Catalog -Users can register additional catalogs into an existing Flink session. +Users have access to a default in-memory catalog named `default_catalog`, that is always created by default. This catalog by default has a single database called `default_database`. +Users can also register additional catalogs into an existing Flink session.
    @@ -124,7 +124,7 @@ Metadata from catalogs that are not the current catalog are accessible by provid
    {% highlight java %} -tableEnv.scan("not_the_current_catalog", "not_the_current_db", "my_table"); +tableEnv.from("not_the_current_catalog.not_the_current_db.my_table"); {% endhighlight %}
    diff --git a/docs/dev/table/common.md b/docs/dev/table/common.md index c8523e415817..bfd49ebc6a4e 100644 --- a/docs/dev/table/common.md +++ b/docs/dev/table/common.md @@ -52,16 +52,15 @@ All Table API and SQL programs for batch and streaming follow the same pattern. // create a TableEnvironment for specific planner batch or streaming TableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section -// register a Table -tableEnv.registerTable("table1", ...) // or -tableEnv.registerTableSource("table2", ...); +// create a Table +tableEnv.connect(...).createTemporaryTable("table1"); // register an output Table -tableEnv.registerTableSink("outputTable", ...); +tableEnv.connect(...).createTemporaryTable("outputTable"); -// create a Table from a Table API query -Table tapiResult = tableEnv.scan("table1").select(...); -// create a Table from a SQL query -Table sqlResult = tableEnv.sqlQuery("SELECT ... FROM table2 ... "); +// create a Table object from a Table API query +Table tapiResult = tableEnv.from("table1").select(...); +// create a Table object from a SQL query +Table sqlResult = tableEnv.sqlQuery("SELECT ... FROM table1 ... "); // emit a Table API result Table to a TableSink, same for SQL result tapiResult.insertInto("outputTable"); @@ -78,16 +77,15 @@ tableEnv.execute("java_job"); // create a TableEnvironment for specific planner batch or streaming val tableEnv = ... // see "Create a TableEnvironment" section -// register a Table -tableEnv.registerTable("table1", ...) // or -tableEnv.registerTableSource("table2", ...) +// create a Table +tableEnv.connect(...).createTemporaryTable("table1") // register an output Table -tableEnv.registerTableSink("outputTable", ...); +tableEnv.connect(...).createTemporaryTable("outputTable") // create a Table from a Table API query -val tapiResult = tableEnv.scan("table1").select(...) +val tapiResult = tableEnv.from("table1").select(...) // create a Table from a SQL query -val sqlResult = tableEnv.sqlQuery("SELECT ... FROM table2 ...") +val sqlResult = tableEnv.sqlQuery("SELECT ... FROM table1 ...") // emit a Table API result Table to a TableSink, same for SQL result tapiResult.insertInto("outputTable") @@ -105,16 +103,15 @@ tableEnv.execute("scala_job") table_env = ... # see "Create a TableEnvironment" section # register a Table -table_env.register_table("table1", ...) # or -table_env.register_table_source("table2", ...) +table_env.connect(...).create_temporary_table("table1") # register an output Table -table_env.register_table_sink("outputTable", ...); +table_env.connect(...).create_temporary_table("outputTable") # create a Table from a Table API query -tapi_result = table_env.scan("table1").select(...) +tapi_result = table_env.from_path("table1").select(...) # create a Table from a SQL query -sql_result = table_env.sql_query("SELECT ... FROM table2 ...") +sql_result = table_env.sql_query("SELECT ... FROM table1 ...") # emit a Table API result Table to a TableSink, same for SQL result tapi_result.insert_into("outputTable") @@ -292,25 +289,51 @@ b_b_t_env = BatchTableEnvironment.create(environment_settings=b_b_settings) **Note:** If there is only one planner jar in `/lib` directory, you can use `useAnyPlanner` (`use_any_planner` for python) to create specific `EnvironmentSettings`. - {% top %} -Register Tables in the Catalog +Create Tables in the Catalog ------------------------------- -A `TableEnvironment` maintains a catalog of tables which are registered by name. There are two types of tables, *input tables* and *output tables*. Input tables can be referenced in Table API and SQL queries and provide input data. Output tables can be used to emit the result of a Table API or SQL query to an external system. +A `TableEnvironment` maintains a map of catalogs of tables which are created with an identifier. Each +identifier consists of 3 parts: catalog name, database name and object name. If a catalog or database is not +specified, the current default value will be used (see examples in the [Table identifier expanding](#table-identifier-expanding) section). + +Tables can be either virtual (`VIEWS`) or regular (`TABLES`). `VIEWS` can be created from an +existing `Table` object, usually the result of a Table API or SQL query. `TABLES` describe +external data, such as a file, database table, or message queue. + +### Temporary vs Permanent tables. + +Tables may either be temporary, and tied to the lifecycle of a single Flink session, or permanent, +and visible across multiple Flink sessions and clusters. + +Permanent tables require a [catalog]({{ site.baseurl }}/dev/table/catalogs.html) (such as Hive Metastore) +to maintain metadata about the table. Once a permanent table is created, it is visible to any Flink +session that is connected to the catalog and will continue to exist until the table is explicitly +dropped. -An input table can be registered from various sources: +On the other hand, temporary tables are always stored in memory and only exist for the duration of +the Flink session they are created within. These tables are not visible to other sessions. They are +not bound to any catalog or database but can be created in the namespace of one. Temporary tables +are not dropped if their corresponding database is removed. -* an existing `Table` object, usually the result of a Table API or SQL query. -* a `TableSource`, which accesses external data, such as a file, database, or messaging system. -* a `DataStream` or `DataSet` from a DataStream (only for stream job) or DataSet (only for batch job translated from old planner) program. Registering a `DataStream` or `DataSet` is discussed in the [Integration with DataStream and DataSet API](#integration-with-datastream-and-dataset-api) section. +#### Shadowing -An output table can be registered using a `TableSink`. +It is possible to register a temporary table with the same identifier as an existing permanent +table. The temporary table shadows the permanent one and makes the permanent table inaccessible as +long as the temporary one exists. All queries with that identifier will be executed against the +temporary table. -### Register a Table +This might be useful for experimentation. It allows running exactly the same query first against a +temporary table that e.g. has just a subset of data, or the data is obfuscated. Once verified that +the query is correct it can be run against the real production table. -A `Table` is registered in a `TableEnvironment` as follows: +### Create a Table + +#### Virtual Tables + +A `Table` API object corresponds to a `VIEW` (virtual table) in a SQL terms. It encapsulates a logical +query plan. It can be created in a catalog as follows:
    @@ -319,10 +342,10 @@ A `Table` is registered in a `TableEnvironment` as follows: TableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section // table is the result of a simple projection query -Table projTable = tableEnv.scan("X").select(...); +Table projTable = tableEnv.from("X").select(...); // register the Table projTable as table "projectedTable" -tableEnv.registerTable("projectedTable", projTable); +tableEnv.createTemporaryView("projectedTable", projTable); {% endhighlight %}
    @@ -332,10 +355,10 @@ tableEnv.registerTable("projectedTable", projTable); val tableEnv = ... // see "Create a TableEnvironment" section // table is the result of a simple projection query -val projTable: Table = tableEnv.scan("X").select(...) +val projTable: Table = tableEnv.from("X").select(...) // register the Table projTable as table "projectedTable" -tableEnv.registerTable("projectedTable", projTable) +tableEnv.createTemporaryView("projectedTable", projTable) {% endhighlight %}
    @@ -345,7 +368,7 @@ tableEnv.registerTable("projectedTable", projTable) table_env = ... # see "Create a TableEnvironment" section # table is the result of a simple projection query -proj_table = table_env.scan("X").select(...) +proj_table = table_env.from_path("X").select(...) # register the Table projTable as table "projectedTable" table_env.register_table("projectedTable", proj_table) @@ -353,125 +376,131 @@ table_env.register_table("projectedTable", proj_table)
    -**Note:** A registered `Table` is treated similarly to a `VIEW` as known from relational database systems, i.e., the query that defines the `Table` is not optimized but will be inlined when another query references the registered `Table`. If multiple queries reference the same registered `Table`, it will be inlined for each referencing query and executed multiple times, i.e., the result of the registered `Table` will *not* be shared. +**Note:** `Table` objects are similar to `VIEW`'s from relational database +systems, i.e., the query that defines the `Table` is not optimized but will be inlined when another +query references the registered `Table`. If multiple queries reference the same registered `Table`, +it will be inlined for each referencing query and executed multiple times, i.e., the result of the +registered `Table` will *not* be shared. {% top %} -### Register a TableSource - -A `TableSource` provides access to external data which is stored in a storage system such as a database (MySQL, HBase, ...), a file with a specific encoding (CSV, Apache \[Parquet, Avro, ORC\], ...), or a messaging system (Apache Kafka, RabbitMQ, ...). +#### Connector Tables -Flink aims to provide TableSources for common data formats and storage systems. Please have a look at the [Table Sources and Sinks]({{ site.baseurl }}/dev/table/sourceSinks.html) page for a list of supported TableSources and instructions for how to build a custom `TableSource`. - -A `TableSource` is registered in a `TableEnvironment` as follows: +It is also possible to create a `TABLE` as known from relational databases from a [connector]({{ site.baseurl }}/dev/table/connect.html) declaration. +The connector describes the external system that stores the data of a table. Storage systems such as Apacha Kafka or a regular file system can be declared here.
    {% highlight java %} -// get a TableEnvironment -TableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section - -// create a TableSource -TableSource csvSource = new CsvTableSource("/path/to/file", ...); - -// register the TableSource as table "CsvTable" -tableEnv.registerTableSource("CsvTable", csvSource); +tableEnvironment + .connect(...) + .withFormat(...) + .withSchema(...) + .inAppendMode() + .createTemporaryTable("MyTable") {% endhighlight %}
    {% highlight scala %} -// get a TableEnvironment -val tableEnv = ... // see "Create a TableEnvironment" section - -// create a TableSource -val csvSource: TableSource = new CsvTableSource("/path/to/file", ...) - -// register the TableSource as table "CsvTable" -tableEnv.registerTableSource("CsvTable", csvSource) +tableEnvironment + .connect(...) + .withFormat(...) + .withSchema(...) + .inAppendMode() + .createTemporaryTable("MyTable") {% endhighlight %}
    {% highlight python %} -# get a TableEnvironment -table_env = ... # see "Create a TableEnvironment" section - -# create a TableSource -csv_source = CsvTableSource("/path/to/file", ...) +table_environment \ + .connect(...) \ + .with_format(...) \ + .with_schema(...) \ + .in_append_mode() \ + .create_temporary_table("MyTable") +{% endhighlight %} +
    -# register the TableSource as table "csvTable" -table_env.register_table_source("csvTable", csv_source) +
    +{% highlight sql %} +tableEnvironment.sqlUpdate("CREATE [TEMPORARY] TABLE MyTable (...) WITH (...)") {% endhighlight %}
    -**Note:** A `TableEnvironment` used for Blink planner only accepts `StreamTableSource`, `LookupableTableSource` and `InputFormatTableSource`, and a `StreamTableSource` used for Blink planner on batch must be bounded. +### Expanding Table identifiers -{% top %} +Tables are always registered with a 3 part identifier consisting of catalog, database, and +table name. The first two parts are optional and if they are not provided the set default values will +be used. Identifiers follow SQL requirements which means that they can be escaped with a backtick character (`` ` ``). +Additionally all SQL reserved keywords must be escaped. -### Register a TableSink +
    +
    +{% highlight java %} +TableEnvironment tEnv = ...; +tEnv.useCatalog("custom_catalog"); +tEnv.useDatabase("custom_database"); -A registered `TableSink` can be used to [emit the result of a Table API or SQL query](common.html#emit-a-table) to an external storage system, such as a database, key-value store, message queue, or file system (in different encodings, e.g., CSV, Apache \[Parquet, Avro, ORC\], ...). +Table table = ...; -Flink aims to provide TableSinks for common data formats and storage systems. Please see the documentation about [Table Sources and Sinks]({{ site.baseurl }}/dev/table/sourceSinks.html) page for details about available sinks and instructions for how to implement a custom `TableSink`. +// register the view named 'exampleView' in the catalog named 'custom_catalog' +// in the database named 'custom_database' +tableEnv.createTemporaryView("exampleView", table); -A `TableSink` is registered in a `TableEnvironment` as follows: +// register the view named 'exampleView' in the catalog named 'custom_catalog' +// in the database named 'other_database' +tableEnv.createTemporaryView("other_database.exampleView", table); -
    -
    -{% highlight java %} -// get a TableEnvironment -TableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section +// register the view named 'View' in the catalog named 'custom_catalog' in the +// database named 'custom_database'. 'View' is a reserved keyword and must be escaped. +tableEnv.createTemporaryView("`View`", table); -// create a TableSink -TableSink csvSink = new CsvTableSink("/path/to/file", ...); +// register the view named 'example.View' in the catalog named 'custom_catalog' +// in the database named 'custom_database' +tableEnv.createTemporaryView("`example.View`", table); -// define the field names and types -String[] fieldNames = {"a", "b", "c"}; -TypeInformation[] fieldTypes = {Types.INT, Types.STRING, Types.LONG}; +// register the view named 'exampleView' in the catalog named 'other_catalog' +// in the database named 'other_database' +tableEnv.createTemporaryView("other_catalog.other_database.exampleView", table); -// register the TableSink as table "CsvSinkTable" -tableEnv.registerTableSink("CsvSinkTable", fieldNames, fieldTypes, csvSink); {% endhighlight %}
    {% highlight scala %} // get a TableEnvironment -val tableEnv = ... // see "Create a TableEnvironment" section - -// create a TableSink -val csvSink: TableSink = new CsvTableSink("/path/to/file", ...) +val tEnv: TableEnvironment = ...; +tEnv.useCatalog("custom_catalog") +tEnv.useDatabase("custom_database") -// define the field names and types -val fieldNames: Array[String] = Array("a", "b", "c") -val fieldTypes: Array[TypeInformation[_]] = Array(Types.INT, Types.STRING, Types.LONG) +val table: Table = ...; -// register the TableSink as table "CsvSinkTable" -tableEnv.registerTableSink("CsvSinkTable", fieldNames, fieldTypes, csvSink) -{% endhighlight %} -
    +// register the view named 'exampleView' in the catalog named 'custom_catalog' +// in the database named 'custom_database' +tableEnv.createTemporaryView("exampleView", table) -
    -{% highlight python %} -# get a TableEnvironment -table_env = ... # see "Create a TableEnvironment" section +// register the view named 'exampleView' in the catalog named 'custom_catalog' +// in the database named 'other_database' +tableEnv.createTemporaryView("other_database.exampleView", table) -# define the field names and types -field_names = ["a", "b", "c"] -field_types = [DataTypes.INT(), DataTypes.STRING(), DataTypes.BIGINT()] +// register the view named 'View' in the catalog named 'custom_catalog' in the +// database named 'custom_database'. 'View' is a reserved keyword and must be escaped. +tableEnv.createTemporaryView("`View`", table) -# create a TableSink -csv_sink = CsvTableSink(field_names, field_types, "/path/to/file", ...) +// register the view named 'example.View' in the catalog named 'custom_catalog' +// in the database named 'custom_database' +tableEnv.createTemporaryView("`example.View`", table) -# register the TableSink as table "CsvSinkTable" -table_env.register_table_sink("CsvSinkTable", csv_sink) +// register the view named 'exampleView' in the catalog named 'other_catalog' +// in the database named 'other_database' +tableEnv.createTemporaryView("other_catalog.other_database.exampleView", table) {% endhighlight %}
    -
    -{% top %} +
    Query a Table ------------- @@ -495,7 +524,7 @@ TableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section // register Orders table // scan registered Orders table -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); // compute revenue for all customers from France Table revenue = orders .filter("cCountry === 'FRANCE'") @@ -515,7 +544,7 @@ val tableEnv = ... // see "Create a TableEnvironment" section // register Orders table // scan registered Orders table -val orders = tableEnv.scan("Orders") +val orders = tableEnv.from("Orders") // compute revenue for all customers from France val revenue = orders .filter('cCountry === "FRANCE") @@ -537,7 +566,7 @@ table_env = # see "Create a TableEnvironment" section # register Orders table # scan registered Orders table -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") # compute revenue for all customers from France revenue = orders \ .filter("cCountry === 'FRANCE'") \ @@ -721,13 +750,16 @@ The following examples shows how to emit a `Table`: // get a TableEnvironment TableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section -// create a TableSink -TableSink sink = new CsvTableSink("/path/to/file", fieldDelim = "|"); +// create an output Table +final Schema schema = new Schema() + .field("a", DataTypes.INT()) + .field("b", DataTypes.STRING()) + .field("c", DataTypes.LONG()); -// register the TableSink with a specific schema -String[] fieldNames = {"a", "b", "c"}; -TypeInformation[] fieldTypes = {Types.INT, Types.STRING, Types.LONG}; -tableEnv.registerTableSink("CsvSinkTable", fieldNames, fieldTypes, sink); +tableEnv.connect(new FileSystem("/path/to/file")) + .withFormat(new Csv().fieldDelimiter('|').deriveSchema()) + .withSchema(schema) + .createTemporaryTable("CsvSinkTable"); // compute a result Table using Table API operators and/or SQL queries Table result = ... @@ -743,13 +775,16 @@ result.insertInto("CsvSinkTable"); // get a TableEnvironment val tableEnv = ... // see "Create a TableEnvironment" section -// create a TableSink -val sink: TableSink = new CsvTableSink("/path/to/file", fieldDelim = "|") +// create an output Table +val schema = new Schema() + .field("a", DataTypes.INT()) + .field("b", DataTypes.STRING()) + .field("c", DataTypes.LONG()) -// register the TableSink with a specific schema -val fieldNames: Array[String] = Array("a", "b", "c") -val fieldTypes: Array[TypeInformation] = Array(Types.INT, Types.STRING, Types.LONG) -tableEnv.registerTableSink("CsvSinkTable", fieldNames, fieldTypes, sink) +tableEnv.connect(new FileSystem("/path/to/file")) + .withFormat(new Csv().fieldDelimiter('|').deriveSchema()) + .withSchema(schema) + .createTemporaryTable("CsvSinkTable") // compute a result Table using Table API operators and/or SQL queries val result: Table = ... @@ -766,13 +801,16 @@ result.insertInto("CsvSinkTable") # get a TableEnvironment table_env = ... # see "Create a TableEnvironment" section -field_names = ["a", "b", "c"] -field_types = [DataTypes.INT(), DataTypes.STRING(), DataTypes.BIGINT()] - # create a TableSink -sink = CsvTableSink(field_names, field_types, "/path/to/file", "|") - -table_env.register_table_sink("CsvSinkTable", sink) +t_env.connect(FileSystem().path("/path/to/file"))) + .with_format(Csv() + .field_delimiter(',') + .deriveSchema()) + .with_schema(Schema() + .field("a", DataTypes.INT()) + .field("b", DataTypes.STRING()) + .field("c", DataTypes.BIGINT())) + .create_temporary_table("CsvSinkTable") # compute a result Table using Table API operators and/or SQL queries result = ... @@ -847,9 +885,11 @@ This interaction can be achieved by converting a `DataStream` or `DataSet` into The Scala Table API features implicit conversions for the `DataSet`, `DataStream`, and `Table` classes. These conversions are enabled by importing the package `org.apache.flink.table.api.scala._` in addition to `org.apache.flink.api.scala._` for the Scala DataStream API. -### Register a DataStream or DataSet as Table +### Create a View from a DataStream or DataSet + +A `DataStream` or `DataSet` can be registered in a `TableEnvironment` as a View. The schema of the resulting view depends on the data type of the registered `DataStream` or `DataSet`. Please check the section about [mapping of data types to table schema](#mapping-of-data-types-to-table-schema) for details. -A `DataStream` or `DataSet` can be registered in a `TableEnvironment` as a Table. The schema of the resulting table depends on the data type of the registered `DataStream` or `DataSet`. Please check the section about [mapping of data types to table schema](#mapping-of-data-types-to-table-schema) for details. +**Note:** Views created from a `DataStream` or `DataSet` can be registered as temporary views only.
    @@ -860,11 +900,11 @@ StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" sectio DataStream> stream = ... -// register the DataStream as Table "myTable" with fields "f0", "f1" -tableEnv.registerDataStream("myTable", stream); +// register the DataStream as View "myTable" with fields "f0", "f1" +tableEnv.createTemporaryView("myTable", stream); -// register the DataStream as table "myTable2" with fields "myLong", "myString" -tableEnv.registerDataStream("myTable2", stream, "myLong, myString"); +// register the DataStream as View "myTable2" with fields "myLong", "myString" +tableEnv.createTemporaryView("myTable2", stream, "myLong, myString"); {% endhighlight %}
    @@ -876,17 +916,15 @@ val tableEnv: StreamTableEnvironment = ... // see "Create a TableEnvironment" se val stream: DataStream[(Long, String)] = ... -// register the DataStream as Table "myTable" with fields "f0", "f1" -tableEnv.registerDataStream("myTable", stream) +// register the DataStream as View "myTable" with fields "f0", "f1" +tableEnv.createTemporaryView("myTable", stream) -// register the DataStream as table "myTable2" with fields "myLong", "myString" -tableEnv.registerDataStream("myTable2", stream, 'myLong, 'myString) +// register the DataStream as View "myTable2" with fields "myLong", "myString" +tableEnv.createTemporaryView("myTable2", stream, 'myLong, 'myString) {% endhighlight %}
    -**Note:** The name of a `DataStream` `Table` must not match the `^_DataStreamTable_[0-9]+` pattern and the name of a `DataSet` `Table` must not match the `^_DataSetTable_[0-9]+` pattern. These patterns are reserved for internal use only. - {% top %} ### Convert a DataStream or DataSet into a Table @@ -1601,17 +1639,31 @@ The following code shows an example and the corresponding output for multiple-si EnvironmentSettings settings = EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build(); TableEnvironment tEnv = TableEnvironment.create(settings); -String[] fieldNames = { "count", "word" }; -TypeInformation[] fieldTypes = { Types.INT, Types.STRING }; -tEnv.registerTableSource("MySource1", new CsvTableSource("/source/path1", fieldNames, fieldTypes)); -tEnv.registerTableSource("MySource2", new CsvTableSource("/source/path2", fieldNames, fieldTypes)); -tEnv.registerTableSink("MySink1", new CsvTableSink("/sink/path1").configure(fieldNames, fieldTypes)); -tEnv.registerTableSink("MySink2", new CsvTableSink("/sink/path2").configure(fieldNames, fieldTypes)); - -Table table1 = tEnv.scan("MySource1").where("LIKE(word, 'F%')"); +final Schema schema = new Schema() + .field("count", DataTypes.INT()) + .field("word", DataTypes.STRING()); + +tEnv.connect(new FileSystem("/source/path1")) + .withFormat(new Csv().deriveSchema()) + .withSchema(schema) + .createTemporaryTable("MySource1"); +tEnv.connect(new FileSystem("/source/path2")) + .withFormat(new Csv().deriveSchema()) + .withSchema(schema) + .createTemporaryTable("MySource2"); +tEnv.connect(new FileSystem("/sink/path1")) + .withFormat(new Csv().deriveSchema()) + .withSchema(schema) + .createTemporaryTable("MySink1"); +tEnv.connect(new FileSystem("/sink/path2")) + .withFormat(new Csv().deriveSchema()) + .withSchema(schema) + .createTemporaryTable("MySink2"); + +Table table1 = tEnv.from("MySource1").where("LIKE(word, 'F%')"); table1.insertInto("MySink1"); -Table table2 = table1.unionAll(tEnv.scan("MySource2")); +Table table2 = table1.unionAll(tEnv.from("MySource2")); table2.insertInto("MySink2"); String explanation = tEnv.explain(false); @@ -1625,17 +1677,31 @@ System.out.println(explanation); val settings = EnvironmentSettings.newInstance.useBlinkPlanner.inStreamingMode.build val tEnv = TableEnvironment.create(settings) -val fieldNames = Array("count", "word") -val fieldTypes = Array[TypeInformation[_]](Types.INT, Types.STRING) -tEnv.registerTableSource("MySource1", new CsvTableSource("/source/path1", fieldNames, fieldTypes)) -tEnv.registerTableSource("MySource2", new CsvTableSource("/source/path2",fieldNames, fieldTypes)) -tEnv.registerTableSink("MySink1", new CsvTableSink("/sink/path1").configure(fieldNames, fieldTypes)) -tEnv.registerTableSink("MySink2", new CsvTableSink("/sink/path2").configure(fieldNames, fieldTypes)) - -val table1 = tEnv.scan("MySource1").where("LIKE(word, 'F%')") +val schema = new Schema() + .field("count", DataTypes.INT()) + .field("word", DataTypes.STRING()) + +tEnv.connect(new FileSystem("/source/path1")) + .withFormat(new Csv().deriveSchema()) + .withSchema(schema) + .createTemporaryTable("MySource1") +tEnv.connect(new FileSystem("/source/path2")) + .withFormat(new Csv().deriveSchema()) + .withSchema(schema) + .createTemporaryTable("MySource2") +tEnv.connect(new FileSystem("/sink/path1")) + .withFormat(new Csv().deriveSchema()) + .withSchema(schema) + .createTemporaryTable("MySink1") +tEnv.connect(new FileSystem("/sink/path2")) + .withFormat(new Csv().deriveSchema()) + .withSchema(schema) + .createTemporaryTable("MySink2") + +val table1 = tEnv.from("MySource1").where("LIKE(word, 'F%')") table1.insertInto("MySink1") -val table2 = table1.unionAll(tEnv.scan("MySource2")) +val table2 = table1.unionAll(tEnv.from("MySource2")) table2.insertInto("MySink2") val explanation = tEnv.explain(false) @@ -1649,17 +1715,31 @@ println(explanation) settings = EnvironmentSettings.new_instance().use_blink_planner().in_streaming_mode().build() t_env = TableEnvironment.create(environment_settings=settings) -field_names = ["count", "word"] -field_types = [DataTypes.INT(), DataTypes.STRING()] -t_env.register_table_source("MySource1", CsvTableSource("/source/path1", field_names, field_types)) -t_env.register_table_source("MySource2", CsvTableSource("/source/path2", field_names, field_types)) -t_env.register_table_sink("MySink1", CsvTableSink("/sink/path1", field_names, field_types)) -t_env.register_table_sink("MySink2", CsvTableSink("/sink/path2", field_names, field_types)) - -table1 = t_env.scan("MySource1").where("LIKE(word, 'F%')") +schema = Schema() + .field("count", DataTypes.INT()) + .field("word", DataTypes.STRING()) + +t_env.connect(FileSystem().path("/source/path1"))) + .with_format(Csv().deriveSchema()) + .with_schema(schema) + .create_temporary_table("MySource1") +t_env.connect(FileSystem().path("/source/path2"))) + .with_format(Csv().deriveSchema()) + .with_schema(schema) + .create_temporary_table("MySource2") +t_env.connect(FileSystem().path("/sink/path1"))) + .with_format(Csv().deriveSchema()) + .with_schema(schema) + .create_temporary_table("MySink1") +t_env.connect(FileSystem().path("/sink/path2"))) + .with_format(Csv().deriveSchema()) + .with_schema(schema) + .create_temporary_table("MySink2") + +table1 = t_env.from_path("MySource1").where("LIKE(word, 'F%')") table1.insert_into("MySink1") -table2 = table1.union_all(t_env.scan("MySource2")) +table2 = table1.union_all(t_env.from_path("MySource2")) table2.insert_into("MySink2") explanation = t_env.explain() diff --git a/docs/dev/table/common.zh.md b/docs/dev/table/common.zh.md index f031a7da1428..bfd49ebc6a4e 100644 --- a/docs/dev/table/common.zh.md +++ b/docs/dev/table/common.zh.md @@ -1,5 +1,5 @@ --- -title: "概念与通用 API" +title: "Concepts & Common API" nav-parent_id: tableapi nav-pos: 0 --- @@ -30,8 +30,8 @@ The Table API and SQL are integrated in a joint API. The central concept of this Main Differences Between the Two Planners ----------------------------------------- -1. Blink treats batch jobs as a special case of streaming. As such, the conversion between Table and DataSet is not supported, and batch jobs will not be translated into `DateSet` programs but translated into `DataStream` programs, the same as the streaming jobs. -2. The Blink planner does not support `BatchTableSource`, uses bounded `StreamTableSource` instead of it. +1. Blink treats batch jobs as a special case of streaming. As such, the conversion between Table and DataSet is also not supported, and batch jobs will not be translated into `DateSet` programs but translated into `DataStream` programs, the same as the streaming jobs. +2. The Blink planner does not support `BatchTableSource`, use bounded `StreamTableSource` instead of it. 3. The Blink planner only support the brand new `Catalog` and does not support `ExternalCatalog` which is deprecated. 4. The implementations of `FilterableTableSource` for the old planner and the Blink planner are incompatible. The old planner will push down `PlannerExpression`s into `FilterableTableSource`, while the Blink planner will push down `Expression`s. 5. String based key-value config options (Please see the documentation about [Configuration]({{ site.baseurl }}/dev/table/config.html) for details) are only used for the Blink planner. @@ -52,16 +52,15 @@ All Table API and SQL programs for batch and streaming follow the same pattern. // create a TableEnvironment for specific planner batch or streaming TableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section -// register a Table -tableEnv.registerTable("table1", ...) // or -tableEnv.registerTableSource("table2", ...); +// create a Table +tableEnv.connect(...).createTemporaryTable("table1"); // register an output Table -tableEnv.registerTableSink("outputTable", ...); +tableEnv.connect(...).createTemporaryTable("outputTable"); -// create a Table from a Table API query -Table tapiResult = tableEnv.scan("table1").select(...); -// create a Table from a SQL query -Table sqlResult = tableEnv.sqlQuery("SELECT ... FROM table2 ... "); +// create a Table object from a Table API query +Table tapiResult = tableEnv.from("table1").select(...); +// create a Table object from a SQL query +Table sqlResult = tableEnv.sqlQuery("SELECT ... FROM table1 ... "); // emit a Table API result Table to a TableSink, same for SQL result tapiResult.insertInto("outputTable"); @@ -78,16 +77,15 @@ tableEnv.execute("java_job"); // create a TableEnvironment for specific planner batch or streaming val tableEnv = ... // see "Create a TableEnvironment" section -// register a Table -tableEnv.registerTable("table1", ...) // or -tableEnv.registerTableSource("table2", ...) +// create a Table +tableEnv.connect(...).createTemporaryTable("table1") // register an output Table -tableEnv.registerTableSink("outputTable", ...); +tableEnv.connect(...).createTemporaryTable("outputTable") // create a Table from a Table API query -val tapiResult = tableEnv.scan("table1").select(...) +val tapiResult = tableEnv.from("table1").select(...) // create a Table from a SQL query -val sqlResult = tableEnv.sqlQuery("SELECT ... FROM table2 ...") +val sqlResult = tableEnv.sqlQuery("SELECT ... FROM table1 ...") // emit a Table API result Table to a TableSink, same for SQL result tapiResult.insertInto("outputTable") @@ -105,16 +103,15 @@ tableEnv.execute("scala_job") table_env = ... # see "Create a TableEnvironment" section # register a Table -table_env.register_table("table1", ...) # or -table_env.register_table_source("table2", ...) +table_env.connect(...).create_temporary_table("table1") # register an output Table -table_env.register_table_sink("outputTable", ...); +table_env.connect(...).create_temporary_table("outputTable") # create a Table from a Table API query -tapi_result = table_env.scan("table1").select(...) +tapi_result = table_env.from_path("table1").select(...) # create a Table from a SQL query -sql_result = table_env.sql_query("SELECT ... FROM table2 ...") +sql_result = table_env.sql_query("SELECT ... FROM table1 ...") # emit a Table API result Table to a TableSink, same for SQL result tapi_result.insert_into("outputTable") @@ -292,25 +289,51 @@ b_b_t_env = BatchTableEnvironment.create(environment_settings=b_b_settings) **Note:** If there is only one planner jar in `/lib` directory, you can use `useAnyPlanner` (`use_any_planner` for python) to create specific `EnvironmentSettings`. - {% top %} -Register Tables in the Catalog +Create Tables in the Catalog ------------------------------- -A `TableEnvironment` maintains a catalog of tables which are registered by name. There are two types of tables, *input tables* and *output tables*. Input tables can be referenced in Table API and SQL queries and provide input data. Output tables can be used to emit the result of a Table API or SQL query to an external system. +A `TableEnvironment` maintains a map of catalogs of tables which are created with an identifier. Each +identifier consists of 3 parts: catalog name, database name and object name. If a catalog or database is not +specified, the current default value will be used (see examples in the [Table identifier expanding](#table-identifier-expanding) section). + +Tables can be either virtual (`VIEWS`) or regular (`TABLES`). `VIEWS` can be created from an +existing `Table` object, usually the result of a Table API or SQL query. `TABLES` describe +external data, such as a file, database table, or message queue. + +### Temporary vs Permanent tables. + +Tables may either be temporary, and tied to the lifecycle of a single Flink session, or permanent, +and visible across multiple Flink sessions and clusters. + +Permanent tables require a [catalog]({{ site.baseurl }}/dev/table/catalogs.html) (such as Hive Metastore) +to maintain metadata about the table. Once a permanent table is created, it is visible to any Flink +session that is connected to the catalog and will continue to exist until the table is explicitly +dropped. -An input table can be registered from various sources: +On the other hand, temporary tables are always stored in memory and only exist for the duration of +the Flink session they are created within. These tables are not visible to other sessions. They are +not bound to any catalog or database but can be created in the namespace of one. Temporary tables +are not dropped if their corresponding database is removed. -* an existing `Table` object, usually the result of a Table API or SQL query. -* a `TableSource`, which accesses external data, such as a file, database, or messaging system. -* a `DataStream` or `DataSet` from a DataStream (only for stream job) or DataSet (only for batch job translated from old planner) program. Registering a `DataStream` or `DataSet` is discussed in the [Integration with DataStream and DataSet API](#integration-with-datastream-and-dataset-api) section. +#### Shadowing -An output table can be registered using a `TableSink`. +It is possible to register a temporary table with the same identifier as an existing permanent +table. The temporary table shadows the permanent one and makes the permanent table inaccessible as +long as the temporary one exists. All queries with that identifier will be executed against the +temporary table. -### Register a Table +This might be useful for experimentation. It allows running exactly the same query first against a +temporary table that e.g. has just a subset of data, or the data is obfuscated. Once verified that +the query is correct it can be run against the real production table. -A `Table` is registered in a `TableEnvironment` as follows: +### Create a Table + +#### Virtual Tables + +A `Table` API object corresponds to a `VIEW` (virtual table) in a SQL terms. It encapsulates a logical +query plan. It can be created in a catalog as follows:
    @@ -318,11 +341,11 @@ A `Table` is registered in a `TableEnvironment` as follows: // get a TableEnvironment TableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section -// table is the result of a simple projection query -Table projTable = tableEnv.scan("X").select(...); +// table is the result of a simple projection query +Table projTable = tableEnv.from("X").select(...); // register the Table projTable as table "projectedTable" -tableEnv.registerTable("projectedTable", projTable); +tableEnv.createTemporaryView("projectedTable", projTable); {% endhighlight %}
    @@ -331,11 +354,11 @@ tableEnv.registerTable("projectedTable", projTable); // get a TableEnvironment val tableEnv = ... // see "Create a TableEnvironment" section -// table is the result of a simple projection query -val projTable: Table = tableEnv.scan("X").select(...) +// table is the result of a simple projection query +val projTable: Table = tableEnv.from("X").select(...) // register the Table projTable as table "projectedTable" -tableEnv.registerTable("projectedTable", projTable) +tableEnv.createTemporaryView("projectedTable", projTable) {% endhighlight %}
    @@ -344,8 +367,8 @@ tableEnv.registerTable("projectedTable", projTable) # get a TableEnvironment table_env = ... # see "Create a TableEnvironment" section -# table is the result of a simple projection query -proj_table = table_env.scan("X").select(...) +# table is the result of a simple projection query +proj_table = table_env.from_path("X").select(...) # register the Table projTable as table "projectedTable" table_env.register_table("projectedTable", proj_table) @@ -353,127 +376,133 @@ table_env.register_table("projectedTable", proj_table)
    -**Note:** A registered `Table` is treated similarly to a `VIEW` as known from relational database systems, i.e., the query that defines the `Table` is not optimized but will be inlined when another query references the registered `Table`. If multiple queries reference the same registered `Table`, it will be inlined for each referencing query and executed multiple times, i.e., the result of the registered `Table` will *not* be shared. +**Note:** `Table` objects are similar to `VIEW`'s from relational database +systems, i.e., the query that defines the `Table` is not optimized but will be inlined when another +query references the registered `Table`. If multiple queries reference the same registered `Table`, +it will be inlined for each referencing query and executed multiple times, i.e., the result of the +registered `Table` will *not* be shared. {% top %} -### Register a TableSource - -A `TableSource` provides access to external data which is stored in a storage system such as a database (MySQL, HBase, ...), a file with a specific encoding (CSV, Apache \[Parquet, Avro, ORC\], ...), or a messaging system (Apache Kafka, RabbitMQ, ...). +#### Connector Tables -Flink aims to provide TableSources for common data formats and storage systems. Please have a look at the [Table Sources and Sinks]({{ site.baseurl }}/dev/table/sourceSinks.html) page for a list of supported TableSources and instructions for how to build a custom `TableSource`. - -A `TableSource` is registered in a `TableEnvironment` as follows: +It is also possible to create a `TABLE` as known from relational databases from a [connector]({{ site.baseurl }}/dev/table/connect.html) declaration. +The connector describes the external system that stores the data of a table. Storage systems such as Apacha Kafka or a regular file system can be declared here.
    {% highlight java %} -// get a TableEnvironment -TableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section - -// create a TableSource -TableSource csvSource = new CsvTableSource("/path/to/file", ...); - -// register the TableSource as table "CsvTable" -tableEnv.registerTableSource("CsvTable", csvSource); +tableEnvironment + .connect(...) + .withFormat(...) + .withSchema(...) + .inAppendMode() + .createTemporaryTable("MyTable") {% endhighlight %}
    {% highlight scala %} -// get a TableEnvironment -val tableEnv = ... // see "Create a TableEnvironment" section - -// create a TableSource -val csvSource: TableSource = new CsvTableSource("/path/to/file", ...) - -// register the TableSource as table "CsvTable" -tableEnv.registerTableSource("CsvTable", csvSource) +tableEnvironment + .connect(...) + .withFormat(...) + .withSchema(...) + .inAppendMode() + .createTemporaryTable("MyTable") {% endhighlight %}
    {% highlight python %} -# get a TableEnvironment -table_env = ... # see "Create a TableEnvironment" section - -# create a TableSource -csv_source = CsvTableSource("/path/to/file", ...) +table_environment \ + .connect(...) \ + .with_format(...) \ + .with_schema(...) \ + .in_append_mode() \ + .create_temporary_table("MyTable") +{% endhighlight %} +
    -# register the TableSource as table "csvTable" -table_env.register_table_source("csvTable", csv_source) +
    +{% highlight sql %} +tableEnvironment.sqlUpdate("CREATE [TEMPORARY] TABLE MyTable (...) WITH (...)") {% endhighlight %}
    -**Note:** A `TableEnvironment` used for Blink planner only accepts `StreamTableSource`, `LookupableTableSource` and `InputFormatTableSource`, and a `StreamTableSource` used for Blink planner on batch must be bounded. +### Expanding Table identifiers -{% top %} +Tables are always registered with a 3 part identifier consisting of catalog, database, and +table name. The first two parts are optional and if they are not provided the set default values will +be used. Identifiers follow SQL requirements which means that they can be escaped with a backtick character (`` ` ``). +Additionally all SQL reserved keywords must be escaped. -### Register a TableSink +
    +
    +{% highlight java %} +TableEnvironment tEnv = ...; +tEnv.useCatalog("custom_catalog"); +tEnv.useDatabase("custom_database"); -A registered `TableSink` can be used to [emit the result of a Table API or SQL query](common.html#emit-a-table) to an external storage system, such as a database, key-value store, message queue, or file system (in different encodings, e.g., CSV, Apache \[Parquet, Avro, ORC\], ...). +Table table = ...; -Flink aims to provide TableSinks for common data formats and storage systems. Please see the documentation about [Table Sources and Sinks]({{ site.baseurl }}/dev/table/sourceSinks.html) page for details about available sinks and instructions for how to implement a custom `TableSink`. +// register the view named 'exampleView' in the catalog named 'custom_catalog' +// in the database named 'custom_database' +tableEnv.createTemporaryView("exampleView", table); -A `TableSink` is registered in a `TableEnvironment` as follows: +// register the view named 'exampleView' in the catalog named 'custom_catalog' +// in the database named 'other_database' +tableEnv.createTemporaryView("other_database.exampleView", table); -
    -
    -{% highlight java %} -// get a TableEnvironment -TableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section +// register the view named 'View' in the catalog named 'custom_catalog' in the +// database named 'custom_database'. 'View' is a reserved keyword and must be escaped. +tableEnv.createTemporaryView("`View`", table); -// create a TableSink -TableSink csvSink = new CsvTableSink("/path/to/file", ...); +// register the view named 'example.View' in the catalog named 'custom_catalog' +// in the database named 'custom_database' +tableEnv.createTemporaryView("`example.View`", table); -// define the field names and types -String[] fieldNames = {"a", "b", "c"}; -TypeInformation[] fieldTypes = {Types.INT, Types.STRING, Types.LONG}; +// register the view named 'exampleView' in the catalog named 'other_catalog' +// in the database named 'other_database' +tableEnv.createTemporaryView("other_catalog.other_database.exampleView", table); -// register the TableSink as table "CsvSinkTable" -tableEnv.registerTableSink("CsvSinkTable", fieldNames, fieldTypes, csvSink); {% endhighlight %}
    {% highlight scala %} // get a TableEnvironment -val tableEnv = ... // see "Create a TableEnvironment" section - -// create a TableSink -val csvSink: TableSink = new CsvTableSink("/path/to/file", ...) +val tEnv: TableEnvironment = ...; +tEnv.useCatalog("custom_catalog") +tEnv.useDatabase("custom_database") -// define the field names and types -val fieldNames: Array[String] = Array("a", "b", "c") -val fieldTypes: Array[TypeInformation[_]] = Array(Types.INT, Types.STRING, Types.LONG) +val table: Table = ...; -// register the TableSink as table "CsvSinkTable" -tableEnv.registerTableSink("CsvSinkTable", fieldNames, fieldTypes, csvSink) -{% endhighlight %} -
    +// register the view named 'exampleView' in the catalog named 'custom_catalog' +// in the database named 'custom_database' +tableEnv.createTemporaryView("exampleView", table) -
    -{% highlight python %} -# get a TableEnvironment -table_env = ... # see "Create a TableEnvironment" section +// register the view named 'exampleView' in the catalog named 'custom_catalog' +// in the database named 'other_database' +tableEnv.createTemporaryView("other_database.exampleView", table) -# define the field names and types -field_names = ["a", "b", "c"] -field_types = [DataTypes.INT(), DataTypes.STRING(), DataTypes.BIGINT()] +// register the view named 'View' in the catalog named 'custom_catalog' in the +// database named 'custom_database'. 'View' is a reserved keyword and must be escaped. +tableEnv.createTemporaryView("`View`", table) -# create a TableSink -csv_sink = CsvTableSink(field_names, field_types, "/path/to/file", ...) +// register the view named 'example.View' in the catalog named 'custom_catalog' +// in the database named 'custom_database' +tableEnv.createTemporaryView("`example.View`", table) -# register the TableSink as table "CsvSinkTable" -table_env.register_table_sink("CsvSinkTable", csv_sink) +// register the view named 'exampleView' in the catalog named 'other_catalog' +// in the database named 'other_database' +tableEnv.createTemporaryView("other_catalog.other_database.exampleView", table) {% endhighlight %}
    -
    -{% top %} +
    -Query a Table +Query a Table ------------- ### Table API @@ -495,7 +524,7 @@ TableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section // register Orders table // scan registered Orders table -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); // compute revenue for all customers from France Table revenue = orders .filter("cCountry === 'FRANCE'") @@ -515,7 +544,7 @@ val tableEnv = ... // see "Create a TableEnvironment" section // register Orders table // scan registered Orders table -val orders = tableEnv.scan("Orders") +val orders = tableEnv.from("Orders") // compute revenue for all customers from France val revenue = orders .filter('cCountry === "FRANCE") @@ -537,7 +566,7 @@ table_env = # see "Create a TableEnvironment" section # register Orders table # scan registered Orders table -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") # compute revenue for all customers from France revenue = orders \ .filter("cCountry === 'FRANCE'") \ @@ -721,13 +750,16 @@ The following examples shows how to emit a `Table`: // get a TableEnvironment TableEnvironment tableEnv = ...; // see "Create a TableEnvironment" section -// create a TableSink -TableSink sink = new CsvTableSink("/path/to/file", fieldDelim = "|"); +// create an output Table +final Schema schema = new Schema() + .field("a", DataTypes.INT()) + .field("b", DataTypes.STRING()) + .field("c", DataTypes.LONG()); -// register the TableSink with a specific schema -String[] fieldNames = {"a", "b", "c"}; -TypeInformation[] fieldTypes = {Types.INT, Types.STRING, Types.LONG}; -tableEnv.registerTableSink("CsvSinkTable", fieldNames, fieldTypes, sink); +tableEnv.connect(new FileSystem("/path/to/file")) + .withFormat(new Csv().fieldDelimiter('|').deriveSchema()) + .withSchema(schema) + .createTemporaryTable("CsvSinkTable"); // compute a result Table using Table API operators and/or SQL queries Table result = ... @@ -743,13 +775,16 @@ result.insertInto("CsvSinkTable"); // get a TableEnvironment val tableEnv = ... // see "Create a TableEnvironment" section -// create a TableSink -val sink: TableSink = new CsvTableSink("/path/to/file", fieldDelim = "|") +// create an output Table +val schema = new Schema() + .field("a", DataTypes.INT()) + .field("b", DataTypes.STRING()) + .field("c", DataTypes.LONG()) -// register the TableSink with a specific schema -val fieldNames: Array[String] = Array("a", "b", "c") -val fieldTypes: Array[TypeInformation] = Array(Types.INT, Types.STRING, Types.LONG) -tableEnv.registerTableSink("CsvSinkTable", fieldNames, fieldTypes, sink) +tableEnv.connect(new FileSystem("/path/to/file")) + .withFormat(new Csv().fieldDelimiter('|').deriveSchema()) + .withSchema(schema) + .createTemporaryTable("CsvSinkTable") // compute a result Table using Table API operators and/or SQL queries val result: Table = ... @@ -766,13 +801,16 @@ result.insertInto("CsvSinkTable") # get a TableEnvironment table_env = ... # see "Create a TableEnvironment" section -field_names = ["a", "b", "c"] -field_types = [DataTypes.INT(), DataTypes.STRING(), DataTypes.BIGINT()] - # create a TableSink -sink = CsvTableSink(field_names, field_types, "/path/to/file", "|") - -table_env.register_table_sink("CsvSinkTable", sink) +t_env.connect(FileSystem().path("/path/to/file"))) + .with_format(Csv() + .field_delimiter(',') + .deriveSchema()) + .with_schema(Schema() + .field("a", DataTypes.INT()) + .field("b", DataTypes.STRING()) + .field("c", DataTypes.BIGINT())) + .create_temporary_table("CsvSinkTable") # compute a result Table using Table API operators and/or SQL queries result = ... @@ -847,9 +885,11 @@ This interaction can be achieved by converting a `DataStream` or `DataSet` into The Scala Table API features implicit conversions for the `DataSet`, `DataStream`, and `Table` classes. These conversions are enabled by importing the package `org.apache.flink.table.api.scala._` in addition to `org.apache.flink.api.scala._` for the Scala DataStream API. -### Register a DataStream or DataSet as Table +### Create a View from a DataStream or DataSet + +A `DataStream` or `DataSet` can be registered in a `TableEnvironment` as a View. The schema of the resulting view depends on the data type of the registered `DataStream` or `DataSet`. Please check the section about [mapping of data types to table schema](#mapping-of-data-types-to-table-schema) for details. -A `DataStream` or `DataSet` can be registered in a `TableEnvironment` as a Table. The schema of the resulting table depends on the data type of the registered `DataStream` or `DataSet`. Please check the section about [mapping of data types to table schema](#mapping-of-data-types-to-table-schema) for details. +**Note:** Views created from a `DataStream` or `DataSet` can be registered as temporary views only.
    @@ -860,11 +900,11 @@ StreamTableEnvironment tableEnv = ...; // see "Create a TableEnvironment" sectio DataStream> stream = ... -// register the DataStream as Table "myTable" with fields "f0", "f1" -tableEnv.registerDataStream("myTable", stream); +// register the DataStream as View "myTable" with fields "f0", "f1" +tableEnv.createTemporaryView("myTable", stream); -// register the DataStream as table "myTable2" with fields "myLong", "myString" -tableEnv.registerDataStream("myTable2", stream, "myLong, myString"); +// register the DataStream as View "myTable2" with fields "myLong", "myString" +tableEnv.createTemporaryView("myTable2", stream, "myLong, myString"); {% endhighlight %}
    @@ -876,17 +916,15 @@ val tableEnv: StreamTableEnvironment = ... // see "Create a TableEnvironment" se val stream: DataStream[(Long, String)] = ... -// register the DataStream as Table "myTable" with fields "f0", "f1" -tableEnv.registerDataStream("myTable", stream) +// register the DataStream as View "myTable" with fields "f0", "f1" +tableEnv.createTemporaryView("myTable", stream) -// register the DataStream as table "myTable2" with fields "myLong", "myString" -tableEnv.registerDataStream("myTable2", stream, 'myLong, 'myString) +// register the DataStream as View "myTable2" with fields "myLong", "myString" +tableEnv.createTemporaryView("myTable2", stream, 'myLong, 'myString) {% endhighlight %}
    -**Note:** The name of a `DataStream` `Table` must not match the `^_DataStreamTable_[0-9]+` pattern and the name of a `DataSet` `Table` must not match the `^_DataSetTable_[0-9]+` pattern. These patterns are reserved for internal use only. - {% top %} ### Convert a DataStream or DataSet into a Table @@ -1601,17 +1639,31 @@ The following code shows an example and the corresponding output for multiple-si EnvironmentSettings settings = EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build(); TableEnvironment tEnv = TableEnvironment.create(settings); -String[] fieldNames = { "count", "word" }; -TypeInformation[] fieldTypes = { Types.INT, Types.STRING }; -tEnv.registerTableSource("MySource1", new CsvTableSource("/source/path1", fieldNames, fieldTypes)); -tEnv.registerTableSource("MySource2", new CsvTableSource("/source/path2", fieldNames, fieldTypes)); -tEnv.registerTableSink("MySink1", new CsvTableSink("/sink/path1").configure(fieldNames, fieldTypes)); -tEnv.registerTableSink("MySink2", new CsvTableSink("/sink/path2").configure(fieldNames, fieldTypes)); - -Table table1 = tEnv.scan("MySource1").where("LIKE(word, 'F%')"); +final Schema schema = new Schema() + .field("count", DataTypes.INT()) + .field("word", DataTypes.STRING()); + +tEnv.connect(new FileSystem("/source/path1")) + .withFormat(new Csv().deriveSchema()) + .withSchema(schema) + .createTemporaryTable("MySource1"); +tEnv.connect(new FileSystem("/source/path2")) + .withFormat(new Csv().deriveSchema()) + .withSchema(schema) + .createTemporaryTable("MySource2"); +tEnv.connect(new FileSystem("/sink/path1")) + .withFormat(new Csv().deriveSchema()) + .withSchema(schema) + .createTemporaryTable("MySink1"); +tEnv.connect(new FileSystem("/sink/path2")) + .withFormat(new Csv().deriveSchema()) + .withSchema(schema) + .createTemporaryTable("MySink2"); + +Table table1 = tEnv.from("MySource1").where("LIKE(word, 'F%')"); table1.insertInto("MySink1"); -Table table2 = table1.unionAll(tEnv.scan("MySource2")); +Table table2 = table1.unionAll(tEnv.from("MySource2")); table2.insertInto("MySink2"); String explanation = tEnv.explain(false); @@ -1625,17 +1677,31 @@ System.out.println(explanation); val settings = EnvironmentSettings.newInstance.useBlinkPlanner.inStreamingMode.build val tEnv = TableEnvironment.create(settings) -val fieldNames = Array("count", "word") -val fieldTypes = Array[TypeInformation[_]](Types.INT, Types.STRING) -tEnv.registerTableSource("MySource1", new CsvTableSource("/source/path1", fieldNames, fieldTypes)) -tEnv.registerTableSource("MySource2", new CsvTableSource("/source/path2",fieldNames, fieldTypes)) -tEnv.registerTableSink("MySink1", new CsvTableSink("/sink/path1").configure(fieldNames, fieldTypes)) -tEnv.registerTableSink("MySink2", new CsvTableSink("/sink/path2").configure(fieldNames, fieldTypes)) - -val table1 = tEnv.scan("MySource1").where("LIKE(word, 'F%')") +val schema = new Schema() + .field("count", DataTypes.INT()) + .field("word", DataTypes.STRING()) + +tEnv.connect(new FileSystem("/source/path1")) + .withFormat(new Csv().deriveSchema()) + .withSchema(schema) + .createTemporaryTable("MySource1") +tEnv.connect(new FileSystem("/source/path2")) + .withFormat(new Csv().deriveSchema()) + .withSchema(schema) + .createTemporaryTable("MySource2") +tEnv.connect(new FileSystem("/sink/path1")) + .withFormat(new Csv().deriveSchema()) + .withSchema(schema) + .createTemporaryTable("MySink1") +tEnv.connect(new FileSystem("/sink/path2")) + .withFormat(new Csv().deriveSchema()) + .withSchema(schema) + .createTemporaryTable("MySink2") + +val table1 = tEnv.from("MySource1").where("LIKE(word, 'F%')") table1.insertInto("MySink1") -val table2 = table1.unionAll(tEnv.scan("MySource2")) +val table2 = table1.unionAll(tEnv.from("MySource2")) table2.insertInto("MySink2") val explanation = tEnv.explain(false) @@ -1649,17 +1715,31 @@ println(explanation) settings = EnvironmentSettings.new_instance().use_blink_planner().in_streaming_mode().build() t_env = TableEnvironment.create(environment_settings=settings) -field_names = ["count", "word"] -field_types = [DataTypes.INT(), DataTypes.STRING()] -t_env.register_table_source("MySource1", CsvTableSource("/source/path1", field_names, field_types)) -t_env.register_table_source("MySource2", CsvTableSource("/source/path2", field_names, field_types)) -t_env.register_table_sink("MySink1", CsvTableSink("/sink/path1", field_names, field_types)) -t_env.register_table_sink("MySink2", CsvTableSink("/sink/path2", field_names, field_types)) - -table1 = t_env.scan("MySource1").where("LIKE(word, 'F%')") +schema = Schema() + .field("count", DataTypes.INT()) + .field("word", DataTypes.STRING()) + +t_env.connect(FileSystem().path("/source/path1"))) + .with_format(Csv().deriveSchema()) + .with_schema(schema) + .create_temporary_table("MySource1") +t_env.connect(FileSystem().path("/source/path2"))) + .with_format(Csv().deriveSchema()) + .with_schema(schema) + .create_temporary_table("MySource2") +t_env.connect(FileSystem().path("/sink/path1"))) + .with_format(Csv().deriveSchema()) + .with_schema(schema) + .create_temporary_table("MySink1") +t_env.connect(FileSystem().path("/sink/path2"))) + .with_format(Csv().deriveSchema()) + .with_schema(schema) + .create_temporary_table("MySink2") + +table1 = t_env.from_path("MySource1").where("LIKE(word, 'F%')") table1.insert_into("MySink1") -table2 = table1.union_all(t_env.scan("MySource2")) +table2 = table1.union_all(t_env.from_path("MySource2")) table2.insert_into("MySink2") explanation = t_env.explain() diff --git a/docs/dev/table/connect.md b/docs/dev/table/connect.md index e760059cd507..3add088d5c11 100644 --- a/docs/dev/table/connect.md +++ b/docs/dev/table/connect.md @@ -99,7 +99,7 @@ tableEnvironment .withFormat(...) .withSchema(...) .inAppendMode() - .registerTableSource("MyTable") + .createTemporaryTable("MyTable") {% endhighlight %}
    @@ -110,7 +110,7 @@ table_environment \ .with_format(...) \ .with_schema(...) \ .in_append_mode() \ - .register_table_source("MyTable") + .create_temporary_table("MyTable") {% endhighlight %}
    @@ -184,8 +184,8 @@ tableEnvironment // specify the update-mode for streaming tables .inAppendMode() - // register as source, sink, or both and under a name - .registerTableSource("MyUserTable"); + // create a table with given name + .createTemporaryTable("MyUserTable"); {% endhighlight %}
    @@ -227,7 +227,7 @@ table_environment \ .field("message", DataTypes.STRING()) ) \ .in_append_mode() \ - .register_table_source("MyUserTable") + .create_temporary_table("MyUserTable") # specify the update-mode for streaming tables and # register as source, sink, or both and under a name {% endhighlight %} diff --git a/docs/dev/table/connect.zh.md b/docs/dev/table/connect.zh.md index 130f55ebf94f..dd2e8b15ef07 100644 --- a/docs/dev/table/connect.zh.md +++ b/docs/dev/table/connect.zh.md @@ -99,7 +99,7 @@ tableEnvironment .withFormat(...) .withSchema(...) .inAppendMode() - .registerTableSource("MyTable") + .createTemporaryTable("MyTable") {% endhighlight %}
    @@ -110,7 +110,7 @@ table_environment \ .with_format(...) \ .with_schema(...) \ .in_append_mode() \ - .register_table_source("MyTable") + .create_temporary_table("MyTable") {% endhighlight %}
    @@ -184,8 +184,8 @@ tableEnvironment // specify the update-mode for streaming tables .inAppendMode() - // register as source, sink, or both and under a name - .registerTableSource("MyUserTable"); + // create a table with given name + .createTemporaryTable("MyUserTable"); {% endhighlight %}
    @@ -227,7 +227,7 @@ table_environment \ .field("message", DataTypes.STRING()) ) \ .in_append_mode() \ - .register_table_source("MyUserTable") + .create_temporary_table("MyUserTable") # specify the update-mode for streaming tables and # register as source, sink, or both and under a name {% endhighlight %} @@ -1857,6 +1857,7 @@ Use the old one for stream/batch filesystem operations for now. new OldCsv() .field("field1", Types.STRING) // required: ordered format fields .field("field2", Types.TIMESTAMP) + .deriveSchema() // or use the table's schema .fieldDelimiter(",") // optional: string delimiter "," by default .lineDelimiter("\n") // optional: string delimiter "\n" by default .quoteCharacter('"') // optional: single character for string values, empty by default @@ -1892,6 +1893,7 @@ format: type: VARCHAR - name: field2 type: TIMESTAMP + derive-schema: true # or use the table's schema field-delimiter: "," # optional: string delimiter "," by default line-delimiter: "\n" # optional: string delimiter "\n" by default quote-character: '"' # optional: single character for string values, empty by default @@ -1913,6 +1915,8 @@ CREATE TABLE MyUserTable ( 'format.fields.1.name' = 'rideTime', 'format.fields.1.type' = 'TIMESTAMP', + 'format.derive-schema' = 'true', -- or use the table's schema' + 'format.field-delimiter' = ',', -- optional: string delimiter "," by default 'format.line-delimiter' = '\n', -- optional: string delimiter "\n" by default 'format.quote-character' = '"', -- optional: single character for string values, empty by default diff --git a/docs/dev/table/sourceSinks.md b/docs/dev/table/sourceSinks.md index 8da8046ee42f..d0e4781c80b0 100644 --- a/docs/dev/table/sourceSinks.md +++ b/docs/dev/table/sourceSinks.md @@ -775,8 +775,9 @@ StreamTableEnvironment tableEnv = // ... tableEnv .connect(new MySystemConnector(true)) + .withSchema(...) .inAppendMode() - .registerTableSource("MySystemTable"); + .createTemporaryTable("MySystemTable"); {% endhighlight %} @@ -786,8 +787,9 @@ val tableEnv: StreamTableEnvironment = // ... tableEnv .connect(new MySystemConnector(isDebug = true)) + .withSchema(...) .inAppendMode() - .registerTableSource("MySystemTable") + .createTemporaryTable("MySystemTable") {% endhighlight %} diff --git a/docs/dev/table/sourceSinks.zh.md b/docs/dev/table/sourceSinks.zh.md index 82398a44d873..d0e4781c80b0 100644 --- a/docs/dev/table/sourceSinks.zh.md +++ b/docs/dev/table/sourceSinks.zh.md @@ -36,9 +36,9 @@ Have a look at the [common concepts and API](common.html) page for details how t Define a TableSource -------------------- -A `TableSource` is a generic interface that gives Table API and SQL queries access to data stored in an external system. It provides the schema of the table and the records that are mapped to rows with the table's schema. Depending on whether the `TableSource` is used in a streaming or batch query, the records are produced as a `DataSet` or `DataStream`. +A `TableSource` is a generic interface that gives Table API and SQL queries access to data stored in an external system. It provides the schema of the table and the records that are mapped to rows with the table's schema. Depending on whether the `TableSource` is used in a streaming or batch query, the records are produced as a `DataSet` or `DataStream`. -If a `TableSource` is used in a streaming query it must implement the `StreamTableSource` interface, if it is used in a batch query it must implement the `BatchTableSource` interface. A `TableSource` can also implement both interfaces and be used in streaming and batch queries. +If a `TableSource` is used in a streaming query it must implement the `StreamTableSource` interface, if it is used in a batch query it must implement the `BatchTableSource` interface. A `TableSource` can also implement both interfaces and be used in streaming and batch queries. `StreamTableSource` and `BatchTableSource` extend the base interface `TableSource` that defines the following methods: @@ -77,7 +77,7 @@ TableSource[T] { * `explainSource()`: Returns a String that describes the `TableSource`. This method is optional and used for display purposes only. -The `TableSource` interface separates the logical table schema from the physical type of the returned `DataStream` or `DataSet`. As a consequence, all fields of the table schema (`getTableSchema()`) must be mapped to a field with corresponding type of the physical return type (`getReturnType()`). By default, this mapping is done based on field names. For example, a `TableSource` that defines a table schema with two fields `[name: String, size: Integer]` requires a `TypeInformation` with at least two fields called `name` and `size` of type `String` and `Integer`, respectively. This could be a `PojoTypeInfo` or a `RowTypeInfo` that have two fields named `name` and `size` with matching types. +The `TableSource` interface separates the logical table schema from the physical type of the returned `DataStream` or `DataSet`. As a consequence, all fields of the table schema (`getTableSchema()`) must be mapped to a field with corresponding type of the physical return type (`getReturnType()`). By default, this mapping is done based on field names. For example, a `TableSource` that defines a table schema with two fields `[name: String, size: Integer]` requires a `TypeInformation` with at least two fields called `name` and `size` of type `String` and `Integer`, respectively. This could be a `PojoTypeInfo` or a `RowTypeInfo` that have two fields named `name` and `size` with matching types. However, some types, such as Tuple or CaseClass types, do support custom field names. If a `TableSource` returns a `DataStream` or `DataSet` of a type with fixed field names, it can implement the `DefinedFieldMapping` interface to map field names from the table schema to field names of the physical return type. @@ -111,7 +111,7 @@ BatchTableSource[T] extends TableSource[T] { ### Defining a StreamTableSource -The `StreamTableSource` interface extends the `TableSource` interface and defines one additional method: +The `StreamTableSource` interface extends the `TableSource` interface and defines one additional method:
    @@ -175,9 +175,9 @@ DefinedProctimeAttribute { [Rowtime attributes](streaming/time_attributes.html#event-time) are attributes of type `TIMESTAMP` and handled in a unified way in stream and batch queries. -A table schema field of type `SQL_TIMESTAMP` can be declared as rowtime attribute by specifying +A table schema field of type `SQL_TIMESTAMP` can be declared as rowtime attribute by specifying -* the name of the field, +* the name of the field, * a `TimestampExtractor` that computes the actual value for the attribute (usually from one or more other fields), and * a `WatermarkStrategy` that specifies how watermarks are generated for the the rowtime attribute. @@ -263,7 +263,7 @@ 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. +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: @@ -287,7 +287,7 @@ 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`. +* `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. @@ -323,7 +323,7 @@ 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. +* `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. @@ -775,8 +775,9 @@ StreamTableEnvironment tableEnv = // ... tableEnv .connect(new MySystemConnector(true)) + .withSchema(...) .inAppendMode() - .registerTableSource("MySystemTable"); + .createTemporaryTable("MySystemTable"); {% endhighlight %} @@ -786,8 +787,9 @@ val tableEnv: StreamTableEnvironment = // ... tableEnv .connect(new MySystemConnector(isDebug = true)) + .withSchema(...) .inAppendMode() - .registerTableSource("MySystemTable") + .createTemporaryTable("MySystemTable") {% endhighlight %} diff --git a/docs/dev/table/sql.md b/docs/dev/table/sql.md index c3d0cd39f095..eadb00891201 100644 --- a/docs/dev/table/sql.md +++ b/docs/dev/table/sql.md @@ -54,18 +54,23 @@ Table result = tableEnv.sqlQuery( "SELECT SUM(amount) FROM " + table + " WHERE product LIKE '%Rubber%'"); // SQL query with a registered table -// register the DataStream as table "Orders" -tableEnv.registerDataStream("Orders", ds, "user, product, amount"); +// register the DataStream as view "Orders" +tableEnv.createTemporaryView("Orders", ds, "user, product, amount"); // run a SQL query on the Table and retrieve the result as a new Table Table result2 = tableEnv.sqlQuery( "SELECT product, amount FROM Orders WHERE product LIKE '%Rubber%'"); // SQL update with a registered table // create and register a TableSink -TableSink csvSink = new CsvTableSink("/path/to/file", ...); -String[] fieldNames = {"product", "amount"}; -TypeInformation[] fieldTypes = {Types.STRING, Types.INT}; -tableEnv.registerTableSink("RubberOrders", fieldNames, fieldTypes, csvSink); +final Schema schema = new Schema() + .field("product", DataTypes.STRING()) + .field("amount", DataTypes.INT()); + +tableEnv.connect(new FileSystem("/path/to/file")) + .withFormat(...) + .withSchema(schema) + .createTemporaryTable("RubberOrders"); + // run a SQL update query on the Table and emit the result to the TableSink tableEnv.sqlUpdate( "INSERT INTO RubberOrders SELECT product, amount FROM Orders WHERE product LIKE '%Rubber%'"); @@ -87,17 +92,22 @@ val result = tableEnv.sqlQuery( // SQL query with a registered table // register the DataStream under the name "Orders" -tableEnv.registerDataStream("Orders", ds, 'user, 'product, 'amount) +tableEnv.createTemporaryView("Orders", ds, 'user, 'product, 'amount) // run a SQL query on the Table and retrieve the result as a new Table val result2 = tableEnv.sqlQuery( "SELECT product, amount FROM Orders WHERE product LIKE '%Rubber%'") // SQL update with a registered table // create and register a TableSink -val csvSink: CsvTableSink = new CsvTableSink("/path/to/file", ...) -val fieldNames: Array[String] = Array("product", "amount") -val fieldTypes: Array[TypeInformation[_]] = Array(Types.STRING, Types.INT) -tableEnv.registerTableSink("RubberOrders", fieldNames, fieldTypes, csvSink) +val schema = new Schema() + .field("product", DataTypes.STRING()) + .field("amount", DataTypes.INT()) + +tableEnv.connect(new FileSystem("/path/to/file")) + .withFormat(...) + .withSchema(schema) + .createTemporaryTable("RubberOrders") + // run a SQL update query on the Table and emit the result to the TableSink tableEnv.sqlUpdate( "INSERT INTO RubberOrders SELECT product, amount FROM Orders WHERE product LIKE '%Rubber%'") @@ -117,11 +127,15 @@ result = table_env \ # SQL update with a registered table # create and register a TableSink -table_env.register_table("Orders", table) -field_names = ["product", "amount"] -field_types = [DataTypes.STRING(), DataTypes.BIGINT()] -csv_sink = CsvTableSink(field_names, field_types, "/path/to/file", ...) -table_env.register_table_sink("RubberOrders", csv_sink) +t_env.connect(FileSystem().path("/path/to/file"))) + .with_format(Csv() + .field_delimiter(',') + .deriveSchema()) + .with_schema(Schema() + .field("product", DataTypes.STRING()) + .field("amount", DataTypes.BIGINT())) + .create_temporary_table("RubberOrders") + # run a SQL update query on the Table and emit the result to the TableSink table_env \ .sql_update("INSERT INTO RubberOrders SELECT product, amount FROM Orders WHERE product LIKE '%Rubber%'") @@ -880,7 +894,7 @@ StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); // ingest a DataStream from an external source DataStream> ds = env.addSource(...); // register the DataStream as table "ShopSales" -tableEnv.registerDataStream("ShopSales", ds, "product_id, category, product_name, sales"); +tableEnv.createTemporaryView("ShopSales", ds, "product_id, category, product_name, sales"); // select top-5 products per category which have the maximum sales. Table result1 = tableEnv.sqlQuery( @@ -901,7 +915,7 @@ val tableEnv = TableEnvironment.getTableEnvironment(env) // read a DataStream from an external source val ds: DataStream[(String, String, String, Long)] = env.addSource(...) // register the DataStream under the name "ShopSales" -tableEnv.registerDataStream("ShopSales", ds, 'product_id, 'category, 'product_name, 'sales) +tableEnv.createTemporaryView("ShopSales", ds, 'product_id, 'category, 'product_name, 'sales) // select top-5 products per category which have the maximum sales. @@ -935,7 +949,7 @@ StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); // ingest a DataStream from an external source DataStream> ds = env.addSource(...); // register the DataStream as table "ShopSales" -tableEnv.registerDataStream("ShopSales", ds, "product_id, category, product_name, sales"); +tableEnv.createTemporaryView("ShopSales", ds, "product_id, category, product_name, sales"); // select top-5 products per category which have the maximum sales. Table result1 = tableEnv.sqlQuery( @@ -956,7 +970,7 @@ val tableEnv = TableEnvironment.getTableEnvironment(env) // read a DataStream from an external source val ds: DataStream[(String, String, String, Long)] = env.addSource(...) // register the DataStream under the name "ShopSales" -tableEnv.registerDataStream("ShopSales", ds, 'product_id, 'category, 'product_name, 'sales) +tableEnv.createTemporaryView("ShopSales", ds, 'product_id, 'category, 'product_name, 'sales) // select top-5 products per category which have the maximum sales. @@ -1015,7 +1029,7 @@ StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); // ingest a DataStream from an external source DataStream> ds = env.addSource(...); // register the DataStream as table "Orders" -tableEnv.registerDataStream("Orders", ds, "order_id, user, product, number, proctime.proctime"); +tableEnv.createTemporaryView("Orders", ds, "order_id, user, product, number, proctime.proctime"); // remove duplicate rows on order_id and keep the first occurrence row, // because there shouldn't be two orders with the same order_id. @@ -1037,7 +1051,7 @@ val tableEnv = TableEnvironment.getTableEnvironment(env) // read a DataStream from an external source val ds: DataStream[(String, String, String, Int)] = env.addSource(...) // register the DataStream under the name "Orders" -tableEnv.registerDataStream("Orders", ds, 'order_id, 'user, 'product, 'number, 'proctime.proctime) +tableEnv.createTemporaryView("Orders", ds, 'order_id, 'user, 'product, 'number, 'proctime.proctime) // remove duplicate rows on order_id and keep the first occurrence row, // because there shouldn't be two orders with the same order_id. @@ -1186,7 +1200,7 @@ StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env); // ingest a DataStream from an external source DataStream> ds = env.addSource(...); // register the DataStream as table "Orders" -tableEnv.registerDataStream("Orders", ds, "user, product, amount, proctime.proctime, rowtime.rowtime"); +tableEnv.createTemporaryView("Orders", ds, "user, product, amount, proctime.proctime, rowtime.rowtime"); // compute SUM(amount) per day (in event-time) Table result1 = tableEnv.sqlQuery( @@ -1223,7 +1237,7 @@ val tableEnv = StreamTableEnvironment.create(env) // read a DataStream from an external source val ds: DataStream[(Long, String, Int)] = env.addSource(...) // register the DataStream under the name "Orders" -tableEnv.registerDataStream("Orders", ds, 'user, 'product, 'amount, 'proctime.proctime, 'rowtime.rowtime) +tableEnv.createTemporaryView("Orders", ds, 'user, 'product, 'amount, 'proctime.proctime, 'rowtime.rowtime) // compute SUM(amount) per day (in event-time) val result1 = tableEnv.sqlQuery( diff --git a/docs/dev/table/sql.zh.md b/docs/dev/table/sql.zh.md index 84b744e65dd8..eadb00891201 100644 --- a/docs/dev/table/sql.zh.md +++ b/docs/dev/table/sql.zh.md @@ -54,18 +54,23 @@ Table result = tableEnv.sqlQuery( "SELECT SUM(amount) FROM " + table + " WHERE product LIKE '%Rubber%'"); // SQL query with a registered table -// register the DataStream as table "Orders" -tableEnv.registerDataStream("Orders", ds, "user, product, amount"); +// register the DataStream as view "Orders" +tableEnv.createTemporaryView("Orders", ds, "user, product, amount"); // run a SQL query on the Table and retrieve the result as a new Table Table result2 = tableEnv.sqlQuery( "SELECT product, amount FROM Orders WHERE product LIKE '%Rubber%'"); // SQL update with a registered table // create and register a TableSink -TableSink csvSink = new CsvTableSink("/path/to/file", ...); -String[] fieldNames = {"product", "amount"}; -TypeInformation[] fieldTypes = {Types.STRING, Types.INT}; -tableEnv.registerTableSink("RubberOrders", fieldNames, fieldTypes, csvSink); +final Schema schema = new Schema() + .field("product", DataTypes.STRING()) + .field("amount", DataTypes.INT()); + +tableEnv.connect(new FileSystem("/path/to/file")) + .withFormat(...) + .withSchema(schema) + .createTemporaryTable("RubberOrders"); + // run a SQL update query on the Table and emit the result to the TableSink tableEnv.sqlUpdate( "INSERT INTO RubberOrders SELECT product, amount FROM Orders WHERE product LIKE '%Rubber%'"); @@ -87,17 +92,22 @@ val result = tableEnv.sqlQuery( // SQL query with a registered table // register the DataStream under the name "Orders" -tableEnv.registerDataStream("Orders", ds, 'user, 'product, 'amount) +tableEnv.createTemporaryView("Orders", ds, 'user, 'product, 'amount) // run a SQL query on the Table and retrieve the result as a new Table val result2 = tableEnv.sqlQuery( "SELECT product, amount FROM Orders WHERE product LIKE '%Rubber%'") // SQL update with a registered table // create and register a TableSink -val csvSink: CsvTableSink = new CsvTableSink("/path/to/file", ...) -val fieldNames: Array[String] = Array("product", "amount") -val fieldTypes: Array[TypeInformation[_]] = Array(Types.STRING, Types.INT) -tableEnv.registerTableSink("RubberOrders", fieldNames, fieldTypes, csvSink) +val schema = new Schema() + .field("product", DataTypes.STRING()) + .field("amount", DataTypes.INT()) + +tableEnv.connect(new FileSystem("/path/to/file")) + .withFormat(...) + .withSchema(schema) + .createTemporaryTable("RubberOrders") + // run a SQL update query on the Table and emit the result to the TableSink tableEnv.sqlUpdate( "INSERT INTO RubberOrders SELECT product, amount FROM Orders WHERE product LIKE '%Rubber%'") @@ -117,11 +127,15 @@ result = table_env \ # SQL update with a registered table # create and register a TableSink -table_env.register_table("Orders", table) -field_names = ["product", "amount"] -field_types = [DataTypes.STRING(), DataTypes.BIGINT()] -csv_sink = CsvTableSink(field_names, field_types, "/path/to/file", ...) -table_env.register_table_sink("RubberOrders", csv_sink) +t_env.connect(FileSystem().path("/path/to/file"))) + .with_format(Csv() + .field_delimiter(',') + .deriveSchema()) + .with_schema(Schema() + .field("product", DataTypes.STRING()) + .field("amount", DataTypes.BIGINT())) + .create_temporary_table("RubberOrders") + # run a SQL update query on the Table and emit the result to the TableSink table_env \ .sql_update("INSERT INTO RubberOrders SELECT product, amount FROM Orders WHERE product LIKE '%Rubber%'") @@ -142,7 +156,7 @@ The following BNF-grammar describes the superset of supported SQL features in ba insert: INSERT INTO tableReference query - + query: values | { @@ -168,7 +182,7 @@ select: [ GROUP BY { groupItem [, groupItem ]* } ] [ HAVING booleanExpression ] [ WINDOW windowName AS windowSpec [, windowName AS windowSpec ]* ] - + selectWithoutFrom: SELECT [ ALL | DISTINCT ] { * | projectItem [, projectItem ]* } @@ -440,11 +454,11 @@ SELECT COUNT(amount) OVER ( FROM Orders SELECT COUNT(amount) OVER w, SUM(amount) OVER w -FROM Orders +FROM Orders WINDOW w AS ( PARTITION BY user ORDER BY proctime - ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) + ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) {% endhighlight %} @@ -819,7 +833,7 @@ ORDER BY orderTime Batch -Note: The LIMIT clause requires an ORDER BY clause. +Note: The LIMIT clause requires an ORDER BY clause. {% highlight sql %} SELECT * FROM Orders @@ -880,7 +894,7 @@ StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); // ingest a DataStream from an external source DataStream> ds = env.addSource(...); // register the DataStream as table "ShopSales" -tableEnv.registerDataStream("ShopSales", ds, "product_id, category, product_name, sales"); +tableEnv.createTemporaryView("ShopSales", ds, "product_id, category, product_name, sales"); // select top-5 products per category which have the maximum sales. Table result1 = tableEnv.sqlQuery( @@ -901,7 +915,7 @@ val tableEnv = TableEnvironment.getTableEnvironment(env) // read a DataStream from an external source val ds: DataStream[(String, String, String, Long)] = env.addSource(...) // register the DataStream under the name "ShopSales" -tableEnv.registerDataStream("ShopSales", ds, 'product_id, 'category, 'product_name, 'sales) +tableEnv.createTemporaryView("ShopSales", ds, 'product_id, 'category, 'product_name, 'sales) // select top-5 products per category which have the maximum sales. @@ -935,7 +949,7 @@ StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); // ingest a DataStream from an external source DataStream> ds = env.addSource(...); // register the DataStream as table "ShopSales" -tableEnv.registerDataStream("ShopSales", ds, "product_id, category, product_name, sales"); +tableEnv.createTemporaryView("ShopSales", ds, "product_id, category, product_name, sales"); // select top-5 products per category which have the maximum sales. Table result1 = tableEnv.sqlQuery( @@ -956,7 +970,7 @@ val tableEnv = TableEnvironment.getTableEnvironment(env) // read a DataStream from an external source val ds: DataStream[(String, String, String, Long)] = env.addSource(...) // register the DataStream under the name "ShopSales" -tableEnv.registerDataStream("ShopSales", ds, 'product_id, 'category, 'product_name, 'sales) +tableEnv.createTemporaryView("ShopSales", ds, 'product_id, 'category, 'product_name, 'sales) // select top-5 products per category which have the maximum sales. @@ -1015,7 +1029,7 @@ StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env); // ingest a DataStream from an external source DataStream> ds = env.addSource(...); // register the DataStream as table "Orders" -tableEnv.registerDataStream("Orders", ds, "order_id, user, product, number, proctime.proctime"); +tableEnv.createTemporaryView("Orders", ds, "order_id, user, product, number, proctime.proctime"); // remove duplicate rows on order_id and keep the first occurrence row, // because there shouldn't be two orders with the same order_id. @@ -1037,7 +1051,7 @@ val tableEnv = TableEnvironment.getTableEnvironment(env) // read a DataStream from an external source val ds: DataStream[(String, String, String, Int)] = env.addSource(...) // register the DataStream under the name "Orders" -tableEnv.registerDataStream("Orders", ds, 'order_id, 'user, 'product, 'number, 'proctime.proctime) +tableEnv.createTemporaryView("Orders", ds, 'order_id, 'user, 'product, 'number, 'proctime.proctime) // remove duplicate rows on order_id and keep the first occurrence row, // because there shouldn't be two orders with the same order_id. @@ -1186,7 +1200,7 @@ StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env); // ingest a DataStream from an external source DataStream> ds = env.addSource(...); // register the DataStream as table "Orders" -tableEnv.registerDataStream("Orders", ds, "user, product, amount, proctime.proctime, rowtime.rowtime"); +tableEnv.createTemporaryView("Orders", ds, "user, product, amount, proctime.proctime, rowtime.rowtime"); // compute SUM(amount) per day (in event-time) Table result1 = tableEnv.sqlQuery( @@ -1223,7 +1237,7 @@ val tableEnv = StreamTableEnvironment.create(env) // read a DataStream from an external source val ds: DataStream[(Long, String, Int)] = env.addSource(...) // register the DataStream under the name "Orders" -tableEnv.registerDataStream("Orders", ds, 'user, 'product, 'amount, 'proctime.proctime, 'rowtime.rowtime) +tableEnv.createTemporaryView("Orders", ds, 'user, 'product, 'amount, 'proctime.proctime, 'rowtime.rowtime) // compute SUM(amount) per day (in event-time) val result1 = tableEnv.sqlQuery( diff --git a/docs/dev/table/streaming/temporal_tables.md b/docs/dev/table/streaming/temporal_tables.md index baecbd07d553..bc7addd345cc 100644 --- a/docs/dev/table/streaming/temporal_tables.md +++ b/docs/dev/table/streaming/temporal_tables.md @@ -178,7 +178,7 @@ ratesHistoryData.add(Tuple2.of("Euro", 119L)); DataStream> ratesHistoryStream = env.fromCollection(ratesHistoryData); Table ratesHistory = tEnv.fromDataStream(ratesHistoryStream, "r_currency, r_rate, r_proctime.proctime"); -tEnv.registerTable("RatesHistory", ratesHistory); +tEnv.createTemporaryView("RatesHistory", ratesHistory); // Create and register a temporal table function. // Define "r_proctime" as the time attribute and "r_currency" as the primary key. @@ -206,7 +206,7 @@ val ratesHistory = env .fromCollection(ratesHistoryData) .toTable(tEnv, 'r_currency, 'r_rate, 'r_proctime.proctime) -tEnv.registerTable("RatesHistory", ratesHistory) +tEnv.createTemporaryView("RatesHistory", ratesHistory) // Create and register TemporalTableFunction. // Define "r_proctime" as the time attribute and "r_currency" as the primary key. diff --git a/docs/dev/table/streaming/time_attributes.md b/docs/dev/table/streaming/time_attributes.md index 20e4301bc25c..3840573231cd 100644 --- a/docs/dev/table/streaming/time_attributes.md +++ b/docs/dev/table/streaming/time_attributes.md @@ -151,7 +151,7 @@ public class UserActionSource implements StreamTableSource, DefinedProctime tEnv.registerTableSource("UserActions", new UserActionSource()); WindowedTable windowedTable = tEnv - .scan("UserActions") + .from("UserActions") .window(Tumble.over("10.minutes").on("UserActionTime").as("userActionWindow")); {% endhighlight %} @@ -182,7 +182,7 @@ class UserActionSource extends StreamTableSource[Row] with DefinedProctimeAttrib tEnv.registerTableSource("UserActions", new UserActionSource) val windowedTable = tEnv - .scan("UserActions") + .from("UserActions") .window(Tumble over 10.minutes on 'UserActionTime as 'userActionWindow) {% endhighlight %} @@ -314,7 +314,7 @@ public class UserActionSource implements StreamTableSource, DefinedRowtimeA tEnv.registerTableSource("UserActions", new UserActionSource()); WindowedTable windowedTable = tEnv - .scan("UserActions") + .from("UserActions") .window(Tumble.over("10.minutes").on("UserActionTime").as("userActionWindow")); {% endhighlight %} @@ -353,7 +353,7 @@ class UserActionSource extends StreamTableSource[Row] with DefinedRowtimeAttribu tEnv.registerTableSource("UserActions", new UserActionSource) val windowedTable = tEnv - .scan("UserActions") + .from("UserActions") .window(Tumble over 10.minutes on 'UserActionTime as 'userActionWindow) {% endhighlight %} diff --git a/docs/dev/table/streaming/time_attributes.zh.md b/docs/dev/table/streaming/time_attributes.zh.md index 20e4301bc25c..3840573231cd 100644 --- a/docs/dev/table/streaming/time_attributes.zh.md +++ b/docs/dev/table/streaming/time_attributes.zh.md @@ -151,7 +151,7 @@ public class UserActionSource implements StreamTableSource, DefinedProctime tEnv.registerTableSource("UserActions", new UserActionSource()); WindowedTable windowedTable = tEnv - .scan("UserActions") + .from("UserActions") .window(Tumble.over("10.minutes").on("UserActionTime").as("userActionWindow")); {% endhighlight %} @@ -182,7 +182,7 @@ class UserActionSource extends StreamTableSource[Row] with DefinedProctimeAttrib tEnv.registerTableSource("UserActions", new UserActionSource) val windowedTable = tEnv - .scan("UserActions") + .from("UserActions") .window(Tumble over 10.minutes on 'UserActionTime as 'userActionWindow) {% endhighlight %} @@ -314,7 +314,7 @@ public class UserActionSource implements StreamTableSource, DefinedRowtimeA tEnv.registerTableSource("UserActions", new UserActionSource()); WindowedTable windowedTable = tEnv - .scan("UserActions") + .from("UserActions") .window(Tumble.over("10.minutes").on("UserActionTime").as("userActionWindow")); {% endhighlight %} @@ -353,7 +353,7 @@ class UserActionSource extends StreamTableSource[Row] with DefinedRowtimeAttribu tEnv.registerTableSource("UserActions", new UserActionSource) val windowedTable = tEnv - .scan("UserActions") + .from("UserActions") .window(Tumble over 10.minutes on 'UserActionTime as 'userActionWindow) {% endhighlight %} diff --git a/docs/dev/table/tableApi.md b/docs/dev/table/tableApi.md index c677de62ab10..a0426cd7530f 100644 --- a/docs/dev/table/tableApi.md +++ b/docs/dev/table/tableApi.md @@ -55,7 +55,7 @@ BatchTableEnvironment tEnv = BatchTableEnvironment.create(env); // ... // specify table program -Table orders = tEnv.scan("Orders"); // schema (a, b, c, rowtime) +Table orders = tEnv.from("Orders"); // schema (a, b, c, rowtime) Table counts = orders .groupBy("a") @@ -87,7 +87,7 @@ val tEnv = BatchTableEnvironment.create(env) // ... // specify table program -val orders = tEnv.scan("Orders") // schema (a, b, c, rowtime) +val orders = tEnv.from("Orders") // schema (a, b, c, rowtime) val result = orders .groupBy('a) @@ -115,7 +115,7 @@ t_env = TableEnvironment.create(env, TableConfig()) # ... # specify table program -orders = t_env.scan("Orders") # schema (a, b, c, rowtime) +orders = t_env.from_path("Orders") # schema (a, b, c, rowtime) orders.group_by("a").select("a, b.count as cnt").insert_into("result") @@ -136,7 +136,7 @@ The next example shows a more complex Table API program. The program scans again // ... // specify table program -Table orders = tEnv.scan("Orders"); // schema (a, b, c, rowtime) +Table orders = tEnv.from("Orders"); // schema (a, b, c, rowtime) Table result = orders .filter("a.isNotNull && b.isNotNull && c.isNotNull") @@ -155,7 +155,7 @@ Table result = orders // ... // specify table program -val orders: Table = tEnv.scan("Orders") // schema (a, b, c, rowtime) +val orders: Table = tEnv.from("Orders") // schema (a, b, c, rowtime) val result: Table = orders .filter('a.isNotNull && 'b.isNotNull && 'c.isNotNull) @@ -174,7 +174,7 @@ val result: Table = orders # ... # specify table program -orders = t_env.scan("Orders") # schema (a, b, c, rowtime) +orders = t_env.from_path("Orders") # schema (a, b, c, rowtime) result = orders.filter("a.isNotNull && b.isNotNull && c.isNotNull") \ .select("a.lowerCase() as a, b, rowtime") \ @@ -210,13 +210,13 @@ The Table API supports the following operations. Please note that not all operat - Scan
    + From
    Batch Streaming

    Similar to the FROM clause in a SQL query. Performs a scan of a registered table.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); {% endhighlight %} @@ -228,7 +228,7 @@ Table orders = tableEnv.scan("Orders");

    Similar to a SQL SELECT statement. Performs a select operation.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders.select("a, c as d"); {% endhighlight %}

    You can use star (*) to act as a wild card, selecting all of the columns in the table.

    @@ -245,7 +245,7 @@ Table result = orders.select("*");

    Renames fields.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders.as("x, y, z, t"); {% endhighlight %} @@ -259,12 +259,12 @@ Table result = orders.as("x, y, z, t");

    Similar to a SQL WHERE clause. Filters out rows that do not pass the filter predicate.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders.where("b === 'red'"); {% endhighlight %} or {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders.filter("a % 2 === 0"); {% endhighlight %} @@ -285,13 +285,13 @@ Table result = orders.filter("a % 2 === 0"); - Scan
    + From
    Batch Streaming

    Similar to the FROM clause in a SQL query. Performs a scan of a registered table.

    {% highlight scala %} -val orders: Table = tableEnv.scan("Orders") +val orders: Table = tableEnv.from("Orders") {% endhighlight %} @@ -303,12 +303,12 @@ val orders: Table = tableEnv.scan("Orders")

    Similar to a SQL SELECT statement. Performs a select operation.

    {% highlight scala %} -val orders: Table = tableEnv.scan("Orders") +val orders: Table = tableEnv.from("Orders") val result = orders.select('a, 'c as 'd) {% endhighlight %}

    You can use star (*) to act as a wild card, selecting all of the columns in the table.

    {% highlight scala %} -val orders: Table = tableEnv.scan("Orders") +val orders: Table = tableEnv.from("Orders") val result = orders.select('*) {% endhighlight %} @@ -321,7 +321,7 @@ val result = orders.select('*)

    Renames fields.

    {% highlight scala %} -val orders: Table = tableEnv.scan("Orders").as('x, 'y, 'z, 't) +val orders: Table = tableEnv.from("Orders").as('x, 'y, 'z, 't) {% endhighlight %} @@ -334,12 +334,12 @@ val orders: Table = tableEnv.scan("Orders").as('x, 'y, 'z, 't)

    Similar to a SQL WHERE clause. Filters out rows that do not pass the filter predicate.

    {% highlight scala %} -val orders: Table = tableEnv.scan("Orders") +val orders: Table = tableEnv.from("Orders") val result = orders.filter('a % 2 === 0) {% endhighlight %} or {% highlight scala %} -val orders: Table = tableEnv.scan("Orders") +val orders: Table = tableEnv.from("Orders") val result = orders.where('b === "red") {% endhighlight %} @@ -365,7 +365,7 @@ val result = orders.where('b === "red")

    Similar to the FROM clause in a SQL query. Performs a scan of a registered table.

    {% highlight python %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") {% endhighlight %} @@ -377,7 +377,7 @@ orders = table_env.scan("Orders")

    Similar to a SQL SELECT statement. Performs a select operation.

    {% highlight python %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.select("a, c as d") {% endhighlight %}

    You can use star (*) to act as a wild card, selecting all of the columns in the table.

    @@ -394,7 +394,7 @@ result = orders.select("*")

    Renames fields.

    {% highlight python %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.alias("x, y, z, t") {% endhighlight %} @@ -408,12 +408,12 @@ result = orders.alias("x, y, z, t")

    Similar to a SQL WHERE clause. Filters out rows that do not pass the filter predicate.

    {% highlight python %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.where("b === 'red'") {% endhighlight %} or {% highlight python %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.filter("a % 2 === 0") {% endhighlight %} @@ -446,7 +446,7 @@ result = orders.filter("a % 2 === 0")

    Performs a field add operation. It will throw an exception if the added fields already exist.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders.addColumns("concat(c, 'sunny')"); {% endhighlight %} @@ -460,7 +460,7 @@ Table result = orders.addColumns("concat(c, 'sunny')");

    Performs a field add operation. Existing fields will be replaced if add columns name is the same as the existing column name. Moreover, if the added fields have duplicate field name, then the last one is used.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders.addOrReplaceColumns("concat(c, 'sunny') as desc"); {% endhighlight %} @@ -473,7 +473,7 @@ Table result = orders.addOrReplaceColumns("concat(c, 'sunny') as desc");

    Performs a field drop operation. The field expressions should be field reference expressions, and only existing fields can be dropped.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders.dropColumns("b, c"); {% endhighlight %} @@ -486,7 +486,7 @@ Table result = orders.dropColumns("b, c");

    Performs a field rename operation. The field expressions should be alias expressions, and only the existing fields can be renamed.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders.renameColumns("b as b2, c as c2"); {% endhighlight %} @@ -513,7 +513,7 @@ Table result = orders.renameColumns("b as b2, c as c2");

    Performs a field add operation. It will throw an exception if the added fields already exist.

    {% highlight scala %} -val orders = tableEnv.scan("Orders"); +val orders = tableEnv.from("Orders"); val result = orders.addColumns(concat('c, "Sunny")) {% endhighlight %} @@ -526,7 +526,7 @@ val result = orders.addColumns(concat('c, "Sunny"))

    Performs a field add operation. Existing fields will be replaced if add columns name is the same as the existing column name. Moreover, if the added fields have duplicate field name, then the last one is used.

    {% highlight scala %} -val orders = tableEnv.scan("Orders"); +val orders = tableEnv.from("Orders"); val result = orders.addOrReplaceColumns(concat('c, "Sunny") as 'desc) {% endhighlight %} @@ -539,7 +539,7 @@ val result = orders.addOrReplaceColumns(concat('c, "Sunny") as 'desc)

    Performs a field drop operation. The field expressions should be field reference expressions, and only existing fields can be dropped.

    {% highlight scala %} -val orders = tableEnv.scan("Orders"); +val orders = tableEnv.from("Orders"); val result = orders.dropColumns('b, 'c) {% endhighlight %} @@ -552,7 +552,7 @@ val result = orders.dropColumns('b, 'c)

    Performs a field rename operation. The field expressions should be alias expressions, and only the existing fields can be renamed.

    {% highlight scala %} -val orders = tableEnv.scan("Orders"); +val orders = tableEnv.from("Orders"); val result = orders.renameColumns('b as 'b2, 'c as 'c2) {% endhighlight %} @@ -578,7 +578,7 @@ val result = orders.renameColumns('b as 'b2, 'c as 'c2)

    Performs a field add operation. It will throw an exception if the added fields already exist.

    {% highlight python %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.add_columns("concat(c, 'sunny')") {% endhighlight %} @@ -592,7 +592,7 @@ result = orders.add_columns("concat(c, 'sunny')")

    Performs a field add operation. Existing fields will be replaced if add columns name is the same as the existing column name. Moreover, if the added fields have duplicate field name, then the last one is used.

    {% highlight python %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.add_or_replace_columns("concat(c, 'sunny') as desc") {% endhighlight %} @@ -605,7 +605,7 @@ result = orders.add_or_replace_columns("concat(c, 'sunny') as desc")

    Performs a field drop operation. The field expressions should be field reference expressions, and only existing fields can be dropped.

    {% highlight python %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.drop_columns("b, c") {% endhighlight %} @@ -618,7 +618,7 @@ result = orders.drop_columns("b, c")

    Performs a field rename operation. The field expressions should be alias expressions, and only the existing fields can be renamed.

    {% highlight python %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.rename_columns("b as b2, c as c2") {% endhighlight %} @@ -653,7 +653,7 @@ result = orders.rename_columns("b as b2, c as c2")

    Similar to a SQL GROUP BY clause. Groups the rows on the grouping keys with a following running aggregation operator to aggregate rows group-wise.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders.groupBy("a").select("a, b.sum as d"); {% endhighlight %}

    Note: For streaming queries the required state to compute the query result might grow infinitely depending on the type of aggregation and the number of distinct grouping keys. Please provide a query configuration with valid retention interval to prevent excessive state size. See Query Configuration for details.

    @@ -667,7 +667,7 @@ Table result = orders.groupBy("a").select("a, b.sum as d");

    Groups and aggregates a table on a group window and possibly one or more grouping keys.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders .window(Tumble.over("5.minutes").on("rowtime").as("w")) // define window .groupBy("a, w") // group by key and window @@ -683,7 +683,7 @@ Table result = orders

    Similar to a SQL OVER clause. Over window aggregates are computed for each row, based on a window (range) of preceding and succeeding rows. See the over windows section for more details.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders // define window .window(Over @@ -706,7 +706,7 @@ Table result = orders

    Similar to a SQL DISTINCT aggregation clause such as COUNT(DISTINCT a). Distinct aggregation declares that an aggregation function (built-in or user-defined) is only applied on distinct input values. Distinct can be applied to GroupBy Aggregation, GroupBy Window Aggregation and Over Window Aggregation.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); // Distinct aggregation on group by Table groupByDistinctResult = orders .groupBy("a") @@ -726,7 +726,7 @@ Table result = orders {% endhighlight %}

    User-defined aggregation function can also be used with DISTINCT modifiers. To calculate the aggregate results only for distinct values, simply add the distinct modifier towards the aggregation function.

    {% highlight java %} -Table orders = tEnv.scan("Orders"); +Table orders = tEnv.from("Orders"); // Use distinct aggregation for user-defined aggregate functions tEnv.registerFunction("myUdagg", new MyUdagg()); @@ -744,7 +744,7 @@ orders.groupBy("users").select("users, myUdagg.distinct(points) as myDistinctRes

    Similar to a SQL DISTINCT clause. Returns records with distinct value combinations.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders.distinct(); {% endhighlight %}

    Note: For streaming queries the required state to compute the query result might grow infinitely depending on the number of distinct fields. Please provide a query configuration with valid retention interval to prevent excessive state size. If state cleaning is enabled, distinct have to emit messages to prevent too early state eviction of downstream operators which makes distinct contains result updating. See Query Configuration for details.

    @@ -774,7 +774,7 @@ Table result = orders.distinct();

    Similar to a SQL GROUP BY clause. Groups the rows on the grouping keys with a following running aggregation operator to aggregate rows group-wise.

    {% highlight scala %} -val orders: Table = tableEnv.scan("Orders") +val orders: Table = tableEnv.from("Orders") val result = orders.groupBy('a).select('a, 'b.sum as 'd) {% endhighlight %}

    Note: For streaming queries the required state to compute the query result might grow infinitely depending on the type of aggregation and the number of distinct grouping keys. Please provide a query configuration with valid retention interval to prevent excessive state size. See Query Configuration for details.

    @@ -788,7 +788,7 @@ val result = orders.groupBy('a).select('a, 'b.sum as 'd)

    Groups and aggregates a table on a group window and possibly one or more grouping keys.

    {% highlight scala %} -val orders: Table = tableEnv.scan("Orders") +val orders: Table = tableEnv.from("Orders") val result: Table = orders .window(Tumble over 5.minutes on 'rowtime as 'w) // define window .groupBy('a, 'w) // group by key and window @@ -804,7 +804,7 @@ val result: Table = orders

    Similar to a SQL OVER clause. Over window aggregates are computed for each row, based on a window (range) of preceding and succeeding rows. See the over windows section for more details.

    {% highlight scala %} -val orders: Table = tableEnv.scan("Orders") +val orders: Table = tableEnv.from("Orders") val result: Table = orders // define window .window(Over @@ -827,7 +827,7 @@ val result: Table = orders

    Similar to a SQL DISTINCT AGGREGATION clause such as COUNT(DISTINCT a). Distinct aggregation declares that an aggregation function (built-in or user-defined) is only applied on distinct input values. Distinct can be applied to GroupBy Aggregation, GroupBy Window Aggregation and Over Window Aggregation.

    {% highlight scala %} -val orders: Table = tableEnv.scan("Orders"); +val orders: Table = tableEnv.from("Orders"); // Distinct aggregation on group by val groupByDistinctResult = orders .groupBy('a) @@ -847,7 +847,7 @@ val result = orders {% endhighlight %}

    User-defined aggregation function can also be used with DISTINCT modifiers. To calculate the aggregate results only for distinct values, simply add the distinct modifier towards the aggregation function.

    {% highlight scala %} -val orders: Table = tEnv.scan("Orders"); +val orders: Table = tEnv.from("Orders"); // Use distinct aggregation for user-defined aggregate functions val myUdagg = new MyUdagg(); @@ -865,7 +865,7 @@ orders.groupBy('users).select('users, myUdagg.distinct('points) as 'myDistinctRe

    Similar to a SQL DISTINCT clause. Returns records with distinct value combinations.

    {% highlight scala %} -val orders: Table = tableEnv.scan("Orders") +val orders: Table = tableEnv.from("Orders") val result = orders.distinct() {% endhighlight %}

    Note: For streaming queries the required state to compute the query result might grow infinitely depending on the number of distinct fields. Please provide a query configuration with valid retention interval to prevent excessive state size. If state cleaning is enabled, distinct have to emit messages to prevent too early state eviction of downstream operators which makes distinct contains result updating. See Query Configuration for details.

    @@ -893,7 +893,7 @@ val result = orders.distinct()

    Similar to a SQL GROUP BY clause. Groups the rows on the grouping keys with a following running aggregation operator to aggregate rows group-wise.

    {% highlight python %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.group_by("a").select("a, b.sum as d") {% endhighlight %}

    Note: For streaming queries the required state to compute the query result might grow infinitely depending on the type of aggregation and the number of distinct grouping keys. Please provide a query configuration with valid retention interval to prevent excessive state size. See Query Configuration for details.

    @@ -907,7 +907,7 @@ result = orders.group_by("a").select("a, b.sum as d")

    Groups and aggregates a table on a group window and possibly one or more grouping keys.

    {% highlight python %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.window(Tumble.over("5.minutes").on("rowtime").alias("w")) \ .group_by("a, w") \ .select("a, w.start, w.end, w.rowtime, b.sum as d") @@ -922,7 +922,7 @@ result = orders.window(Tumble.over("5.minutes").on("rowtime").alias("w")) \

    Similar to a SQL OVER clause. Over window aggregates are computed for each row, based on a window (range) of preceding and succeeding rows. See the over windows section for more details.

    {% highlight python %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.over_window(Over.partition_by("a").order_by("rowtime") .preceding("UNBOUNDED_RANGE").following("CURRENT_RANGE") .alias("w")) \ @@ -940,7 +940,7 @@ result = orders.over_window(Over.partition_by("a").order_by("rowtime")

    Similar to a SQL DISTINCT aggregation clause such as COUNT(DISTINCT a). Distinct aggregation declares that an aggregation function (built-in or user-defined) is only applied on distinct input values. Distinct can be applied to GroupBy Aggregation, GroupBy Window Aggregation and Over Window Aggregation.

    {% highlight python %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") # Distinct aggregation on group by group_by_distinct_result = orders.group_by("a") \ .select("a, b.sum.distinct as d") @@ -969,7 +969,7 @@ result = orders.over_window(Over

    Similar to a SQL DISTINCT clause. Returns records with distinct value combinations.

    {% highlight java %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.distinct() {% endhighlight %}

    Note: For streaming queries the required state to compute the query result might grow infinitely depending on the number of distinct fields. Please provide a query configuration with valid retention interval to prevent excessive state size. See Query Configuration for details.

    @@ -1073,7 +1073,7 @@ TableFunction split = new MySplitUDTF(); tableEnv.registerFunction("split", split); // join -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders .joinLateral("split(c).as(s, t, v)") .select("a, b, s, t, v"); @@ -1094,7 +1094,7 @@ TableFunction split = new MySplitUDTF(); tableEnv.registerFunction("split", split); // join -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders .leftOuterJoinLateral("split(c).as(s, t, v)") .select("a, b, s, t, v"); @@ -1113,7 +1113,7 @@ Table result = orders

    Currently only inner joins with temporal tables are supported.

    {% highlight java %} -Table ratesHistory = tableEnv.scan("RatesHistory"); +Table ratesHistory = tableEnv.from("RatesHistory"); // register temporal table function with a time attribute and primary key TemporalTableFunction rates = ratesHistory.createTemporalTableFunction( @@ -1122,7 +1122,7 @@ TemporalTableFunction rates = ratesHistory.createTemporalTableFunction( tableEnv.registerFunction("rates", rates); // join with "Orders" based on the time attribute and key -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders .joinLateral("rates(o_proctime)", "o_currency = r_currency") {% endhighlight %} @@ -1257,13 +1257,13 @@ val result: Table = table

    Currently only inner joins with temporal tables are supported.

    {% highlight scala %} -val ratesHistory = tableEnv.scan("RatesHistory") +val ratesHistory = tableEnv.from("RatesHistory") // register temporal table function with a time attribute and primary key val rates = ratesHistory.createTemporalTableFunction('r_proctime, 'r_currency) // join with "Orders" based on the time attribute and key -val orders = tableEnv.scan("Orders") +val orders = tableEnv.from("Orders") val result = orders .joinLateral(rates('o_rowtime), 'r_currency === 'o_currency) {% endhighlight %} @@ -1293,8 +1293,8 @@ val result = orders

    Similar to a SQL JOIN clause. Joins two tables. Both tables must have distinct field names and at least one equality join predicate must be defined through join operator or using a where or filter operator.

    {% highlight python %} -left = table_env.scan("Source1").select("a, b, c") -right = table_env.scan("Source2").select("d, e, f") +left = table_env.from_path("Source1").select("a, b, c") +right = table_env.from_path("Source2").select("d, e, f") result = left.join(right).where("a = d").select("a, b, e") {% endhighlight %}

    Note: For streaming queries the required state to compute the query result might grow infinitely depending on the number of distinct input rows. Please provide a query configuration with valid retention interval to prevent excessive state size. See Query Configuration for details.

    @@ -1311,8 +1311,8 @@ result = left.join(right).where("a = d").select("a, b, e")

    Similar to SQL LEFT/RIGHT/FULL OUTER JOIN clauses. Joins two tables. Both tables must have distinct field names and at least one equality join predicate must be defined.

    {% highlight python %} -left = table_env.scan("Source1").select("a, b, c") -right = table_env.scan("Source2").select("d, e, f") +left = table_env.from_path("Source1").select("a, b, c") +right = table_env.from_path("Source2").select("d, e, f") left_outer_result = left.left_outer_join(right, "a = d").select("a, b, e") right_outer_result = left.right_outer_join(right, "a = d").select("a, b, e") @@ -1343,7 +1343,7 @@ full_outer_result = left.full_outer_join(right, "a = d").select("a, b, e") table_env.register_java_function("split", "com.my.udf.MySplitUDTF") # join -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.join_lateral("split(c).as(s, t, v)").select("a, b, s, t, v") {% endhighlight %} @@ -1361,7 +1361,7 @@ result = orders.join_lateral("split(c).as(s, t, v)").select("a, b, s, t, v") table_env.register_java_function("split", "com.my.udf.MySplitUDTF") # join -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.left_outer_join_lateral("split(c).as(s, t, v)").select("a, b, s, t, v") {% endhighlight %} @@ -1502,7 +1502,7 @@ Table right = ds2.toTable(tableEnv, "a"); Table result = left.select("a, b, c").where("a.in(" + right + ")"); // using explicit registration -tableEnv.registerTable("RightTable", right); +tableEnv.createTemporaryView("RightTable", right); Table result = left.select("a, b, c").where("a.in(RightTable)"); {% endhighlight %} @@ -1651,8 +1651,8 @@ val result = left.select('a, 'b, 'c).where('a.in(right))

    Similar to a SQL UNION clause. Unions two tables with duplicate records removed. Both tables must have identical field types.

    {% highlight python %} -left = table_env.scan("Source1").select("a, b, c") -right = table_env.scan("Source2").select("a, b, c") +left = table_env.from_path("Source1").select("a, b, c") +right = table_env.from_path("Source2").select("a, b, c") result = left.union(right) {% endhighlight %} @@ -1666,8 +1666,8 @@ result = left.union(right)

    Similar to a SQL UNION ALL clause. Unions two tables. Both tables must have identical field types.

    {% highlight python %} -left = table_env.scan("Source1").select("a, b, c") -right = table_env.scan("Source2").select("a, b, c") +left = table_env.from_path("Source1").select("a, b, c") +right = table_env.from_path("Source2").select("a, b, c") result = left.union_all(right) {% endhighlight %} @@ -1681,8 +1681,8 @@ result = left.union_all(right)

    Similar to a SQL INTERSECT clause. Intersect returns records that exist in both tables. If a record is present one or both tables more than once, it is returned just once, i.e., the resulting table has no duplicate records. Both tables must have identical field types.

    {% highlight python %} -left = table_env.scan("Source1").select("a, b, c") -right = table_env.scan("Source2").select("a, b, c") +left = table_env.from_path("Source1").select("a, b, c") +right = table_env.from_path("Source2").select("a, b, c") result = left.intersect(right) {% endhighlight %} @@ -1696,8 +1696,8 @@ result = left.intersect(right)

    Similar to a SQL INTERSECT ALL clause. IntersectAll returns records that exist in both tables. If a record is present in both tables more than once, it is returned as many times as it is present in both tables, i.e., the resulting table might have duplicate records. Both tables must have identical field types.

    {% highlight python %} -left = table_env.scan("Source1").select("a, b, c") -right = table_env.scan("Source2").select("a, b, c") +left = table_env.from_path("Source1").select("a, b, c") +right = table_env.from_path("Source2").select("a, b, c") result = left.intersect_all(right) {% endhighlight %} @@ -1711,8 +1711,8 @@ result = left.intersect_all(right)

    Similar to a SQL EXCEPT clause. Minus returns records from the left table that do not exist in the right table. Duplicate records in the left table are returned exactly once, i.e., duplicates are removed. Both tables must have identical field types.

    {% highlight python %} -left = table_env.scan("Source1").select("a, b, c") -right = table_env.scan("Source2").select("a, b, c") +left = table_env.from_path("Source1").select("a, b, c") +right = table_env.from_path("Source2").select("a, b, c") result = left.minus(right); {% endhighlight %} @@ -1726,8 +1726,8 @@ result = left.minus(right);

    Similar to a SQL EXCEPT ALL clause. MinusAll returns the records that do not exist in the right table. A record that is present n times in the left table and m times in the right table is returned (n - m) times, i.e., as many duplicates as are present in the right table are removed. Both tables must have identical field types.

    {% highlight python %} -left = table_env.scan("Source1").select("a, b, c") -right = table_env.scan("Source2").select("a, b, c") +left = table_env.from_path("Source1").select("a, b, c") +right = table_env.from_path("Source2").select("a, b, c") result = left.minus_all(right) {% endhighlight %} @@ -1741,8 +1741,8 @@ result = left.minus_all(right)

    Similar to a SQL IN clause. In returns true if an expression exists in a given table sub-query. The sub-query table must consist of one column. This column must have the same data type as the expression.

    {% highlight python %} -left = table_env.scan("Source1").select("a, b, c") -right = table_env.scan("Source2").select("a") +left = table_env.from_path("Source1").select("a, b, c") +right = table_env.from_path("Source2").select("a") # using implicit registration result = left.select("a, b, c").where("a.in(%s)" % right) @@ -1880,7 +1880,7 @@ val result3: Table = in.orderBy('a.asc).offset(10).fetch(5)

    Similar to a SQL ORDER BY clause. Returns records globally sorted across all parallel partitions.

    {% highlight python %} -in = table_env.scan("Source1").select("a, b, c") +in = table_env.from_path("Source1").select("a, b, c") result = in.order_by("a.asc") {% endhighlight %} @@ -1894,7 +1894,7 @@ result = in.order_by("a.asc")

    Similar to the SQL OFFSET and FETCH clauses. Offset and Fetch limit the number of records returned from a sorted result. Offset and Fetch are technically part of the Order By operator and thus must be preceded by it.

    {% highlight python %} -in = table_env.scan("Source1").select("a, b, c") +in = table_env.from_path("Source1").select("a, b, c") # returns the first 5 records from the sorted result result1 = in.order_by("a.asc").fetch(5) @@ -1934,10 +1934,10 @@ result3 = in.order_by("a.asc").offset(10).fetch(5)

    Similar to the INSERT INTO clause in a SQL query. Performs a insertion into a registered output table.

    -

    Output tables must be registered in the TableEnvironment (see Register a TableSink). Moreover, the schema of the registered table must match the schema of the query.

    +

    Output tables must be registered in the TableEnvironment (see Connector tables). Moreover, the schema of the registered table must match the schema of the query.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); orders.insertInto("OutOrders"); {% endhighlight %} @@ -1964,10 +1964,10 @@ orders.insertInto("OutOrders");

    Similar to the INSERT INTO clause in a SQL query. Performs a insertion into a registered output table.

    -

    Output tables must be registered in the TableEnvironment (see Register a TableSink). Moreover, the schema of the registered table must match the schema of the query.

    +

    Output tables must be registered in the TableEnvironment (see Connector tables). Moreover, the schema of the registered table must match the schema of the query.

    {% highlight scala %} -val orders: Table = tableEnv.scan("Orders") +val orders: Table = tableEnv.from("Orders") orders.insertInto("OutOrders") {% endhighlight %} @@ -1997,7 +1997,7 @@ orders.insertInto("OutOrders")

    Output tables must be registered in the TableEnvironment (see Register a TableSink). Moreover, the schema of the registered table must match the schema of the query.

    {% highlight python %} -orders = table_env.scan("Orders"); +orders = table_env.from_path("Orders"); orders.insert_into("OutOrders"); {% endhighlight %} @@ -2739,7 +2739,7 @@ public class Top2 extends TableAggregateFunction, Top2A } tEnv.registerFunction("top2", new Top2()); -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders .groupBy("key") .flatAggregate("top2(a) as (v, rank)") @@ -2759,7 +2759,7 @@ Table result = orders

    Groups and aggregates a table on a group window and possibly one or more grouping keys. You have to close the "flatAggregate" with a select statement. And the select statement does not support aggregate functions.

    {% highlight java %} tableEnv.registerFunction("top2", new Top2()); -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders .window(Tumble.over("5.minutes").on("rowtime").as("w")) // define window .groupBy("a, w") // group by key and window @@ -2967,7 +2967,7 @@ class Top2 extends TableAggregateFunction[JTuple2[JInteger, JInteger], Top2Accum } val top2 = new Top2 -val orders: Table = tableEnv.scan("Orders") +val orders: Table = tableEnv.from("Orders") val result = orders .groupBy('key) .flatAggregate(top2('a) as ('v, 'rank)) @@ -2986,7 +2986,7 @@ val result = orders

    Groups and aggregates a table on a group window and possibly one or more grouping keys. You have to close the "flatAggregate" with a select statement. And the select statement does not support aggregate functions.

    {% highlight scala %} val top2 = new Top2 -val orders: Table = tableEnv.scan("Orders") +val orders: Table = tableEnv.from("Orders") val result = orders .window(Tumble over 5.minutes on 'rowtime as 'w) // define window .groupBy('a, 'w) // group by key and window diff --git a/docs/dev/table/tableApi.zh.md b/docs/dev/table/tableApi.zh.md index c8cf4d57e3ae..42508d23edd5 100644 --- a/docs/dev/table/tableApi.zh.md +++ b/docs/dev/table/tableApi.zh.md @@ -55,7 +55,7 @@ BatchTableEnvironment tEnv = BatchTableEnvironment.create(env); // ... // specify table program -Table orders = tEnv.scan("Orders"); // schema (a, b, c, rowtime) +Table orders = tEnv.from("Orders"); // schema (a, b, c, rowtime) Table counts = orders .groupBy("a") @@ -87,7 +87,7 @@ val tEnv = BatchTableEnvironment.create(env) // ... // specify table program -val orders = tEnv.scan("Orders") // schema (a, b, c, rowtime) +val orders = tEnv.from("Orders") // schema (a, b, c, rowtime) val result = orders .groupBy('a) @@ -115,7 +115,7 @@ t_env = TableEnvironment.create(env, TableConfig()) # ... # specify table program -orders = t_env.scan("Orders") # schema (a, b, c, rowtime) +orders = t_env.from_path("Orders") # schema (a, b, c, rowtime) orders.group_by("a").select("a, b.count as cnt").insert_into("result") @@ -136,7 +136,7 @@ The next example shows a more complex Table API program. The program scans again // ... // specify table program -Table orders = tEnv.scan("Orders"); // schema (a, b, c, rowtime) +Table orders = tEnv.from("Orders"); // schema (a, b, c, rowtime) Table result = orders .filter("a.isNotNull && b.isNotNull && c.isNotNull") @@ -155,7 +155,7 @@ Table result = orders // ... // specify table program -val orders: Table = tEnv.scan("Orders") // schema (a, b, c, rowtime) +val orders: Table = tEnv.from("Orders") // schema (a, b, c, rowtime) val result: Table = orders .filter('a.isNotNull && 'b.isNotNull && 'c.isNotNull) @@ -174,7 +174,7 @@ val result: Table = orders # ... # specify table program -orders = t_env.scan("Orders") # schema (a, b, c, rowtime) +orders = t_env.from_path("Orders") # schema (a, b, c, rowtime) result = orders.filter("a.isNotNull && b.isNotNull && c.isNotNull") \ .select("a.lowerCase() as a, b, rowtime") \ @@ -210,13 +210,13 @@ The Table API supports the following operations. Please note that not all operat - Scan
    + From
    Batch Streaming

    Similar to the FROM clause in a SQL query. Performs a scan of a registered table.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); {% endhighlight %} @@ -228,7 +228,7 @@ Table orders = tableEnv.scan("Orders");

    Similar to a SQL SELECT statement. Performs a select operation.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders.select("a, c as d"); {% endhighlight %}

    You can use star (*) to act as a wild card, selecting all of the columns in the table.

    @@ -245,7 +245,7 @@ Table result = orders.select("*");

    Renames fields.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders.as("x, y, z, t"); {% endhighlight %} @@ -259,12 +259,12 @@ Table result = orders.as("x, y, z, t");

    Similar to a SQL WHERE clause. Filters out rows that do not pass the filter predicate.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders.where("b === 'red'"); {% endhighlight %} or {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders.filter("a % 2 === 0"); {% endhighlight %} @@ -285,13 +285,13 @@ Table result = orders.filter("a % 2 === 0"); - Scan
    + From
    Batch Streaming

    Similar to the FROM clause in a SQL query. Performs a scan of a registered table.

    {% highlight scala %} -val orders: Table = tableEnv.scan("Orders") +val orders: Table = tableEnv.from("Orders") {% endhighlight %} @@ -303,12 +303,12 @@ val orders: Table = tableEnv.scan("Orders")

    Similar to a SQL SELECT statement. Performs a select operation.

    {% highlight scala %} -val orders: Table = tableEnv.scan("Orders") +val orders: Table = tableEnv.from("Orders") val result = orders.select('a, 'c as 'd) {% endhighlight %}

    You can use star (*) to act as a wild card, selecting all of the columns in the table.

    {% highlight scala %} -val orders: Table = tableEnv.scan("Orders") +val orders: Table = tableEnv.from("Orders") val result = orders.select('*) {% endhighlight %} @@ -321,7 +321,7 @@ val result = orders.select('*)

    Renames fields.

    {% highlight scala %} -val orders: Table = tableEnv.scan("Orders").as('x, 'y, 'z, 't) +val orders: Table = tableEnv.from("Orders").as('x, 'y, 'z, 't) {% endhighlight %} @@ -334,12 +334,12 @@ val orders: Table = tableEnv.scan("Orders").as('x, 'y, 'z, 't)

    Similar to a SQL WHERE clause. Filters out rows that do not pass the filter predicate.

    {% highlight scala %} -val orders: Table = tableEnv.scan("Orders") +val orders: Table = tableEnv.from("Orders") val result = orders.filter('a % 2 === 0) {% endhighlight %} or {% highlight scala %} -val orders: Table = tableEnv.scan("Orders") +val orders: Table = tableEnv.from("Orders") val result = orders.where('b === "red") {% endhighlight %} @@ -365,7 +365,7 @@ val result = orders.where('b === "red")

    类似于SQL请求中的FROM子句,将一个环境中已注册的表转换成Table对象。

    {% highlight python %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") {% endhighlight %} @@ -377,7 +377,7 @@ orders = table_env.scan("Orders")

    类似于SQL请求中的SELECT子句,执行一个select操作。

    {% highlight python %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.select("a, c as d") {% endhighlight %}

    您可以使用星号 (*) 表示选择表中的所有列。

    @@ -394,7 +394,7 @@ result = orders.select("*")

    重命名字段。

    {% highlight python %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.alias("x, y, z, t") {% endhighlight %} @@ -408,12 +408,12 @@ result = orders.alias("x, y, z, t")

    类似于SQL请求中的WHERE子句,过滤掉表中不满足条件的行。

    {% highlight python %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.where("b === 'red'") {% endhighlight %} or {% highlight python %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.filter("a % 2 === 0") {% endhighlight %} @@ -446,7 +446,7 @@ result = orders.filter("a % 2 === 0")

    Performs a field add operation. It will throw an exception if the added fields already exist.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders.addColumns("concat(c, 'sunny')"); {% endhighlight %} @@ -460,7 +460,7 @@ Table result = orders.addColumns("concat(c, 'sunny')");

    Performs a field add operation. Existing fields will be replaced if add columns name is the same as the existing column name. Moreover, if the added fields have duplicate field name, then the last one is used.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders.addOrReplaceColumns("concat(c, 'sunny') as desc"); {% endhighlight %} @@ -473,7 +473,7 @@ Table result = orders.addOrReplaceColumns("concat(c, 'sunny') as desc");

    Performs a field drop operation. The field expressions should be field reference expressions, and only existing fields can be dropped.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders.dropColumns("b, c"); {% endhighlight %} @@ -486,7 +486,7 @@ Table result = orders.dropColumns("b, c");

    Performs a field rename operation. The field expressions should be alias expressions, and only the existing fields can be renamed.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders.renameColumns("b as b2, c as c2"); {% endhighlight %} @@ -513,7 +513,7 @@ Table result = orders.renameColumns("b as b2, c as c2");

    Performs a field add operation. It will throw an exception if the added fields already exist.

    {% highlight scala %} -val orders = tableEnv.scan("Orders"); +val orders = tableEnv.from("Orders"); val result = orders.addColumns(concat('c, "Sunny")) {% endhighlight %} @@ -526,7 +526,7 @@ val result = orders.addColumns(concat('c, "Sunny"))

    Performs a field add operation. Existing fields will be replaced if add columns name is the same as the existing column name. Moreover, if the added fields have duplicate field name, then the last one is used.

    {% highlight scala %} -val orders = tableEnv.scan("Orders"); +val orders = tableEnv.from("Orders"); val result = orders.addOrReplaceColumns(concat('c, "Sunny") as 'desc) {% endhighlight %} @@ -539,7 +539,7 @@ val result = orders.addOrReplaceColumns(concat('c, "Sunny") as 'desc)

    Performs a field drop operation. The field expressions should be field reference expressions, and only existing fields can be dropped.

    {% highlight scala %} -val orders = tableEnv.scan("Orders"); +val orders = tableEnv.from("Orders"); val result = orders.dropColumns('b, 'c) {% endhighlight %} @@ -552,7 +552,7 @@ val result = orders.dropColumns('b, 'c)

    Performs a field rename operation. The field expressions should be alias expressions, and only the existing fields can be renamed.

    {% highlight scala %} -val orders = tableEnv.scan("Orders"); +val orders = tableEnv.from("Orders"); val result = orders.renameColumns('b as 'b2, 'c as 'c2) {% endhighlight %} @@ -578,7 +578,7 @@ val result = orders.renameColumns('b as 'b2, 'c as 'c2)

    执行新增字段操作。如果欲添加字段已经存在,将会抛出异常。

    {% highlight python %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.add_columns("concat(c, 'sunny')") {% endhighlight %} @@ -592,7 +592,7 @@ result = orders.add_columns("concat(c, 'sunny')")

    执行新增字段操作。如果欲添加字段已经存在,将会替换该字段。如果新增字段列表中有同名字段,取最靠后的为有效字段。

    {% highlight python %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.add_or_replace_columns("concat(c, 'sunny') as desc") {% endhighlight %} @@ -605,7 +605,7 @@ result = orders.add_or_replace_columns("concat(c, 'sunny') as desc")

    执行删除字段操作。参数必须是字段列表,并且必须是已经存在的字段才能被删除。

    {% highlight python %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.drop_columns("b, c") {% endhighlight %} @@ -618,7 +618,7 @@ result = orders.drop_columns("b, c")

    执行重命名字段操作。参数必须是字段别名(例:b as b2)列表,并且必须是已经存在的字段才能被重命名。

    {% highlight python %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.rename_columns("b as b2, c as c2") {% endhighlight %} @@ -653,7 +653,7 @@ result = orders.rename_columns("b as b2, c as c2")

    Similar to a SQL GROUP BY clause. Groups the rows on the grouping keys with a following running aggregation operator to aggregate rows group-wise.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders.groupBy("a").select("a, b.sum as d"); {% endhighlight %}

    Note: For streaming queries the required state to compute the query result might grow infinitely depending on the type of aggregation and the number of distinct grouping keys. Please provide a query configuration with valid retention interval to prevent excessive state size. See Query Configuration for details.

    @@ -667,7 +667,7 @@ Table result = orders.groupBy("a").select("a, b.sum as d");

    Groups and aggregates a table on a group window and possibly one or more grouping keys.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders .window(Tumble.over("5.minutes").on("rowtime").as("w")) // define window .groupBy("a, w") // group by key and window @@ -683,7 +683,7 @@ Table result = orders

    Similar to a SQL OVER clause. Over window aggregates are computed for each row, based on a window (range) of preceding and succeeding rows. See the over windows section for more details.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders // define window .window(Over @@ -706,7 +706,7 @@ Table result = orders

    Similar to a SQL DISTINCT aggregation clause such as COUNT(DISTINCT a). Distinct aggregation declares that an aggregation function (built-in or user-defined) is only applied on distinct input values. Distinct can be applied to GroupBy Aggregation, GroupBy Window Aggregation and Over Window Aggregation.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); // Distinct aggregation on group by Table groupByDistinctResult = orders .groupBy("a") @@ -726,7 +726,7 @@ Table result = orders {% endhighlight %}

    User-defined aggregation function can also be used with DISTINCT modifiers. To calculate the aggregate results only for distinct values, simply add the distinct modifier towards the aggregation function.

    {% highlight java %} -Table orders = tEnv.scan("Orders"); +Table orders = tEnv.from("Orders"); // Use distinct aggregation for user-defined aggregate functions tEnv.registerFunction("myUdagg", new MyUdagg()); @@ -744,7 +744,7 @@ orders.groupBy("users").select("users, myUdagg.distinct(points) as myDistinctRes

    Similar to a SQL DISTINCT clause. Returns records with distinct value combinations.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders.distinct(); {% endhighlight %}

    Note: For streaming queries the required state to compute the query result might grow infinitely depending on the number of distinct fields. Please provide a query configuration with valid retention interval to prevent excessive state size. If state cleaning is enabled, distinct have to emit messages to prevent too early state eviction of downstream operators which makes distinct contains result updating. See Query Configuration for details.

    @@ -774,7 +774,7 @@ Table result = orders.distinct();

    Similar to a SQL GROUP BY clause. Groups the rows on the grouping keys with a following running aggregation operator to aggregate rows group-wise.

    {% highlight scala %} -val orders: Table = tableEnv.scan("Orders") +val orders: Table = tableEnv.from("Orders") val result = orders.groupBy('a).select('a, 'b.sum as 'd) {% endhighlight %}

    Note: For streaming queries the required state to compute the query result might grow infinitely depending on the type of aggregation and the number of distinct grouping keys. Please provide a query configuration with valid retention interval to prevent excessive state size. See Query Configuration for details.

    @@ -788,7 +788,7 @@ val result = orders.groupBy('a).select('a, 'b.sum as 'd)

    Groups and aggregates a table on a group window and possibly one or more grouping keys.

    {% highlight scala %} -val orders: Table = tableEnv.scan("Orders") +val orders: Table = tableEnv.from("Orders") val result: Table = orders .window(Tumble over 5.minutes on 'rowtime as 'w) // define window .groupBy('a, 'w) // group by key and window @@ -804,7 +804,7 @@ val result: Table = orders

    Similar to a SQL OVER clause. Over window aggregates are computed for each row, based on a window (range) of preceding and succeeding rows. See the over windows section for more details.

    {% highlight scala %} -val orders: Table = tableEnv.scan("Orders") +val orders: Table = tableEnv.from("Orders") val result: Table = orders // define window .window(Over @@ -827,7 +827,7 @@ val result: Table = orders

    Similar to a SQL DISTINCT AGGREGATION clause such as COUNT(DISTINCT a). Distinct aggregation declares that an aggregation function (built-in or user-defined) is only applied on distinct input values. Distinct can be applied to GroupBy Aggregation, GroupBy Window Aggregation and Over Window Aggregation.

    {% highlight scala %} -val orders: Table = tableEnv.scan("Orders"); +val orders: Table = tableEnv.from("Orders"); // Distinct aggregation on group by val groupByDistinctResult = orders .groupBy('a) @@ -847,7 +847,7 @@ val result = orders {% endhighlight %}

    User-defined aggregation function can also be used with DISTINCT modifiers. To calculate the aggregate results only for distinct values, simply add the distinct modifier towards the aggregation function.

    {% highlight scala %} -val orders: Table = tEnv.scan("Orders"); +val orders: Table = tEnv.from("Orders"); // Use distinct aggregation for user-defined aggregate functions val myUdagg = new MyUdagg(); @@ -864,7 +864,7 @@ orders.groupBy('users).select('users, myUdagg.distinct('points) as 'myDistinctRe

    Similar to a SQL DISTINCT clause. Returns records with distinct value combinations.

    {% highlight scala %} -val orders: Table = tableEnv.scan("Orders") +val orders: Table = tableEnv.from("Orders") val result = orders.distinct() {% endhighlight %}

    Note: For streaming queries the required state to compute the query result might grow infinitely depending on the number of distinct fields. Please provide a query configuration with valid retention interval to prevent excessive state size. See Query Configuration for details.

    @@ -892,7 +892,7 @@ val result = orders.distinct()

    类似于SQL的GROUP BY子句。将数据按照指定字段进行分组,之后对各组内数据执行聚合操作。

    {% highlight python %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.group_by("a").select("a, b.sum as d") {% endhighlight %}

    注意: 对于流式查询,计算查询结果所需的状态(state)可能会无限增长,具体情况取决于聚合操作的类型和分组的数量。您可能需要在查询配置中设置状态保留时间,以防止状态过大。详情请看查询配置

    @@ -906,7 +906,7 @@ result = orders.group_by("a").select("a, b.sum as d")

    在一个窗口上分组和聚合数据,可包含其它分组字段。

    {% highlight python %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.window(Tumble.over("5.minutes").on("rowtime").alias("w")) \ .group_by("a, w") \ .select("a, w.start, w.end, w.rowtime, b.sum as d") @@ -921,7 +921,7 @@ result = orders.window(Tumble.over("5.minutes").on("rowtime").alias("w")) \

    类似于SQL中的OVER开窗函数。Over窗口聚合对每一行都进行一次聚合计算,聚合的对象是以当前行的位置为基准,向前向后取一个区间范围内的所有数据。详情请见Over窗口一节。

    {% highlight python %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.over_window(Over.partition_by("a").order_by("rowtime") .preceding("UNBOUNDED_RANGE").following("CURRENT_RANGE") .alias("w")) \ @@ -939,7 +939,7 @@ result = orders.over_window(Over.partition_by("a").order_by("rowtime")

    类似于SQL聚合函数中的的DISTINCT关键字比如COUNT(DISTINCT a)。带有distinct标记的聚合函数只会接受不重复的输入,重复输入将被丢弃。这个去重特性可以在分组聚合(GroupBy Aggregation)分组窗口聚合(GroupBy Window Aggregation)以及Over窗口聚合(Over Window Aggregation)上使用。

    {% highlight python %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") # Distinct aggregation on group by group_by_distinct_result = orders.group_by("a") \ .select("a, b.sum.distinct as d") @@ -968,7 +968,7 @@ result = orders.over_window(Over

    类似于SQL中的DISTINCT子句。返回去重后的数据。

    {% highlight java %} -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.distinct() {% endhighlight %}

    注意: 对于流式查询,计算查询结果所需的状态(state)可能会无限增长,具体情况取决于执行去重判断时参与判断的字段的数量。您可能需要在查询配置中设置状态保留时间,以防止状态过大。详情请看查询配置

    @@ -1072,7 +1072,7 @@ TableFunction split = new MySplitUDTF(); tableEnv.registerFunction("split", split); // join -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders .joinLateral("split(c).as(s, t, v)") .select("a, b, s, t, v"); @@ -1093,7 +1093,7 @@ TableFunction split = new MySplitUDTF(); tableEnv.registerFunction("split", split); // join -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders .leftOuterJoinLateral("split(c).as(s, t, v)") .select("a, b, s, t, v"); @@ -1112,7 +1112,7 @@ Table result = orders

    Currently only inner joins with temporal tables are supported.

    {% highlight java %} -Table ratesHistory = tableEnv.scan("RatesHistory"); +Table ratesHistory = tableEnv.from("RatesHistory"); // register temporal table function with a time attribute and primary key TemporalTableFunction rates = ratesHistory.createTemporalTableFunction( @@ -1121,7 +1121,7 @@ TemporalTableFunction rates = ratesHistory.createTemporalTableFunction( tableEnv.registerFunction("rates", rates); // join with "Orders" based on the time attribute and key -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders .joinLateral("rates(o_proctime)", "o_currency = r_currency") {% endhighlight %} @@ -1256,13 +1256,13 @@ val result: Table = table

    Currently only inner joins with temporal tables are supported.

    {% highlight scala %} -val ratesHistory = tableEnv.scan("RatesHistory") +val ratesHistory = tableEnv.from("RatesHistory") // register temporal table function with a time attribute and primary key val rates = ratesHistory.createTemporalTableFunction('r_proctime, 'r_currency) // join with "Orders" based on the time attribute and key -val orders = tableEnv.scan("Orders") +val orders = tableEnv.from("Orders") val result = orders .joinLateral(rates('o_rowtime), 'r_currency === 'o_currency) {% endhighlight %} @@ -1292,8 +1292,8 @@ val result = orders

    类似于SQL的JOIN子句。对两张表执行内连接操作。两张表必须具有不同的字段名称,并且必须在join方法或者随后的where或filter方法中定义至少一个等值连接条件。

    {% highlight python %} -left = table_env.scan("Source1").select("a, b, c") -right = table_env.scan("Source2").select("d, e, f") +left = table_env.from_path("Source1").select("a, b, c") +right = table_env.from_path("Source2").select("d, e, f") result = left.join(right).where("a = d").select("a, b, e") {% endhighlight %}

    注意: 对于流式查询,计算查询结果所需的状态(state)可能会无限增长,具体取决于不重复的输入行的数量。您可能需要在查询配置中设置状态保留时间,以防止状态过大。详情请看查询配置

    @@ -1310,8 +1310,8 @@ result = left.join(right).where("a = d").select("a, b, e")

    类似于SQL的LEFT/RIGHT/FULL OUTER JOIN子句。对两张表执行外连接操作。两张表必须具有不同的字段名称,并且必须定义至少一个等值连接条件。

    {% highlight python %} -left = table_env.scan("Source1").select("a, b, c") -right = table_env.scan("Source2").select("d, e, f") +left = table_env.from_path("Source1").select("a, b, c") +right = table_env.from_path("Source2").select("d, e, f") left_outer_result = left.left_outer_join(right, "a = d").select("a, b, e") right_outer_result = left.right_outer_join(right, "a = d").select("a, b, e") @@ -1342,7 +1342,7 @@ full_outer_result = left.full_outer_join(right, "a = d").select("a, b, e") table_env.register_java_function("split", "com.my.udf.MySplitUDTF") # join -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.join_lateral("split(c).as(s, t, v)").select("a, b, s, t, v") {% endhighlight %} @@ -1360,7 +1360,7 @@ result = orders.join_lateral("split(c).as(s, t, v)").select("a, b, s, t, v") table_env.register_java_function("split", "com.my.udf.MySplitUDTF") # join -orders = table_env.scan("Orders") +orders = table_env.from_path("Orders") result = orders.left_outer_join_lateral("split(c).as(s, t, v)").select("a, b, s, t, v") {% endhighlight %} @@ -1501,7 +1501,7 @@ Table right = ds2.toTable(tableEnv, "a"); Table result = left.select("a, b, c").where("a.in(" + right + ")"); // using explicit registration -tableEnv.registerTable("RightTable", right); +tableEnv.createTemporaryView("RightTable", right); Table result = left.select("a, b, c").where("a.in(RightTable)"); {% endhighlight %} @@ -1650,8 +1650,8 @@ val result = left.select('a, 'b, 'c).where('a.in(right))

    类似于SQL的UNION子句。将两张表组合成一张表,这张表拥有二者去除重复后的全部数据。两张表的字段和类型必须完全一致。

    {% highlight python %} -left = table_env.scan("Source1").select("a, b, c") -right = table_env.scan("Source2").select("a, b, c") +left = table_env.from_path("Source1").select("a, b, c") +right = table_env.from_path("Source2").select("a, b, c") result = left.union(right) {% endhighlight %} @@ -1665,8 +1665,8 @@ result = left.union(right)

    类似于SQL的UNION ALL子句。将两张表组合成一张表,这张表拥有二者的全部数据。两张表的字段和类型必须完全一致。

    {% highlight python %} -left = table_env.scan("Source1").select("a, b, c") -right = table_env.scan("Source2").select("a, b, c") +left = table_env.from_path("Source1").select("a, b, c") +right = table_env.from_path("Source2").select("a, b, c") result = left.union_all(right) {% endhighlight %} @@ -1680,8 +1680,8 @@ result = left.union_all(right)

    类似于SQL的INTERSECT子句。Intersect返回在两张表中都存在的数据。如果一个记录在两张表中不止出现一次,则只返回一次,即结果表没有重复记录。两张表的字段和类型必须完全一致。

    {% highlight python %} -left = table_env.scan("Source1").select("a, b, c") -right = table_env.scan("Source2").select("a, b, c") +left = table_env.from_path("Source1").select("a, b, c") +right = table_env.from_path("Source2").select("a, b, c") result = left.intersect(right) {% endhighlight %} @@ -1695,8 +1695,8 @@ result = left.intersect(right)

    类似于SQL的INTERSECT ALL子句。IntersectAll返回在两张表中都存在的数据。如果一个记录在两张表中不止出现一次,则按照它在两张表中都出现的次数返回,即结果表可能包含重复数据。两张表的字段和类型必须完全一致。

    {% highlight python %} -left = table_env.scan("Source1").select("a, b, c") -right = table_env.scan("Source2").select("a, b, c") +left = table_env.from_path("Source1").select("a, b, c") +right = table_env.from_path("Source2").select("a, b, c") result = left.intersect_all(right) {% endhighlight %} @@ -1710,8 +1710,8 @@ result = left.intersect_all(right)

    类似于SQL的EXCEPT子句。Minus返回仅存在于左表,不存在于右表中的数据。左表中的相同数据只会返回一次,即数据会被去重。两张表的字段和类型必须完全一致。

    {% highlight python %} -left = table_env.scan("Source1").select("a, b, c") -right = table_env.scan("Source2").select("a, b, c") +left = table_env.from_path("Source1").select("a, b, c") +right = table_env.from_path("Source2").select("a, b, c") result = left.minus(right); {% endhighlight %} @@ -1725,8 +1725,8 @@ result = left.minus(right);

    类似于SQL的EXCEPT ALL子句。MinusAll返回仅存在于左表,不存在于右表中的数据。如果一条数据在左表中出现了n次,在右表中出现了m次,最终这条数据将会被返回(n - m)次,即按右表中出现的次数来移除数据。两张表的字段和类型必须完全一致。

    {% highlight python %} -left = table_env.scan("Source1").select("a, b, c") -right = table_env.scan("Source2").select("a, b, c") +left = table_env.from_path("Source1").select("a, b, c") +right = table_env.from_path("Source2").select("a, b, c") result = left.minus_all(right) {% endhighlight %} @@ -1740,8 +1740,8 @@ result = left.minus_all(right)

    类似于SQL的IN子句。如果In左边表达式的值在给定的子查询结果中则返回true。子查询的结果必须为单列。此列数据类型必须和表达式一致。

    {% highlight python %} -left = table_env.scan("Source1").select("a, b, c") -right = table_env.scan("Source2").select("a") +left = table_env.from_path("Source1").select("a, b, c") +right = table_env.from_path("Source2").select("a") # using implicit registration result = left.select("a, b, c").where("a.in(%s)" % right) @@ -1879,7 +1879,7 @@ val result3: Table = in.orderBy('a.asc).offset(10).fetch(5)

    类似于SQL的ORDER BY子句。返回包括所有子并发分区内所有数据的全局排序结果。

    {% highlight python %} -in = table_env.scan("Source1").select("a, b, c") +in = table_env.from_path("Source1").select("a, b, c") result = in.order_by("a.asc") {% endhighlight %} @@ -1893,7 +1893,7 @@ result = in.order_by("a.asc")

    类似于SQL的OFFSET和FETCH子句。Offset和Fetch从已排序的结果中返回指定数量的数据。Offset和Fetch在技术上是Order By操作的一部分,因此必须紧跟其后出现。

    {% highlight python %} -in = table_env.scan("Source1").select("a, b, c") +in = table_env.from_path("Source1").select("a, b, c") # returns the first 5 records from the sorted result result1 = in.order_by("a.asc").fetch(5) @@ -1936,7 +1936,7 @@ result3 = in.order_by("a.asc").offset(10).fetch(5)

    Output tables must be registered in the TableEnvironment (see Register a TableSink). Moreover, the schema of the registered table must match the schema of the query.

    {% highlight java %} -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); orders.insertInto("OutOrders"); {% endhighlight %} @@ -1963,10 +1963,10 @@ orders.insertInto("OutOrders");

    Similar to the INSERT INTO clause in a SQL query. Performs a insertion into a registered output table.

    -

    Output tables must be registered in the TableEnvironment (see Register a TableSink). Moreover, the schema of the registered table must match the schema of the query.

    +

    Output tables must be registered in the TableEnvironment (see Connector tables). Moreover, the schema of the registered table must match the schema of the query.

    {% highlight scala %} -val orders: Table = tableEnv.scan("Orders") +val orders: Table = tableEnv.from("Orders") orders.insertInto("OutOrders") {% endhighlight %} @@ -1996,7 +1996,7 @@ orders.insertInto("OutOrders")

    输出表必须先在TableEnvironment中注册(详见注册一个TableSink)。此外,注册的表的模式(schema)必须和请求的结果的模式(schema)相匹配。

    {% highlight python %} -orders = table_env.scan("Orders"); +orders = table_env.from_path("Orders"); orders.insert_into("OutOrders"); {% endhighlight %} @@ -2738,7 +2738,7 @@ public class Top2 extends TableAggregateFunction, Top2A } tEnv.registerFunction("top2", new Top2()); -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders .groupBy("key") .flatAggregate("top2(a) as (v, rank)") @@ -2758,7 +2758,7 @@ Table result = orders

    Groups and aggregates a table on a group window and possibly one or more grouping keys. You have to close the "flatAggregate" with a select statement. And the select statement does not support aggregate functions.

    {% highlight java %} tableEnv.registerFunction("top2", new Top2()); -Table orders = tableEnv.scan("Orders"); +Table orders = tableEnv.from("Orders"); Table result = orders .window(Tumble.over("5.minutes").on("rowtime").as("w")) // define window .groupBy("a, w") // group by key and window @@ -2966,7 +2966,7 @@ class Top2 extends TableAggregateFunction[JTuple2[JInteger, JInteger], Top2Accum } val top2 = new Top2 -val orders: Table = tableEnv.scan("Orders") +val orders: Table = tableEnv.from("Orders") val result = orders .groupBy('key) .flatAggregate(top2('a) as ('v, 'rank)) @@ -2985,7 +2985,7 @@ val result = orders

    Groups and aggregates a table on a group window and possibly one or more grouping keys. You have to close the "flatAggregate" with a select statement. And the select statement does not support aggregate functions.

    {% highlight scala %} val top2 = new Top2 -val orders: Table = tableEnv.scan("Orders") +val orders: Table = tableEnv.from("Orders") val result = orders .window(Tumble over 5.minutes on 'rowtime as 'w) // define window .groupBy('a, 'w) // group by key and window diff --git a/docs/ops/scala_shell.md b/docs/ops/scala_shell.md index 50987af2ae9e..386df146c473 100644 --- a/docs/ops/scala_shell.md +++ b/docs/ops/scala_shell.md @@ -151,7 +151,7 @@ Scala-Flink> val textSource = stenv.fromDataStream( "The slings and arrows of outrageous fortune", "Or to take arms against a sea of troubles,"), 'text) -Scala-Flink> stenv.registerTable("text_source", textSource) +Scala-Flink> stenv.createTemporaryView("text_source", textSource) Scala-Flink> class $Split extends TableFunction[String] { def eval(s: String): Unit = { s.toLowerCase.split("\\W+").foreach(collect) @@ -177,7 +177,7 @@ Scala-Flink> val textSource = btenv.fromDataSet( "The slings and arrows of outrageous fortune", "Or to take arms against a sea of troubles,"), 'text) -Scala-Flink> btenv.registerTable("text_source", textSource) +Scala-Flink> btenv.createTemporaryView("text_source", textSource) Scala-Flink> class $Split extends TableFunction[String] { def eval(s: String): Unit = { s.toLowerCase.split("\\W+").foreach(collect) diff --git a/docs/ops/scala_shell.zh.md b/docs/ops/scala_shell.zh.md index 50987af2ae9e..386df146c473 100644 --- a/docs/ops/scala_shell.zh.md +++ b/docs/ops/scala_shell.zh.md @@ -151,7 +151,7 @@ Scala-Flink> val textSource = stenv.fromDataStream( "The slings and arrows of outrageous fortune", "Or to take arms against a sea of troubles,"), 'text) -Scala-Flink> stenv.registerTable("text_source", textSource) +Scala-Flink> stenv.createTemporaryView("text_source", textSource) Scala-Flink> class $Split extends TableFunction[String] { def eval(s: String): Unit = { s.toLowerCase.split("\\W+").foreach(collect) @@ -177,7 +177,7 @@ Scala-Flink> val textSource = btenv.fromDataSet( "The slings and arrows of outrageous fortune", "Or to take arms against a sea of troubles,"), 'text) -Scala-Flink> btenv.registerTable("text_source", textSource) +Scala-Flink> btenv.createTemporaryView("text_source", textSource) Scala-Flink> class $Split extends TableFunction[String] { def eval(s: String): Unit = { s.toLowerCase.split("\\W+").foreach(collect) diff --git a/docs/tutorials/python_table_api.md b/docs/tutorials/python_table_api.md index f882830df579..715ba9ddffa6 100644 --- a/docs/tutorials/python_table_api.md +++ b/docs/tutorials/python_table_api.md @@ -65,7 +65,7 @@ t_env.connect(FileSystem().path('/tmp/input')) \ .field('word', DataTypes.STRING())) \ .with_schema(Schema() .field('word', DataTypes.STRING())) \ - .register_table_source('mySource') + .create_temporary_table('mySource') t_env.connect(FileSystem().path('/tmp/output')) \ .with_format(OldCsv() @@ -75,7 +75,7 @@ t_env.connect(FileSystem().path('/tmp/output')) \ .with_schema(Schema() .field('word', DataTypes.STRING()) .field('count', DataTypes.BIGINT())) \ - .register_table_sink('mySink') + .create_temporary_table('mySink') {% endhighlight %} This registers a table named `mySource` and a table named `mySink` in the @@ -87,7 +87,7 @@ Then we need to create a job which reads input from table `mySource`, preforms s operations and writes the results to table `mySink`. {% highlight python %} -t_env.scan('mySource') \ +t_env.from_path('mySource') \ .group_by('word') \ .select('word, count(1)') \ .insert_into('mySink') @@ -120,7 +120,7 @@ t_env.connect(FileSystem().path('/tmp/input')) \ .field('word', DataTypes.STRING())) \ .with_schema(Schema() .field('word', DataTypes.STRING())) \ - .register_table_source('mySource') + .create_temporary_table('mySource') t_env.connect(FileSystem().path('/tmp/output')) \ .with_format(OldCsv() @@ -130,9 +130,9 @@ t_env.connect(FileSystem().path('/tmp/output')) \ .with_schema(Schema() .field('word', DataTypes.STRING()) .field('count', DataTypes.BIGINT())) \ - .register_table_sink('mySink') + .create_temporary_table('mySink') -t_env.scan('mySource') \ +t_env.from_path('mySource') \ .group_by('word') \ .select('word, count(1)') \ .insert_into('mySink') From 7c4fb30f514bf3d57985bb81cc5627f0c8007470 Mon Sep 17 00:00:00 2001 From: Rui Li Date: Tue, 12 Nov 2019 16:48:17 +0800 Subject: [PATCH 577/746] [FLINK-14715][hive] HiveModuleTest fails with Hive-3.1.1 this closes #10159. --- .../org/apache/flink/table/module/hive/HiveModuleTest.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleTest.java index 7fefe3b685c9..cea529f14141 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleTest.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/module/hive/HiveModuleTest.java @@ -58,8 +58,10 @@ public void testNumberOfBuiltinFunctions() { assertEquals(243, new HiveModule(HiveShimLoader.getHiveVersion()).listFunctions().size()); } else if (hiveVersion.equals(HIVE_VERSION_V2_1_1) || hiveVersion.equals(HIVE_VERSION_V2_2_0)) { assertEquals(253, new HiveModule(HiveShimLoader.getHiveVersion()).listFunctions().size()); - } else if (hiveVersion.equals(HIVE_VERSION_V2_3_4) || hiveVersion.equals(HIVE_VERSION_V3_1_1)) { + } else if (hiveVersion.equals(HIVE_VERSION_V2_3_4)) { assertEquals(287, new HiveModule(HiveShimLoader.getHiveVersion()).listFunctions().size()); + } else if (hiveVersion.equals(HIVE_VERSION_V3_1_1)) { + assertEquals(306, new HiveModule(HiveShimLoader.getHiveVersion()).listFunctions().size()); } } From 0022d4aad53ce28d1201e97538644d6e3d8ab0e5 Mon Sep 17 00:00:00 2001 From: zjuwangg Date: Mon, 11 Nov 2019 16:58:12 +0800 Subject: [PATCH 578/746] [FLINK-14689][table] Add catalog related commands support in sql parser. This closes #10149. --- .../src/main/codegen/data/Parser.tdd | 14 +++- .../src/main/codegen/includes/parserImpls.ftl | 40 +++++++++++ .../flink/sql/parser/ddl/SqlUseCatalog.java | 68 +++++++++++++++++++ .../sql/parser/dql/SqlDescribeCatalog.java | 68 +++++++++++++++++++ .../flink/sql/parser/dql/SqlShowCatalogs.java | 60 ++++++++++++++++ .../sql/parser/FlinkSqlParserImplTest.java | 22 ++++++ 6 files changed, 270 insertions(+), 2 deletions(-) create mode 100644 flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlUseCatalog.java create mode 100644 flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlDescribeCatalog.java create mode 100644 flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowCatalogs.java diff --git a/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd b/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd index 20ed012a5a70..dcb6ecdb6561 100644 --- a/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd +++ b/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd @@ -32,8 +32,11 @@ "org.apache.flink.sql.parser.ddl.SqlTableColumn", "org.apache.flink.sql.parser.ddl.SqlTableOption", "org.apache.flink.sql.parser.ddl.SqlWatermark", + "org.apache.flink.sql.parser.ddl.SqlUseCatalog", "org.apache.flink.sql.parser.dml.RichSqlInsert", "org.apache.flink.sql.parser.dml.RichSqlInsertKeyword", + "org.apache.flink.sql.parser.dql.SqlShowCatalogs", + "org.apache.flink.sql.parser.dql.SqlDescribeCatalog", "org.apache.flink.sql.parser.type.ExtendedSqlBasicTypeNameSpec", "org.apache.flink.sql.parser.type.ExtendedSqlCollectionTypeNameSpec", "org.apache.flink.sql.parser.utils.SqlTimeUnit", @@ -55,7 +58,9 @@ "WATERMARK", "OVERWRITE", "STRING", - "BYTES" + "BYTES", + "CATALOGS", + "USE" ] # List of keywords from "keywords" section that are not reserved. @@ -81,6 +86,7 @@ "C" "CASCADE" "CATALOG" + "CATALOGS" "CATALOG_NAME" "CENTURY" "CHAIN" @@ -347,6 +353,7 @@ "UNDER" "UNNAMED" "USAGE" + "USE" "USER_DEFINED_TYPE_CATALOG" "USER_DEFINED_TYPE_CODE" "USER_DEFINED_TYPE_NAME" @@ -373,7 +380,10 @@ # Return type of method implementation should be 'SqlNode'. # Example: SqlShowDatabases(), SqlShowTables(). statementParserMethods: [ - "RichSqlInsert()" + "RichSqlInsert()", + "SqlShowCatalogs()", + "SqlDescribeCatalog()", + "SqlUseCatalog()" ] # List of methods for parsing custom literals. 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 e3ff107b4a8a..421c88192752 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 @@ -15,6 +15,46 @@ // limitations under the License. --> +/** +* Parse a "Show Catalogs" metadata query command. +*/ +SqlShowCatalogs SqlShowCatalogs() : +{ +} +{ + + { + return new SqlShowCatalogs(getPos()); + } +} + +SqlDescribeCatalog SqlDescribeCatalog() : +{ + SqlIdentifier catalogName; + SqlParserPos pos; +} +{ + { pos = getPos();} + catalogName = SimpleIdentifier() + { + return new SqlDescribeCatalog(pos, catalogName); + } + +} + +SqlUseCatalog SqlUseCatalog() : +{ +SqlIdentifier catalogName; +SqlParserPos pos; +} +{ + { pos = getPos();} + catalogName = SimpleIdentifier() + { + return new SqlUseCatalog(pos, catalogName); + } +} + void TableColumn(TableCreationContext context) : { } diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlUseCatalog.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlUseCatalog.java new file mode 100644 index 000000000000..6fa2fa0d126e --- /dev/null +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlUseCatalog.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.sql.parser.ddl; + +import org.apache.calcite.sql.SqlCall; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.SqlSpecialOperator; +import org.apache.calcite.sql.SqlWriter; +import org.apache.calcite.sql.parser.SqlParserPos; + +import java.util.Collections; +import java.util.List; + +/** + * USE CATALOG sql call. + */ +public class SqlUseCatalog extends SqlCall { + + public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("USE CATALOG", SqlKind.OTHER); + private final SqlIdentifier catalogName; + + public SqlUseCatalog(SqlParserPos pos, SqlIdentifier catalogName) { + super(pos); + this.catalogName = catalogName; + } + + @Override + public SqlOperator getOperator() { + return OPERATOR; + } + + @Override + public List getOperandList() { + return Collections.singletonList(catalogName); + } + + public String getCatalogName() { + return catalogName.getSimple(); + } + + @Override + public void unparse( + SqlWriter writer, + int leftPrec, + int rightPrec) { + writer.keyword("USE CATALOG"); + catalogName.unparse(writer, leftPrec, rightPrec); + } +} diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlDescribeCatalog.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlDescribeCatalog.java new file mode 100644 index 000000000000..28ddbdbfea8c --- /dev/null +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlDescribeCatalog.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.sql.parser.dql; + +import org.apache.calcite.sql.SqlCall; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.SqlSpecialOperator; +import org.apache.calcite.sql.SqlWriter; +import org.apache.calcite.sql.parser.SqlParserPos; + +import java.util.Collections; +import java.util.List; + +/** + * DESCRIBE CATALOG sql call. + */ +public class SqlDescribeCatalog extends SqlCall { + + public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("DESCRIBE CATALOG", SqlKind.OTHER); + private final SqlIdentifier catalogName; + + public SqlDescribeCatalog(SqlParserPos pos, SqlIdentifier catalogName) { + super(pos); + this.catalogName = catalogName; + } + + @Override + public SqlOperator getOperator() { + return OPERATOR; + } + + @Override + public List getOperandList() { + return Collections.singletonList(catalogName); + } + + public String getCatalogName() { + return catalogName.getSimple(); + } + + @Override + public void unparse( + SqlWriter writer, + int leftPrec, + int rightPrec) { + writer.keyword("DESCRIBE CATALOG"); + catalogName.unparse(writer, leftPrec, rightPrec); + } +} diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowCatalogs.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowCatalogs.java new file mode 100644 index 000000000000..0821c1801a2a --- /dev/null +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowCatalogs.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.sql.parser.dql; + +import org.apache.calcite.sql.SqlCall; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.SqlSpecialOperator; +import org.apache.calcite.sql.SqlWriter; +import org.apache.calcite.sql.parser.SqlParserPos; + +import java.util.Collections; +import java.util.List; + +/** + * SHOW CATALOGS sql call. + */ +public class SqlShowCatalogs extends SqlCall { + + public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("SHOW CATALOGS", SqlKind.OTHER); + + public SqlShowCatalogs(SqlParserPos pos) { + super(pos); + } + + @Override + public SqlOperator getOperator() { + return OPERATOR; + } + + @Override + public List getOperandList() { + return Collections.EMPTY_LIST; + } + + @Override + public void unparse( + SqlWriter writer, + int leftPrec, + int rightPrec) { + writer.keyword("SHOW CATALOGS"); + } +} 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 f4371ee7d7dd..7d390a2ab3fd 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 @@ -73,6 +73,28 @@ public void before() { conformance0 = null; } + @Test + public void testShowCatalogs() { + check("show catalogs", "SHOW CATALOGS"); + } + + @Test + public void testDescribeCatalog() { + check("describe catalog a", "DESCRIBE CATALOG `A`"); + } + + /** + * Here we override the super method to avoid test error from `describe schema` supported in original calcite. + */ + @Override + public void testDescribeSchema() { + } + + @Test + public void testUseCatalog() { + check("use catalog a", "USE CATALOG `A`"); + } + @Test public void testCreateTable() { check("CREATE TABLE tbl1 (\n" + From a8e3a2029733a5223f2f9217ed68ef86499d058d Mon Sep 17 00:00:00 2001 From: yanghua Date: Wed, 6 Nov 2019 16:55:02 +0800 Subject: [PATCH 579/746] [FLINK-14612][runtime] Change Type of Field intermediateResults from ConcurrentHashMap to HashMap This closes #10102. --- .../apache/flink/runtime/executiongraph/ExecutionGraph.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index f787ba2a2ff8..bb51ebe79f3f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -199,7 +199,7 @@ public class ExecutionGraph implements AccessExecutionGraph { private final List verticesInCreationOrder; /** All intermediate results that are part of this graph. */ - private final ConcurrentHashMap intermediateResults; + private final Map intermediateResults; /** The currently executed tasks, for callbacks. */ private final ConcurrentHashMap currentExecutions; @@ -463,7 +463,7 @@ public ExecutionGraph( this.userClassLoader = Preconditions.checkNotNull(userClassLoader, "userClassLoader"); this.tasks = new ConcurrentHashMap<>(16); - this.intermediateResults = new ConcurrentHashMap<>(16); + this.intermediateResults = new HashMap<>(16); this.verticesInCreationOrder = new ArrayList<>(16); this.currentExecutions = new ConcurrentHashMap<>(16); From 512b4f7ccb604d903ec0a3205f89b358c5405434 Mon Sep 17 00:00:00 2001 From: yanghua Date: Wed, 6 Nov 2019 16:45:27 +0800 Subject: [PATCH 580/746] [FLINK-14602][runtime] Change Type of Field tasks from ConcurrentHashMap to HashMap This closes #10101. --- .../apache/flink/runtime/executiongraph/ExecutionGraph.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index bb51ebe79f3f..966728517ed9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -193,7 +193,7 @@ public class ExecutionGraph implements AccessExecutionGraph { private boolean isStoppable = true; /** All job vertices that are part of this graph. */ - private final ConcurrentHashMap tasks; + private final Map tasks; /** All vertices, in the order in which they were created. **/ private final List verticesInCreationOrder; @@ -462,7 +462,7 @@ public ExecutionGraph( allocationTimeout); this.userClassLoader = Preconditions.checkNotNull(userClassLoader, "userClassLoader"); - this.tasks = new ConcurrentHashMap<>(16); + this.tasks = new HashMap<>(16); this.intermediateResults = new HashMap<>(16); this.verticesInCreationOrder = new ArrayList<>(16); this.currentExecutions = new ConcurrentHashMap<>(16); From 4d97e2fece250b8c1fbfee4ef94f276d9623ddcf Mon Sep 17 00:00:00 2001 From: yanghua Date: Thu, 7 Nov 2019 15:22:12 +0800 Subject: [PATCH 581/746] [FLINK-14640][runtime] Change Type of Field currentExecutions from ConcurrentHashMap to HashMap This closes #10112. --- .../executiongraph/ExecutionGraph.java | 5 ++- .../ExecutionGraphDeploymentTest.java | 32 ++++++++++++------- 2 files changed, 22 insertions(+), 15 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 966728517ed9..98a018974a5d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -111,7 +111,6 @@ import java.util.Optional; import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; @@ -202,7 +201,7 @@ public class ExecutionGraph implements AccessExecutionGraph { private final Map intermediateResults; /** The currently executed tasks, for callbacks. */ - private final ConcurrentHashMap currentExecutions; + private final Map currentExecutions; /** Listeners that receive messages when the entire job switches it status * (such as from RUNNING to FINISHED). */ @@ -465,7 +464,7 @@ public ExecutionGraph( this.tasks = new HashMap<>(16); this.intermediateResults = new HashMap<>(16); this.verticesInCreationOrder = new ArrayList<>(16); - this.currentExecutions = new ConcurrentHashMap<>(16); + this.currentExecutions = new HashMap<>(16); this.jobStatusListeners = new CopyOnWriteArrayList<>(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java index 439093718cc8..c5a5f637cc8e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java @@ -246,13 +246,15 @@ public void testRegistrationOfExecutionsFinishing() { JobVertex v1 = new JobVertex("v1", jid1); JobVertex v2 = new JobVertex("v2", jid2); - Map executions = setupExecution(v1, 7650, v2, 2350).f1; + Tuple2> graphExecutionsTuple = setupExecution(v1, 7650, v2, 2350); + ExecutionGraph testExecutionGraph = graphExecutionsTuple.f0; + Collection executions = new ArrayList<>(graphExecutionsTuple.f1.values()); - for (Execution e : executions.values()) { + for (Execution e : executions) { e.markFinished(); } - assertEquals(0, executions.size()); + assertEquals(0, testExecutionGraph.getRegisteredExecutions().size()); } catch (Exception e) { e.printStackTrace(); @@ -270,13 +272,15 @@ public void testRegistrationOfExecutionsFailing() { JobVertex v1 = new JobVertex("v1", jid1); JobVertex v2 = new JobVertex("v2", jid2); - Map executions = setupExecution(v1, 7, v2, 6).f1; + Tuple2> graphExecutionsTuple = setupExecution(v1, 7, v2, 6); + ExecutionGraph testExecutionGraph = graphExecutionsTuple.f0; + Collection executions = new ArrayList<>(graphExecutionsTuple.f1.values()); - for (Execution e : executions.values()) { + for (Execution e : executions) { e.markFailed(null); } - assertEquals(0, executions.size()); + assertEquals(0, testExecutionGraph.getRegisteredExecutions().size()); } catch (Exception e) { e.printStackTrace(); @@ -294,13 +298,15 @@ public void testRegistrationOfExecutionsFailedExternally() { JobVertex v1 = new JobVertex("v1", jid1); JobVertex v2 = new JobVertex("v2", jid2); - Map executions = setupExecution(v1, 7, v2, 6).f1; + Tuple2> graphExecutionsTuple = setupExecution(v1, 7, v2, 6); + ExecutionGraph testExecutionGraph = graphExecutionsTuple.f0; + Collection executions = new ArrayList<>(graphExecutionsTuple.f1.values()); - for (Execution e : executions.values()) { + for (Execution e : executions) { e.fail(null); } - assertEquals(0, executions.size()); + assertEquals(0, testExecutionGraph.getRegisteredExecutions().size()); } catch (Exception e) { e.printStackTrace(); @@ -397,14 +403,16 @@ public void testRegistrationOfExecutionsCanceled() { JobVertex v1 = new JobVertex("v1", jid1); JobVertex v2 = new JobVertex("v2", jid2); - Map executions = setupExecution(v1, 19, v2, 37).f1; + Tuple2> graphExecutionsTuple = setupExecution(v1, 19, v2, 37); + ExecutionGraph testExecutionGraph = graphExecutionsTuple.f0; + Collection executions = new ArrayList<>(graphExecutionsTuple.f1.values()); - for (Execution e : executions.values()) { + for (Execution e : executions) { e.cancel(); e.completeCancelling(); } - assertEquals(0, executions.size()); + assertEquals(0, testExecutionGraph.getRegisteredExecutions().size()); } catch (Exception e) { e.printStackTrace(); From 58f22cc1c662a479c031f5a868753b51eac63b71 Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Wed, 6 Nov 2019 15:26:15 -0800 Subject: [PATCH 582/746] [FLINK-14638][doc] move functions related docs to a new single, unified dir /dev/table/functions --- docs/dev/table/{functions.md => functions/builtinFunctions.md} | 0 .../table/{functions.zh.md => functions/builtinFunctions.zh.md} | 0 docs/dev/table/{ => functions}/udfs.md | 0 docs/dev/table/{ => functions}/udfs.zh.md | 0 4 files changed, 0 insertions(+), 0 deletions(-) rename docs/dev/table/{functions.md => functions/builtinFunctions.md} (100%) rename docs/dev/table/{functions.zh.md => functions/builtinFunctions.zh.md} (100%) rename docs/dev/table/{ => functions}/udfs.md (100%) rename docs/dev/table/{ => functions}/udfs.zh.md (100%) diff --git a/docs/dev/table/functions.md b/docs/dev/table/functions/builtinFunctions.md similarity index 100% rename from docs/dev/table/functions.md rename to docs/dev/table/functions/builtinFunctions.md diff --git a/docs/dev/table/functions.zh.md b/docs/dev/table/functions/builtinFunctions.zh.md similarity index 100% rename from docs/dev/table/functions.zh.md rename to docs/dev/table/functions/builtinFunctions.zh.md diff --git a/docs/dev/table/udfs.md b/docs/dev/table/functions/udfs.md similarity index 100% rename from docs/dev/table/udfs.md rename to docs/dev/table/functions/udfs.md diff --git a/docs/dev/table/udfs.zh.md b/docs/dev/table/functions/udfs.zh.md similarity index 100% rename from docs/dev/table/udfs.zh.md rename to docs/dev/table/functions/udfs.zh.md From a20eb65cf7b93e5d2b19885cbbe44f98031c1b2e Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Wed, 6 Nov 2019 15:27:26 -0800 Subject: [PATCH 583/746] [FLINK-14301][doc] add documentation for functions categories and new function resolution orders this closes #10110. --- docs/dev/table/functions/index.md | 94 ++++++++++++++++++ docs/dev/table/functions/index.zh.md | 97 +++++++++++++++++++ ...builtinFunctions.md => systemFunctions.md} | 4 +- ...nFunctions.zh.md => systemFunctions.zh.md} | 4 +- docs/dev/table/functions/udfs.md | 2 +- docs/dev/table/functions/udfs.zh.md | 2 +- 6 files changed, 197 insertions(+), 6 deletions(-) create mode 100644 docs/dev/table/functions/index.md create mode 100644 docs/dev/table/functions/index.zh.md rename docs/dev/table/functions/{builtinFunctions.md => systemFunctions.md} (99%) rename docs/dev/table/functions/{builtinFunctions.zh.md => systemFunctions.zh.md} (99%) diff --git a/docs/dev/table/functions/index.md b/docs/dev/table/functions/index.md new file mode 100644 index 000000000000..16a8b8720c30 --- /dev/null +++ b/docs/dev/table/functions/index.md @@ -0,0 +1,94 @@ +--- +title: "Functions" +nav-id: table_functions +nav-parent_id: table_functions +nav-pos: 40 +nav-show_overview: true +--- + + +Flink Table API & SQL empowers users to do data transformations with functions. + +* This will be replaced by the TOC +{:toc} + +Types of Functions +------------------ + +There are two dimensions to classify functions in Flink. + +One dimension is system (or built-in) functions v.s. catalog functions. System functions have no namespace and can be +referenced with just their names. Catalog functions belong to a catalog and database therefore they have catalog and database +namespaces, they can be referenced by either fully/partially qualified name (`catalog.db.func` or `db.func`) or just the +function name. + +The other dimension is temporary functions v.s. persistent functions. Temporary functions are volatile and only live up to + lifespan of a session, they are always created by users. Persistent functions live across lifespan of sessions, they are either + provided by the system or persisted in catalogs. + +The two dimensions give Flink users 4 categories of functions: + +1. Temporary system functions +2. System functions +3. Temporary catalog functions +4. Catalog functions + +Referencing Functions +--------------------- + +There are two ways users can reference a function in Flink - referencing function precisely or ambiguously. + +## Precise Function Reference + +Precise function reference empowers users to use catalog functions specifically, and across catalog and across database, +e.g. `select mycatalog.mydb.myfunc(x) from mytable` and `select mydb.myfunc(x) from mytable`. + +This is only supported starting from Flink 1.10. + +## Ambiguous Function Reference + +In ambiguous function reference, users just specify the function's name in SQL query, e.g. `select myfunc(x) from mytable`. + + +Function Resolution Order +------------------------- + +The resolution order only matters when there are functions of different types but the same name, +e.g. when there’re three functions all named “myfunc” but are of temporary catalog, catalog, and system function respectively. +If there’s no function name collision, functions will just be resolved to the sole one. + +## Precise Function Reference + +Because system functions don’t have namespaces, a precise function reference in Flink must be pointing to either a temporary catalog +function or a catalog function. + +The resolution order is: + +1. Temporary catalog function +2. Catalog function + +## Ambiguous Function Reference + +The resolution order is: + +1. Temporary system function +2. System function +3. Temporary catalog function, in the current catalog and current database of the session +4. Catalog function, in the current catalog and current database of the session diff --git a/docs/dev/table/functions/index.zh.md b/docs/dev/table/functions/index.zh.md new file mode 100644 index 000000000000..3ff1deac24e3 --- /dev/null +++ b/docs/dev/table/functions/index.zh.md @@ -0,0 +1,97 @@ +--- +title: "Functions" +nav-id: table_functions +nav-parent_id: table_functions +nav-pos: 40 +nav-show_overview: true +--- + + +Flink Table API & SQL empowers users to do data transformations with functions. + +* This will be replaced by the TOC +{:toc} + +Types of Functions +------------------ + +There are two dimensions to classify functions in Flink. + +One dimension is system (or built-in) functions v.s. catalog functions. System functions have no namespace and can be +referenced with just their names. Catalog functions belong to a catalog and database therefore they have catalog and database +namespaces, they can be referenced by either fully/partially qualified name (`catalog.db.func` or `db.func`) or just the +function name. + +The other dimension is temporary functions v.s. persistent functions. Temporary functions are volatile and only live up to + lifespan of a session, they are always created by users. Persistent functions live across lifespan of sessions, they are either + provided by the system or persisted in catalogs. + +The two dimensions give Flink users 4 categories of functions: + +1. Temporary system functions +2. System functions +3. Temporary catalog functions +4. Catalog functions + +Note that system functions always precede catalog's, and temporary functions always precede persistent on their own dimension +in function resolution order explained below. + +Referencing Functions +--------------------- + +There are two ways users can reference a function in Flink - referencing function precisely or ambiguously. + +## Precise Function Reference + +Precise function reference empowers users to use catalog functions specifically, and across catalog and across database, +e.g. `select mycatalog.mydb.myfunc(x) from mytable` and `select mydb.myfunc(x) from mytable`. + +This is only supported starting from Flink 1.10. + +## Ambiguous Function Reference + +In ambiguous function reference, users just specify the function's name in SQL query, e.g. `select myfunc(x) from mytable`. + + +Function Resolution Order +------------------------- + +The resolution order only matters when there are functions of different types but the same name, +e.g. when there’re three functions all named “myfunc” but are of temporary catalog, catalog, and system function respectively. +If there’s no function name collision, functions will just be resolved to the sole one. + +## Precise Function Reference + +Because system functions don’t have namespaces, a precise function reference in Flink must be pointing to either a temporary catalog +function or a catalog function. + +The resolution order is: + +1. Temporary catalog function +2. Catalog function + +## Ambiguous Function Reference + +The resolution order is: + +1. Temporary system function +2. System function +3. Temporary catalog function, in the current catalog and current database of the session +4. Catalog function, in the current catalog and current database of the session diff --git a/docs/dev/table/functions/builtinFunctions.md b/docs/dev/table/functions/systemFunctions.md similarity index 99% rename from docs/dev/table/functions/builtinFunctions.md rename to docs/dev/table/functions/systemFunctions.md index c2fcaa570602..58ff844d90c4 100644 --- a/docs/dev/table/functions/builtinFunctions.md +++ b/docs/dev/table/functions/systemFunctions.md @@ -1,6 +1,6 @@ --- -title: "Built-In Functions" -nav-parent_id: tableapi +title: "System (Built-in) Functions" +nav-parent_id: table_functions nav-pos: 31 --- + -If the samples show that a task Thread is stuck in a certain internal method call (requesting buffers from the network stack), this indicates that there is back pressure for the task. +Internally, back pressure is judged based on the availability of output buffers. If there is no available buffer (at least one) for output, then it indicates that there is back pressure for the task. -By default, the job manager triggers 100 stack traces every 50ms for each task in order to determine back pressure. The ratio you see in the web interface tells you how many of these stack traces were stuck in the internal method call, e.g. `0.01` indicates that only 1 in 100 was stuck in that method. +By default, the job manager triggers 100 samples every 50ms for each task in order to determine back pressure. The ratio you see in the web interface tells you how many of these samples were indicating back pressure, e.g. `0.01` indicates that only 1 in 100 was back pressured. - **OK**: 0 <= Ratio <= 0.10 - **LOW**: 0.10 < Ratio <= 0.5 - **HIGH**: 0.5 < Ratio <= 1 -In order to not overload the task managers with stack trace samples, the web interface refreshes samples only after 60 seconds. +In order to not overload the task managers with back pressure samples, the web interface refreshes samples only after 60 seconds. ## Configuration You can configure the number of samples for the job manager with the following configuration keys: - `web.backpressure.refresh-interval`: Time after which available stats are deprecated and need to be refreshed (DEFAULT: 60000, 1 min). -- `web.backpressure.num-samples`: Number of stack trace samples to take to determine back pressure (DEFAULT: 100). -- `web.backpressure.delay-between-samples`: Delay between stack trace samples to determine back pressure (DEFAULT: 50, 50 ms). +- `web.backpressure.num-samples`: Number of samples to take to determine back pressure (DEFAULT: 100). +- `web.backpressure.delay-between-samples`: Delay between samples to determine back pressure (DEFAULT: 50, 50 ms). ## Example @@ -66,7 +66,7 @@ You can find the *Back Pressure* tab next to the job overview. ### Sampling In Progress -This means that the JobManager triggered a stack trace sample of the running tasks. With the default configuration, this takes about 5 seconds to complete. +This means that the JobManager triggered a back pressure sample of the running tasks. With the default configuration, this takes about 5 seconds to complete. Note that clicking the row, you trigger the sample for all subtasks of this operator. diff --git a/docs/monitoring/back_pressure.zh.md b/docs/monitoring/back_pressure.zh.md index 127f7ffd1af5..4dd171d3644e 100644 --- a/docs/monitoring/back_pressure.zh.md +++ b/docs/monitoring/back_pressure.zh.md @@ -36,31 +36,31 @@ Flink Web 界面提供了一个选项卡来监控正在运行 Job 的反压行 `Sink` 正在向上游的 `Source` 算子产生反压。 -## 采样线程 +## 反压采样 -通过不断对每个 Task 的 stack trace 采样来进行反压监控。JobManager 会触发对 Task `Thread.getStackTrace()` 的重复调用。 +通过不断对每个 Task 的反压状态采样来进行反压监控。JobManager 会触发对 Task `Task.isBackPressured()` 的重复调用。 - + -如果样本显示 Task 线程卡在某个内部方法调用中(例如:从网络堆栈请求缓冲区),则表示这个 Task 存在反压。 +Task 是否反压是基于输出 Buffer 的可用性判断的,如果一个用于数据输出的 Buffer 都没有了,则表明 Task 被反压了。 -默认情况下,JobManager 会触发 100 次 stack trace 采样,每次间隔 50ms 来确定反压。 -你在 Web 界面看到的比率表示在内部方法调用中有多少 stack trace 被卡住,例如: `0.01` 表示在该方法中 100 个只有 1 个被卡住了。 +默认情况下,JobManager 会触发 100 次采样,每次间隔 50ms 来确定反压。 +你在 Web 界面看到的比率表示在获得的样本中有多少表明 Task 正在被反压,例如: `0.01` 表示 100 个样本中只有 1 个反压了。 - **OK**: 0 <= 比例 <= 0.10 - **LOW**: 0.10 < 比例 <= 0.5 - **HIGH**: 0.5 < 比例 <= 1 -为了不因为 stack trace 导致 TaskManager 产生负载,Web 界面仅在 60 秒后重新采样。 +为了不因为采样导致 TaskManager 负载过重,Web 界面仅在每 60 秒后重新采样。 ## 配置参数 你可以使用以下键来配置 JobManager 的样本数: -- `web.backpressure.refresh-interval`: 有效统计被废弃并重新进行采样的时间 (默认: 60000, 1 min)。 -- `web.backpressure.num-samples`: 用于确定反压的 stack trace 样本数 (默认: 100)。 -- `web.backpressure.delay-between-samples`: 用于确定反压的 stack trace 采样间隔时间 (默认: 50, 50 ms)。 +- `web.backpressure.refresh-interval`: 有效的反压结果被废弃并重新进行采样的时间 (默认: 60000, 1 min)。 +- `web.backpressure.num-samples`: 用于确定反压采样的样本数 (默认: 100)。 +- `web.backpressure.delay-between-samples`: 用于确定反压采样的间隔时间 (默认: 50, 50 ms)。 ## 示例 @@ -69,7 +69,7 @@ Flink Web 界面提供了一个选项卡来监控正在运行 Job 的反压行 ### 采样进行中 -这意味着 JobManager 对正在运行的 Task 触发了 stack trace 采样。默认情况下,大约需要 5 秒完成采样。 +这意味着 JobManager 对正在运行的 Task 触发了反压采样。默认情况下,大约需要 5 秒完成采样。 注意,点击该行,可触发该算子所有 SubTask 的采样。 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 b156ec844e55..4bf79349e31f 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 @@ -813,7 +813,7 @@ public final class ConfigConstants { public static final String JOB_MANAGER_WEB_BACK_PRESSURE_REFRESH_INTERVAL = "jobmanager.web.backpressure.refresh-interval"; /** - * Number of stack trace samples to take to determine back pressure. + * Number of samples to take to determine back pressure. * * @deprecated Use {@link WebOptions#BACKPRESSURE_NUM_SAMPLES} instead. */ @@ -821,7 +821,7 @@ public final class ConfigConstants { public static final String JOB_MANAGER_WEB_BACK_PRESSURE_NUM_SAMPLES = "jobmanager.web.backpressure.num-samples"; /** - * Delay between stack trace samples to determine back pressure. + * Delay between samples to determine back pressure. * * @deprecated Use {@link WebOptions#BACKPRESSURE_DELAY} instead. */ diff --git a/flink-core/src/main/java/org/apache/flink/configuration/WebOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/WebOptions.java index 0b630e92ef7e..d7a41b11f8da 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/WebOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/WebOptions.java @@ -154,22 +154,22 @@ public class WebOptions { " (by resampling)."); /** - * Number of stack trace samples to take to determine back pressure. + * Number of samples to take to determine back pressure. */ public static final ConfigOption BACKPRESSURE_NUM_SAMPLES = key("web.backpressure.num-samples") .defaultValue(100) .withDeprecatedKeys("jobmanager.web.backpressure.num-samples") - .withDescription("Number of stack trace samples to take to determine back pressure."); + .withDescription("Number of samples to take to determine back pressure."); /** - * Delay between stack trace samples to determine back pressure in milliseconds. + * Delay between samples to determine back pressure in milliseconds. */ public static final ConfigOption BACKPRESSURE_DELAY = key("web.backpressure.delay-between-samples") .defaultValue(50) .withDeprecatedKeys("jobmanager.web.backpressure.delay-between-samples") - .withDescription("Delay between stack trace samples to determine back pressure in milliseconds."); + .withDescription("Delay between samples to determine back pressure in milliseconds."); /** * Timeout for asynchronous operations by the web monitor in milliseconds. 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 7338a4488606..339241753f60 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 @@ -52,7 +52,7 @@ import org.apache.flink.runtime.jobmaster.LogicalSlot; import org.apache.flink.runtime.jobmaster.SlotRequestId; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.messages.StackTraceSampleResponse; +import org.apache.flink.runtime.messages.TaskBackPressureResponse; import org.apache.flink.runtime.shuffle.PartitionDescriptor; import org.apache.flink.runtime.shuffle.ProducerDescriptor; import org.apache.flink.runtime.shuffle.ShuffleDescriptor; @@ -921,34 +921,20 @@ public void fail(Throwable t) { } /** - * Request a stack trace sample from the task of this execution. + * Request the back pressure ratio from the task of this execution. * - * @param sampleId of the stack trace sample - * @param numSamples the sample should contain - * @param delayBetweenSamples to wait - * @param maxStackTraceDepth of the samples - * @param timeout until the request times out - * @return Future stack trace sample response + * @param requestId id of the request. + * @param timeout the request times out. + * @return A future of the task back pressure result. */ - public CompletableFuture requestStackTraceSample( - int sampleId, - int numSamples, - Time delayBetweenSamples, - int maxStackTraceDepth, - Time timeout) { + public CompletableFuture requestBackPressure(int requestId, Time timeout) { final LogicalSlot slot = assignedResource; if (slot != null) { final TaskManagerGateway taskManagerGateway = slot.getTaskManagerGateway(); - return taskManagerGateway.requestStackTraceSample( - attemptId, - sampleId, - numSamples, - delayBetweenSamples, - maxStackTraceDepth, - timeout); + return taskManagerGateway.requestTaskBackPressure(attemptId, requestId, timeout); } else { return FutureUtils.completedExceptionally(new Exception("The execution has no slot assigned.")); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java index d60a31903b8b..b03fe7984ed0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java @@ -24,9 +24,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.execution.Environment; -import org.apache.flink.runtime.taskmanager.Task; -import java.util.Optional; import java.util.concurrent.Future; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -127,16 +125,6 @@ public boolean shouldInterruptOnCancel() { return shouldInterruptOnCancel; } - /** - * If the invokable implementation executes user code in a thread other than, - * {@link Task#getExecutingThread()}, this method returns that executing thread. - * - * @see Task#getStackTraceOfExecutingThread() - */ - public Optional getExecutingThread() { - return Optional.empty(); - } - // ------------------------------------------------------------------------ // Access to Environment and Configuration // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/TaskManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/TaskManagerGateway.java index 6e51b1e1aa22..38e6a582fc03 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/TaskManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/slots/TaskManagerGateway.java @@ -27,7 +27,7 @@ import org.apache.flink.runtime.executiongraph.PartitionInfo; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.messages.StackTraceSampleResponse; +import org.apache.flink.runtime.messages.TaskBackPressureResponse; import org.apache.flink.runtime.rpc.RpcTimeout; import java.util.Set; @@ -46,23 +46,17 @@ public interface TaskManagerGateway { String getAddress(); /** - * Request a stack trace sample from the given task. + * Request the back pressure ratio for the given task. * - * @param executionAttemptID identifying the task to sample - * @param sampleId of the sample - * @param numSamples to take from the given task - * @param delayBetweenSamples to wait for - * @param maxStackTraceDepth of the returned sample - * @param timeout of the request - * @return Future of stack trace sample response + * @param executionAttemptID identifying the task to request. + * @param requestId id of the request. + * @param timeout rpc request timeout. + * @return A future of the task back pressure result. */ - CompletableFuture requestStackTraceSample( - final ExecutionAttemptID executionAttemptID, - final int sampleId, - final int numSamples, - final Time delayBetweenSamples, - final int maxStackTraceDepth, - final Time timeout); + CompletableFuture requestTaskBackPressure( + ExecutionAttemptID executionAttemptID, + int requestId, + Time timeout); /** * Submit a task to the task manager. 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 72e870cfea38..19557d28977e 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 @@ -18,7 +18,6 @@ package org.apache.flink.runtime.jobmaster; -import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.IllegalConfigurationException; @@ -32,7 +31,7 @@ import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory; import org.apache.flink.runtime.rest.handler.legacy.backpressure.BackPressureStatsTracker; import org.apache.flink.runtime.rest.handler.legacy.backpressure.BackPressureStatsTrackerImpl; -import org.apache.flink.runtime.rest.handler.legacy.backpressure.StackTraceSampleCoordinator; +import org.apache.flink.runtime.rest.handler.legacy.backpressure.BackPressureRequestCoordinator; import org.apache.flink.runtime.util.ExecutorThreadFactory; import org.apache.flink.runtime.util.Hardware; import org.apache.flink.util.ExceptionUtils; @@ -58,7 +57,7 @@ public class JobManagerSharedServices { private final RestartStrategyFactory restartStrategyFactory; - private final StackTraceSampleCoordinator stackTraceSampleCoordinator; + private final BackPressureRequestCoordinator backPressureSampleCoordinator; private final BackPressureStatsTracker backPressureStatsTracker; @@ -69,14 +68,14 @@ public JobManagerSharedServices( ScheduledExecutorService scheduledExecutorService, LibraryCacheManager libraryCacheManager, RestartStrategyFactory restartStrategyFactory, - StackTraceSampleCoordinator stackTraceSampleCoordinator, + BackPressureRequestCoordinator backPressureSampleCoordinator, BackPressureStatsTracker backPressureStatsTracker, @Nonnull BlobWriter blobWriter) { this.scheduledExecutorService = checkNotNull(scheduledExecutorService); this.libraryCacheManager = checkNotNull(libraryCacheManager); this.restartStrategyFactory = checkNotNull(restartStrategyFactory); - this.stackTraceSampleCoordinator = checkNotNull(stackTraceSampleCoordinator); + this.backPressureSampleCoordinator = checkNotNull(backPressureSampleCoordinator); this.backPressureStatsTracker = checkNotNull(backPressureStatsTracker); this.blobWriter = blobWriter; } @@ -121,7 +120,7 @@ public void shutdown() throws Exception { } libraryCacheManager.shutdown(); - stackTraceSampleCoordinator.shutDown(); + backPressureSampleCoordinator.shutDown(); backPressureStatsTracker.shutDown(); if (firstException != null) { @@ -151,9 +150,9 @@ public static JobManagerSharedServices fromConfiguration( FlinkUserCodeClassLoaders.ResolveOrder.fromString(classLoaderResolveOrder), alwaysParentFirstLoaderPatterns); - final Duration timeout; + final Duration akkaTimeout; try { - timeout = AkkaUtils.getTimeout(config); + akkaTimeout = AkkaUtils.getTimeout(config); } catch (NumberFormatException e) { throw new IllegalConfigurationException(AkkaUtils.formatDurationParsingErrorMessage()); } @@ -162,15 +161,17 @@ public static JobManagerSharedServices fromConfiguration( Hardware.getNumberCPUCores(), new ExecutorThreadFactory("jobmanager-future")); - final StackTraceSampleCoordinator stackTraceSampleCoordinator = - new StackTraceSampleCoordinator(futureExecutor, timeout.toMillis()); + final int numSamples = config.getInteger(WebOptions.BACKPRESSURE_NUM_SAMPLES); + final long delayBetweenSamples = config.getInteger(WebOptions.BACKPRESSURE_DELAY); + final BackPressureRequestCoordinator coordinator = new BackPressureRequestCoordinator( + futureExecutor, + akkaTimeout.toMillis() + numSamples * delayBetweenSamples); + final int cleanUpInterval = config.getInteger(WebOptions.BACKPRESSURE_CLEANUP_INTERVAL); final BackPressureStatsTrackerImpl backPressureStatsTracker = new BackPressureStatsTrackerImpl( - stackTraceSampleCoordinator, + coordinator, cleanUpInterval, - config.getInteger(WebOptions.BACKPRESSURE_NUM_SAMPLES), - config.getInteger(WebOptions.BACKPRESSURE_REFRESH_INTERVAL), - Time.milliseconds(config.getInteger(WebOptions.BACKPRESSURE_DELAY))); + config.getInteger(WebOptions.BACKPRESSURE_REFRESH_INTERVAL)); futureExecutor.scheduleWithFixedDelay( backPressureStatsTracker::cleanUpOperatorStatsCache, @@ -182,7 +183,7 @@ public static JobManagerSharedServices fromConfiguration( futureExecutor, libraryCacheManager, RestartStrategyFactory.createRestartStrategyFactory(config), - stackTraceSampleCoordinator, + coordinator, backPressureStatsTracker, blobServer); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/RpcTaskManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/RpcTaskManagerGateway.java index 96265431e5d1..bb0af2befd0c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/RpcTaskManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/RpcTaskManagerGateway.java @@ -28,7 +28,7 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.messages.StackTraceSampleResponse; +import org.apache.flink.runtime.messages.TaskBackPressureResponse; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.apache.flink.util.Preconditions; @@ -56,21 +56,12 @@ public String getAddress() { } @Override - public CompletableFuture requestStackTraceSample( + public CompletableFuture requestTaskBackPressure( ExecutionAttemptID executionAttemptID, - int sampleId, - int numSamples, - Time delayBetweenSamples, - int maxStackTraceDepth, + int requestId, Time timeout) { - return taskExecutorGateway.requestStackTraceSample( - executionAttemptID, - sampleId, - numSamples, - delayBetweenSamples, - maxStackTraceDepth, - timeout); + return taskExecutorGateway.requestTaskBackPressure(executionAttemptID, requestId, timeout); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/messages/StackTraceSampleResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/TaskBackPressureResponse.java similarity index 72% rename from flink-runtime/src/main/java/org/apache/flink/runtime/messages/StackTraceSampleResponse.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/messages/TaskBackPressureResponse.java index 52dbe777f41d..8ebc9d7fddd9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/messages/StackTraceSampleResponse.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/TaskBackPressureResponse.java @@ -22,39 +22,38 @@ import org.apache.flink.util.Preconditions; import java.io.Serializable; -import java.util.List; /** - * Response to the TriggerStackTraceSample message. + * Response to the task back pressure request rpc call. */ -public class StackTraceSampleResponse implements Serializable { +public class TaskBackPressureResponse implements Serializable { private static final long serialVersionUID = -4786454630050578031L; - private final int sampleId; + private final int requestId; private final ExecutionAttemptID executionAttemptID; - private final List samples; + private final double backPressureRatio; - public StackTraceSampleResponse( - int sampleId, + public TaskBackPressureResponse( + int requestId, ExecutionAttemptID executionAttemptID, - List samples) { - this.sampleId = sampleId; + double backPressureRatio) { + this.requestId = requestId; this.executionAttemptID = Preconditions.checkNotNull(executionAttemptID); - this.samples = Preconditions.checkNotNull(samples); + this.backPressureRatio = backPressureRatio; } - public int getSampleId() { - return sampleId; + public int getRequestId() { + return requestId; } public ExecutionAttemptID getExecutionAttemptID() { return executionAttemptID; } - public List getSamples() { - return samples; + public double getBackPressureRatio() { + return backPressureRatio; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinator.java new file mode 100644 index 000000000000..9c3de60345af --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinator.java @@ -0,0 +1,369 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.rest.handler.legacy.backpressure; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.Execution; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.messages.TaskBackPressureResponse; + +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import javax.annotation.concurrent.GuardedBy; + +import java.util.ArrayDeque; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A coordinator for triggering and collecting back pressure stats + * of running tasks. + */ +public class BackPressureRequestCoordinator { + + private static final Logger LOG = LoggerFactory.getLogger(BackPressureRequestCoordinator.class); + + private static final int NUM_GHOST_REQUEST_IDS = 10; + + private final Object lock = new Object(); + + /** Executor used to run the futures. */ + private final Executor executor; + + /** Request time out of a triggered back pressure request. */ + private final Time requestTimeout; + + /** In progress back pressure requests. */ + @GuardedBy("lock") + private final Map pendingRequests = new HashMap<>(); + + /** A list of recent request IDs to identify late messages vs. invalid ones. */ + private final ArrayDeque recentPendingRequests = new ArrayDeque<>(NUM_GHOST_REQUEST_IDS); + + /** Request ID counter. */ + @GuardedBy("lock") + private int requestIdCounter; + + /** Flag indicating whether the coordinator is still running. */ + @GuardedBy("lock") + private boolean isShutDown; + + /** + * Creates a new coordinator for the cluster. + * + * @param executor Used to execute the futures. + * @param requestTimeout Request time out of a triggered back pressure request. + */ + public BackPressureRequestCoordinator( + Executor executor, + long requestTimeout) { + + checkArgument(requestTimeout >= 0L, "The request timeout must be non-negative."); + + this.executor = checkNotNull(executor); + this.requestTimeout = Time.milliseconds(requestTimeout); + } + + /** + * Triggers a task back pressure stats request to all tasks. + * + * @param tasks Tasks to request. + * @return A future of the completed task back pressure stats. + */ + CompletableFuture triggerBackPressureRequest(ExecutionVertex[] tasks) { + checkNotNull(tasks, "Tasks to request must not be null."); + checkArgument(tasks.length >= 1, "No tasks to request."); + + // Execution IDs of running tasks + ExecutionAttemptID[] triggerIds = new ExecutionAttemptID[tasks.length]; + Execution[] executions = new Execution[tasks.length]; + + // Check that all tasks are RUNNING before triggering anything. The + // triggering can still fail. + for (int i = 0; i < triggerIds.length; i++) { + Execution execution = tasks[i].getCurrentExecutionAttempt(); + if (execution != null && execution.getState() == ExecutionState.RUNNING) { + executions[i] = execution; + triggerIds[i] = execution.getAttemptId(); + } else { + return FutureUtils.completedExceptionally(new IllegalStateException("Task " + tasks[i] + .getTaskNameWithSubtaskIndex() + " is not running.")); + } + } + + synchronized (lock) { + if (isShutDown) { + return FutureUtils.completedExceptionally(new IllegalStateException("Shut down.")); + } + + int requestId = requestIdCounter++; + + LOG.debug("Triggering task back pressure request {}.", requestId); + + PendingBackPressureRequest pending = new PendingBackPressureRequest(requestId, triggerIds); + + // Add the pending request before scheduling the discard task to + // prevent races with removing it again. + pendingRequests.put(requestId, pending); + + requestBackPressure(executions, requestId); + + return pending.getBackPressureStatsFuture(); + } + } + + /** + * Requests back pressure stats from all the given executions. The response would + * be ignored if it does not return within timeout. + */ + private void requestBackPressure(Execution[] executions, int requestId) { + assert Thread.holdsLock(lock); + + for (Execution execution: executions) { + CompletableFuture taskBackPressureFuture = + execution.requestBackPressure(requestId, requestTimeout); + + taskBackPressureFuture.handleAsync( + (TaskBackPressureResponse taskBackPressureResponse, Throwable throwable) -> { + if (taskBackPressureResponse != null) { + handleSuccessfulTaskBackPressureResponse(taskBackPressureResponse); + } else { + handleFailedTaskBackPressureResponse(requestId, throwable); + } + + return null; + }, + executor); + } + } + + /** + * Handles the failed task back pressure response by canceling the corresponding unfinished + * pending back pressure request. + * + * @param requestId ID of the request. + * @param cause Cause of the failure (can be null). + */ + private void handleFailedTaskBackPressureResponse(int requestId, @Nullable Throwable cause) { + synchronized (lock) { + if (isShutDown) { + return; + } + + PendingBackPressureRequest pendingRequest = pendingRequests.remove(requestId); + if (pendingRequest != null) { + if (cause != null) { + LOG.info(String.format("Cancelling back pressure request %d.", requestId), cause); + } else { + LOG.info("Cancelling back pressure request {}.", requestId); + } + + pendingRequest.discard(cause); + rememberRecentRequestId(requestId); + } + } + } + + /** + * Shuts down the coordinator. + * + *

    After shut down, no further operations are executed. + */ + public void shutDown() { + synchronized (lock) { + if (!isShutDown) { + LOG.info("Shutting down back pressure request coordinator."); + + for (PendingBackPressureRequest pending : pendingRequests.values()) { + pending.discard(new RuntimeException("Shut down.")); + } + + pendingRequests.clear(); + recentPendingRequests.clear(); + + isShutDown = true; + } + } + } + + /** + * Handles the successfully returned task back pressure response by collecting + * the corresponding back pressure ratio. + * + * @param taskBackPressureResponse The returned task back pressure response. + * @throws IllegalStateException If the request ID is unknown and not recently + * finished or the request has been cancelled. + */ + private void handleSuccessfulTaskBackPressureResponse(TaskBackPressureResponse taskBackPressureResponse) { + int requestId = taskBackPressureResponse.getRequestId(); + ExecutionAttemptID executionId = taskBackPressureResponse.getExecutionAttemptID(); + double taskBackPressureRatio = taskBackPressureResponse.getBackPressureRatio(); + + synchronized (lock) { + if (isShutDown) { + return; + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Collecting back pressure response of request {} from task {}.", requestId, executionId); + } + + PendingBackPressureRequest pending = pendingRequests.get(requestId); + + if (pending != null) { + pending.collectBackPressureStats(executionId, taskBackPressureRatio); + + // Publish the request result + if (pending.isComplete()) { + pendingRequests.remove(requestId); + rememberRecentRequestId(requestId); + + pending.completePromiseAndDiscard(); + } + } else if (recentPendingRequests.contains(requestId)) { + if (LOG.isDebugEnabled()) { + LOG.debug("Received late back pressure request {} result of task {}.", + requestId, executionId); + } + } else { + if (LOG.isDebugEnabled()) { + LOG.debug(String.format("Unknown request ID %d.", requestId)); + } + } + } + } + + private void rememberRecentRequestId(int requestId) { + if (recentPendingRequests.size() >= NUM_GHOST_REQUEST_IDS) { + recentPendingRequests.removeFirst(); + } + recentPendingRequests.addLast(requestId); + } + + @VisibleForTesting + int getNumberOfPendingRequests() { + synchronized (lock) { + return pendingRequests.size(); + } + } + + // ------------------------------------------------------------------------ + + /** + * A pending task back pressure request, which collects task back pressure + * ratio and owns a {@link BackPressureStats} promise. + * + *

    Access pending request in lock scope. + */ + private static class PendingBackPressureRequest { + + private final int requestId; + private final long startTime; + private final Set pendingTasks; + private final Map backPressureRatios; + private final CompletableFuture backPressureStatsFuture; + + private boolean isDiscarded; + + PendingBackPressureRequest( + int requestId, + ExecutionAttemptID[] tasksToCollect) { + + this.requestId = requestId; + this.startTime = System.currentTimeMillis(); + this.pendingTasks = new HashSet<>(Arrays.asList(tasksToCollect)); + this.backPressureRatios = Maps.newHashMapWithExpectedSize(tasksToCollect.length); + this.backPressureStatsFuture = new CompletableFuture<>(); + } + + private boolean isComplete() { + checkDiscarded(); + + return pendingTasks.isEmpty(); + } + + private void discard(Throwable cause) { + if (!isDiscarded) { + pendingTasks.clear(); + backPressureRatios.clear(); + + backPressureStatsFuture.completeExceptionally(new RuntimeException("Discarded.", cause)); + + isDiscarded = true; + } + } + + private void collectBackPressureStats(ExecutionAttemptID executionId, double backPressureRatio) { + checkDiscarded(); + checkCompleted(); + + if (pendingTasks.remove(executionId)) { + backPressureRatios.put(executionId, backPressureRatio); + } else { + throw new IllegalArgumentException(String.format("Unknown task %s.", executionId)); + } + } + + private void completePromiseAndDiscard() { + isDiscarded = true; + + long endTime = System.currentTimeMillis(); + + BackPressureStats backPressureStats = new BackPressureStats( + requestId, + startTime, + endTime, + backPressureRatios); + + backPressureStatsFuture.complete(backPressureStats); + } + + private CompletableFuture getBackPressureStatsFuture() { + return backPressureStatsFuture; + } + + private void checkCompleted() { + if (pendingTasks.isEmpty()) { + throw new IllegalStateException("Completed."); + } + } + + private void checkDiscarded() { + if (isDiscarded) { + throw new IllegalStateException("Discarded."); + } + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStats.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStats.java new file mode 100644 index 000000000000..cfd0477121d0 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStats.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.handler.legacy.backpressure; + +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; + +import javax.annotation.Nonnegative; + +import java.util.Collections; +import java.util.Map; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Back pressure stats for one or more tasks. + * + *

    The stats are collected by request triggered in {@link BackPressureRequestCoordinator}. + */ +public class BackPressureStats { + + /** ID of the request (unique per job). */ + private final int requestId; + + /** Time stamp, when the request was triggered. */ + private final long startTime; + + /** Time stamp, when all back pressure stats were collected at the BackPressureRequestCoordinator. */ + private final long endTime; + + /** Map of back pressure ratios by execution ID. */ + private final Map backPressureRatios; + + public BackPressureStats( + @Nonnegative int requestId, + @Nonnegative long startTime, + @Nonnegative long endTime, + Map backPressureRatios) { + checkArgument(endTime >= startTime, "End time must not before start time."); + + this.requestId = requestId; + this.startTime = startTime; + this.endTime = endTime; + this.backPressureRatios = Collections.unmodifiableMap(checkNotNull(backPressureRatios)); + } + + public int getRequestId() { + return requestId; + } + + public long getStartTime() { + return startTime; + } + + public long getEndTime() { + return endTime; + } + + public Map getBackPressureRatios() { + return backPressureRatios; + } + + @Override + public String toString() { + return "BackPressureStats{" + + "requestId=" + requestId + + ", startTime=" + startTime + + ", endTime=" + endTime + + '}'; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTracker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTracker.java index 7c447c05d006..c9d03cd0c37e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTracker.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTracker.java @@ -29,7 +29,7 @@ public interface BackPressureStatsTracker { /** - * Returns back pressure statistics for a operator. Automatically triggers stack trace sampling + * Returns back pressure statistics for an operator. Automatically triggers back pressure request * if statistics are not available or outdated. * * @param vertex Operator to get the stats for. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImpl.java index bbcf38099a7b..3c4068e4f8cb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImpl.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.rest.handler.legacy.backpressure; -import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.executiongraph.ExecutionVertex; @@ -31,9 +30,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.concurrent.GuardedBy; + import java.util.Arrays; import java.util.HashSet; -import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Optional; @@ -47,108 +47,69 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** - * Back pressure statistics tracker. - * - *

    Back pressure is determined by sampling running tasks. If a task is - * slowed down by back pressure it will be stuck in memory requests to a - * {@link org.apache.flink.runtime.io.network.buffer.LocalBufferPool}. - * - *

    The back pressured stack traces look like this: - * - *

    - * java.lang.Object.wait(Native Method)
    - * o.a.f.[...].LocalBufferPool.requestBuffer(LocalBufferPool.java:163)
    - * o.a.f.[...].LocalBufferPool.requestBufferBlocking(LocalBufferPool.java:133) <--- BLOCKING
    - * request
    - * [...]
    - * 
    + * Back pressure statistics tracker. See {@link org.apache.flink.runtime.taskexecutor.BackPressureSampleService} + * for more details about how back pressure ratio of a task is calculated. */ public class BackPressureStatsTrackerImpl implements BackPressureStatsTracker { private static final Logger LOG = LoggerFactory.getLogger(BackPressureStatsTrackerImpl.class); - /** Maximum stack trace depth for samples. */ - static final int MAX_STACK_TRACE_DEPTH = 8; - - /** Expected class name for back pressure indicating stack trace element. */ - static final String EXPECTED_CLASS_NAME = "org.apache.flink.runtime.io.network.buffer.LocalBufferPool"; - - /** Expected method name for back pressure indicating stack trace element. */ - static final String EXPECTED_METHOD_NAME = "requestBufferBuilderBlocking"; - /** Lock guarding trigger operations. */ private final Object lock = new Object(); - /* Stack trace sample coordinator. */ - private final StackTraceSampleCoordinator coordinator; + /** Coordinator for back pressure stats request. */ + private final BackPressureRequestCoordinator coordinator; /** * Completed stats. Important: Job vertex IDs need to be scoped by job ID, - * because they are potentially constant across runs messing up the cached - * data. + * because they are potentially constant across runs which may mess up the + * cached data. */ private final Cache operatorStatsCache; - /** Pending in progress stats. Important: Job vertex IDs need to be scoped - * by job ID, because they are potentially constant across runs messing up - * the cached data.*/ + /** + * Pending in progress stats. Important: Job vertex IDs need to be scoped + * by job ID, because they are potentially constant across runs which may + * mess up the cached data. + */ private final Set pendingStats = new HashSet<>(); - /** Cleanup interval for completed stats cache. */ - private final int cleanUpInterval; - - private final int numSamples; - + /** + * Time interval, in milliseconds, after which the available back pressure + * stats are deprecated and need to be refreshed. + */ private final int backPressureStatsRefreshInterval; - private final Time delayBetweenSamples; - /** Flag indicating whether the stats tracker has been shut down. */ + @GuardedBy("lock") private boolean shutDown; /** * Creates a back pressure statistics tracker. * - * @param cleanUpInterval Clean up interval for completed stats. - * @param numSamples Number of stack trace samples when determining back pressure. - * @param delayBetweenSamples Delay between samples when determining back pressure. + * @param coordinator Coordinator for back pressure stats request. + * @param cleanUpInterval Clean up interval for completed stats. + * @param refreshInterval Time interval after which the available back pressure + * stats are deprecated and need to be refreshed. */ public BackPressureStatsTrackerImpl( - StackTraceSampleCoordinator coordinator, + BackPressureRequestCoordinator coordinator, int cleanUpInterval, - int numSamples, - int backPressureStatsRefreshInterval, - Time delayBetweenSamples) { - - this.coordinator = checkNotNull(coordinator, "Stack trace sample coordinator"); - - checkArgument(cleanUpInterval >= 0, "Clean up interval"); - this.cleanUpInterval = cleanUpInterval; - - checkArgument(numSamples >= 1, "Number of samples"); - this.numSamples = numSamples; - - checkArgument( - backPressureStatsRefreshInterval >= 0, - "backPressureStatsRefreshInterval must be greater than or equal to 0"); - this.backPressureStatsRefreshInterval = backPressureStatsRefreshInterval; - - this.delayBetweenSamples = checkNotNull(delayBetweenSamples, "Delay between samples"); + int refreshInterval) { + checkArgument(cleanUpInterval >= 0, "The cleanup interval must be non-negative."); + checkArgument(refreshInterval >= 0, "The back pressure stats refresh interval must be non-negative."); + this.coordinator = checkNotNull(coordinator); + this.backPressureStatsRefreshInterval = refreshInterval; this.operatorStatsCache = CacheBuilder.newBuilder() .concurrencyLevel(1) .expireAfterAccess(cleanUpInterval, TimeUnit.MILLISECONDS) .build(); } - /** Cleanup interval for completed stats cache. */ - public long getCleanUpInterval() { - return cleanUpInterval; - } - /** - * Returns back pressure statistics for a operator. Automatically triggers stack trace sampling - * if statistics are not available or outdated. + * Returns back pressure statistics for a operator. Automatically triggers task back pressure + * sampling if statistics are not available or outdated. * * @param vertex Operator to get the stats for. * @return Back pressure statistics for an operator @@ -157,70 +118,44 @@ public Optional getOperatorBackPressureStats(Executio synchronized (lock) { final OperatorBackPressureStats stats = operatorStatsCache.getIfPresent(vertex); if (stats == null || backPressureStatsRefreshInterval <= System.currentTimeMillis() - stats.getEndTimestamp()) { - triggerStackTraceSampleInternal(vertex); + triggerBackPressureRequestInternal(vertex); } return Optional.ofNullable(stats); } } /** - * Triggers a stack trace sample for a operator to gather the back pressure - * statistics. If there is a sample in progress for the operator, the call + * Triggers a back pressure request for a vertex to gather the back pressure + * statistics. If there is a request in progress for the vertex, the call * is ignored. * - * @param vertex Operator to get the stats for. - * @return Flag indicating whether a sample with triggered. + * @param vertex Vertex to get the stats for. */ - private boolean triggerStackTraceSampleInternal(final ExecutionJobVertex vertex) { + private void triggerBackPressureRequestInternal(final ExecutionJobVertex vertex) { assert(Thread.holdsLock(lock)); if (shutDown) { - return false; + return; } - if (!pendingStats.contains(vertex) && - !vertex.getGraph().getState().isGloballyTerminalState()) { + if (!pendingStats.contains(vertex) && !vertex.getGraph().getState().isGloballyTerminalState()) { Executor executor = vertex.getGraph().getFutureExecutor(); - // Only trigger if still active job + // Only trigger for still active job if (executor != null) { pendingStats.add(vertex); if (LOG.isDebugEnabled()) { - LOG.debug("Triggering stack trace sample for tasks: " + Arrays.toString(vertex.getTaskVertices())); + LOG.debug("Triggering back pressure request for tasks: " + Arrays.toString(vertex.getTaskVertices())); } - CompletableFuture sample = coordinator.triggerStackTraceSample( - vertex.getTaskVertices(), - numSamples, - delayBetweenSamples, - MAX_STACK_TRACE_DEPTH); - - sample.handleAsync(new StackTraceSampleCompletionCallback(vertex), executor); + CompletableFuture statsFuture = + coordinator.triggerBackPressureRequest(vertex.getTaskVertices()); - return true; + statsFuture.handleAsync(new BackPressureRequestCompletionCallback(vertex), executor); } } - - return false; - } - - /** - * Triggers a stack trace sample for a operator to gather the back pressure - * statistics. If there is a sample in progress for the operator, the call - * is ignored. - * - * @param vertex Operator to get the stats for. - * @return Flag indicating whether a sample with triggered. - * @deprecated {@link #getOperatorBackPressureStats(ExecutionJobVertex)} will trigger - * stack trace sampling automatically. - */ - @Deprecated - public boolean triggerStackTraceSample(ExecutionJobVertex vertex) { - synchronized (lock) { - return triggerStackTraceSampleInternal(vertex); - } } /** @@ -250,25 +185,18 @@ public void shutDown() { } /** - * Invalidates the cache (irrespective of clean up interval). + * Callback on completed back pressure request. */ - void invalidateOperatorStatsCache() { - operatorStatsCache.invalidateAll(); - } - - /** - * Callback on completed stack trace sample. - */ - class StackTraceSampleCompletionCallback implements BiFunction { + private class BackPressureRequestCompletionCallback implements BiFunction { private final ExecutionJobVertex vertex; - public StackTraceSampleCompletionCallback(ExecutionJobVertex vertex) { + BackPressureRequestCompletionCallback(ExecutionJobVertex vertex) { this.vertex = vertex; } @Override - public Void apply(StackTraceSample stackTraceSample, Throwable throwable) { + public Void apply(BackPressureStats backPressureStats, Throwable throwable) { synchronized (lock) { try { if (shutDown) { @@ -278,12 +206,12 @@ public Void apply(StackTraceSample stackTraceSample, Throwable throwable) { // Job finished, ignore. JobStatus jobState = vertex.getGraph().getState(); if (jobState.isGloballyTerminalState()) { - LOG.debug("Ignoring sample, because job is in state " + jobState + "."); - } else if (stackTraceSample != null) { - OperatorBackPressureStats stats = createStatsFromSample(stackTraceSample); + LOG.debug("Ignoring stats, because job is in state " + jobState + "."); + } else if (backPressureStats != null) { + OperatorBackPressureStats stats = createOperatorBackPressureStats(backPressureStats); operatorStatsCache.put(vertex, stats); } else { - LOG.debug("Failed to gather stack trace sample.", throwable); + LOG.debug("Failed to gather back pressure stats.", throwable); } } catch (Throwable t) { LOG.error("Error during stats completion.", t); @@ -296,68 +224,41 @@ public Void apply(StackTraceSample stackTraceSample, Throwable throwable) { } /** - * Creates the back pressure stats from a stack trace sample. - * - * @param sample Stack trace sample to base stats on. - * - * @return Back pressure stats + * Creates {@link OperatorBackPressureStats} from {@link BackPressureStats}. */ - private OperatorBackPressureStats createStatsFromSample(StackTraceSample sample) { - Map> traces = sample.getStackTraces(); + private OperatorBackPressureStats createOperatorBackPressureStats(BackPressureStats stats) { + Map backPressureRatiosByTask = stats.getBackPressureRatios(); // Map task ID to subtask index, because the web interface expects // it like that. Map subtaskIndexMap = Maps - .newHashMapWithExpectedSize(traces.size()); + .newHashMapWithExpectedSize(backPressureRatiosByTask.size()); - Set sampledTasks = sample.getStackTraces().keySet(); + Set tasks = backPressureRatiosByTask.keySet(); for (ExecutionVertex task : vertex.getTaskVertices()) { ExecutionAttemptID taskId = task.getCurrentExecutionAttempt().getAttemptId(); - if (sampledTasks.contains(taskId)) { + if (tasks.contains(taskId)) { subtaskIndexMap.put(taskId, task.getParallelSubtaskIndex()); } else { - LOG.debug("Outdated sample. A task, which is part of the " + - "sample has been reset."); + LOG.debug("Outdated stats. A task, which is part of the " + + "request has been reset."); } } - // Ratio of blocked samples to total samples per sub task. Array - // position corresponds to sub task index. - double[] backPressureRatio = new double[traces.size()]; - - for (Entry> entry : traces.entrySet()) { - int backPressureSamples = 0; - - List taskTraces = entry.getValue(); - - for (StackTraceElement[] trace : taskTraces) { - for (int i = trace.length - 1; i >= 0; i--) { - StackTraceElement elem = trace[i]; - - if (elem.getClassName().equals(EXPECTED_CLASS_NAME) && - elem.getMethodName().equals(EXPECTED_METHOD_NAME)) { - - backPressureSamples++; - break; // Continue with next stack trace - } - } - } + // Back pressure ratios of all tasks. Array position corresponds + // to sub task index. + double[] backPressureRatios = new double[backPressureRatiosByTask.size()]; + for (Entry entry : backPressureRatiosByTask.entrySet()) { int subtaskIndex = subtaskIndexMap.get(entry.getKey()); - - int size = taskTraces.size(); - double ratio = (size > 0) - ? ((double) backPressureSamples) / size - : 0; - - backPressureRatio[subtaskIndex] = ratio; + backPressureRatios[subtaskIndex] = entry.getValue(); } return new OperatorBackPressureStats( - sample.getSampleId(), - sample.getEndTime(), - backPressureRatio); + stats.getRequestId(), + stats.getEndTime(), + backPressureRatios); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/OperatorBackPressureStats.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/OperatorBackPressureStats.java index 49e5fcec8b04..c0710d899ae8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/OperatorBackPressureStats.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/OperatorBackPressureStats.java @@ -25,40 +25,39 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** - * Back pressure statistics of multiple tasks. - * - *

    Statistics are gathered by sampling stack traces of running tasks. The - * back pressure ratio denotes the ratio of traces indicating back pressure - * to the total number of sampled traces. + * Back pressure statistics of multiple tasks generated by {@link BackPressureStatsTrackerImpl}. */ public class OperatorBackPressureStats implements Serializable { private static final long serialVersionUID = 1L; - /** ID of the corresponding sample. */ - private final int sampleId; + /** ID of the corresponding request. */ + private final int requestId; - /** End time stamp of the corresponding sample. */ + /** + * End time stamp when all responses of the back pressure request + * were collected at BackPressureRequestCoordinator. + */ private final long endTimestamp; /** Back pressure ratio per subtask. */ - private final double[] subTaskBackPressureRatio; + private final double[] subTaskBackPressureRatios; /** Maximum back pressure ratio. */ private final double maxSubTaskBackPressureRatio; public OperatorBackPressureStats( - int sampleId, + int requestId, long endTimestamp, - double[] subTaskBackPressureRatio) { + double[] subTaskBackPressureRatios) { - this.sampleId = sampleId; + this.requestId = requestId; this.endTimestamp = endTimestamp; - this.subTaskBackPressureRatio = checkNotNull(subTaskBackPressureRatio, "Sub task back pressure ratio"); - checkArgument(subTaskBackPressureRatio.length >= 1, "No Sub task back pressure ratio specified"); + this.subTaskBackPressureRatios = checkNotNull(subTaskBackPressureRatios); + checkArgument(subTaskBackPressureRatios.length >= 1, "No Sub task back pressure ratio specified."); double max = 0; - for (double ratio : subTaskBackPressureRatio) { + for (double ratio : subTaskBackPressureRatios) { if (ratio > max) { max = ratio; } @@ -67,52 +66,27 @@ public OperatorBackPressureStats( maxSubTaskBackPressureRatio = max; } - /** - * Returns the ID of the sample. - * - * @return ID of the sample - */ - public int getSampleId() { - return sampleId; + public int getRequestId() { + return requestId; } - /** - * Returns the time stamp, when all stack traces were collected at the - * JobManager. - * - * @return Time stamp, when all stack traces were collected at the - * JobManager - */ public long getEndTimestamp() { return endTimestamp; } - /** - * Returns the number of sub tasks. - * - * @return Number of sub tasks. - */ public int getNumberOfSubTasks() { - return subTaskBackPressureRatio.length; + return subTaskBackPressureRatios.length; } /** - * Returns the ratio of stack traces indicating back pressure to total - * number of sampled stack traces. - * - * @param index Subtask index. - * - * @return Ratio of stack traces indicating back pressure to total number - * of sampled stack traces. + * Returns the back pressure ratio of the given subtask index. */ public double getBackPressureRatio(int index) { - return subTaskBackPressureRatio[index]; + return subTaskBackPressureRatios[index]; } /** * Returns the maximum back pressure ratio of all sub tasks. - * - * @return Maximum back pressure ratio of all sub tasks. */ public double getMaxBackPressureRatio() { return maxSubTaskBackPressureRatio; @@ -121,9 +95,9 @@ public double getMaxBackPressureRatio() { @Override public String toString() { return "OperatorBackPressureStats{" + - "sampleId=" + sampleId + + "requestId=" + requestId + ", endTimestamp=" + endTimestamp + - ", subTaskBackPressureRatio=" + Arrays.toString(subTaskBackPressureRatio) + + ", subTaskBackPressureRatios=" + Arrays.toString(subTaskBackPressureRatios) + '}'; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/StackTraceSample.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/StackTraceSample.java deleted file mode 100644 index dda4e33f7329..000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/StackTraceSample.java +++ /dev/null @@ -1,119 +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.rest.handler.legacy.backpressure; - -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; - -import java.util.Collections; -import java.util.List; -import java.util.Map; - -import static org.apache.flink.util.Preconditions.checkArgument; - -/** - * A sample of stack traces for one or more tasks. - * - *

    The sampling is triggered in {@link StackTraceSampleCoordinator}. - */ -public class StackTraceSample { - - /** ID of this sample (unique per job). */ - private final int sampleId; - - /** Time stamp, when the sample was triggered. */ - private final long startTime; - - /** Time stamp, when all stack traces were collected at the JobManager. */ - private final long endTime; - - /** Map of stack traces by execution ID. */ - private final Map> stackTracesByTask; - - /** - * Creates a stack trace sample. - * - * @param sampleId ID of the sample. - * @param startTime Time stamp, when the sample was triggered. - * @param endTime Time stamp, when all stack traces were - * collected at the JobManager. - * @param stackTracesByTask Map of stack traces by execution ID. - */ - public StackTraceSample( - int sampleId, - long startTime, - long endTime, - Map> stackTracesByTask) { - - checkArgument(sampleId >= 0, "Negative sample ID"); - checkArgument(startTime >= 0, "Negative start time"); - checkArgument(endTime >= startTime, "End time before start time"); - - this.sampleId = sampleId; - this.startTime = startTime; - this.endTime = endTime; - this.stackTracesByTask = Collections.unmodifiableMap(stackTracesByTask); - } - - /** - * Returns the ID of the sample. - * - * @return ID of the sample - */ - public int getSampleId() { - return sampleId; - } - - /** - * Returns the time stamp, when the sample was triggered. - * - * @return Time stamp, when the sample was triggered - */ - public long getStartTime() { - return startTime; - } - - /** - * Returns the time stamp, when all stack traces were collected at the - * JobManager. - * - * @return Time stamp, when all stack traces were collected at the - * JobManager - */ - public long getEndTime() { - return endTime; - } - - /** - * Returns the a map of stack traces by execution ID. - * - * @return Map of stack traces by execution ID - */ - public Map> getStackTraces() { - return stackTracesByTask; - } - - @Override - public String toString() { - return "StackTraceSample{" + - "sampleId=" + sampleId + - ", startTime=" + startTime + - ", endTime=" + endTime + - '}'; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/StackTraceSampleCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/StackTraceSampleCoordinator.java deleted file mode 100644 index 8c2ec6e7b086..000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/StackTraceSampleCoordinator.java +++ /dev/null @@ -1,392 +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.rest.handler.legacy.backpressure; - -import org.apache.flink.api.common.time.Time; -import org.apache.flink.runtime.concurrent.FutureUtils; -import org.apache.flink.runtime.execution.ExecutionState; -import org.apache.flink.runtime.executiongraph.Execution; -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; -import org.apache.flink.runtime.executiongraph.ExecutionVertex; -import org.apache.flink.runtime.messages.StackTraceSampleResponse; -import org.apache.flink.util.Preconditions; - -import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayDeque; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -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.Executor; - -import static org.apache.flink.util.Preconditions.checkArgument; -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * A coordinator for triggering and collecting stack traces of running tasks. - */ -public class StackTraceSampleCoordinator { - - private static final Logger LOG = LoggerFactory.getLogger(StackTraceSampleCoordinator.class); - - private static final int NUM_GHOST_SAMPLE_IDS = 10; - - private final Object lock = new Object(); - - /** Executor used to run the futures. */ - private final Executor executor; - - /** Time out after the expected sampling duration. */ - private final long sampleTimeout; - - /** In progress samples (guarded by lock). */ - private final Map pendingSamples = new HashMap<>(); - - /** A list of recent sample IDs to identify late messages vs. invalid ones. */ - private final ArrayDeque recentPendingSamples = new ArrayDeque<>(NUM_GHOST_SAMPLE_IDS); - - /** Sample ID counter (guarded by lock). */ - private int sampleIdCounter; - - /** - * Flag indicating whether the coordinator is still running (guarded by - * lock). - */ - private boolean isShutDown; - - /** - * Creates a new coordinator for the job. - * - * @param executor to use to execute the futures - * @param sampleTimeout Time out after the expected sampling duration. - * This is added to the expected duration of a - * sample, which is determined by the number of - * samples and the delay between each sample. - */ - public StackTraceSampleCoordinator(Executor executor, long sampleTimeout) { - checkArgument(sampleTimeout >= 0L); - this.executor = Preconditions.checkNotNull(executor); - this.sampleTimeout = sampleTimeout; - } - - /** - * Triggers a stack trace sample to all tasks. - * - * @param tasksToSample Tasks to sample. - * @param numSamples Number of stack trace samples to collect. - * @param delayBetweenSamples Delay between consecutive samples. - * @param maxStackTraceDepth Maximum depth of the stack trace. 0 indicates - * no maximum and keeps the complete stack trace. - * @return A future of the completed stack trace sample - */ - @SuppressWarnings("unchecked") - public CompletableFuture triggerStackTraceSample( - ExecutionVertex[] tasksToSample, - int numSamples, - Time delayBetweenSamples, - int maxStackTraceDepth) { - - checkNotNull(tasksToSample, "Tasks to sample"); - checkArgument(tasksToSample.length >= 1, "No tasks to sample"); - checkArgument(numSamples >= 1, "No number of samples"); - checkArgument(maxStackTraceDepth >= 0, "Negative maximum stack trace depth"); - - // Execution IDs of running tasks - ExecutionAttemptID[] triggerIds = new ExecutionAttemptID[tasksToSample.length]; - Execution[] executions = new Execution[tasksToSample.length]; - - // Check that all tasks are RUNNING before triggering anything. The - // triggering can still fail. - for (int i = 0; i < triggerIds.length; i++) { - Execution execution = tasksToSample[i].getCurrentExecutionAttempt(); - if (execution != null && execution.getState() == ExecutionState.RUNNING) { - executions[i] = execution; - triggerIds[i] = execution.getAttemptId(); - } else { - return FutureUtils.completedExceptionally(new IllegalStateException("Task " + tasksToSample[i] - .getTaskNameWithSubtaskIndex() + " is not running.")); - } - } - - synchronized (lock) { - if (isShutDown) { - return FutureUtils.completedExceptionally(new IllegalStateException("Shut down")); - } - - final int sampleId = sampleIdCounter++; - - LOG.debug("Triggering stack trace sample {}", sampleId); - - final PendingStackTraceSample pending = new PendingStackTraceSample( - sampleId, triggerIds); - - // Discard the sample if it takes too long. We don't send cancel - // messages to the task managers, but only wait for the responses - // and then ignore them. - long expectedDuration = numSamples * delayBetweenSamples.toMilliseconds(); - Time timeout = Time.milliseconds(expectedDuration + sampleTimeout); - - // Add the pending sample before scheduling the discard task to - // prevent races with removing it again. - pendingSamples.put(sampleId, pending); - - // Trigger all samples - for (Execution execution: executions) { - final CompletableFuture stackTraceSampleFuture = execution.requestStackTraceSample( - sampleId, - numSamples, - delayBetweenSamples, - maxStackTraceDepth, - timeout); - - stackTraceSampleFuture.handleAsync( - (StackTraceSampleResponse stackTraceSampleResponse, Throwable throwable) -> { - if (stackTraceSampleResponse != null) { - collectStackTraces( - stackTraceSampleResponse.getSampleId(), - stackTraceSampleResponse.getExecutionAttemptID(), - stackTraceSampleResponse.getSamples()); - } else { - cancelStackTraceSample(sampleId, throwable); - } - - return null; - }, - executor); - } - - return pending.getStackTraceSampleFuture(); - } - } - - /** - * Cancels a pending sample. - * - * @param sampleId ID of the sample to cancel. - * @param cause Cause of the cancelling (can be null). - */ - public void cancelStackTraceSample(int sampleId, Throwable cause) { - synchronized (lock) { - if (isShutDown) { - return; - } - - PendingStackTraceSample sample = pendingSamples.remove(sampleId); - if (sample != null) { - if (cause != null) { - LOG.info("Cancelling sample " + sampleId, cause); - } else { - LOG.info("Cancelling sample {}", sampleId); - } - - sample.discard(cause); - rememberRecentSampleId(sampleId); - } - } - } - - /** - * Shuts down the coordinator. - * - *

    After shut down, no further operations are executed. - */ - public void shutDown() { - synchronized (lock) { - if (!isShutDown) { - LOG.info("Shutting down stack trace sample coordinator."); - - for (PendingStackTraceSample pending : pendingSamples.values()) { - pending.discard(new RuntimeException("Shut down")); - } - - pendingSamples.clear(); - - isShutDown = true; - } - } - } - - /** - * Collects stack traces of a task. - * - * @param sampleId ID of the sample. - * @param executionId ID of the sampled task. - * @param stackTraces Stack traces of the sampled task. - * - * @throws IllegalStateException If unknown sample ID and not recently - * finished or cancelled sample. - */ - public void collectStackTraces( - int sampleId, - ExecutionAttemptID executionId, - List stackTraces) { - - synchronized (lock) { - if (isShutDown) { - return; - } - - if (LOG.isDebugEnabled()) { - LOG.debug("Collecting stack trace sample {} of task {}", sampleId, executionId); - } - - PendingStackTraceSample pending = pendingSamples.get(sampleId); - - if (pending != null) { - pending.collectStackTraces(executionId, stackTraces); - - // Publish the sample - if (pending.isComplete()) { - pendingSamples.remove(sampleId); - rememberRecentSampleId(sampleId); - - pending.completePromiseAndDiscard(); - } - } else if (recentPendingSamples.contains(sampleId)) { - if (LOG.isDebugEnabled()) { - LOG.debug("Received late stack trace sample {} of task {}", - sampleId, executionId); - } - } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Unknown sample ID " + sampleId); - } - } - } - } - - private void rememberRecentSampleId(int sampleId) { - if (recentPendingSamples.size() >= NUM_GHOST_SAMPLE_IDS) { - recentPendingSamples.removeFirst(); - } - recentPendingSamples.addLast(sampleId); - } - - int getNumberOfPendingSamples() { - synchronized (lock) { - return pendingSamples.size(); - } - } - - // ------------------------------------------------------------------------ - - /** - * A pending stack trace sample, which collects stack traces and owns a - * {@link StackTraceSample} promise. - * - *

    Access pending sample in lock scope. - */ - private static class PendingStackTraceSample { - - private final int sampleId; - private final long startTime; - private final Set pendingTasks; - private final Map> stackTracesByTask; - private final CompletableFuture stackTraceFuture; - - private boolean isDiscarded; - - PendingStackTraceSample( - int sampleId, - ExecutionAttemptID[] tasksToCollect) { - - this.sampleId = sampleId; - this.startTime = System.currentTimeMillis(); - this.pendingTasks = new HashSet<>(Arrays.asList(tasksToCollect)); - this.stackTracesByTask = Maps.newHashMapWithExpectedSize(tasksToCollect.length); - this.stackTraceFuture = new CompletableFuture<>(); - } - - int getSampleId() { - return sampleId; - } - - long getStartTime() { - return startTime; - } - - boolean isDiscarded() { - return isDiscarded; - } - - boolean isComplete() { - if (isDiscarded) { - throw new IllegalStateException("Discarded"); - } - - return pendingTasks.isEmpty(); - } - - void discard(Throwable cause) { - if (!isDiscarded) { - pendingTasks.clear(); - stackTracesByTask.clear(); - - stackTraceFuture.completeExceptionally(new RuntimeException("Discarded", cause)); - - isDiscarded = true; - } - } - - void collectStackTraces(ExecutionAttemptID executionId, List stackTraces) { - if (isDiscarded) { - throw new IllegalStateException("Discarded"); - } - - if (pendingTasks.remove(executionId)) { - stackTracesByTask.put(executionId, Collections.unmodifiableList(stackTraces)); - } else if (isComplete()) { - throw new IllegalStateException("Completed"); - } else { - throw new IllegalArgumentException("Unknown task " + executionId); - } - } - - void completePromiseAndDiscard() { - if (isComplete()) { - isDiscarded = true; - - long endTime = System.currentTimeMillis(); - - StackTraceSample stackTraceSample = new StackTraceSample( - sampleId, - startTime, - endTime, - stackTracesByTask); - - stackTraceFuture.complete(stackTraceSample); - } else { - throw new IllegalStateException("Not completed yet"); - } - } - - @SuppressWarnings("unchecked") - CompletableFuture getStackTraceSampleFuture() { - return stackTraceFuture; - } - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/BackPressureSampleService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/BackPressureSampleService.java new file mode 100644 index 000000000000..39bd64a21676 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/BackPressureSampleService.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.taskexecutor; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.concurrent.ScheduledExecutor; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Samples whether a task is back pressured multi times. The total number of samples + * divided by the number of back pressure samples reaches the back pressure ratio. + */ +public class BackPressureSampleService { + + /** Number of samples to take when determining the back pressure of a task. */ + private final int numSamples; + + /** Time to wait between samples when determining the back pressure of a task. */ + private final Time delayBetweenSamples; + + /** Executor to run back pressures sample tasks. */ + private final ScheduledExecutor scheduledExecutor; + + BackPressureSampleService( + int numSamples, + Time delayBetweenSamples, + ScheduledExecutor scheduledExecutor) { + + checkArgument(numSamples >= 1, "Illegal number of samples: " + numSamples); + + this.numSamples = numSamples; + this.delayBetweenSamples = checkNotNull(delayBetweenSamples); + this.scheduledExecutor = checkNotNull(scheduledExecutor); + } + + /** + * Schedules to sample the task back pressure and returns a future that completes + * with the back pressure ratio. + * + * @param task The task to be sampled. + * @return A future containing the task back pressure ratio. + */ + public CompletableFuture sampleTaskBackPressure(BackPressureSampleableTask task) { + if (!task.isRunning()) { + throw new IllegalStateException( + String.format("Cannot sample task. Because the sampled task %s is not running.", task)); + } + + return sampleTaskBackPressure( + checkNotNull(task), + numSamples, + new ArrayList<>(numSamples), + new CompletableFuture<>()); + } + + private CompletableFuture sampleTaskBackPressure( + BackPressureSampleableTask task, + int remainingNumSamples, + List taskBackPressureSamples, + CompletableFuture resultFuture) { + taskBackPressureSamples.add(task.isBackPressured()); + + if (task.isRunning() && remainingNumSamples > 1) { + scheduledExecutor.schedule( + () -> sampleTaskBackPressure( + task, + remainingNumSamples - 1, + taskBackPressureSamples, + resultFuture), + delayBetweenSamples.getSize(), + delayBetweenSamples.getUnit()); + } else { + resultFuture.complete(calculateTaskBackPressureRatio(taskBackPressureSamples)); + } + + return resultFuture; + } + + private double calculateTaskBackPressureRatio(final List taskBackPressureSamples) { + double backPressureCount = 0.0; + for (Boolean isBackPressured: taskBackPressureSamples) { + if (isBackPressured) { + ++backPressureCount; + } + } + return taskBackPressureSamples.isEmpty() ? 0.0 : backPressureCount / taskBackPressureSamples.size(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/StackTraceSampleableTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/BackPressureSampleableTask.java similarity index 76% rename from flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/StackTraceSampleableTask.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/BackPressureSampleableTask.java index 272f61b006e8..3b6573261dae 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/StackTraceSampleableTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/BackPressureSampleableTask.java @@ -19,17 +19,13 @@ package org.apache.flink.runtime.taskexecutor; -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; - /** - * Task interface used by {@link StackTraceSampleService} for back pressure tracking. + * Task interface used by {@link BackPressureSampleService} for back pressure tracking. */ -interface StackTraceSampleableTask { +public interface BackPressureSampleableTask { boolean isRunning(); - StackTraceElement[] getStackTrace(); - - ExecutionAttemptID getExecutionId(); + boolean isBackPressured(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/StackTraceSampleService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/StackTraceSampleService.java deleted file mode 100644 index 9b3272ae4a69..000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/StackTraceSampleService.java +++ /dev/null @@ -1,128 +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.taskexecutor; - -import org.apache.flink.api.common.time.Time; -import org.apache.flink.runtime.concurrent.ScheduledExecutor; - -import javax.annotation.Nonnegative; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Optional; -import java.util.concurrent.CompletableFuture; - -import static java.util.Objects.requireNonNull; -import static org.apache.flink.util.Preconditions.checkArgument; -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * Samples the stack traces of tasks for back pressure tracking. - * - * @see org.apache.flink.runtime.rest.handler.legacy.backpressure.BackPressureStatsTracker - */ -class StackTraceSampleService { - - private final ScheduledExecutor scheduledExecutor; - - StackTraceSampleService(final ScheduledExecutor scheduledExecutor) { - this.scheduledExecutor = requireNonNull(scheduledExecutor, "scheduledExecutor must not be null"); - } - - /** - * Returns a future that completes with a given number of stack trace samples of a task thread. - * - * @param task The task to be sampled from. - * @param numSamples The number of samples. - * @param delayBetweenSamples The time to wait between taking samples. - * @param maxStackTraceDepth The maximum depth of the returned stack traces. - * Negative means unlimited. - * @return A future containing the stack trace samples. - */ - public CompletableFuture> requestStackTraceSample( - final StackTraceSampleableTask task, - @Nonnegative final int numSamples, - final Time delayBetweenSamples, - final int maxStackTraceDepth) { - - checkNotNull(task, "task must not be null"); - checkArgument(numSamples > 0, "numSamples must be positive"); - checkNotNull(delayBetweenSamples, "delayBetweenSamples must not be null"); - - return requestStackTraceSample( - task, - numSamples, - delayBetweenSamples, - maxStackTraceDepth, - new ArrayList<>(numSamples), - new CompletableFuture<>()); - } - - private CompletableFuture> requestStackTraceSample( - final StackTraceSampleableTask task, - final int numSamples, - final Time delayBetweenSamples, - final int maxStackTraceDepth, - final List currentTraces, - final CompletableFuture> resultFuture) { - - final Optional stackTrace = getStackTrace(task, maxStackTraceDepth); - if (stackTrace.isPresent()) { - currentTraces.add(stackTrace.get()); - } else if (!currentTraces.isEmpty()) { - resultFuture.complete(currentTraces); - return resultFuture; - } else { - throw new IllegalStateException(String.format("Cannot sample task %s. " + - "The task is not running.", - task.getExecutionId())); - } - - if (numSamples > 1) { - scheduledExecutor.schedule(() -> requestStackTraceSample( - task, - numSamples - 1, - delayBetweenSamples, - maxStackTraceDepth, - currentTraces, - resultFuture), delayBetweenSamples.getSize(), delayBetweenSamples.getUnit()); - } else { - resultFuture.complete(currentTraces); - } - return resultFuture; - } - - private Optional getStackTrace(final StackTraceSampleableTask task, final int maxStackTraceDepth) { - if (!task.isRunning()) { - return Optional.empty(); - } - - final StackTraceElement[] stackTrace = task.getStackTrace(); - return Optional.of(truncateStackTrace(stackTrace, maxStackTraceDepth)); - } - - private StackTraceElement[] truncateStackTrace(final StackTraceElement[] stackTrace, final int maxStackTraceDepth) { - if (maxStackTraceDepth > 0) { - return Arrays.copyOfRange(stackTrace, 0, Math.min(maxStackTraceDepth, stackTrace.length)); - } - return stackTrace; - } -} 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 b59bfb242f5c..2767a8c62f55 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 @@ -65,7 +65,7 @@ import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.messages.StackTraceSampleResponse; +import org.apache.flink.runtime.messages.TaskBackPressureResponse; import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.query.KvStateClientProxy; @@ -77,6 +77,7 @@ import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.RpcTimeout; import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils; import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.shuffle.ShuffleEnvironment; @@ -212,6 +213,8 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { private final TaskExecutorPartitionTracker partitionTracker; + private final BackPressureSampleService backPressureSampleService; + // --------- resource manager -------- @Nullable @@ -226,8 +229,6 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { @Nullable private UUID currentRegistrationTimeoutId; - private final StackTraceSampleService stackTraceSampleService; - private Map>> taskResultPartitionCleanupFuturesPerJob = new HashMap<>(8); public TaskExecutor( @@ -240,7 +241,8 @@ public TaskExecutor( String metricQueryServiceAddress, BlobCacheService blobCacheService, FatalErrorHandler fatalErrorHandler, - TaskExecutorPartitionTracker partitionTracker) { + TaskExecutorPartitionTracker partitionTracker, + BackPressureSampleService backPressureSampleService) { super(rpcService, AkkaRpcServiceUtils.createRandomName(TASK_MANAGER_NAME)); @@ -254,6 +256,7 @@ public TaskExecutor( this.taskManagerMetricGroup = checkNotNull(taskManagerMetricGroup); this.blobCacheService = checkNotNull(blobCacheService); this.metricQueryServiceAddress = checkNotNull(metricQueryServiceAddress); + this.backPressureSampleService = checkNotNull(backPressureSampleService); this.taskSlotTable = taskExecutorServices.getTaskSlotTable(); this.jobManagerTable = taskExecutorServices.getJobManagerTable(); @@ -272,8 +275,6 @@ public TaskExecutor( this.resourceManagerAddress = null; this.resourceManagerConnection = null; this.currentRegistrationTimeoutId = null; - - this.stackTraceSampleService = new StackTraceSampleService(rpcService.getScheduledExecutor()); this.taskCompletionTracker = new TaskCompletionTracker(); final ResourceID resourceId = taskExecutorServices.getTaskManagerLocation().getResourceID(); @@ -435,29 +436,22 @@ private void stopTaskExecutorServices() throws Exception { // ====================================================================== @Override - public CompletableFuture requestStackTraceSample( - final ExecutionAttemptID executionAttemptId, - final int sampleId, - final int numSamples, - final Time delayBetweenSamples, - final int maxStackTraceDepth, - final Time timeout) { + public CompletableFuture requestTaskBackPressure( + ExecutionAttemptID executionAttemptId, + int requestId, + @RpcTimeout Time timeout) { final Task task = taskSlotTable.getTask(executionAttemptId); if (task == null) { return FutureUtils.completedExceptionally( - new IllegalStateException(String.format("Cannot sample task %s. " + + new IllegalStateException(String.format("Cannot request back pressure of task %s. " + "Task is not known to the task manager.", executionAttemptId))); } + final CompletableFuture backPressureRatioFuture = + backPressureSampleService.sampleTaskBackPressure(task); - final CompletableFuture> stackTracesFuture = stackTraceSampleService.requestStackTraceSample( - TaskStackTraceSampleableTaskAdapter.fromTask(task), - numSamples, - delayBetweenSamples, - maxStackTraceDepth); - - return stackTracesFuture.thenApply(stackTraces -> - new StackTraceSampleResponse(sampleId, executionAttemptId, stackTraces)); + return backPressureRatioFuture.thenApply(backPressureRatio -> + new TaskBackPressureResponse(requestId, executionAttemptId, backPressureRatio)); } // ---------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java index 9e50ada6cad0..142bb82ac55e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java @@ -33,7 +33,7 @@ import org.apache.flink.runtime.jobmaster.AllocatedSlotReport; import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.messages.StackTraceSampleResponse; +import org.apache.flink.runtime.messages.TaskBackPressureResponse; import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcTimeout; @@ -67,12 +67,9 @@ CompletableFuture requestSlot( ResourceManagerId resourceManagerId, @RpcTimeout Time timeout); - CompletableFuture requestStackTraceSample( + CompletableFuture requestTaskBackPressure( ExecutionAttemptID executionAttemptId, - int sampleId, - int numSamples, - Time delayBetweenSamples, - int maxStackTraceDepth, + int requestId, @RpcTimeout Time timeout); /** 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 8afa4f8fedf5..2c8b98b3e02a 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 @@ -25,6 +25,7 @@ import org.apache.flink.configuration.ConfigurationUtils; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.configuration.WebOptions; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.plugin.PluginUtils; import org.apache.flink.metrics.MetricGroup; @@ -32,6 +33,7 @@ import org.apache.flink.runtime.blob.BlobCacheService; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.entrypoint.ClusterConfiguration; import org.apache.flink.runtime.entrypoint.ClusterConfigurationParserFactory; import org.apache.flink.runtime.entrypoint.FlinkParseException; @@ -387,7 +389,17 @@ public static TaskExecutor startTaskManager( metricQueryServiceAddress, blobCacheService, fatalErrorHandler, - new TaskExecutorPartitionTrackerImpl(taskManagerServices.getShuffleEnvironment())); + new TaskExecutorPartitionTrackerImpl(taskManagerServices.getShuffleEnvironment()), + createBackPressureSampleService(configuration, rpcService.getScheduledExecutor())); + } + + static BackPressureSampleService createBackPressureSampleService( + Configuration configuration, + ScheduledExecutor scheduledExecutor) { + return new BackPressureSampleService( + configuration.getInteger(WebOptions.BACKPRESSURE_NUM_SAMPLES), + Time.milliseconds(configuration.getInteger(WebOptions.BACKPRESSURE_DELAY)), + scheduledExecutor); } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskStackTraceSampleableTaskAdapter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskStackTraceSampleableTaskAdapter.java deleted file mode 100644 index 2297c3accfb6..000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskStackTraceSampleableTaskAdapter.java +++ /dev/null @@ -1,58 +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.taskexecutor; - -import org.apache.flink.runtime.execution.ExecutionState; -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; -import org.apache.flink.runtime.taskmanager.Task; - -import static java.util.Objects.requireNonNull; - -/** - * Adapts {@link Task} to {@link StackTraceSampleableTask}. - */ -class TaskStackTraceSampleableTaskAdapter implements StackTraceSampleableTask { - - private final Task task; - - static TaskStackTraceSampleableTaskAdapter fromTask(final Task task) { - return new TaskStackTraceSampleableTaskAdapter(task); - } - - private TaskStackTraceSampleableTaskAdapter(final Task task) { - this.task = requireNonNull(task, "task must not be null"); - } - - @Override - public boolean isRunning() { - return task.getExecutionState() == ExecutionState.RUNNING; - } - - @Override - public StackTraceElement[] getStackTrace() { - return task.getStackTraceOfExecutingThread(); - } - - @Override - public ExecutionAttemptID getExecutionId() { - return task.getExecutionId(); - } - -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index 47b253e8856a..7848171ab15e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -66,6 +66,7 @@ import org.apache.flink.runtime.shuffle.ShuffleIOOwnerContext; import org.apache.flink.runtime.state.CheckpointListener; import org.apache.flink.runtime.state.TaskStateManager; +import org.apache.flink.runtime.taskexecutor.BackPressureSampleableTask; import org.apache.flink.runtime.taskexecutor.GlobalAggregateManager; import org.apache.flink.runtime.taskexecutor.KvStateService; import org.apache.flink.runtime.taskexecutor.PartitionProducerStateChecker; @@ -122,7 +123,7 @@ * *

    Each Task is run by one dedicated thread. */ -public class Task implements Runnable, TaskActions, PartitionProducerStateProvider, CheckpointListener { +public class Task implements Runnable, TaskActions, PartitionProducerStateProvider, CheckpointListener, BackPressureSampleableTask { /** The class logger. */ private static final Logger LOG = LoggerFactory.getLogger(Task.class); @@ -461,16 +462,16 @@ AbstractInvokable getInvokable() { return invokable; } - public StackTraceElement[] getStackTraceOfExecutingThread() { - final AbstractInvokable invokable = this.invokable; - - if (invokable == null) { - return new StackTraceElement[0]; + @Override + public boolean isBackPressured() { + if (invokable == null || consumableNotifyingPartitionWriters.length == 0 || !isRunning()) { + return false; } - - return invokable.getExecutingThread() - .orElse(executingThread) - .getStackTrace(); + final CompletableFuture[] outputFutures = new CompletableFuture[consumableNotifyingPartitionWriters.length]; + for (int i = 0; i < outputFutures.length; ++i) { + outputFutures[i] = consumableNotifyingPartitionWriters[i].isAvailable(); + } + return !CompletableFuture.allOf(outputFutures).isDone(); } // ------------------------------------------------------------------------ @@ -495,6 +496,11 @@ public boolean isCanceledOrFailed() { executionState == ExecutionState.FAILED; } + @Override + public boolean isRunning() { + return executionState == ExecutionState.RUNNING; + } + /** * If the task has failed, this method gets the exception that caused this task to fail. * Otherwise this method returns null. diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertexTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertexTest.java index 78c87b7ed994..3c350640f503 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertexTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertexTest.java @@ -20,16 +20,14 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.JobException; -import org.apache.flink.runtime.concurrent.Executors; +import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.testutils.DirectScheduledExecutorService; import org.junit.Assert; import org.junit.Test; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - public class ExecutionJobVertexTest { private static final int NOT_CONFIGURED = -1; @@ -122,10 +120,9 @@ public void testMaxParallelismDefaulting() throws Exception { //------------------------------------------------------------------------------------------------------ - private static ExecutionJobVertex createExecutionJobVertex( + public static ExecutionJobVertex createExecutionJobVertex( int parallelism, - int preconfiguredMaxParallelism) throws JobException { - + int preconfiguredMaxParallelism) throws JobException, JobExecutionException { JobVertex jobVertex = new JobVertex("testVertex"); jobVertex.setInvokableClass(AbstractInvokable.class); jobVertex.setParallelism(parallelism); @@ -134,11 +131,16 @@ private static ExecutionJobVertex createExecutionJobVertex( jobVertex.setMaxParallelism(preconfiguredMaxParallelism); } - ExecutionGraph executionGraphMock = mock(ExecutionGraph.class); - when(executionGraphMock.getFutureExecutor()).thenReturn(Executors.directExecutor()); - ExecutionJobVertex executionJobVertex = - new ExecutionJobVertex(executionGraphMock, jobVertex, 1, Time.seconds(10)); + ExecutionGraph executionGraph = createExecutionGraph(); + return new ExecutionJobVertex(executionGraph, jobVertex, 1, Time.seconds(10)); + } - return executionJobVertex; + private static ExecutionGraph createExecutionGraph() throws JobException, JobExecutionException { + final ExecutionGraph executionGraph = TestingExecutionGraphBuilder + .newBuilder() + .setFutureExecutor(new DirectScheduledExecutorService()) + .build(); + executionGraph.transitionToRunning(); + return executionGraph; } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleAckingTaskManagerGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleAckingTaskManagerGateway.java index 3408e1132db9..a0d2dfb8b7f3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleAckingTaskManagerGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleAckingTaskManagerGateway.java @@ -29,7 +29,7 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.messages.StackTraceSampleResponse; +import org.apache.flink.runtime.messages.TaskBackPressureResponse; import java.util.Collection; import java.util.Set; @@ -89,12 +89,9 @@ public String getAddress() { } @Override - public CompletableFuture requestStackTraceSample( + public CompletableFuture requestTaskBackPressure( ExecutionAttemptID executionAttemptID, - int sampleId, - int numSamples, - Time delayBetweenSamples, - int maxStackTraceDepth, + int requestId, Time timeout) { return FutureUtils.completedExceptionally(new UnsupportedOperationException()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerSharedServicesBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerSharedServicesBuilder.java index 1edffe411c0e..7ec3196db24b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerSharedServicesBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerSharedServicesBuilder.java @@ -25,14 +25,12 @@ import org.apache.flink.runtime.executiongraph.restart.NoOrFixedIfCheckpointingEnabledRestartStrategyFactory; import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory; import org.apache.flink.runtime.rest.handler.legacy.backpressure.BackPressureStatsTracker; -import org.apache.flink.runtime.rest.handler.legacy.backpressure.StackTraceSampleCoordinator; +import org.apache.flink.runtime.rest.handler.legacy.backpressure.BackPressureRequestCoordinator; import org.apache.flink.runtime.rest.handler.legacy.backpressure.VoidBackPressureStatsTracker; import org.apache.flink.runtime.testingUtils.TestingUtils; import java.util.concurrent.ScheduledExecutorService; -import static org.mockito.Mockito.mock; - /** * Builder for the {@link JobManagerSharedServices}. */ @@ -44,7 +42,7 @@ public class TestingJobManagerSharedServicesBuilder { private RestartStrategyFactory restartStrategyFactory; - private StackTraceSampleCoordinator stackTraceSampleCoordinator; + private BackPressureRequestCoordinator backPressureSampleCoordinator; private BackPressureStatsTracker backPressureStatsTracker; @@ -54,7 +52,7 @@ public TestingJobManagerSharedServicesBuilder() { scheduledExecutorService = TestingUtils.defaultExecutor(); libraryCacheManager = ContextClassLoaderLibraryCacheManager.INSTANCE; restartStrategyFactory = new NoOrFixedIfCheckpointingEnabledRestartStrategyFactory(); - stackTraceSampleCoordinator = mock(StackTraceSampleCoordinator.class); + backPressureSampleCoordinator = new BackPressureRequestCoordinator(Runnable::run, 10000); backPressureStatsTracker = VoidBackPressureStatsTracker.INSTANCE; blobWriter = VoidBlobWriter.getInstance(); } @@ -75,8 +73,8 @@ public TestingJobManagerSharedServicesBuilder setRestartStrategyFactory(RestartS return this; } - public TestingJobManagerSharedServicesBuilder setStackTraceSampleCoordinator(StackTraceSampleCoordinator stackTraceSampleCoordinator) { - this.stackTraceSampleCoordinator = stackTraceSampleCoordinator; + public TestingJobManagerSharedServicesBuilder setBackPressureSampleCoordinator(BackPressureRequestCoordinator backPressureSampleCoordinator) { + this.backPressureSampleCoordinator = backPressureSampleCoordinator; return this; } @@ -95,7 +93,7 @@ public JobManagerSharedServices build() { scheduledExecutorService, libraryCacheManager, restartStrategyFactory, - stackTraceSampleCoordinator, + backPressureSampleCoordinator, backPressureStatsTracker, blobWriter); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java new file mode 100644 index 000000000000..00bac1dc780c --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureRequestCoordinatorTest.java @@ -0,0 +1,343 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.rest.handler.legacy.backpressure; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.Execution; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.ExecutionJobVertexTest; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.IntermediateResult; +import org.apache.flink.runtime.messages.TaskBackPressureResponse; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.TestLogger; + +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * Tests for the {@link BackPressureRequestCoordinator}. + */ +public class BackPressureRequestCoordinatorTest extends TestLogger { + + private static final long requestTimeout = 10000; + private static final double backPressureRatio = 0.5; + private static final String requestTimeoutMessage = "Request timeout."; + + private static ScheduledExecutorService executorService; + private BackPressureRequestCoordinator coordinator; + + @Rule + public Timeout caseTimeout = new Timeout(10, TimeUnit.SECONDS); + + @BeforeClass + public static void setUp() throws Exception { + executorService = new ScheduledThreadPoolExecutor(1); + } + + @AfterClass + public static void tearDown() throws Exception { + if (executorService != null) { + executorService.shutdown(); + } + } + + @Before + public void initCoordinator() throws Exception { + coordinator = new BackPressureRequestCoordinator(executorService, requestTimeout); + } + + @After + public void shutdownCoordinator() throws Exception { + if (coordinator != null) { + // verify no more pending request + assertEquals(0, coordinator.getNumberOfPendingRequests()); + coordinator.shutDown(); + } + } + + /** + * Tests request of task back pressure stats and verifies the response. + */ + @Test + public void testSuccessfulBackPressureRequest() throws Exception { + ExecutionVertex[] vertices = createExecutionVertices(ExecutionState.RUNNING, CompletionType.SUCCESSFULLY); + + CompletableFuture requestFuture = coordinator.triggerBackPressureRequest(vertices); + BackPressureStats backPressureStats = requestFuture.get(); + + // verify the request result + assertEquals(0, backPressureStats.getRequestId()); + Map backPressureRatios = backPressureStats.getBackPressureRatios(); + for (ExecutionVertex executionVertex : vertices) { + ExecutionAttemptID executionId = executionVertex.getCurrentExecutionAttempt().getAttemptId(); + assertEquals(backPressureRatio, backPressureRatios.get(executionId), 0.0); + } + } + + /** + * Tests back pressure request of non-running tasks fails the future. + */ + @Test + public void testRequestNotRunningTasks() throws Exception { + ExecutionVertex[] vertices = createExecutionVertices(ExecutionState.DEPLOYING, CompletionType.SUCCESSFULLY); + + CompletableFuture requestFuture = coordinator.triggerBackPressureRequest(vertices); + assertTrue(requestFuture.isDone()); + try { + requestFuture.get(); + fail("Exception expected."); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof IllegalStateException); + } + } + + /** + * Tests failed request to execution fails the future. + */ + @Test + public void testBackPressureRequestWithException() throws Exception { + ExecutionVertex[] vertices = createExecutionVertices(ExecutionState.RUNNING, CompletionType.EXCEPTIONALLY); + + CompletableFuture requestFuture = coordinator.triggerBackPressureRequest(vertices); + try { + requestFuture.get(); + fail("Exception expected."); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof RuntimeException); + } + } + + /** + * Tests that request timeout if not finished in time. + */ + @Test + public void testBackPressureRequestTimeout() throws Exception { + long requestTimeout = 100; + ExecutionVertex[] vertices = createExecutionVertices(ExecutionState.RUNNING, CompletionType.TIMEOUT, requestTimeout); + BackPressureRequestCoordinator coordinator = new BackPressureRequestCoordinator(executorService, requestTimeout); + + try { + CompletableFuture requestFuture = coordinator.triggerBackPressureRequest(vertices); + requestFuture.get(); + fail("Exception expected."); + } catch (ExecutionException e) { + assertTrue(ExceptionUtils.findThrowableWithMessage(e, requestTimeoutMessage).isPresent()); + } finally { + coordinator.shutDown(); + } + } + + /** + * Tests shutdown fails all pending requests and future request triggers. + */ + @Test + public void testShutDown() throws Exception { + ExecutionVertex[] vertices = createExecutionVertices(ExecutionState.RUNNING, CompletionType.NEVER_COMPLETE); + + List> requestFutures = new ArrayList<>(); + + // trigger request + requestFutures.add(coordinator.triggerBackPressureRequest(vertices)); + requestFutures.add(coordinator.triggerBackPressureRequest(vertices)); + + for (CompletableFuture future : requestFutures) { + assertFalse(future.isDone()); + } + + // shut down + coordinator.shutDown(); + + // verify all completed + for (CompletableFuture future : requestFutures) { + assertTrue(future.isCompletedExceptionally()); + } + + // verify new trigger returns failed future + CompletableFuture future = coordinator.triggerBackPressureRequest(vertices); + assertTrue(future.isCompletedExceptionally()); + } + + private ExecutionVertex[] createExecutionVertices( + ExecutionState state, + CompletionType completionType) throws Exception { + return createExecutionVertices(state, completionType, requestTimeout); + } + + private ExecutionVertex[] createExecutionVertices( + ExecutionState state, + CompletionType completionType, + long requestTimeout) throws Exception { + return new ExecutionVertex[] { + createExecutionVertex(0, ExecutionState.RUNNING, CompletionType.SUCCESSFULLY, requestTimeout), + createExecutionVertex(1, state, completionType, requestTimeout), + createExecutionVertex(2, ExecutionState.RUNNING, CompletionType.SUCCESSFULLY, requestTimeout) + }; + } + + private ExecutionVertex createExecutionVertex( + int subTaskIndex, + ExecutionState state, + CompletionType completionType, + long requestTimeout) throws Exception { + return new TestingExecutionVertex( + ExecutionJobVertexTest.createExecutionJobVertex(4, 4), + subTaskIndex, + Time.seconds(10), + 1L, + System.currentTimeMillis(), + state, + completionType, + requestTimeout); + } + + /** + * Completion types of the request future. + */ + private enum CompletionType { + SUCCESSFULLY, + EXCEPTIONALLY, + TIMEOUT, + NEVER_COMPLETE + } + + /** + * A testing {@link ExecutionVertex} implementation used to wrap {@link TestingExecution}. + */ + private static class TestingExecutionVertex extends ExecutionVertex { + + private final Execution execution; + + TestingExecutionVertex( + ExecutionJobVertex jobVertex, + int subTaskIndex, + Time timeout, + long initialGlobalModVersion, + long createTimestamp, + ExecutionState state, + CompletionType completionType, + long requestTimeout) { + + super( + jobVertex, + subTaskIndex, + new IntermediateResult[0], + timeout, + initialGlobalModVersion, + createTimestamp, + JobManagerOptions.MAX_ATTEMPTS_HISTORY_SIZE.defaultValue()); + execution = new TestingExecution( + Runnable::run, + this, + 0, + initialGlobalModVersion, + createTimestamp, + timeout, + state, + completionType, + requestTimeout); + } + + @Override + public Execution getCurrentExecutionAttempt() { + return execution; + } + } + + /** + * A testing implementation of {@link Execution} which acts differently according to + * the given {@link ExecutionState} and {@link CompletionType}. + */ + private static class TestingExecution extends Execution { + + private final ExecutionState state; + private final CompletionType completionType; + private final long requestTimeout; + + TestingExecution( + Executor executor, + ExecutionVertex vertex, + int attemptNumber, + long globalModVersion, + long startTimestamp, + Time rpcTimeout, + ExecutionState state, + CompletionType completionType, + long requestTimeout) { + super(executor, vertex, attemptNumber, globalModVersion, startTimestamp, rpcTimeout); + this.state = checkNotNull(state); + this.completionType = checkNotNull(completionType); + this.requestTimeout = requestTimeout; + } + + @Override + public CompletableFuture requestBackPressure(int requestId, Time timeout) { + CompletableFuture responseFuture = new CompletableFuture<>(); + switch (completionType) { + case SUCCESSFULLY: + responseFuture.complete(new TaskBackPressureResponse(0, getAttemptId(), backPressureRatio)); + break; + case EXCEPTIONALLY: + responseFuture.completeExceptionally(new RuntimeException("Request failed.")); + break; + case TIMEOUT: + executorService.schedule( + () -> responseFuture.completeExceptionally(new TimeoutException(requestTimeoutMessage)), + requestTimeout, + TimeUnit.MILLISECONDS); + break; + case NEVER_COMPLETE: + break; // do nothing + default: + throw new RuntimeException("Unknown completion type."); + } + return responseFuture; + } + + @Override + public ExecutionState getState() { + return state; + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImplTest.java index e418980efd53..a79ce99d5c11 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/BackPressureStatsTrackerImplTest.java @@ -18,166 +18,209 @@ package org.apache.flink.runtime.rest.handler.legacy.backpressure; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.time.Time; -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.ExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.ExecutionJobVertexTest; import org.apache.flink.runtime.executiongraph.ExecutionVertex; -import org.apache.flink.runtime.jobgraph.JobStatus; -import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.util.TestLogger; -import org.junit.Assert; +import org.junit.Rule; import org.junit.Test; -import org.mockito.Matchers; -import org.mockito.Mockito; +import org.junit.rules.Timeout; -import java.util.ArrayList; import java.util.HashMap; -import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; /** - * Tests for the BackPressureStatsTrackerImpl. + * Tests for the {@link BackPressureStatsTrackerImpl}. */ public class BackPressureStatsTrackerImplTest extends TestLogger { - /** Tests simple statistics with fake stack traces. */ + private static final int requestId = 0; + private static final double backPressureRatio = 0.1; + private static final ExecutionJobVertex executionJobVertex = createExecutionJobVertex(); + private static final ExecutionVertex[] taskVertices = executionJobVertex.getTaskVertices(); + private static final BackPressureStats backPressureStats = createBackPressureStats(requestId, 1, backPressureRatio); + private static final int cleanUpInterval = 60000; + private static final int backPressureStatsRefreshInterval = 60000; + private static final long timeGap = 60000; + + @Rule + public Timeout caseTimeout = new Timeout(10, TimeUnit.SECONDS); + @Test - @SuppressWarnings("unchecked") - public void testTriggerStackTraceSample() throws Exception { - CompletableFuture sampleFuture = new CompletableFuture<>(); - - StackTraceSampleCoordinator sampleCoordinator = Mockito.mock(StackTraceSampleCoordinator.class); - Mockito.when(sampleCoordinator.triggerStackTraceSample( - Matchers.any(ExecutionVertex[].class), - Matchers.anyInt(), - Matchers.any(Time.class), - Matchers.anyInt())).thenReturn(sampleFuture); - - ExecutionGraph graph = Mockito.mock(ExecutionGraph.class); - Mockito.when(graph.getState()).thenReturn(JobStatus.RUNNING); - - // Same Thread execution context - Mockito.when(graph.getFutureExecutor()).thenReturn(new Executor() { - - @Override - public void execute(Runnable runnable) { - runnable.run(); - } - }); - - ExecutionVertex[] taskVertices = new ExecutionVertex[4]; - - ExecutionJobVertex jobVertex = Mockito.mock(ExecutionJobVertex.class); - Mockito.when(jobVertex.getJobId()).thenReturn(new JobID()); - Mockito.when(jobVertex.getJobVertexId()).thenReturn(new JobVertexID()); - Mockito.when(jobVertex.getGraph()).thenReturn(graph); - Mockito.when(jobVertex.getTaskVertices()).thenReturn(taskVertices); - - taskVertices[0] = mockExecutionVertex(jobVertex, 0); - taskVertices[1] = mockExecutionVertex(jobVertex, 1); - taskVertices[2] = mockExecutionVertex(jobVertex, 2); - taskVertices[3] = mockExecutionVertex(jobVertex, 3); - - int numSamples = 100; - Time delayBetweenSamples = Time.milliseconds(100L); - - BackPressureStatsTrackerImpl tracker = new BackPressureStatsTrackerImpl( - sampleCoordinator, 9999, numSamples, Integer.MAX_VALUE, delayBetweenSamples); - - // getOperatorBackPressureStats triggers stack trace sampling - Assert.assertFalse(tracker.getOperatorBackPressureStats(jobVertex).isPresent()); - - Mockito.verify(sampleCoordinator, Mockito.times(1)).triggerStackTraceSample( - Matchers.eq(taskVertices), - Matchers.eq(numSamples), - Matchers.eq(delayBetweenSamples), - Matchers.eq(BackPressureStatsTrackerImpl.MAX_STACK_TRACE_DEPTH)); - - // Request back pressure stats again. This should not trigger another sample request - Assert.assertTrue(!tracker.getOperatorBackPressureStats(jobVertex).isPresent()); - - Mockito.verify(sampleCoordinator, Mockito.times(1)).triggerStackTraceSample( - Matchers.eq(taskVertices), - Matchers.eq(numSamples), - Matchers.eq(delayBetweenSamples), - Matchers.eq(BackPressureStatsTrackerImpl.MAX_STACK_TRACE_DEPTH)); - - Assert.assertTrue(!tracker.getOperatorBackPressureStats(jobVertex).isPresent()); - - // Complete the future - Map> traces = new HashMap<>(); - for (ExecutionVertex vertex : taskVertices) { - List taskTraces = new ArrayList<>(); + public void testGetOperatorBackPressureStats() throws Exception { + doInitialRequestAndVerifyResult(createBackPressureTracker()); + } - for (int i = 0; i < taskVertices.length; i++) { - // Traces until sub task index are back pressured - taskTraces.add(createStackTrace(i <= vertex.getParallelSubtaskIndex())); - } + @Test + public void testCachedStatsNotUpdatedWithinRefreshInterval() throws Exception { + final double backPressureRatio2 = 0.2; + final int requestId2 = 1; + final BackPressureStats backPressureStats2 = createBackPressureStats(requestId2, timeGap, backPressureRatio2); + + final BackPressureStatsTracker tracker = createBackPressureTracker( + cleanUpInterval, backPressureStatsRefreshInterval, backPressureStats, backPressureStats2); + doInitialRequestAndVerifyResult(tracker); + // verify that no new back pressure request is triggered + checkOperatorBackPressureStats(tracker.getOperatorBackPressureStats(executionJobVertex)); + } - traces.put(vertex.getCurrentExecutionAttempt().getAttemptId(), taskTraces); - } + @Test + public void testCachedStatsUpdatedAfterRefreshInterval() throws Exception { + final int backPressureStatsRefreshInterval2 = 10; + final long waitingTime = backPressureStatsRefreshInterval2 + 10; + final double backPressureRatio2 = 0.2; + final int requestId2 = 1; + final BackPressureStats backPressureStats2 = createBackPressureStats(requestId2, timeGap, backPressureRatio2); + + final BackPressureStatsTracker tracker = createBackPressureTracker( + cleanUpInterval, backPressureStatsRefreshInterval2, backPressureStats, backPressureStats2); + doInitialRequestAndVerifyResult(tracker); + + // ensure that we are ready for next request + Thread.sleep(waitingTime); + + // trigger next back pressure stats request and verify the result + assertTrue(tracker.getOperatorBackPressureStats(executionJobVertex).isPresent()); + checkOperatorBackPressureStats(backPressureRatio2, backPressureStats2, tracker.getOperatorBackPressureStats(executionJobVertex)); + } - int sampleId = 1231; - int endTime = 841; + @Test + public void testShutDown() throws Exception { + final BackPressureStatsTracker tracker = createBackPressureTracker(); + doInitialRequestAndVerifyResult(tracker); - StackTraceSample sample = new StackTraceSample( - sampleId, - 0, - endTime, - traces); + // shutdown directly + tracker.shutDown(); - // Succeed the promise - sampleFuture.complete(sample); + // verify that the previous cached result is invalid and trigger another request + assertFalse(tracker.getOperatorBackPressureStats(executionJobVertex).isPresent()); + // verify no response after shutdown + assertFalse(tracker.getOperatorBackPressureStats(executionJobVertex).isPresent()); + } - Assert.assertTrue(tracker.getOperatorBackPressureStats(jobVertex).isPresent()); + @Test + public void testCachedStatsNotCleanedWithinCleanupInterval() throws Exception { + final BackPressureStatsTracker tracker = createBackPressureTracker(); + doInitialRequestAndVerifyResult(tracker); - OperatorBackPressureStats stats = tracker.getOperatorBackPressureStats(jobVertex).get(); + tracker.cleanUpOperatorStatsCache(); + // the back pressure stats should be still there + checkOperatorBackPressureStats(tracker.getOperatorBackPressureStats(executionJobVertex)); + } - // Verify the stats - Assert.assertEquals(sampleId, stats.getSampleId()); - Assert.assertEquals(endTime, stats.getEndTimestamp()); - Assert.assertEquals(taskVertices.length, stats.getNumberOfSubTasks()); + @Test + public void testCachedStatsCleanedAfterCleanupInterval() throws Exception { + final int cleanUpInterval2 = 10; + final long waitingTime = cleanUpInterval2 + 10; - for (int i = 0; i < taskVertices.length; i++) { - double ratio = stats.getBackPressureRatio(i); - // Traces until sub task index are back pressured - Assert.assertEquals((i + 1) / ((double) 4), ratio, 0.0); - } + final BackPressureStatsTracker tracker = createBackPressureTracker( + cleanUpInterval2, backPressureStatsRefreshInterval, backPressureStats); + doInitialRequestAndVerifyResult(tracker); + + // wait until we are ready to cleanup + Thread.sleep(waitingTime); + + // cleanup the cached back pressure stats + tracker.cleanUpOperatorStatsCache(); + assertFalse(tracker.getOperatorBackPressureStats(executionJobVertex).isPresent()); } - private StackTraceElement[] createStackTrace(boolean isBackPressure) { - if (isBackPressure) { - return new StackTraceElement[] { new StackTraceElement( - BackPressureStatsTrackerImpl.EXPECTED_CLASS_NAME, - BackPressureStatsTrackerImpl.EXPECTED_METHOD_NAME, - "LocalBufferPool.java", - 133) }; - } else { - return Thread.currentThread().getStackTrace(); + private void doInitialRequestAndVerifyResult(BackPressureStatsTracker tracker) { + // trigger back pressure stats request + assertFalse(tracker.getOperatorBackPressureStats(executionJobVertex).isPresent()); + // verify the result + checkOperatorBackPressureStats(tracker.getOperatorBackPressureStats(executionJobVertex)); + } + + private void checkOperatorBackPressureStats(Optional optionalStats) { + checkOperatorBackPressureStats(backPressureRatio, backPressureStats, optionalStats); + } + + private void checkOperatorBackPressureStats( + double backPressureRatio, + BackPressureStats backPressureStats, + Optional optionalStats) { + assertTrue(optionalStats.isPresent()); + OperatorBackPressureStats stats = optionalStats.get(); + + assertEquals(backPressureStats.getRequestId(), stats.getRequestId()); + assertEquals(backPressureStats.getEndTime(), stats.getEndTimestamp()); + assertEquals(taskVertices.length, stats.getNumberOfSubTasks()); + + for (int i = 0; i < stats.getNumberOfSubTasks(); i++) { + assertEquals(backPressureRatio, stats.getBackPressureRatio(i), 0.0); } } - private ExecutionVertex mockExecutionVertex( - ExecutionJobVertex jobVertex, - int subTaskIndex) { + private BackPressureStatsTracker createBackPressureTracker() { + return createBackPressureTracker(cleanUpInterval, backPressureStatsRefreshInterval, backPressureStats); + } + + private BackPressureStatsTracker createBackPressureTracker( + int cleanUpInterval, + int backPressureStatsRefreshInterval, + BackPressureStats... stats) { + + final BackPressureRequestCoordinator coordinator = + new TestingBackPressureRequestCoordinator(Runnable::run, 10000, stats); + return new BackPressureStatsTrackerImpl( + coordinator, + cleanUpInterval, + backPressureStatsRefreshInterval); + } - Execution exec = Mockito.mock(Execution.class); - Mockito.when(exec.getAttemptId()).thenReturn(new ExecutionAttemptID()); + private static BackPressureStats createBackPressureStats( + int requestId, + long timeGap, + double backPressureRatio) { + long startTime = System.currentTimeMillis(); + long endTime = startTime + timeGap; - JobVertexID id = jobVertex.getJobVertexId(); + final Map backPressureRatiosByTask = new HashMap<>(); + for (ExecutionVertex vertex : taskVertices) { + backPressureRatiosByTask.put(vertex.getCurrentExecutionAttempt().getAttemptId(), backPressureRatio); + } - ExecutionVertex vertex = Mockito.mock(ExecutionVertex.class); - Mockito.when(vertex.getJobvertexId()).thenReturn(id); - Mockito.when(vertex.getCurrentExecutionAttempt()).thenReturn(exec); - Mockito.when(vertex.getParallelSubtaskIndex()).thenReturn(subTaskIndex); + return new BackPressureStats(requestId, startTime, endTime, backPressureRatiosByTask); + } - return vertex; + private static ExecutionJobVertex createExecutionJobVertex() { + try { + return ExecutionJobVertexTest.createExecutionJobVertex(4, 4); + } catch (Exception e) { + throw new RuntimeException("Failed to create ExecutionJobVertex."); + } } + /** + * A {@link BackPressureRequestCoordinator} which returns the pre-generated back pressure stats directly. + */ + private static class TestingBackPressureRequestCoordinator extends BackPressureRequestCoordinator { + + private final BackPressureStats[] backPressureStats; + private int counter = 0; + + TestingBackPressureRequestCoordinator( + Executor executor, + long requestTimeout, + BackPressureStats... backPressureStats) { + super(executor, requestTimeout); + this.backPressureStats = backPressureStats; + } + + @Override + CompletableFuture triggerBackPressureRequest(ExecutionVertex[] tasks) { + return CompletableFuture.completedFuture(backPressureStats[(counter++) % backPressureStats.length]); + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/StackTraceSampleCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/StackTraceSampleCoordinatorTest.java deleted file mode 100644 index 8693b1636d28..000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/backpressure/StackTraceSampleCoordinatorTest.java +++ /dev/null @@ -1,421 +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.rest.handler.legacy.backpressure; - -import org.apache.flink.api.common.time.Time; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.runtime.execution.ExecutionState; -import org.apache.flink.runtime.executiongraph.Execution; -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; -import org.apache.flink.runtime.executiongraph.ExecutionVertex; -import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.messages.StackTraceSampleResponse; -import org.apache.flink.util.TestLogger; - -import akka.actor.ActorSystem; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; -import org.mockito.Matchers; -import org.mockito.Mockito; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; - -/** - * Test for the {@link StackTraceSampleCoordinator}. - */ -public class StackTraceSampleCoordinatorTest extends TestLogger { - - private static ActorSystem system; - - private StackTraceSampleCoordinator coord; - - @BeforeClass - public static void setUp() throws Exception { - system = AkkaUtils.createLocalActorSystem(new Configuration()); - } - - @AfterClass - public static void tearDown() throws Exception { - if (system != null) { - system.terminate(); - } - } - - @Before - public void init() throws Exception { - this.coord = new StackTraceSampleCoordinator(system.dispatcher(), 60000); - } - - /** Tests simple trigger and collect of stack trace samples. */ - @Test - public void testTriggerStackTraceSample() throws Exception { - ExecutionVertex[] vertices = new ExecutionVertex[] { - mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true), - mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true), - mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true), - mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true) - }; - - int numSamples = 1; - Time delayBetweenSamples = Time.milliseconds(100L); - int maxStackTraceDepth = 0; - - CompletableFuture sampleFuture = coord.triggerStackTraceSample( - vertices, numSamples, delayBetweenSamples, maxStackTraceDepth); - - // Verify messages have been sent - for (ExecutionVertex vertex : vertices) { - Mockito.verify(vertex.getCurrentExecutionAttempt()) - .requestStackTraceSample(Matchers.eq(0), Matchers.eq(numSamples), Matchers.eq(delayBetweenSamples), Matchers.eq(maxStackTraceDepth), Matchers.any(Time.class)); - } - - Assert.assertFalse(sampleFuture.isDone()); - - StackTraceElement[] stackTraceSample = Thread.currentThread().getStackTrace(); - List traces = new ArrayList<>(); - traces.add(stackTraceSample); - traces.add(stackTraceSample); - traces.add(stackTraceSample); - - // Collect stack traces - for (int i = 0; i < vertices.length; i++) { - ExecutionAttemptID executionId = vertices[i].getCurrentExecutionAttempt().getAttemptId(); - coord.collectStackTraces(0, executionId, traces); - - if (i == vertices.length - 1) { - Assert.assertTrue(sampleFuture.isDone()); - } else { - Assert.assertFalse(sampleFuture.isDone()); - } - } - - // Verify completed stack trace sample - StackTraceSample sample = sampleFuture.get(); - - Assert.assertEquals(0, sample.getSampleId()); - Assert.assertTrue(sample.getEndTime() >= sample.getStartTime()); - - Map> tracesByTask = sample.getStackTraces(); - - for (ExecutionVertex vertex : vertices) { - ExecutionAttemptID executionId = vertex.getCurrentExecutionAttempt().getAttemptId(); - List sampleTraces = tracesByTask.get(executionId); - - Assert.assertNotNull("Task not found", sampleTraces); - Assert.assertTrue(traces.equals(sampleTraces)); - } - - // Verify no more pending sample - Assert.assertEquals(0, coord.getNumberOfPendingSamples()); - - // Verify no error on late collect - coord.collectStackTraces(0, vertices[0].getCurrentExecutionAttempt().getAttemptId(), traces); - } - - /** Tests triggering for non-running tasks fails the future. */ - @Test - public void testTriggerStackTraceSampleNotRunningTasks() throws Exception { - ExecutionVertex[] vertices = new ExecutionVertex[] { - mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true), - mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.DEPLOYING, true) - }; - - CompletableFuture sampleFuture = coord.triggerStackTraceSample( - vertices, - 1, - Time.milliseconds(100L), - 0); - - Assert.assertTrue(sampleFuture.isDone()); - - try { - sampleFuture.get(); - Assert.fail("Expected exception."); - } catch (ExecutionException e) { - Assert.assertTrue(e.getCause() instanceof IllegalStateException); - } - } - - /** Tests triggering for reset tasks fails the future. */ - @Test(timeout = 1000L) - public void testTriggerStackTraceSampleResetRunningTasks() throws Exception { - ExecutionVertex[] vertices = new ExecutionVertex[] { - mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true), - // Fails to send the message to the execution (happens when execution is reset) - mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, false) - }; - - CompletableFuture sampleFuture = coord.triggerStackTraceSample( - vertices, - 1, - Time.milliseconds(100L), - 0); - - try { - sampleFuture.get(); - Assert.fail("Expected exception."); - } catch (ExecutionException e) { - Assert.assertTrue(e.getCause() instanceof RuntimeException); - } - } - - /** Tests that samples time out if they don't finish in time. */ - @Test(timeout = 1000L) - public void testTriggerStackTraceSampleTimeout() throws Exception { - int timeout = 100; - - coord = new StackTraceSampleCoordinator(system.dispatcher(), timeout); - - final ScheduledExecutorService scheduledExecutorService = new ScheduledThreadPoolExecutor(1); - - try { - - ExecutionVertex[] vertices = new ExecutionVertex[]{ - mockExecutionVertexWithTimeout( - new ExecutionAttemptID(), - ExecutionState.RUNNING, - scheduledExecutorService, - timeout) - }; - - CompletableFuture sampleFuture = coord.triggerStackTraceSample( - vertices, 1, Time.milliseconds(100L), 0); - - // Wait for the timeout - Thread.sleep(timeout * 2); - - boolean success = false; - for (int i = 0; i < 10; i++) { - if (sampleFuture.isDone()) { - success = true; - break; - } - - Thread.sleep(timeout); - } - - Assert.assertTrue("Sample did not time out", success); - - try { - sampleFuture.get(); - Assert.fail("Expected exception."); - } catch (ExecutionException e) { - Assert.assertTrue(e.getCause().getCause().getMessage().contains("Timeout")); - } - - // Collect after the timeout (should be ignored) - ExecutionAttemptID executionId = vertices[0].getCurrentExecutionAttempt().getAttemptId(); - coord.collectStackTraces(0, executionId, new ArrayList()); - } finally { - scheduledExecutorService.shutdownNow(); - } - } - - /** Tests that collecting an unknown sample is ignored. */ - @Test - public void testCollectStackTraceForUnknownSample() throws Exception { - coord.collectStackTraces(0, new ExecutionAttemptID(), new ArrayList()); - } - - /** Tests cancelling of a pending sample. */ - @Test - public void testCancelStackTraceSample() throws Exception { - ExecutionVertex[] vertices = new ExecutionVertex[] { - mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true), - }; - - CompletableFuture sampleFuture = coord.triggerStackTraceSample( - vertices, 1, Time.milliseconds(100L), 0); - - Assert.assertFalse(sampleFuture.isDone()); - - // Cancel - coord.cancelStackTraceSample(0, null); - - // Verify completed - Assert.assertTrue(sampleFuture.isDone()); - - // Verify no more pending samples - Assert.assertEquals(0, coord.getNumberOfPendingSamples()); - } - - /** Tests that collecting for a cancelled sample throws no Exception. */ - @Test - public void testCollectStackTraceForCanceledSample() throws Exception { - ExecutionVertex[] vertices = new ExecutionVertex[] { - mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true), - }; - - CompletableFuture sampleFuture = coord.triggerStackTraceSample( - vertices, 1, Time.milliseconds(100L), 0); - - Assert.assertFalse(sampleFuture.isDone()); - - coord.cancelStackTraceSample(0, null); - - Assert.assertTrue(sampleFuture.isDone()); - - // Verify no error on late collect - ExecutionAttemptID executionId = vertices[0].getCurrentExecutionAttempt().getAttemptId(); - coord.collectStackTraces(0, executionId, new ArrayList()); - } - - /** Tests that collecting for a cancelled sample throws no Exception. */ - @Test - public void testCollectForDiscardedPendingSample() throws Exception { - ExecutionVertex[] vertices = new ExecutionVertex[] { - mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true), - }; - - CompletableFuture sampleFuture = coord.triggerStackTraceSample( - vertices, 1, Time.milliseconds(100L), 0); - - Assert.assertFalse(sampleFuture.isDone()); - - coord.cancelStackTraceSample(0, null); - - Assert.assertTrue(sampleFuture.isDone()); - - // Verify no error on late collect - ExecutionAttemptID executionId = vertices[0].getCurrentExecutionAttempt().getAttemptId(); - coord.collectStackTraces(0, executionId, new ArrayList()); - } - - - /** Tests that collecting for a unknown task fails. */ - @Test(expected = IllegalArgumentException.class) - public void testCollectStackTraceForUnknownTask() throws Exception { - ExecutionVertex[] vertices = new ExecutionVertex[] { - mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true), - }; - - coord.triggerStackTraceSample(vertices, 1, Time.milliseconds(100L), 0); - - coord.collectStackTraces(0, new ExecutionAttemptID(), new ArrayList()); - } - - /** Tests that shut down fails all pending samples and future sample triggers. */ - @Test - public void testShutDown() throws Exception { - ExecutionVertex[] vertices = new ExecutionVertex[] { - mockExecutionVertex(new ExecutionAttemptID(), ExecutionState.RUNNING, true), - }; - - List> sampleFutures = new ArrayList<>(); - - // Trigger - sampleFutures.add(coord.triggerStackTraceSample( - vertices, 1, Time.milliseconds(100L), 0)); - - sampleFutures.add(coord.triggerStackTraceSample( - vertices, 1, Time.milliseconds(100L), 0)); - - for (CompletableFuture future : sampleFutures) { - Assert.assertFalse(future.isDone()); - } - - // Shut down - coord.shutDown(); - - // Verify all completed - for (CompletableFuture future : sampleFutures) { - Assert.assertTrue(future.isDone()); - } - - // Verify new trigger returns failed future - CompletableFuture future = coord.triggerStackTraceSample( - vertices, 1, Time.milliseconds(100L), 0); - - Assert.assertTrue(future.isDone()); - - try { - future.get(); - Assert.fail("Expected exception."); - } catch (ExecutionException e) { - // we expected an exception here :-) - } - - } - - // ------------------------------------------------------------------------ - - private ExecutionVertex mockExecutionVertex( - ExecutionAttemptID executionId, - ExecutionState state, - boolean sendSuccess) { - - Execution exec = Mockito.mock(Execution.class); - CompletableFuture failedFuture = new CompletableFuture<>(); - failedFuture.completeExceptionally(new Exception("Send failed.")); - - Mockito.when(exec.getAttemptId()).thenReturn(executionId); - Mockito.when(exec.getState()).thenReturn(state); - Mockito.when(exec.requestStackTraceSample(Matchers.anyInt(), Matchers.anyInt(), Matchers.any(Time.class), Matchers.anyInt(), Matchers.any(Time.class))) - .thenReturn( - sendSuccess ? - CompletableFuture.completedFuture(Mockito.mock(StackTraceSampleResponse.class)) : - failedFuture); - - ExecutionVertex vertex = Mockito.mock(ExecutionVertex.class); - Mockito.when(vertex.getJobvertexId()).thenReturn(new JobVertexID()); - Mockito.when(vertex.getCurrentExecutionAttempt()).thenReturn(exec); - - return vertex; - } - - private ExecutionVertex mockExecutionVertexWithTimeout( - ExecutionAttemptID executionId, - ExecutionState state, - ScheduledExecutorService scheduledExecutorService, - int timeout) { - - final CompletableFuture future = new CompletableFuture<>(); - - Execution exec = Mockito.mock(Execution.class); - Mockito.when(exec.getAttemptId()).thenReturn(executionId); - Mockito.when(exec.getState()).thenReturn(state); - Mockito.when(exec.requestStackTraceSample(Matchers.anyInt(), Matchers.anyInt(), Matchers.any(Time.class), Matchers.anyInt(), Matchers.any(Time.class))) - .thenReturn(future); - - scheduledExecutorService.schedule(new Runnable() { - @Override - public void run() { - future.completeExceptionally(new TimeoutException("Timeout")); - } - }, timeout, TimeUnit.MILLISECONDS); - - ExecutionVertex vertex = Mockito.mock(ExecutionVertex.class); - Mockito.when(vertex.getJobvertexId()).thenReturn(new JobVertexID()); - Mockito.when(vertex.getCurrentExecutionAttempt()).thenReturn(exec); - - return vertex; - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/BackPressureSampleServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/BackPressureSampleServiceTest.java new file mode 100644 index 000000000000..d6e79e8c7490 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/BackPressureSampleServiceTest.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.taskexecutor; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.concurrent.ScheduledExecutor; +import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter; +import org.apache.flink.util.TestLogger; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +/** + * Tests for {@link BackPressureSampleService}. + */ +public class BackPressureSampleServiceTest extends TestLogger { + + private static ScheduledExecutorService scheduledExecutorService; + + private static BackPressureSampleService backPressureSampleService; + + @BeforeClass + public static void setUp() throws Exception { + scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(); + final ScheduledExecutor scheduledExecutor = new ScheduledExecutorServiceAdapter(scheduledExecutorService); + + backPressureSampleService = new BackPressureSampleService(10, Time.milliseconds(10), scheduledExecutor); + } + + @AfterClass + public static void tearDown() throws Exception { + if (scheduledExecutorService != null) { + scheduledExecutorService.shutdown(); + } + } + + @Test(timeout = 10000L) + public void testSampleTaskBackPressure() throws Exception { + final double backPressureRatio = backPressureSampleService. + sampleTaskBackPressure(new TestTask()).get(); + + assertEquals(0.5, backPressureRatio, 0.0); + } + + @Test(timeout = 10000L) + public void testTaskStopsWithPartialSampling() throws Exception { + final double backPressureRatio = backPressureSampleService. + sampleTaskBackPressure(new NotRunningAfterFirstSamplingTask()).get(); + + assertEquals(1.0, backPressureRatio, 0.0); + } + + @Test(expected = IllegalStateException.class) + public void testShouldThrowExceptionIfTaskIsNotRunningBeforeSampling() { + backPressureSampleService.sampleTaskBackPressure(new NeverRunningTask()); + + fail("Exception expected."); + } + + /** + * Task that is always running. + */ + private static class TestTask implements BackPressureSampleableTask { + + protected volatile long counter = 0; + + @Override + public boolean isRunning() { + return true; + } + + @Override + public boolean isBackPressured() { + return counter++ % 2 == 0; + } + } + + /** + * Task that stops running after sampled for the first time. + */ + private static class NotRunningAfterFirstSamplingTask extends TestTask { + + @Override + public boolean isRunning() { + return counter == 0; + } + } + + /** + * Task that never runs. + */ + private static class NeverRunningTask extends TestTask { + + @Override + public boolean isRunning() { + return false; + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/StackTraceSampleServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/StackTraceSampleServiceTest.java deleted file mode 100644 index 1d3e83e064b2..000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/StackTraceSampleServiceTest.java +++ /dev/null @@ -1,189 +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.taskexecutor; - -import org.apache.flink.api.common.time.Time; -import org.apache.flink.runtime.concurrent.ScheduledExecutor; -import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter; -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; -import org.apache.flink.util.ExecutorUtils; -import org.apache.flink.util.TestLogger; - -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import java.util.List; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; - -import static org.hamcrest.Matchers.arrayWithSize; -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.hasSize; -import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.lessThan; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.fail; - -/** - * Tests for {@link StackTraceSampleService}. - */ -public class StackTraceSampleServiceTest extends TestLogger { - - private ScheduledExecutorService scheduledExecutorService; - - private StackTraceSampleService stackTraceSampleService; - - @Before - public void setUp() throws Exception { - scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(); - final ScheduledExecutor scheduledExecutor = new ScheduledExecutorServiceAdapter(scheduledExecutorService); - - stackTraceSampleService = new StackTraceSampleService(scheduledExecutor); - } - - @After - public void tearDown() throws Exception { - if (scheduledExecutorService != null) { - ExecutorUtils.gracefulShutdown(10, TimeUnit.SECONDS, scheduledExecutorService); - } - } - - @Test - public void testShouldReturnStackTraces() throws Exception { - final int numSamples = 10; - final List stackTraces = stackTraceSampleService.requestStackTraceSample( - new TestTask(), - numSamples, - Time.milliseconds(0), - -1).get(); - - assertThat(stackTraces, hasSize(numSamples)); - final StackTraceElement[] firstStackTrace = stackTraces.get(0); - assertThat(firstStackTrace[1].getClassName(), is(equalTo((TestTask.class.getName())))); - } - - @Test - public void testShouldThrowExceptionIfNumSamplesIsNegative() { - try { - stackTraceSampleService.requestStackTraceSample( - new TestTask(), - -1, - Time.milliseconds(0), - 10); - fail("Expected exception not thrown"); - } catch (final IllegalArgumentException e) { - assertThat(e.getMessage(), is(equalTo("numSamples must be positive"))); - } - } - - @Test - public void testShouldTruncateStackTraceIfLimitIsSpecified() throws Exception { - final int maxStackTraceDepth = 1; - final List stackTraces = stackTraceSampleService.requestStackTraceSample( - new TestTask(), - 10, - Time.milliseconds(0), - maxStackTraceDepth).get(); - - assertThat(stackTraces.get(0), is(arrayWithSize(maxStackTraceDepth))); - } - - @Test - public void testShouldReturnPartialResultIfTaskStopsRunningDuringSampling() throws Exception { - final List stackTraces = stackTraceSampleService.requestStackTraceSample( - new NotRunningAfterBeingSampledTask(), - 10, - Time.milliseconds(0), - 1).get(); - - assertThat(stackTraces, hasSize(lessThan(10))); - } - - @Test - public void testShouldThrowExceptionIfTaskIsNotRunningBeforeSampling() { - try { - stackTraceSampleService.requestStackTraceSample( - new NotRunningTask(), - 10, - Time.milliseconds(0), - -1); - fail("Expected exception not thrown"); - } catch (final IllegalStateException e) { - assertThat(e.getMessage(), containsString("Cannot sample task")); - } - } - - /** - * Task that is always running. - */ - private static class TestTask implements StackTraceSampleableTask { - - private final ExecutionAttemptID executionAttemptID = new ExecutionAttemptID(); - - @Override - public boolean isRunning() { - return true; - } - - @Override - public StackTraceElement[] getStackTrace() { - return Thread.currentThread().getStackTrace(); - } - - @Override - public ExecutionAttemptID getExecutionId() { - return executionAttemptID; - } - } - - /** - * Task that stops running after being sampled for the first time. - */ - private static class NotRunningAfterBeingSampledTask extends TestTask { - - private volatile boolean stackTraceSampled; - - @Override - public boolean isRunning() { - return !stackTraceSampled; - } - - @Override - public StackTraceElement[] getStackTrace() { - stackTraceSampled = true; - return super.getStackTrace(); - } - } - - /** - * Task that never runs. - */ - private static class NotRunningTask extends TestTask { - - @Override - public boolean isRunning() { - return false; - } - - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java index 41f4ba86ccf4..56361dfcd86e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java @@ -493,20 +493,22 @@ public void invoke() throws Exception { } private TestingTaskExecutor createTestingTaskExecutor(TaskManagerServices taskManagerServices, TaskExecutorPartitionTracker partitionTracker, String metricQueryServiceAddress) throws IOException { + Configuration configuration = new Configuration(); return new TestingTaskExecutor( RPC, - TaskManagerConfiguration.fromConfiguration(new Configuration()), + TaskManagerConfiguration.fromConfiguration(configuration), haServices, taskManagerServices, new HeartbeatServices(10_000L, 30_000L), UnregisteredMetricGroups.createUnregisteredTaskManagerMetricGroup(), metricQueryServiceAddress, new BlobCacheService( - new Configuration(), + configuration, new VoidBlobStore(), null), new TestingFatalErrorHandler(), - partitionTracker); + partitionTracker, + TaskManagerRunner.createBackPressureSampleService(configuration, RPC.getScheduledExecutor())); } private static TaskSlotTable createTaskSlotTable() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java index c0f8632c835b..68fd372db361 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java @@ -24,6 +24,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.configuration.WebOptions; import org.apache.flink.runtime.blob.PermanentBlobKey; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -40,6 +41,7 @@ import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; import org.apache.flink.runtime.metrics.MetricRegistryImpl; import org.apache.flink.runtime.rpc.TestingRpcService; +import org.apache.flink.runtime.messages.TaskBackPressureResponse; import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; @@ -52,13 +54,13 @@ import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway; import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.messages.StackTraceSampleResponse; import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor; import org.apache.flink.runtime.shuffle.PartitionDescriptor; import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable; import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.testtasks.BlockingNoOpInvokable; +import org.apache.flink.runtime.testtasks.OutputBlockedInvokable; import org.apache.flink.runtime.util.NettyShuffleDescriptorBuilder; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.NetUtils; @@ -75,14 +77,13 @@ import java.io.IOException; import java.net.URL; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import static org.apache.flink.runtime.util.NettyShuffleDescriptorBuilder.createRemoteWithIdAndLocation; +import static org.apache.flink.runtime.util.NettyShuffleDescriptorBuilder.newBuilder; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.CoreMatchers.startsWith; import static org.hamcrest.Matchers.instanceOf; @@ -597,139 +598,88 @@ public void testFailingScheduleOrUpdateConsumers() throws Exception { } // ------------------------------------------------------------------------ - // Stack trace sample + // Back pressure request // ------------------------------------------------------------------------ /** - * Tests sampling of task stack traces. + * Tests request of task back pressure. */ - @Test(timeout = 10000L) - @SuppressWarnings("unchecked") - public void testRequestStackTraceSample() throws Exception { - final ExecutionAttemptID eid = new ExecutionAttemptID(); - final TaskDeploymentDescriptor tdd = createTestTaskDeploymentDescriptor("test task", eid, BlockingNoOpInvokable.class); - - final int sampleId1 = 112223; - final int sampleId2 = 19230; - final int sampleId3 = 1337; - final int sampleId4 = 44; + @Test(timeout = 20000L) + public void testRequestTaskBackPressure() throws Exception { + final NettyShuffleDescriptor shuffleDescriptor = newBuilder().buildLocal(); + final TaskDeploymentDescriptor tdd = createSender(shuffleDescriptor, OutputBlockedInvokable.class); + final ExecutionAttemptID executionAttemptID = tdd.getExecutionAttemptId(); final CompletableFuture taskRunningFuture = new CompletableFuture<>(); final CompletableFuture taskCanceledFuture = new CompletableFuture<>(); - try (TaskSubmissionTestEnvironment env = - new TaskSubmissionTestEnvironment.Builder(jobId) - .setSlotSize(1) - .setMetricQueryServiceAddress(metricQueryServiceAddress) - .addTaskManagerActionListener(eid, ExecutionState.RUNNING, taskRunningFuture) - .addTaskManagerActionListener(eid, ExecutionState.CANCELED, taskCanceledFuture) - .build()) { - TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); - TaskSlotTable taskSlotTable = env.getTaskSlotTable(); + final Configuration configuration = new Configuration(); + configuration.set(WebOptions.BACKPRESSURE_NUM_SAMPLES, 40); + configuration.setString(TaskManagerOptions.MEMORY_SEGMENT_SIZE, "4096"); + + try (final TaskSubmissionTestEnvironment env = new TaskSubmissionTestEnvironment.Builder(jobId) + .setSlotSize(1) + .setMetricQueryServiceAddress(metricQueryServiceAddress) + .setConfiguration(configuration) + .useRealNonMockShuffleEnvironment() + .addTaskManagerActionListener(executionAttemptID, ExecutionState.RUNNING, taskRunningFuture) + .addTaskManagerActionListener(executionAttemptID, ExecutionState.CANCELED, taskCanceledFuture) + .build()) { + final TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); + final TaskSlotTable taskSlotTable = env.getTaskSlotTable(); taskSlotTable.allocateSlot(0, jobId, tdd.getAllocationId(), Time.seconds(60)); tmGateway.submitTask(tdd, env.getJobMasterId(), timeout).get(); taskRunningFuture.get(); - // - // 1) Trigger sample for non-existing task - // - ExecutionAttemptID nonExistTaskEid = new ExecutionAttemptID(); + // 1) trigger request for non-existing task. + final int requestId = 1234; + final ExecutionAttemptID nonExistTaskEid = new ExecutionAttemptID(); - CompletableFuture failedSampleFuture = - tmGateway.requestStackTraceSample(nonExistTaskEid, sampleId1, 100, Time.seconds(60L), 0, timeout); + final CompletableFuture failedRequestFuture = + tmGateway.requestTaskBackPressure(nonExistTaskEid, requestId, timeout); try { - failedSampleFuture.get(); + failedRequestFuture.get(); } catch (Exception e) { assertThat(e.getCause(), instanceOf(IllegalStateException.class)); - assertThat(e.getCause().getMessage(), startsWith("Cannot sample task")); + assertThat(e.getCause().getMessage(), startsWith("Cannot request back pressure")); } - // - // 2) Trigger sample for the blocking task - // - int numSamples = 5; + // 2) trigger request for the blocking task. + double backPressureRatio = 0; - CompletableFuture successfulSampleFuture = - tmGateway.requestStackTraceSample(eid, sampleId2, numSamples, Time.milliseconds(100L), 0, timeout); + for (int i = 0; i < 5; ++i) { + CompletableFuture successfulRequestFuture = + tmGateway.requestTaskBackPressure(executionAttemptID, i, timeout); - StackTraceSampleResponse response = successfulSampleFuture.get(); + TaskBackPressureResponse response = successfulRequestFuture.get(); - assertEquals(response.getSampleId(), sampleId2); - assertEquals(response.getExecutionAttemptID(), eid); + assertEquals(response.getRequestId(), i); + assertEquals(response.getExecutionAttemptID(), executionAttemptID); - List traces = response.getSamples(); - - assertEquals("Number of samples", numSamples, traces.size()); - - for (StackTraceElement[] trace : traces) { - boolean success = false; - for (StackTraceElement elem : trace) { - // Look for BlockingNoOpInvokable#invoke - if (elem.getClassName().equals( - BlockingNoOpInvokable.class.getName())) { - - assertEquals("invoke", elem.getMethodName()); - - success = true; - break; - } - // The BlockingNoOpInvokable might not be invoked here - if (elem.getClassName().equals(TestTaskManagerActions.class.getName())) { - - assertEquals("updateTaskExecutionState", elem.getMethodName()); - - success = true; - break; - } - if (elem.getClassName().equals(Thread.class) && elem.getMethodName().equals("setContextClassLoader")) { - success = true; - } + if ((backPressureRatio = response.getBackPressureRatio()) >= 1.0) { + break; } - - assertTrue("Unexpected stack trace: " + - Arrays.toString(trace), success); } - // - // 3) Trigger sample for the blocking task with max depth - // - int maxDepth = 2; - - CompletableFuture successfulSampleFutureWithMaxDepth = - tmGateway.requestStackTraceSample(eid, sampleId3, numSamples, Time.milliseconds(100L), maxDepth, timeout); - - StackTraceSampleResponse responseWithMaxDepth = successfulSampleFutureWithMaxDepth.get(); - - assertEquals(sampleId3, responseWithMaxDepth.getSampleId()); - assertEquals(eid, responseWithMaxDepth.getExecutionAttemptID()); - - List tracesWithMaxDepth = responseWithMaxDepth.getSamples(); - - assertEquals("Number of samples", numSamples, tracesWithMaxDepth.size()); - - for (StackTraceElement[] trace : tracesWithMaxDepth) { - assertEquals("Max depth", maxDepth, trace.length); - } + assertEquals("Task was not back pressured in given time.", 1.0, backPressureRatio, 0.0); - // - // 4) Trigger sample for the blocking task, but cancel it during sampling - // - int sleepTime = 100; - numSamples = 100; + // 3) trigger request for the blocking task, but cancel it before request finishes. + final int sleepTime = 1000; - CompletableFuture canceldSampleFuture = - tmGateway.requestStackTraceSample(eid, sampleId4, numSamples, Time.milliseconds(10L), maxDepth, timeout); + CompletableFuture canceledRequestFuture = + tmGateway.requestTaskBackPressure(executionAttemptID, requestId, timeout); Thread.sleep(sleepTime); - tmGateway.cancelTask(eid, timeout); + tmGateway.cancelTask(executionAttemptID, timeout); taskCanceledFuture.get(); - StackTraceSampleResponse responseAfterCancel = canceldSampleFuture.get(); + TaskBackPressureResponse responseAfterCancel = canceledRequestFuture.get(); - assertEquals(eid, responseAfterCancel.getExecutionAttemptID()); - assertEquals(sampleId4, responseAfterCancel.getSampleId()); + assertEquals(executionAttemptID, responseAfterCancel.getExecutionAttemptID()); + assertEquals(requestId, responseAfterCancel.getRequestId()); + assertTrue(responseAfterCancel.getBackPressureRatio() > 0); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index 0e289d010878..f61cc6de8a24 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -1934,7 +1934,8 @@ private TaskExecutor createTaskExecutor(TaskManagerServices taskManagerServices, metricQueryServiceAddress, dummyBlobCacheService, testingFatalErrorHandler, - new TaskExecutorPartitionTrackerImpl(taskManagerServices.getShuffleEnvironment())); + new TaskExecutorPartitionTrackerImpl(taskManagerServices.getShuffleEnvironment()), + TaskManagerRunner.createBackPressureSampleService(configuration, rpc.getScheduledExecutor())); } private TestingTaskExecutor createTestingTaskExecutor(TaskManagerServices taskManagerServices) { @@ -1952,7 +1953,8 @@ private TestingTaskExecutor createTestingTaskExecutor(TaskManagerServices taskMa metricQueryServiceAddress, dummyBlobCacheService, testingFatalErrorHandler, - new TaskExecutorPartitionTrackerImpl(taskManagerServices.getShuffleEnvironment())); + new TaskExecutorPartitionTrackerImpl(taskManagerServices.getShuffleEnvironment()), + TaskManagerRunner.createBackPressureSampleService(configuration, rpc.getScheduledExecutor())); } private static final class StartStopNotifyingLeaderRetrievalService implements LeaderRetrievalService { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java index 45dcef39e283..902f0c60d5ed 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java @@ -208,8 +208,8 @@ private TestingTaskExecutor createTaskExecutor(TaskManagerServices taskManagerSe metricQueryServiceAddress, blobCacheService, testingFatalErrorHandler, - new TaskExecutorPartitionTrackerImpl(taskManagerServices.getShuffleEnvironment()) - ); + new TaskExecutorPartitionTrackerImpl(taskManagerServices.getShuffleEnvironment()), + TaskManagerRunner.createBackPressureSampleService(configuration, testingRpcService.getScheduledExecutor())); } static JobManagerConnection createJobManagerConnection(JobID jobId, JobMasterGateway jobMasterGateway, RpcService testingRpcService, TaskManagerActions taskManagerActions, Time timeout) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutor.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutor.java index 354a1304ae4f..31e9b7593334 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutor.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutor.java @@ -45,7 +45,8 @@ public TestingTaskExecutor( String metricQueryServiceAddress, BlobCacheService blobCacheService, FatalErrorHandler fatalErrorHandler, - TaskExecutorPartitionTracker partitionTracker) { + TaskExecutorPartitionTracker partitionTracker, + BackPressureSampleService backPressureSampleService) { super( rpcService, taskManagerConfiguration, @@ -56,7 +57,8 @@ public TestingTaskExecutor( metricQueryServiceAddress, blobCacheService, fatalErrorHandler, - partitionTracker); + partitionTracker, + backPressureSampleService); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGateway.java index ac8e3f088c52..ea9519346485 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGateway.java @@ -34,8 +34,9 @@ import org.apache.flink.runtime.jobmaster.AllocatedSlotReport; import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.messages.StackTraceSampleResponse; +import org.apache.flink.runtime.messages.TaskBackPressureResponse; import org.apache.flink.runtime.resourcemanager.ResourceManagerId; +import org.apache.flink.runtime.rpc.RpcTimeout; import org.apache.flink.types.SerializableOptional; import org.apache.flink.util.Preconditions; import org.apache.flink.util.function.TriConsumer; @@ -110,13 +111,7 @@ public CompletableFuture requestSlot(SlotID slotId, JobID jobId, Al } @Override - public CompletableFuture requestStackTraceSample( - final ExecutionAttemptID executionAttemptId, - final int sampleId, - final int numSamples, - final Time delayBetweenSamples, - final int maxStackTraceDepth, - final Time timeout) { + public CompletableFuture requestTaskBackPressure(ExecutionAttemptID executionAttemptId, int requestId, @RpcTimeout Time timeout) { throw new UnsupportedOperationException(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java index 30c96cd93be3..b34a77367ceb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java @@ -923,10 +923,9 @@ public void testTerminationFutureCompletesOnErrorInInvoke() throws Exception { } @Test - public void testReturnsEmptyStackTraceIfTaskNotStarted() throws Exception { + public void testNoBackPressureIfTaskNotStarted() throws Exception { final Task task = createTaskBuilder().build(); - final StackTraceElement[] actualStackTrace = task.getStackTraceOfExecutingThread(); - assertEquals(0, actualStackTrace.length); + assertFalse(task.isBackPressured()); } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/OutputBlockedInvokable.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/OutputBlockedInvokable.java new file mode 100644 index 000000000000..09d8284d5867 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/OutputBlockedInvokable.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.testtasks; + +import org.apache.flink.runtime.execution.Environment; +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.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.types.IntValue; + +/** + * A simple task that emits int value until the result partition is unavailable for output. + */ +public class OutputBlockedInvokable extends AbstractInvokable { + + public OutputBlockedInvokable(Environment environment) { + super(environment); + } + + @Override + public void invoke() throws Exception { + final IntValue value = new IntValue(1234); + final ResultPartitionWriter resultPartitionWriter = getEnvironment().getWriter(0); + final RecordWriter writer = new RecordWriterBuilder().build(resultPartitionWriter); + + while (true) { + writer.emit(value); + } + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java index 85e285e96e91..3507bb5e5d08 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java @@ -28,7 +28,6 @@ import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; import org.apache.flink.util.FlinkException; -import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; @@ -143,11 +142,6 @@ protected void finishTask() throws Exception { cancelTask(); } - @Override - public Optional getExecutingThread() { - return Optional.of(sourceThread); - } - // ------------------------------------------------------------------------ // Checkpointing // ------------------------------------------------------------------------ From 60f7aa03b5127cb1e100ea4e039eced0416fa536 Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Wed, 13 Nov 2019 17:15:34 +0800 Subject: [PATCH 620/746] [FLINK-14708][runtime] Introduce RestartAllStrategy for NG scheduler --- .../failover/flip1/RestartAllStrategy.java | 66 +++++++++++++++++++ .../flip1/RestartAllStrategyTest.java | 55 ++++++++++++++++ 2 files changed, 121 insertions(+) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartAllStrategy.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartAllStrategyTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartAllStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartAllStrategy.java new file mode 100644 index 000000000000..dda7baefa9a9 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartAllStrategy.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.runtime.executiongraph.failover.flip1; + +import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.util.IterableUtils; + +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A failover strategy that proposes to restart all vertices when a vertex fails. + */ +public class RestartAllStrategy implements FailoverStrategy { + + private final FailoverTopology topology; + + public RestartAllStrategy(final FailoverTopology topology) { + this.topology = checkNotNull(topology); + } + + /** + * Returns all vertices on any task failure. + * + * @param executionVertexId ID of the failed task + * @param cause cause of the failure + * @return set of IDs of vertices to restart + */ + @Override + public Set getTasksNeedingRestart(ExecutionVertexID executionVertexId, Throwable cause) { + return IterableUtils.toStream(topology.getVertices()) + .map(FailoverVertex::getId) + .collect(Collectors.toSet()); + } + + /** + * The factory to instantiate {@link RestartAllStrategy}. + */ + public static class Factory implements FailoverStrategy.Factory { + + @Override + public FailoverStrategy create( + final FailoverTopology topology, + final ResultPartitionAvailabilityChecker resultPartitionAvailabilityChecker) { + + return new RestartAllStrategy(topology); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartAllStrategyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartAllStrategyTest.java new file mode 100644 index 000000000000..a7149134e060 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/RestartAllStrategyTest.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.executiongraph.failover.flip1; + +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import java.util.Arrays; +import java.util.HashSet; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link RestartAllStrategy}. + */ +public class RestartAllStrategyTest extends TestLogger { + + @Test + public void testGetTasksNeedingRestart() { + final TestFailoverTopology.Builder topologyBuilder = new TestFailoverTopology.Builder(); + + final TestFailoverTopology.TestFailoverVertex v1 = topologyBuilder.newVertex(); + final TestFailoverTopology.TestFailoverVertex v2 = topologyBuilder.newVertex(); + final TestFailoverTopology.TestFailoverVertex v3 = topologyBuilder.newVertex(); + + topologyBuilder.connect(v1, v2, ResultPartitionType.PIPELINED); + topologyBuilder.connect(v2, v3, ResultPartitionType.BLOCKING); + + final TestFailoverTopology topology = topologyBuilder.build(); + + final RestartAllStrategy strategy = new RestartAllStrategy(topology); + + assertEquals( + new HashSet<>(Arrays.asList(v1.getId(), v2.getId(), v3.getId())), + strategy.getTasksNeedingRestart(v1.getId(), new Exception("Test failure"))); + } +} From 56c46646a51d5c3a4cb0347fb97f5d0807922aeb Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Wed, 13 Nov 2019 17:48:13 +0800 Subject: [PATCH 621/746] [FLINK-14131][runtime] Introduce FailoverStrategyFactoryLoader to load NG failover strategy factories --- .../flip1/FailoverStrategyFactoryLoader.java | 69 ++++++++++++++++++ .../FailoverStrategyFactoryLoaderTest.java | 72 +++++++++++++++++++ 2 files changed, 141 insertions(+) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverStrategyFactoryLoader.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverStrategyFactoryLoaderTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverStrategyFactoryLoader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverStrategyFactoryLoader.java new file mode 100644 index 000000000000..f0318b340cdd --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverStrategyFactoryLoader.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.executiongraph.failover.flip1; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.configuration.JobManagerOptions; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A utility class to load NG failover strategy factories from the configuration. + */ +public final class FailoverStrategyFactoryLoader { + + /** Config name for the {@link RestartAllStrategy}. */ + public static final String FULL_RESTART_STRATEGY_NAME = "full"; + + /** Config name for the {@link RestartPipelinedRegionStrategy}. */ + public static final String PIPELINED_REGION_RESTART_STRATEGY_NAME = "region"; + + private FailoverStrategyFactoryLoader() { + } + + /** + * Loads a {@link FailoverStrategy.Factory} from the given configuration. + * + * @param config which specifies the failover strategy factory to load + * @return failover strategy factory loaded + */ + public static FailoverStrategy.Factory loadFailoverStrategyFactory(final Configuration config) { + checkNotNull(config); + + // the default NG failover strategy is the region failover strategy. + // TODO: Remove the overridden default value when removing legacy scheduler + // and change the default value of JobManagerOptions.EXECUTION_FAILOVER_STRATEGY + // to be "region" + final String strategyParam = config.getString( + JobManagerOptions.EXECUTION_FAILOVER_STRATEGY, + PIPELINED_REGION_RESTART_STRATEGY_NAME); + + switch (strategyParam.toLowerCase()) { + case FULL_RESTART_STRATEGY_NAME: + return new RestartAllStrategy.Factory(); + + case PIPELINED_REGION_RESTART_STRATEGY_NAME: + return new RestartPipelinedRegionStrategy.Factory(); + + default: + throw new IllegalConfigurationException("Unknown failover strategy: " + strategyParam); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverStrategyFactoryLoaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverStrategyFactoryLoaderTest.java new file mode 100644 index 000000000000..40014a644e61 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/failover/flip1/FailoverStrategyFactoryLoaderTest.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.runtime.executiongraph.failover.flip1; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import static org.hamcrest.Matchers.instanceOf; +import static org.junit.Assert.assertThat; + +/** + * Tests for {@link FailoverStrategyFactoryLoader}. + */ +public class FailoverStrategyFactoryLoaderTest extends TestLogger { + + @Test + public void testLoadRestartAllStrategyFactory() { + final Configuration config = new Configuration(); + config.setString( + JobManagerOptions.EXECUTION_FAILOVER_STRATEGY, + FailoverStrategyFactoryLoader.FULL_RESTART_STRATEGY_NAME); + assertThat( + FailoverStrategyFactoryLoader.loadFailoverStrategyFactory(config), + instanceOf(RestartAllStrategy.Factory.class)); + } + + @Test + public void testLoadRestartPipelinedRegionStrategyFactory() { + final Configuration config = new Configuration(); + config.setString( + JobManagerOptions.EXECUTION_FAILOVER_STRATEGY, + FailoverStrategyFactoryLoader.PIPELINED_REGION_RESTART_STRATEGY_NAME); + assertThat( + FailoverStrategyFactoryLoader.loadFailoverStrategyFactory(config), + instanceOf(RestartPipelinedRegionStrategy.Factory.class)); + } + + @Test + public void testDefaultFailoverStrategyIsRegion() { + final Configuration config = new Configuration(); + assertThat( + FailoverStrategyFactoryLoader.loadFailoverStrategyFactory(config), + instanceOf(RestartPipelinedRegionStrategy.Factory.class)); + } + + @Test(expected = IllegalConfigurationException.class) + public void testLoadFromInvalidConfiguration() { + final Configuration config = new Configuration(); + config.setString(JobManagerOptions.EXECUTION_FAILOVER_STRATEGY, "invalidStrategy"); + FailoverStrategyFactoryLoader.loadFailoverStrategyFactory(config); + } +} From 6abc6d96e1d03354eaa861fee2665dc25f2c0cba Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Wed, 13 Nov 2019 17:54:51 +0800 Subject: [PATCH 622/746] [FLINK-14131][runtime] DefaultSchedulerFactory uses FailoverStrategyFactoryLoader to load failover strategy factory --- .../apache/flink/runtime/scheduler/DefaultScheduler.java | 7 ++++++- .../flink/runtime/scheduler/DefaultSchedulerFactory.java | 4 ++-- 2 files changed, 8 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 f139edcecf1b..485b1d96c96d 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 @@ -139,9 +139,14 @@ public DefaultScheduler( this.executionVertexOperations = checkNotNull(executionVertexOperations); this.executionVertexVersioner = checkNotNull(executionVertexVersioner); + final FailoverStrategy failoverStrategy = failoverStrategyFactory.create( + getFailoverTopology(), + getResultPartitionAvailabilityChecker()); + log.info("Using failover strategy {} for {} ({}).", failoverStrategy, jobGraph.getName(), jobGraph.getJobID()); + this.executionFailureHandler = new ExecutionFailureHandler( getFailoverTopology(), - failoverStrategyFactory.create(getFailoverTopology(), getResultPartitionAvailabilityChecker()), + failoverStrategy, restartBackoffTimeStrategy); this.schedulingStrategy = schedulingStrategyFactory.createInstance(this, getSchedulingTopology(), getJobGraph()); 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 3058ac0c3295..51cda0eaa1b6 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 @@ -25,9 +25,9 @@ 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.FailoverStrategyFactoryLoader; 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.JobMasterPartitionTracker; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.ScheduleMode; @@ -101,7 +101,7 @@ public SchedulerNG createInstance( shuffleMaster, partitionTracker, schedulingStrategyFactory, - new RestartPipelinedRegionStrategy.Factory(), + FailoverStrategyFactoryLoader.loadFailoverStrategyFactory(jobMasterConfiguration), restartBackoffTimeStrategy, new DefaultExecutionVertexOperations(), new ExecutionVertexVersioner(), From 738cda347c8a6f7059a72cb683104066d6e3ba0b Mon Sep 17 00:00:00 2001 From: Zhu Zhu Date: Wed, 13 Nov 2019 17:58:28 +0800 Subject: [PATCH 623/746] [FLINK-14682][tests] Enable AbstractTaskManagerProcessFailureRecoveryTest to pass with new DefaultScheduler This closes #10173. --- .../recovery/AbstractTaskManagerProcessFailureRecoveryTest.java | 1 + 1 file changed, 1 insertion(+) 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 c8f44e8a275f..63efa507a80d 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 @@ -104,6 +104,7 @@ public void testTaskManagerProcessFailure() throws Exception { config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, 2); config.setString(TaskManagerOptions.LEGACY_MANAGED_MEMORY_SIZE, "4m"); config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, 100); + config.setString(JobManagerOptions.EXECUTION_FAILOVER_STRATEGY, "full"); try (final StandaloneSessionClusterEntrypoint clusterEntrypoint = new StandaloneSessionClusterEntrypoint(config)) { From 966c1f270f553bec7344ec49a0232f85446c493b Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Wed, 6 Nov 2019 16:48:25 +0100 Subject: [PATCH 624/746] [FLINK-14680][runtime,tests] Enable KafkaConsumerTestBase#runFailOnNoBrokerTest to pass with DefaultScheduler This closes #10136. --- .../connectors/kafka/KafkaConsumerTestBase.java | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java index 5e99be73072e..a7f8c3ac7b9c 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java @@ -102,6 +102,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; import java.util.Random; import java.util.Set; @@ -185,17 +186,15 @@ public void runFailOnNoBrokerTest() throws Exception { kafkaServer.getVersion().equals("0.10") || kafkaServer.getVersion().equals("0.11") || kafkaServer.getVersion().equals("2.0")) { - assertTrue(jee.getCause() instanceof TimeoutException); + final Optional optionalTimeoutException = ExceptionUtils.findThrowable(jee, TimeoutException.class); + assertTrue(optionalTimeoutException.isPresent()); - TimeoutException te = (TimeoutException) jee.getCause(); - - assertEquals("Timeout expired while fetching topic metadata", te.getMessage()); + final TimeoutException timeoutException = optionalTimeoutException.get(); + assertEquals("Timeout expired while fetching topic metadata", timeoutException.getMessage()); } else { - assertTrue(jee.getCause() instanceof RuntimeException); - - RuntimeException re = (RuntimeException) jee.getCause(); - - assertTrue(re.getMessage().contains("Unable to retrieve any partitions")); + final Optional optionalThrowable = ExceptionUtils.findThrowableWithMessage(jee, "Unable to retrieve any partitions"); + assertTrue(optionalThrowable.isPresent()); + assertTrue(optionalThrowable.get() instanceof RuntimeException); } } } From 77bc6697e58b8e0f2a8ca0c2c6015fc75b1baf58 Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Tue, 12 Nov 2019 16:26:45 -0800 Subject: [PATCH 625/746] [FLINK-14728][hive][doc] add reminder for users of potential thread safety issues of hive built-in function this closes #10166. --- docs/dev/table/hive/hive_functions.md | 3 +++ docs/dev/table/hive/hive_functions.zh.md | 3 +++ 2 files changed, 6 insertions(+) diff --git a/docs/dev/table/hive/hive_functions.md b/docs/dev/table/hive/hive_functions.md index 56394b508b2b..70e7c6c764bf 100644 --- a/docs/dev/table/hive/hive_functions.md +++ b/docs/dev/table/hive/hive_functions.md @@ -201,3 +201,6 @@ modules: {% endhighlight %} + +* NOTE that some Hive built-in functions in older versions have [thread safety issues](https://issues.apache.org/jira/browse/HIVE-16183). +We recommend users patch their own Hive to fix them. diff --git a/docs/dev/table/hive/hive_functions.zh.md b/docs/dev/table/hive/hive_functions.zh.md index 56394b508b2b..70e7c6c764bf 100644 --- a/docs/dev/table/hive/hive_functions.zh.md +++ b/docs/dev/table/hive/hive_functions.zh.md @@ -201,3 +201,6 @@ modules: {% endhighlight %} + +* NOTE that some Hive built-in functions in older versions have [thread safety issues](https://issues.apache.org/jira/browse/HIVE-16183). +We recommend users patch their own Hive to fix them. From f2c4355e42c55f4059d0fc6d3fca5bb753b9f4c6 Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Tue, 12 Nov 2019 16:19:40 -0800 Subject: [PATCH 626/746] [FLINK-14727][hive][doc] update doc of supported Hive versions this closes #10165. --- docs/dev/table/hive/index.md | 41 +++++++++++++++++++++++++-------- docs/dev/table/hive/index.zh.md | 41 +++++++++++++++++++++++++-------- 2 files changed, 64 insertions(+), 18 deletions(-) diff --git a/docs/dev/table/hive/index.md b/docs/dev/table/hive/index.md index b1cd30ff68b2..cc0c716e641b 100644 --- a/docs/dev/table/hive/index.md +++ b/docs/dev/table/hive/index.md @@ -40,15 +40,38 @@ You do not need to modify your existing Hive Metastore or change the data placem ## Supported Hive Versions -Flink supports Hive `2.3.4` and `1.2.1` and relies on Hive's compatibility guarantee's for other minor versions. - -If you use a different minor Hive version such as `1.2.2` or `2.3.1`, it should also be ok to -choose the closest version `1.2.1` (for `1.2.2`) or `2.3.4` (for `2.3.1`) to workaround. For -example, you want to use Flink to integrate `2.3.1` hive version in sql client, just set the -hive-version to `2.3.4` in YAML config. Similarly pass the version string when creating -HiveCatalog instance via Table API. - -Users are welcome to try out different versions with this workaround. Since only `2.3.4` and `1.2.1` have been tested, there might be unexpected issues. We will test and support more versions in future releases. +Flink supports the following Hive versions. + +- 1.0 + - 1.0.0 + - 1.0.1 +- 1.1 + - 1.1.0 + - 1.1.1 +- 1.2 + - 1.2.0 + - 1.2.1 + - 1.2.2 +- 2.0 + - 2.0.0 + - 2.0.1 +- 2.1 + - 2.1.0 + - 2.1.1 +- 2.2 + - 2.2.0 +- 2.3 + - 2.3.0 + - 2.3.1 + - 2.3.2 + - 2.3.3 + - 2.3.4 + - 2.3.5 + - 2.3.6 +- 3.1 + - 3.1.0 + - 3.1.1 + - 3.1.2 ### Dependencies diff --git a/docs/dev/table/hive/index.zh.md b/docs/dev/table/hive/index.zh.md index 263c8d88c7df..205ecfe5f379 100644 --- a/docs/dev/table/hive/index.zh.md +++ b/docs/dev/table/hive/index.zh.md @@ -40,15 +40,38 @@ You do not need to modify your existing Hive Metastore or change the data placem ## Supported Hive Versions -Flink supports Hive `2.3.4` and `1.2.1` and relies on Hive's compatibility guarantee's for other minor versions. - -If you use a different minor Hive version such as `1.2.2` or `2.3.1`, it should also be ok to -choose the closest version `1.2.1` (for `1.2.2`) or `2.3.4` (for `2.3.1`) to workaround. For -example, you want to use Flink to integrate `2.3.1` hive version in sql client, just set the -hive-version to `2.3.4` in YAML config. Similarly pass the version string when creating -HiveCatalog instance via Table API. - -Users are welcome to try out different versions with this workaround. Since only `2.3.4` and `1.2.1` have been tested, there might be unexpected issues. We will test and support more versions in future releases. +Flink supports the following Hive versions. + +- 1.0 + - 1.0.0 + - 1.0.1 +- 1.1 + - 1.1.0 + - 1.1.1 +- 1.2 + - 1.2.0 + - 1.2.1 + - 1.2.2 +- 2.0 + - 2.0.0 + - 2.0.1 +- 2.1 + - 2.1.0 + - 2.1.1 +- 2.2 + - 2.2.0 +- 2.3 + - 2.3.0 + - 2.3.1 + - 2.3.2 + - 2.3.3 + - 2.3.4 + - 2.3.5 + - 2.3.6 +- 3.1 + - 3.1.0 + - 3.1.1 + - 3.1.2 ### Dependencies From 10d372e1293d3c8bcd109e5fc570bfb368959f2c Mon Sep 17 00:00:00 2001 From: zjuwangg Date: Thu, 14 Nov 2019 17:08:13 +0800 Subject: [PATCH 627/746] [FLINK-14687][sql] Add database related ddl support to SQL Parser this closes #10201. --- .../src/main/codegen/data/Parser.tdd | 24 ++- .../src/main/codegen/includes/parserImpls.ftl | 128 +++++++++++++++- .../sql/parser/ddl/SqlAlterDatabase.java | 97 ++++++++++++ .../sql/parser/ddl/SqlCreateDatabase.java | 138 ++++++++++++++++++ .../flink/sql/parser/ddl/SqlDropDatabase.java | 99 +++++++++++++ .../flink/sql/parser/ddl/SqlUseDatabase.java | 73 +++++++++ .../sql/parser/dql/SqlDescribeDatabase.java | 81 ++++++++++ .../sql/parser/dql/SqlShowDatabases.java | 60 ++++++++ .../sql/parser/FlinkSqlParserImplTest.java | 52 +++++++ 9 files changed, 745 insertions(+), 7 deletions(-) create mode 100644 flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlAlterDatabase.java create mode 100644 flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateDatabase.java create mode 100644 flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlDropDatabase.java create mode 100644 flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlUseDatabase.java create mode 100644 flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlDescribeDatabase.java create mode 100644 flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowDatabases.java diff --git a/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd b/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd index dcb6ecdb6561..c3745dfae1e9 100644 --- a/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd +++ b/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd @@ -33,10 +33,16 @@ "org.apache.flink.sql.parser.ddl.SqlTableOption", "org.apache.flink.sql.parser.ddl.SqlWatermark", "org.apache.flink.sql.parser.ddl.SqlUseCatalog", + "org.apache.flink.sql.parser.ddl.SqlUseDatabase", + "org.apache.flink.sql.parser.ddl.SqlCreateDatabase", + "org.apache.flink.sql.parser.ddl.SqlDropDatabase", + "org.apache.flink.sql.parser.ddl.SqlAlterDatabase", "org.apache.flink.sql.parser.dml.RichSqlInsert", "org.apache.flink.sql.parser.dml.RichSqlInsertKeyword", "org.apache.flink.sql.parser.dql.SqlShowCatalogs", "org.apache.flink.sql.parser.dql.SqlDescribeCatalog", + "org.apache.flink.sql.parser.dql.SqlShowDatabases", + "org.apache.flink.sql.parser.dql.SqlDescribeDatabase", "org.apache.flink.sql.parser.type.ExtendedSqlBasicTypeNameSpec", "org.apache.flink.sql.parser.type.ExtendedSqlCollectionTypeNameSpec", "org.apache.flink.sql.parser.utils.SqlTimeUnit", @@ -60,7 +66,9 @@ "STRING", "BYTES", "CATALOGS", - "USE" + "USE", + "DATABASES", + "EXTENDED" ] # List of keywords from "keywords" section that are not reserved. @@ -118,6 +126,7 @@ "CURSOR_NAME" "DATA" "DATABASE" + "DATABASES" "DATETIME_INTERVAL_CODE" "DATETIME_INTERVAL_PRECISION" "DECADE" @@ -145,6 +154,7 @@ "EXCEPTION" "EXCLUDE" "EXCLUDING" + "EXTENDED" "FINAL" "FIRST" "FOLLOWING" @@ -383,7 +393,11 @@ "RichSqlInsert()", "SqlShowCatalogs()", "SqlDescribeCatalog()", - "SqlUseCatalog()" + "SqlUseCatalog()", + "SqlShowDatabases()", + "SqlUseDatabase()", + "SqlAlterDatabase()", + "SqlDescribeDatabase()" ] # List of methods for parsing custom literals. @@ -418,14 +432,16 @@ # Each must accept arguments "(SqlParserPos pos, boolean replace)". createStatementParserMethods: [ "SqlCreateTable", - "SqlCreateView" + "SqlCreateView", + "SqlCreateDatabase" ] # List of methods for parsing extensions to "DROP" calls. # Each must accept arguments "(Span s)". dropStatementParserMethods: [ "SqlDropTable", - "SqlDropView" + "SqlDropView", + "SqlDropDatabase" ] # Binary operators tokens 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 421c88192752..ab5dcb4968fb 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 @@ -44,17 +44,139 @@ SqlDescribeCatalog SqlDescribeCatalog() : SqlUseCatalog SqlUseCatalog() : { -SqlIdentifier catalogName; -SqlParserPos pos; + SqlIdentifier catalogName; + SqlParserPos pos; } { - { pos = getPos();} + { pos = getPos();} catalogName = SimpleIdentifier() { return new SqlUseCatalog(pos, catalogName); } } +/** +* Parse a "Show Catalogs" metadata query command. +*/ +SqlShowDatabases SqlShowDatabases() : +{ +} +{ + + { + return new SqlShowDatabases(getPos()); + } +} + +SqlUseDatabase SqlUseDatabase() : +{ + SqlIdentifier databaseName; + SqlParserPos pos; +} +{ + { pos = getPos();} + databaseName = CompoundIdentifier() + { + return new SqlUseDatabase(pos, databaseName); + } +} + +/** +* Parses a create database statement. +* CREATE DATABASE database_name [COMMENT database_comment] [WITH (property_name=property_value, ...)]; +*/ +SqlCreate SqlCreateDatabase(Span s, boolean replace) : +{ + SqlParserPos startPos; + SqlIdentifier databaseName; + SqlCharStringLiteral comment = null; + SqlNodeList propertyList = SqlNodeList.EMPTY; + boolean ifNotExists = false; +} +{ + { startPos = getPos(); } + [ { ifNotExists = true; } ] + databaseName = CompoundIdentifier() + [ + { + String p = SqlParserUtil.parseString(token.image); + comment = SqlLiteral.createCharString(p, getPos()); + } + ] + [ + + propertyList = TableProperties() + ] + + { return new SqlCreateDatabase(startPos.plus(getPos()), + databaseName, + propertyList, + comment, + ifNotExists); } + +} + +SqlAlterDatabase SqlAlterDatabase() : +{ + SqlParserPos startPos; + SqlIdentifier databaseName; + SqlNodeList propertyList = SqlNodeList.EMPTY; +} +{ + { startPos = getPos(); } + databaseName = CompoundIdentifier() + + propertyList = TableProperties() + { + return new SqlAlterDatabase(startPos.plus(getPos()), + databaseName, + propertyList); + } +} + +SqlDrop SqlDropDatabase(Span s, boolean replace) : +{ + SqlIdentifier databaseName = null; + boolean ifExists = false; + boolean isRestrict = true; +} +{ + + + ( + { ifExists = true; } + | + { ifExists = false; } + ) + + databaseName = CompoundIdentifier() + [ + { isRestrict = true; } + | + { isRestrict = false; } + ] + + { + return new SqlDropDatabase(s.pos(), databaseName, ifExists, isRestrict); + } +} + +SqlDescribeDatabase SqlDescribeDatabase() : +{ + SqlIdentifier databaseName; + SqlParserPos pos; + boolean isExtended = false; +} +{ + { pos = getPos();} + [ { isExtended = true;} ] + databaseName = CompoundIdentifier() + { + return new SqlDescribeDatabase(pos, databaseName, isExtended); + } + +} + void TableColumn(TableCreationContext context) : { } diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlAlterDatabase.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlAlterDatabase.java new file mode 100644 index 000000000000..6ed0dcea7e5e --- /dev/null +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlAlterDatabase.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.ddl; + +import org.apache.calcite.sql.SqlCall; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlNodeList; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.SqlSpecialOperator; +import org.apache.calcite.sql.SqlWriter; +import org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.calcite.util.ImmutableNullableList; + +import java.util.List; + +import static java.util.Objects.requireNonNull; + +/** + * ALTER Database DDL sql call. + */ +public class SqlAlterDatabase extends SqlCall { + + public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("ALTER DATABASE", SqlKind.OTHER); + + private final SqlIdentifier databaseName; + + private final SqlNodeList propertyList; + + public SqlAlterDatabase( + SqlParserPos pos, + SqlIdentifier databaseName, + SqlNodeList propertyList) { + super(pos); + this.databaseName = requireNonNull(databaseName, "tableName should not be null"); + this.propertyList = requireNonNull(propertyList, "propertyList should not be null"); + } + + @Override + public SqlOperator getOperator() { + return OPERATOR; + } + + @Override + public List getOperandList() { + return ImmutableNullableList.of(databaseName, propertyList); + } + + public SqlIdentifier getDatabaseName() { + return databaseName; + } + + public SqlNodeList getPropertyList() { + return propertyList; + } + + @Override + public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { + writer.keyword("ALTER DATABASE"); + databaseName.unparse(writer, leftPrec, rightPrec); + writer.keyword("SET"); + SqlWriter.Frame withFrame = writer.startList("(", ")"); + for (SqlNode property : propertyList) { + printIndent(writer); + property.unparse(writer, leftPrec, rightPrec); + } + writer.newlineAndIndent(); + writer.endList(withFrame); + } + + private void printIndent(SqlWriter writer) { + writer.sep(",", false); + writer.newlineAndIndent(); + writer.print(" "); + } + + public String[] fullDatabaseName() { + return databaseName.names.toArray(new String[0]); + } +} diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateDatabase.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateDatabase.java new file mode 100644 index 000000000000..87a0686b3b45 --- /dev/null +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateDatabase.java @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.ddl; + +import org.apache.flink.sql.parser.ExtendedSqlNode; +import org.apache.flink.sql.parser.error.SqlValidateException; + +import org.apache.calcite.sql.SqlCharStringLiteral; +import org.apache.calcite.sql.SqlCreate; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlNodeList; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.SqlSpecialOperator; +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; + +/** + * CREATE Database DDL sql call. + */ +public class SqlCreateDatabase extends SqlCreate implements ExtendedSqlNode { + + public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("CREATE DATABASE", SqlKind.OTHER); + + private final SqlIdentifier databaseName; + + private final SqlNodeList propertyList; + + @Nullable + private final SqlCharStringLiteral comment; + + public SqlCreateDatabase( + SqlParserPos pos, + SqlIdentifier databaseName, + SqlNodeList propertyList, + SqlCharStringLiteral comment, + boolean ifNotExists) { + super(OPERATOR, pos, false, ifNotExists); + this.databaseName = requireNonNull(databaseName, "tableName should not be null"); + this.propertyList = requireNonNull(propertyList, "propertyList should not be null"); + this.comment = comment; + } + + @Override + public SqlOperator getOperator() { + return OPERATOR; + } + + @Override + public List getOperandList() { + return ImmutableNullableList.of(databaseName, propertyList, comment); + } + + public SqlIdentifier getDatabaseName() { + return databaseName; + } + + public SqlNodeList getPropertyList() { + return propertyList; + } + + public Optional getComment() { + return Optional.ofNullable(comment); + } + + public boolean isIfNotExists() { + return ifNotExists; + } + + @Override + public void validate() throws SqlValidateException { + + } + + @Override + public void unparse( + SqlWriter writer, + int leftPrec, + int rightPrec) { + writer.keyword("CREATE DATABASE"); + if (isIfNotExists()) { + writer.keyword("IF NOT EXISTS"); + } + databaseName.unparse(writer, leftPrec, rightPrec); + + if (comment != null) { + writer.newlineAndIndent(); + writer.keyword("COMMENT"); + comment.unparse(writer, leftPrec, rightPrec); + } + + if (this.propertyList.size() > 0) { + writer.keyword("WITH"); + SqlWriter.Frame withFrame = writer.startList("(", ")"); + for (SqlNode property : propertyList) { + printIndent(writer); + property.unparse(writer, leftPrec, rightPrec); + } + writer.newlineAndIndent(); + writer.endList(withFrame); + } + } + + private void printIndent(SqlWriter writer) { + writer.sep(",", false); + writer.newlineAndIndent(); + writer.print(" "); + } + + public String[] fullDatabaseName() { + return databaseName.names.toArray(new String[0]); + } +} diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlDropDatabase.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlDropDatabase.java new file mode 100644 index 000000000000..d9a079283d4d --- /dev/null +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlDropDatabase.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.sql.parser.ddl; + +import org.apache.flink.sql.parser.ExtendedSqlNode; + +import org.apache.calcite.sql.SqlDrop; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.SqlSpecialOperator; +import org.apache.calcite.sql.SqlWriter; +import org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.calcite.util.ImmutableNullableList; + +import java.util.List; + +/** + * DROP DATABASE DDL sql call. + */ +public class SqlDropDatabase extends SqlDrop implements ExtendedSqlNode { + private static final SqlOperator OPERATOR = + new SqlSpecialOperator("DROP DATABASE", SqlKind.OTHER); + + private SqlIdentifier databaseName; + private boolean ifExists; + private boolean isRestrict = true; + + public SqlDropDatabase(SqlParserPos pos, + SqlIdentifier databaseName, + boolean ifExists, + boolean isRestrict) { + super(OPERATOR, pos, ifExists); + this.databaseName = databaseName; + this.ifExists = ifExists; + this.isRestrict = isRestrict; + } + + @Override + public List getOperandList() { + return ImmutableNullableList.of(databaseName); + } + + public SqlIdentifier getDatabaseName() { + return databaseName; + } + + public void setDatabaseName(SqlIdentifier viewName) { + this.databaseName = viewName; + } + + public boolean getIfExists() { + return this.ifExists; + } + + public void setIfExists(boolean ifExists) { + this.ifExists = ifExists; + } + + @Override + public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { + writer.keyword("DROP"); + writer.keyword("DATABASE"); + if (ifExists) { + writer.keyword("IF EXISTS"); + } + databaseName.unparse(writer, leftPrec, rightPrec); + if (isRestrict) { + writer.keyword("RESTRICT"); + } else { + writer.keyword("CASCADE"); + } + } + + public void validate() { + // no-op + } + + public String[] fullDatabaseName() { + return databaseName.names.toArray(new String[0]); + } +} diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlUseDatabase.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlUseDatabase.java new file mode 100644 index 000000000000..724b619c2682 --- /dev/null +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlUseDatabase.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.sql.parser.ddl; + +import org.apache.calcite.sql.SqlCall; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.SqlSpecialOperator; +import org.apache.calcite.sql.SqlWriter; +import org.apache.calcite.sql.parser.SqlParserPos; + +import java.util.Collections; +import java.util.List; + +/** + * USE [catalog.]database sql call. + */ +public class SqlUseDatabase extends SqlCall { + + public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("USE DATABASE", SqlKind.OTHER); + private final SqlIdentifier databaseName; + + public SqlUseDatabase(SqlParserPos pos, SqlIdentifier databaseName) { + super(pos); + this.databaseName = databaseName; + + } + + @Override + public SqlOperator getOperator() { + return OPERATOR; + } + + @Override + public List getOperandList() { + return Collections.singletonList(databaseName); + } + + public SqlIdentifier getDatabaseName() { + return databaseName; + } + + public String[] fullDatabaseName() { + return databaseName.names.toArray(new String[0]); + } + + @Override + public void unparse( + SqlWriter writer, + int leftPrec, + int rightPrec) { + writer.keyword("USE"); + databaseName.unparse(writer, leftPrec, rightPrec); + } +} diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlDescribeDatabase.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlDescribeDatabase.java new file mode 100644 index 000000000000..7f2623fc3eaf --- /dev/null +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlDescribeDatabase.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.sql.parser.dql; + +import org.apache.calcite.sql.SqlCall; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.SqlSpecialOperator; +import org.apache.calcite.sql.SqlWriter; +import org.apache.calcite.sql.parser.SqlParserPos; + +import java.util.Collections; +import java.util.List; + +/** + * DESCRIBE DATABASE [ EXTENDED] [ databaseName.] dataBasesName sql call. + */ +public class SqlDescribeDatabase extends SqlCall { + + public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("DESCRIBE DATABASE", SqlKind.OTHER); + private final SqlIdentifier databaseName; + private boolean isExtended = false; + + public SqlDescribeDatabase(SqlParserPos pos, SqlIdentifier databaseName, boolean isExtended) { + super(pos); + this.databaseName = databaseName; + this.isExtended = isExtended; + } + + @Override + public SqlOperator getOperator() { + return OPERATOR; + } + + @Override + public List getOperandList() { + return Collections.singletonList(databaseName); + } + + public String getDatabaseName() { + return databaseName.getSimple(); + } + + public boolean isExtended() { + return isExtended; + } + + public String[] fullDatabaseName() { + return databaseName.names.toArray(new String[0]); + } + + @Override + public void unparse( + SqlWriter writer, + int leftPrec, + int rightPrec) { + writer.keyword("DESCRIBE DATABASE"); + if (isExtended) { + writer.keyword("EXTENDED"); + } + databaseName.unparse(writer, leftPrec, rightPrec); + } +} diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowDatabases.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowDatabases.java new file mode 100644 index 000000000000..512ffaf08642 --- /dev/null +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/dql/SqlShowDatabases.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.sql.parser.dql; + +import org.apache.calcite.sql.SqlCall; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.SqlSpecialOperator; +import org.apache.calcite.sql.SqlWriter; +import org.apache.calcite.sql.parser.SqlParserPos; + +import java.util.Collections; +import java.util.List; + +/** + * SHOW Databases sql call. + */ +public class SqlShowDatabases extends SqlCall { + + public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("SHOW DATABASES", SqlKind.OTHER); + + public SqlShowDatabases(SqlParserPos pos) { + super(pos); + } + + @Override + public SqlOperator getOperator() { + return OPERATOR; + } + + @Override + public List getOperandList() { + return Collections.EMPTY_LIST; + } + + @Override + public void unparse( + SqlWriter writer, + int leftPrec, + int rightPrec) { + writer.keyword("SHOW DATABASES"); + } +} 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 7d390a2ab3fd..c1b42c57a16f 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 @@ -95,6 +95,58 @@ public void testUseCatalog() { check("use catalog a", "USE CATALOG `A`"); } + @Test + public void testShowDataBases() { + check("show databases", "SHOW DATABASES"); + } + + @Test + public void testUseDataBase() { + check("use default_db", "USE `DEFAULT_DB`"); + check("use defaultCatalog.default_db", "USE `DEFAULTCATALOG`.`DEFAULT_DB`"); + } + + @Test + public void testCreateDatabase() { + check("create database db1", "CREATE DATABASE `DB1`"); + check("create database if not exists db1", "CREATE DATABASE IF NOT EXISTS `DB1`"); + check("create database catalog1.db1", "CREATE DATABASE `CATALOG1`.`DB1`"); + check("create database db1 comment 'test create database'", + "CREATE DATABASE `DB1`\n" + + "COMMENT 'test create database'"); + check("create database db1 comment 'test create database'" + + "with ( 'key1' = 'value1', 'key2.a' = 'value2.a')", + "CREATE DATABASE `DB1`\n" + + "COMMENT 'test create database' WITH (\n" + + " 'key1' = 'value1',\n" + + " 'key2.a' = 'value2.a'\n" + + ")"); + } + + @Test + public void testDropDatabase() { + check("drop database db1", "DROP DATABASE `DB1` RESTRICT"); + check("drop database catalog1.db1", "DROP DATABASE `CATALOG1`.`DB1` RESTRICT"); + check("drop database db1 RESTRICT", "DROP DATABASE `DB1` RESTRICT"); + check("drop database db1 CASCADE", "DROP DATABASE `DB1` CASCADE"); + } + + @Test + public void testAlterDatabase() { + check("alter database db1 set ('key1' = 'value1','key2.a' = 'value2.a')", + "ALTER DATABASE `DB1` SET (\n" + + " 'key1' = 'value1',\n" + + " 'key2.a' = 'value2.a'\n" + + ")"); + } + + @Test + public void testDescribeDatabase() { + check("describe database db1", "DESCRIBE DATABASE `DB1`"); + check("describe database catlog1.db1", "DESCRIBE DATABASE `CATLOG1`.`DB1`"); + check("describe database extended db1", "DESCRIBE DATABASE EXTENDED `DB1`"); + } + @Test public void testCreateTable() { check("CREATE TABLE tbl1 (\n" + From feca8a780f71c9b1a0f5d361ad8c5797102187c3 Mon Sep 17 00:00:00 2001 From: Rui Li Date: Tue, 12 Nov 2019 16:03:30 +0800 Subject: [PATCH 628/746] [FLINK-14710][hive] Decide column nullability according to Hive constraints When getting a Hive table, set column nullability according to Hive NOT NULL constraints. this closes #10157. --- .../flink/table/catalog/hive/HiveCatalog.java | 4 ++- .../client/HiveMetastoreClientWrapper.java | 18 ++++++------ .../table/catalog/hive/client/HiveShim.java | 5 ++++ .../catalog/hive/client/HiveShimV100.java | 7 +++++ .../catalog/hive/client/HiveShimV310.java | 28 +++++++++++++++++++ .../hive/util/HiveReflectionUtils.java | 11 ++++++++ .../catalog/hive/util/HiveTableUtil.java | 7 ++++- .../hive/TableEnvHiveConnectorTest.java | 25 ++++++++++++++++- .../flink/table/HiveVersionTestUtil.java | 1 + 9 files changed, 93 insertions(+), 13 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java index 82832e7e1d28..535899dd846e 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalog.java @@ -99,6 +99,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.stream.Collectors; import static org.apache.flink.table.catalog.config.CatalogConfig.FLINK_PROPERTY_PREFIX; @@ -525,8 +526,9 @@ private CatalogBaseTable instantiateCatalogTable(Table hiveTable, HiveConf hiveC // get schema from deserializer fields = hiveShim.getFieldsFromDeserializer(hiveConf, hiveTable, true); } + Set notNullColumns = client.getNotNullColumns(hiveConf, hiveTable.getDbName(), hiveTable.getTableName()); TableSchema tableSchema = - HiveTableUtil.createTableSchema(fields, hiveTable.getPartitionKeys()); + HiveTableUtil.createTableSchema(fields, hiveTable.getPartitionKeys(), notNullColumns); // Partition keys List partitionKeys = new ArrayList<>(); diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveMetastoreClientWrapper.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveMetastoreClientWrapper.java index 3da18751aa51..1e4bfca81d16 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveMetastoreClientWrapper.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveMetastoreClientWrapper.java @@ -22,6 +22,7 @@ import org.apache.flink.util.Preconditions; import org.apache.flink.util.StringUtils; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; @@ -43,6 +44,7 @@ import java.util.List; import java.util.Map; +import java.util.Set; import static org.apache.flink.util.Preconditions.checkArgument; @@ -57,12 +59,12 @@ public class HiveMetastoreClientWrapper implements AutoCloseable { private final IMetaStoreClient client; private final HiveConf hiveConf; - private final String hiveVersion; + private final HiveShim hiveShim; public HiveMetastoreClientWrapper(HiveConf hiveConf, String hiveVersion) { this.hiveConf = Preconditions.checkNotNull(hiveConf, "HiveConf cannot be null"); checkArgument(!StringUtils.isNullOrWhitespaceOnly(hiveVersion), "hiveVersion cannot be null or empty"); - this.hiveVersion = hiveVersion; + hiveShim = HiveShimLoader.loadHiveShim(hiveVersion); client = createMetastoreClient(); } @@ -210,13 +212,15 @@ public List listPartitions(String dbName, String tblName, short max) //-------- Start of shimmed methods ---------- + public Set getNotNullColumns(Configuration conf, String dbName, String tableName) { + return hiveShim.getNotNullColumns(client, conf, dbName, tableName); + } + public List getViews(String databaseName) throws UnknownDBException, TException { - HiveShim hiveShim = HiveShimLoader.loadHiveShim(hiveVersion); return hiveShim.getViews(client, databaseName); } private IMetaStoreClient createMetastoreClient() { - HiveShim hiveShim = HiveShimLoader.loadHiveShim(hiveVersion); return hiveShim.getHiveMetastoreClient(hiveConf); } @@ -238,18 +242,12 @@ public Function getFunction(String databaseName, String functionName) throws Met public void alter_table(String databaseName, String tableName, Table table) throws InvalidOperationException, MetaException, TException { - HiveShim hiveShim = HiveShimLoader.loadHiveShim(hiveVersion); hiveShim.alterTable(client, databaseName, tableName, table); } public void alter_partition(String databaseName, String tableName, Partition partition) throws InvalidOperationException, MetaException, TException { - HiveShim hiveShim = HiveShimLoader.loadHiveShim(hiveVersion); hiveShim.alterPartition(client, databaseName, tableName, partition); } - public String getHiveVersion() { - return hiveVersion; - } - } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShim.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShim.java index 929d90e53d39..c7347fcfd351 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShim.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShim.java @@ -198,4 +198,9 @@ FileSinkOperator.RecordWriter getHiveRecordWriter(JobConf jobConf, String output * Get a Hive built-in function by name. */ Optional getBuiltInFunctionInfo(String name); + + /** + * Get the set of columns that have NOT NULL constraints. + */ + Set getNotNullColumns(IMetaStoreClient client, Configuration conf, String dbName, String tableName); } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV100.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV100.java index e5f3b6770786..652521dfa37e 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV100.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV100.java @@ -76,6 +76,7 @@ import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; @@ -333,4 +334,10 @@ public Optional getBuiltInFunctionInfo(String name) { // prior to Hive 1.2.0 throw new UnsupportedOperationException("Getting built in functions are not supported until Hive 1.2.0"); } + + @Override + public Set getNotNullColumns(IMetaStoreClient client, Configuration conf, String dbName, String tableName) { + // NOT NULL constraints not supported until 3.0.0 -- HIVE-16575 + return Collections.emptySet(); + } } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV310.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV310.java index 62438d46630e..599b6214539e 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV310.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/client/HiveShimV310.java @@ -19,6 +19,7 @@ package org.apache.flink.table.catalog.hive.client; import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.hive.util.HiveReflectionUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -32,6 +33,7 @@ import java.io.IOException; import java.lang.reflect.Method; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -111,4 +113,30 @@ public void makeSpecFromName(Map partSpec, Path currPath) { } } + @Override + public Set getNotNullColumns(IMetaStoreClient client, Configuration conf, String dbName, String tableName) { + try { + // HMS catalog (https://issues.apache.org/jira/browse/HIVE-18685) is an on-going feature and we currently + // just get the default catalog. + String hiveDefaultCatalog = (String) HiveReflectionUtils.invokeMethod(getMetaStoreUtilsClass(), null, + "getDefaultCatalog", new Class[]{Configuration.class}, new Object[]{conf}); + Class requestClz = Class.forName("org.apache.hadoop.hive.metastore.api.NotNullConstraintsRequest"); + Object request = requestClz.getDeclaredConstructor(String.class, String.class, String.class) + .newInstance(hiveDefaultCatalog, dbName, tableName); + List constraints = (List) HiveReflectionUtils.invokeMethod(client.getClass(), client, + "getNotNullConstraints", new Class[]{requestClz}, new Object[]{request}); + Class constraintClz = Class.forName("org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint"); + Method colNameMethod = constraintClz.getDeclaredMethod("getColumn_name"); + Method isRelyMethod = constraintClz.getDeclaredMethod("isRely_cstr"); + Set res = new HashSet<>(); + for (Object constraint : constraints) { + if ((boolean) isRelyMethod.invoke(constraint)) { + res.add((String) colNameMethod.invoke(constraint)); + } + } + return res; + } catch (Exception e) { + throw new CatalogException("Failed to get NOT NULL constraints", e); + } + } } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveReflectionUtils.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveReflectionUtils.java index 5224b781c446..a829d173718b 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveReflectionUtils.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveReflectionUtils.java @@ -91,4 +91,15 @@ public static Object convertToHiveTimestamp(HiveShim hiveShim, String s) throws } } + public static Object invokeMethod(Class clz, Object obj, String methodName, Class[] argClz, Object[] args) + throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { + Method method; + try { + method = clz.getDeclaredMethod(methodName, argClz); + } catch (NoSuchMethodException e) { + method = clz.getMethod(methodName, argClz); + } + return method.invoke(obj, args); + } + } diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveTableUtil.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveTableUtil.java index 58075a9ead07..0251c24f1434 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveTableUtil.java +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveTableUtil.java @@ -34,6 +34,7 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.Set; import static org.apache.flink.table.catalog.hive.HiveCatalogConfig.DEFAULT_LIST_COLUMN_TYPES_SEPARATOR; @@ -48,7 +49,8 @@ private HiveTableUtil() { /** * Create a Flink's TableSchema from Hive table's columns and partition keys. */ - public static TableSchema createTableSchema(List cols, List partitionKeys) { + public static TableSchema createTableSchema(List cols, List partitionKeys, + Set notNullColumns) { List allCols = new ArrayList<>(cols); allCols.addAll(partitionKeys); @@ -60,6 +62,9 @@ public static TableSchema createTableSchema(List cols, List= 0; public static final boolean HIVE_110_OR_LATER = HiveShimLoader.getHiveVersion().compareTo(HiveShimLoader.HIVE_VERSION_V1_1_0) >= 0; + public static final boolean HIVE_310_OR_LATER = HiveShimLoader.getHiveVersion().compareTo(HiveShimLoader.HIVE_VERSION_V3_1_0) >= 0; } From 94c3faf75f01d625ad248f867bf176bc214646a4 Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Thu, 14 Nov 2019 11:38:05 -0800 Subject: [PATCH 629/746] [hotfix][doc] update links to table functions --- docs/dev/table/index.md | 6 +++--- docs/dev/table/index.zh.md | 6 +++--- docs/dev/table/sqlClient.md | 2 +- docs/dev/table/sqlClient.zh.md | 2 +- docs/dev/table/streaming/match_recognize.md | 2 +- docs/dev/table/streaming/match_recognize.zh.md | 2 +- docs/getting-started/walkthroughs/table_api.md | 4 ++-- docs/getting-started/walkthroughs/table_api.zh.md | 4 ++-- 8 files changed, 14 insertions(+), 14 deletions(-) diff --git a/docs/dev/table/index.md b/docs/dev/table/index.md index 3ef2a7cd7ea3..3d2a2b715852 100644 --- a/docs/dev/table/index.md +++ b/docs/dev/table/index.md @@ -111,7 +111,7 @@ Internally, parts of the table ecosystem are implemented in Scala. Therefore, pl ### Extension Dependencies -If you want to implement a [custom format]({{ site.baseurl }}/dev/table/sourceSinks.html#define-a-tablefactory) for interacting with Kafka or a set of [user-defined functions]({{ site.baseurl }}/dev/table/functions.html), the following dependency is sufficient and can be used for JAR files for the SQL Client: +If you want to implement a [custom format]({{ site.baseurl }}/dev/table/sourceSinks.html#define-a-tablefactory) for interacting with Kafka or a set of [user-defined functions]({{ site.baseurl }}/dev/table/systemFunctions.html), the following dependency is sufficient and can be used for JAR files for the SQL Client: {% highlight xml %} @@ -137,10 +137,10 @@ Where to go next? * [Concepts & Common API]({{ site.baseurl }}/dev/table/common.html): Shared concepts and APIs of the Table API and SQL. * [Data Types]({{ site.baseurl }}/dev/table/types.html): Lists pre-defined data types and their properties. * [Streaming Concepts]({{ site.baseurl }}/dev/table/streaming): Streaming-specific documentation for the Table API or SQL such as configuration of time attributes and handling of updating results. -* [Connect to External Systems]({{ site.baseurl }}/dev/table/functions.html): Available connectors and formats for reading and writing data to external systems. +* [Connect to External Systems]({{ site.baseurl }}/dev/table/systemFunctions.html): Available connectors and formats for reading and writing data to external systems. * [Table API]({{ site.baseurl }}/dev/table/tableApi.html): Supported operations and API for the Table API. * [SQL]({{ site.baseurl }}/dev/table/sql.html): Supported operations and syntax for SQL. -* [Built-in Functions]({{ site.baseurl }}/dev/table/functions.html): Supported functions in Table API and SQL. +* [Built-in Functions]({{ site.baseurl }}/dev/table/systemFunctions.html): Supported functions in Table API and SQL. * [SQL Client]({{ site.baseurl }}/dev/table/sqlClient.html): Play around with Flink SQL and submit a table program to a cluster without programming knowledge. {% top %} diff --git a/docs/dev/table/index.zh.md b/docs/dev/table/index.zh.md index 3ef2a7cd7ea3..3d2a2b715852 100644 --- a/docs/dev/table/index.zh.md +++ b/docs/dev/table/index.zh.md @@ -111,7 +111,7 @@ Internally, parts of the table ecosystem are implemented in Scala. Therefore, pl ### Extension Dependencies -If you want to implement a [custom format]({{ site.baseurl }}/dev/table/sourceSinks.html#define-a-tablefactory) for interacting with Kafka or a set of [user-defined functions]({{ site.baseurl }}/dev/table/functions.html), the following dependency is sufficient and can be used for JAR files for the SQL Client: +If you want to implement a [custom format]({{ site.baseurl }}/dev/table/sourceSinks.html#define-a-tablefactory) for interacting with Kafka or a set of [user-defined functions]({{ site.baseurl }}/dev/table/systemFunctions.html), the following dependency is sufficient and can be used for JAR files for the SQL Client: {% highlight xml %} @@ -137,10 +137,10 @@ Where to go next? * [Concepts & Common API]({{ site.baseurl }}/dev/table/common.html): Shared concepts and APIs of the Table API and SQL. * [Data Types]({{ site.baseurl }}/dev/table/types.html): Lists pre-defined data types and their properties. * [Streaming Concepts]({{ site.baseurl }}/dev/table/streaming): Streaming-specific documentation for the Table API or SQL such as configuration of time attributes and handling of updating results. -* [Connect to External Systems]({{ site.baseurl }}/dev/table/functions.html): Available connectors and formats for reading and writing data to external systems. +* [Connect to External Systems]({{ site.baseurl }}/dev/table/systemFunctions.html): Available connectors and formats for reading and writing data to external systems. * [Table API]({{ site.baseurl }}/dev/table/tableApi.html): Supported operations and API for the Table API. * [SQL]({{ site.baseurl }}/dev/table/sql.html): Supported operations and syntax for SQL. -* [Built-in Functions]({{ site.baseurl }}/dev/table/functions.html): Supported functions in Table API and SQL. +* [Built-in Functions]({{ site.baseurl }}/dev/table/systemFunctions.html): Supported functions in Table API and SQL. * [SQL Client]({{ site.baseurl }}/dev/table/sqlClient.html): Play around with Flink SQL and submit a table program to a cluster without programming knowledge. {% top %} diff --git a/docs/dev/table/sqlClient.md b/docs/dev/table/sqlClient.md index 24c63cb62d5b..6bd69e1f09f7 100644 --- a/docs/dev/table/sqlClient.md +++ b/docs/dev/table/sqlClient.md @@ -359,7 +359,7 @@ Both `connector` and `format` allow to define a property version (which is curre The SQL Client allows users to create custom, user-defined functions to be used in SQL queries. Currently, these functions are restricted to be defined programmatically in Java/Scala classes. -In order to provide a user-defined function, you need to first implement and compile a function class that extends `ScalarFunction`, `AggregateFunction` or `TableFunction` (see [User-defined Functions]({{ site.baseurl }}/dev/table/udfs.html)). One or more functions can then be packaged into a dependency JAR for the SQL Client. +In order to provide a user-defined function, you need to first implement and compile a function class that extends `ScalarFunction`, `AggregateFunction` or `TableFunction` (see [User-defined Functions]({{ site.baseurl }}/dev/table/functions/udfs.html)). One or more functions can then be packaged into a dependency JAR for the SQL Client. All functions must be declared in an environment file before being called. For each item in the list of `functions`, one must specify diff --git a/docs/dev/table/sqlClient.zh.md b/docs/dev/table/sqlClient.zh.md index 689ef67c60ec..465532801c6a 100644 --- a/docs/dev/table/sqlClient.zh.md +++ b/docs/dev/table/sqlClient.zh.md @@ -359,7 +359,7 @@ Both `connector` and `format` allow to define a property version (which is curre The SQL Client allows users to create custom, user-defined functions to be used in SQL queries. Currently, these functions are restricted to be defined programmatically in Java/Scala classes. -In order to provide a user-defined function, you need to first implement and compile a function class that extends `ScalarFunction`, `AggregateFunction` or `TableFunction` (see [User-defined Functions]({{ site.baseurl }}/dev/table/udfs.html)). One or more functions can then be packaged into a dependency JAR for the SQL Client. +In order to provide a user-defined function, you need to first implement and compile a function class that extends `ScalarFunction`, `AggregateFunction` or `TableFunction` (see [User-defined Functions]({{ site.baseurl }}/dev/table/functions/udfs.html)). One or more functions can then be packaged into a dependency JAR for the SQL Client. All functions must be declared in an environment file before being called. For each item in the list of `functions`, one must specify diff --git a/docs/dev/table/streaming/match_recognize.md b/docs/dev/table/streaming/match_recognize.md index 663c7d2a5913..f1d014168b49 100644 --- a/docs/dev/table/streaming/match_recognize.md +++ b/docs/dev/table/streaming/match_recognize.md @@ -264,7 +264,7 @@ look at the [event stream navigation](#pattern-navigation) section. Aggregations can be used in `DEFINE` and `MEASURES` clauses. Both [built-in]({{ site.baseurl }}/dev/table/sql.html#built-in-functions) and custom -[user defined]({{ site.baseurl }}/dev/table/udfs.html) functions are supported. +[user defined]({{ site.baseurl }}/dev/table/functions/udfs.html) functions are supported. Aggregate functions are applied to each subset of rows mapped to a match. In order to understand how those subsets are evaluated have a look at the [event stream navigation](#pattern-navigation) diff --git a/docs/dev/table/streaming/match_recognize.zh.md b/docs/dev/table/streaming/match_recognize.zh.md index 663c7d2a5913..f1d014168b49 100644 --- a/docs/dev/table/streaming/match_recognize.zh.md +++ b/docs/dev/table/streaming/match_recognize.zh.md @@ -264,7 +264,7 @@ look at the [event stream navigation](#pattern-navigation) section. Aggregations can be used in `DEFINE` and `MEASURES` clauses. Both [built-in]({{ site.baseurl }}/dev/table/sql.html#built-in-functions) and custom -[user defined]({{ site.baseurl }}/dev/table/udfs.html) functions are supported. +[user defined]({{ site.baseurl }}/dev/table/functions/udfs.html) functions are supported. Aggregate functions are applied to each subset of rows mapped to a match. In order to understand how those subsets are evaluated have a look at the [event stream navigation](#pattern-navigation) diff --git a/docs/getting-started/walkthroughs/table_api.md b/docs/getting-started/walkthroughs/table_api.md index f0cbb6222d82..83487d910eb0 100644 --- a/docs/getting-started/walkthroughs/table_api.md +++ b/docs/getting-started/walkthroughs/table_api.md @@ -189,7 +189,7 @@ The spend report (`spend_report`) table logs each row with log level **INFO**, i #### Registering A UDF -Along with the tables, a [user-defined function]({{ site.baseurl }}/dev/table/udfs.html) is registered for working with timestamps. +Along with the tables, a [user-defined function]({{ site.baseurl }}/dev/table/functions/udfs.html) is registered for working with timestamps. This function takes a timestamp and rounds it down to the nearest hour.

    @@ -256,7 +256,7 @@ Now with the skeleton of a Job set-up, you are ready to add some business logic. The goal is to build a report that shows the total spend for each account across each hour of the day. Just like a SQL query, Flink can select the required fields and group by your keys. Because the timestamp field has millisecond granularity, you can use the UDF to round it down to the nearest hour. -Finally, select all the fields, summing the total spend per account-hour pair with the built-in `sum` [aggregate function]({{ site.baseurl }}/dev/table/functions.html#aggregate-functions). +Finally, select all the fields, summing the total spend per account-hour pair with the built-in `sum` [aggregate function]({{ site.baseurl }}/dev/table/systemFunctions.html#aggregate-functions).
    diff --git a/docs/getting-started/walkthroughs/table_api.zh.md b/docs/getting-started/walkthroughs/table_api.zh.md index 5a3f95b799b4..6161e736d292 100644 --- a/docs/getting-started/walkthroughs/table_api.zh.md +++ b/docs/getting-started/walkthroughs/table_api.zh.md @@ -188,7 +188,7 @@ tEnv.registerTableSink("spend_report", new SpendReportTableSink) #### 注册 UDF -一个用来处理时间戳的[自定义函数]({{ site.baseurl }}/zh/dev/table/udfs.html)随表一起被注册到tEnv中。 +一个用来处理时间戳的[自定义函数]({{ site.baseurl }}/zh/dev/table/functions/udfs.html)随表一起被注册到tEnv中。 此函数将时间戳向下舍入到最接近的小时。
    @@ -256,7 +256,7 @@ env.execute("Spend Report") 目标是建立一个报表来显示每天每小时每个账户的总支出。 就像一个 SQL 查询一样,Flink 可以选取所需的字段并且按键分组。 由于时间戳字段具有毫秒的粒度,你可以使用自定义函数将其舍入到最近的小时。 -最后,选取所有的字段,用内建的 `sum` [聚合函数]({{ site.baseurl }}/zh/dev/table/functions.html#aggregate-functions)函数合计每一个账户每小时的支出。 +最后,选取所有的字段,用内建的 `sum` [聚合函数]({{ site.baseurl }}/zh/dev/table/systemFunctions.html#aggregate-functions)函数合计每一个账户每小时的支出。
    {% highlight java %} From c9c8a29b1b2e4f2886fba1524432f9788b564e61 Mon Sep 17 00:00:00 2001 From: forideal Date: Fri, 15 Nov 2019 09:10:45 +0800 Subject: [PATCH 630/746] [FLINK-14759][coordination] Remove unused class TaskManagerCliOptions This closes #10186 . --- .../taskmanager/TaskManagerCliOptions.java | 37 ------------------- 1 file changed, 37 deletions(-) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerCliOptions.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerCliOptions.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerCliOptions.java deleted file mode 100644 index e7120fd5ea35..000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerCliOptions.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.taskmanager; - -/** - * The command line parameters passed to the TaskManager. - */ -public class TaskManagerCliOptions { - - private String configDir; - - // ------------------------------------------------------------------------ - - public String getConfigDir() { - return configDir; - } - - public void setConfigDir(String configDir) { - this.configDir = configDir; - } -} From 65ecfab8d5a3f9ef7877ea35ff22fb72ec2db0a5 Mon Sep 17 00:00:00 2001 From: Leonard Xu Date: Thu, 14 Nov 2019 23:19:30 +0800 Subject: [PATCH 631/746] [FLINK-14784][table] CsvTableSink miss delimiter when row start with null member. This closes #10199 --- .../flink/table/sinks/CsvTableSink.java | 2 +- .../stream/table/TableSinkITCase.scala | 22 +++++++++--------- .../stream/table/TableSinkITCase.scala | 23 ++++++++++--------- 3 files changed, 24 insertions(+), 23 deletions(-) diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java index 56ae63bf9d7b..7026ebc1e2ef 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/sinks/CsvTableSink.java @@ -178,7 +178,7 @@ public String map(Row row) { StringBuilder builder = new StringBuilder(); Object o; for (int i = 0; i < row.getArity(); i++) { - if (builder.length() != 0) { + if (i > 0) { builder.append(fieldDelim); } if ((o = row.getField(i)) != null) { diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/TableSinkITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/TableSinkITCase.scala index c88914923758..c48cf9fd0fdb 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/TableSinkITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/TableSinkITCase.scala @@ -90,8 +90,8 @@ class TableSinkITCase extends AbstractTestBase { tEnv.registerTableSink( "csvSink", new CsvTableSink(path).configure( - Array[String]("c", "b"), - Array[TypeInformation[_]](Types.STRING, Types.SQL_TIMESTAMP))) + Array[String]("nullableCol", "c", "b"), + Array[TypeInformation[_]](Types.INT, Types.STRING, Types.SQL_TIMESTAMP))) val input = env.fromCollection(tupleData3) .assignAscendingTimestamps(_._2) @@ -99,20 +99,20 @@ class TableSinkITCase extends AbstractTestBase { input.toTable(tEnv, 'a, 'b.rowtime, 'c) .where('a < 5 || 'a > 17) - .select('c, 'b) + .select(ifThenElse('a < 4, nullOf(Types.INT()), 'a), 'c, 'b) .insertInto("csvSink") env.execute() val expected = Seq( - "Hi,1970-01-01 00:00:00.001", - "Hello,1970-01-01 00:00:00.002", - "Hello world,1970-01-01 00:00:00.002", - "Hello world, how are you?,1970-01-01 00:00:00.003", - "Comment#12,1970-01-01 00:00:00.006", - "Comment#13,1970-01-01 00:00:00.006", - "Comment#14,1970-01-01 00:00:00.006", - "Comment#15,1970-01-01 00:00:00.006").mkString("\n") + ",Hello world,1970-01-01 00:00:00.002", + ",Hello,1970-01-01 00:00:00.002", + ",Hi,1970-01-01 00:00:00.001", + "18,Comment#12,1970-01-01 00:00:00.006", + "19,Comment#13,1970-01-01 00:00:00.006", + "20,Comment#14,1970-01-01 00:00:00.006", + "21,Comment#15,1970-01-01 00:00:00.006", + "4,Hello world, how are you?,1970-01-01 00:00:00.003").mkString("\n") TestBaseUtils.compareResultsByLinesInMemory(expected, path) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala index cf3fcbf5a7e3..e6ff9818dbb6 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala @@ -95,8 +95,8 @@ class TableSinkITCase extends AbstractTestBase { tEnv.registerTableSink( "csvSink", new CsvTableSink(path).configure( - Array[String]("c", "b"), - Array[TypeInformation[_]](Types.STRING, Types.SQL_TIMESTAMP))) + Array[String]("nullableCol", "c", "b"), + Array[TypeInformation[_]](Types.INT, Types.STRING, Types.SQL_TIMESTAMP))) val input = StreamTestData.get3TupleDataStream(env) .assignAscendingTimestamps(_._2) @@ -104,20 +104,21 @@ class TableSinkITCase extends AbstractTestBase { input.toTable(tEnv, 'a, 'b.rowtime, 'c) .where('a < 5 || 'a > 17) - .select('c, 'b) + .select(ifThenElse('a < 4, nullOf(Types.INT()), 'a), 'c, 'b) .insertInto("csvSink") env.execute() val expected = Seq( - "Hi,1970-01-01 00:00:00.001", - "Hello,1970-01-01 00:00:00.002", - "Hello world,1970-01-01 00:00:00.002", - "Hello world, how are you?,1970-01-01 00:00:00.003", - "Comment#12,1970-01-01 00:00:00.006", - "Comment#13,1970-01-01 00:00:00.006", - "Comment#14,1970-01-01 00:00:00.006", - "Comment#15,1970-01-01 00:00:00.006").mkString("\n") + ",Hello world,1970-01-01 00:00:00.002", + ",Hello,1970-01-01 00:00:00.002", + ",Hi,1970-01-01 00:00:00.001", + "18,Comment#12,1970-01-01 00:00:00.006", + "19,Comment#13,1970-01-01 00:00:00.006", + "20,Comment#14,1970-01-01 00:00:00.006", + "21,Comment#15,1970-01-01 00:00:00.006", + "4,Hello world, how are you?,1970-01-01 00:00:00.003" + ).mkString("\n") TestBaseUtils.compareResultsByLinesInMemory(expected, path) } From 468415b85a59706c8e06b851ed792c5f5146b1ed Mon Sep 17 00:00:00 2001 From: "danrtsey.wy" Date: Mon, 11 Nov 2019 22:31:24 +0800 Subject: [PATCH 632/746] [FLINK-14382][java] Add MultipleParameterTool to support parsing multiple parameters of args. For example, --multi multiValue1 --multi multiValue2. --- .../java/org/apache/flink/api/java/Utils.java | 27 + .../api/java/utils/AbstractParameterTool.java | 308 ++++++++++ .../api/java/utils/MultipleParameterTool.java | 272 +++++++++ .../flink/api/java/utils/ParameterTool.java | 265 +-------- .../java/utils/AbstractParameterToolTest.java | 547 +++++++++++++++++- .../java/utils/MultipleParameterToolTest.java | 79 +++ .../api/java/utils/ParameterToolTest.java | 492 +--------------- 7 files changed, 1224 insertions(+), 766 deletions(-) create mode 100644 flink-java/src/main/java/org/apache/flink/api/java/utils/AbstractParameterTool.java create mode 100644 flink-java/src/main/java/org/apache/flink/api/java/utils/MultipleParameterTool.java create mode 100644 flink-java/src/test/java/org/apache/flink/api/java/utils/MultipleParameterToolTest.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/Utils.java b/flink-java/src/main/java/org/apache/flink/api/java/Utils.java index c514b333ec96..235283306345 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/Utils.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/Utils.java @@ -36,6 +36,7 @@ import java.io.IOException; import java.lang.reflect.Field; import java.lang.reflect.Modifier; +import java.util.Arrays; import java.util.Optional; import java.util.Random; @@ -317,6 +318,32 @@ public static Optional resolveFactory(ThreadLocal threadLocalFactory, return Optional.ofNullable(factory); } + /** + * Get the key from the given args. Keys have to start with '-' or '--'. For example, --key1 value1 -key2 value2. + * @param args all given args. + * @param index the index of args to be parsed. + * @return the key of the given arg. + */ + public static String getKeyFromArgs(String[] args, int index) { + String key; + if (args[index].startsWith("--")) { + key = args[index].substring(2); + } else if (args[index].startsWith("-")) { + key = args[index].substring(1); + } else { + throw new IllegalArgumentException( + String.format("Error parsing arguments '%s' on '%s'. Please prefix keys with -- or -.", + Arrays.toString(args), args[index])); + } + + if (key.isEmpty()) { + throw new IllegalArgumentException( + "The input " + Arrays.toString(args) + " contains an empty argument"); + } + + return key; + } + /** * Private constructor to prevent instantiation. */ diff --git a/flink-java/src/main/java/org/apache/flink/api/java/utils/AbstractParameterTool.java b/flink-java/src/main/java/org/apache/flink/api/java/utils/AbstractParameterTool.java new file mode 100644 index 000000000000..439e9ff3e83b --- /dev/null +++ b/flink-java/src/main/java/org/apache/flink/api/java/utils/AbstractParameterTool.java @@ -0,0 +1,308 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.java.utils; + +import org.apache.flink.annotation.Public; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.ExecutionConfig; + +import java.io.Serializable; +import java.util.Collections; +import java.util.Map; +import java.util.Set; + +/** + * This class provides common utility methods of {@link ParameterTool} and {@link MultipleParameterTool}. + */ +@Public +public abstract class AbstractParameterTool extends ExecutionConfig.GlobalJobParameters implements Serializable, Cloneable { + private static final long serialVersionUID = 1L; + + protected static final String NO_VALUE_KEY = "__NO_VALUE_KEY"; + protected static final String DEFAULT_UNDEFINED = ""; + + // ------------------ ParameterUtil ------------------------ + + // data which is only used on the client and does not need to be transmitted + protected transient Map defaultData; + protected transient Set unrequestedParameters; + + /** + * Returns the set of parameter names which have not been requested with + * {@link #has(String)} or one of the {@code get} methods. Access to the + * map returned by {@link #toMap()} is not tracked. + */ + @PublicEvolving + public Set getUnrequestedParameters() { + return Collections.unmodifiableSet(unrequestedParameters); + } + + // ------------------ Get data from the util ---------------- + + /** + * Returns number of parameters in {@link AbstractParameterTool}. + */ + protected abstract int getNumberOfParameters(); + + /** + * Returns the String value for the given key. + * If the key does not exist it will return null. + */ + protected abstract String get(String key); + + /** + * Check if value is set. + */ + public abstract boolean has(String value); + + /** + * Returns the String value for the given key. + * If the key does not exist it will throw a {@link RuntimeException}. + */ + public String getRequired(String key) { + addToDefaults(key, null); + String value = get(key); + if (value == null) { + throw new RuntimeException("No data for required key '" + key + "'"); + } + return value; + } + + /** + * Returns the String value for the given key. + * If the key does not exist it will return the given default value. + */ + public String get(String key, String defaultValue) { + addToDefaults(key, defaultValue); + String value = get(key); + if (value == null) { + return defaultValue; + } else { + return value; + } + } + + // -------------- Integer + + /** + * Returns the Integer value for the given key. + * The method fails if the key does not exist or the value is not an Integer. + */ + public int getInt(String key) { + addToDefaults(key, null); + String value = getRequired(key); + return Integer.parseInt(value); + } + + /** + * Returns the Integer value for the given key. If the key does not exists it will return the default value given. + * The method fails if the value is not an Integer. + */ + public int getInt(String key, int defaultValue) { + addToDefaults(key, Integer.toString(defaultValue)); + String value = get(key); + if (value == null) { + return defaultValue; + } + return Integer.parseInt(value); + } + + // -------------- LONG + + /** + * Returns the Long value for the given key. + * The method fails if the key does not exist. + */ + public long getLong(String key) { + addToDefaults(key, null); + String value = getRequired(key); + return Long.parseLong(value); + } + + /** + * Returns the Long value for the given key. If the key does not exists it will return the default value given. + * The method fails if the value is not a Long. + */ + public long getLong(String key, long defaultValue) { + addToDefaults(key, Long.toString(defaultValue)); + String value = get(key); + if (value == null) { + return defaultValue; + } + return Long.parseLong(value); + } + + // -------------- FLOAT + + /** + * Returns the Float value for the given key. + * The method fails if the key does not exist. + */ + public float getFloat(String key) { + addToDefaults(key, null); + String value = getRequired(key); + return Float.valueOf(value); + } + + /** + * Returns the Float value for the given key. If the key does not exists it will return the default value given. + * The method fails if the value is not a Float. + */ + public float getFloat(String key, float defaultValue) { + addToDefaults(key, Float.toString(defaultValue)); + String value = get(key); + if (value == null) { + return defaultValue; + } else { + return Float.valueOf(value); + } + } + + // -------------- DOUBLE + + /** + * Returns the Double value for the given key. + * The method fails if the key does not exist. + */ + public double getDouble(String key) { + addToDefaults(key, null); + String value = getRequired(key); + return Double.valueOf(value); + } + + /** + * Returns the Double value for the given key. If the key does not exists it will return the default value given. + * The method fails if the value is not a Double. + */ + public double getDouble(String key, double defaultValue) { + addToDefaults(key, Double.toString(defaultValue)); + String value = get(key); + if (value == null) { + return defaultValue; + } else { + return Double.valueOf(value); + } + } + + // -------------- BOOLEAN + + /** + * Returns the Boolean value for the given key. + * The method fails if the key does not exist. + */ + public boolean getBoolean(String key) { + addToDefaults(key, null); + String value = getRequired(key); + return Boolean.valueOf(value); + } + + /** + * Returns the Boolean value for the given key. If the key does not exists it will return the default value given. + * The method returns whether the string of the value is "true" ignoring cases. + */ + public boolean getBoolean(String key, boolean defaultValue) { + addToDefaults(key, Boolean.toString(defaultValue)); + String value = get(key); + if (value == null) { + return defaultValue; + } else { + return Boolean.valueOf(value); + } + } + + // -------------- SHORT + + /** + * Returns the Short value for the given key. + * The method fails if the key does not exist. + */ + public short getShort(String key) { + addToDefaults(key, null); + String value = getRequired(key); + return Short.valueOf(value); + } + + /** + * Returns the Short value for the given key. If the key does not exists it will return the default value given. + * The method fails if the value is not a Short. + */ + public short getShort(String key, short defaultValue) { + addToDefaults(key, Short.toString(defaultValue)); + String value = get(key); + if (value == null) { + return defaultValue; + } else { + return Short.valueOf(value); + } + } + + // -------------- BYTE + + /** + * Returns the Byte value for the given key. + * The method fails if the key does not exist. + */ + public byte getByte(String key) { + addToDefaults(key, null); + String value = getRequired(key); + return Byte.valueOf(value); + } + + /** + * Returns the Byte value for the given key. If the key does not exists it will return the default value given. + * The method fails if the value is not a Byte. + */ + public byte getByte(String key, byte defaultValue) { + addToDefaults(key, Byte.toString(defaultValue)); + String value = get(key); + if (value == null) { + return defaultValue; + } else { + return Byte.valueOf(value); + } + } + + // --------------- Internals + + protected void addToDefaults(String key, String value) { + final String currentValue = defaultData.get(key); + if (currentValue == null) { + if (value == null) { + value = DEFAULT_UNDEFINED; + } + defaultData.put(key, value); + } else { + // there is already an entry for this key. Check if the value is the undefined + if (currentValue.equals(DEFAULT_UNDEFINED) && value != null) { + // update key with better default value + defaultData.put(key, value); + } + } + } + + // ------------------------- Export to different targets ------------------------- + + @Override + protected abstract Object clone() throws CloneNotSupportedException; + + // ------------------------- ExecutionConfig.UserConfig interface ------------------------- + + @Override + public abstract Map toMap(); +} diff --git a/flink-java/src/main/java/org/apache/flink/api/java/utils/MultipleParameterTool.java b/flink-java/src/main/java/org/apache/flink/api/java/utils/MultipleParameterTool.java new file mode 100644 index 000000000000..1e68db2bbfa8 --- /dev/null +++ b/flink-java/src/main/java/org/apache/flink/api/java/utils/MultipleParameterTool.java @@ -0,0 +1,272 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.java.utils; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.java.Utils; +import org.apache.flink.util.Preconditions; + +import org.apache.commons.lang3.math.NumberUtils; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +/** + * This class provides simple utility methods for reading and parsing program arguments from different sources. + * Multiple values parameter in args could be supported. For example, --multi multiValue1 --multi multiValue2. + * If {@link MultipleParameterTool} object is used for GlobalJobParameters, the last one of multiple values will be used. + * Navigate to {@link #toMap()} for more information. + */ +@PublicEvolving +public class MultipleParameterTool extends AbstractParameterTool { + private static final long serialVersionUID = 1L; + + // ------------------ Constructors ------------------------ + + /** + * Returns {@link MultipleParameterTool} for the given arguments. The arguments are keys followed by values. + * Keys have to start with '-' or '--' + * + *

    Example arguments: + * --key1 value1 --key2 value2 -key3 value3 + * --multi multiValue1 --multi multiValue2 + * + * @param args Input array arguments + * @return A {@link MultipleParameterTool} + */ + public static MultipleParameterTool fromArgs(String[] args) { + final Map> map = new HashMap<>(args.length / 2); + + int i = 0; + while (i < args.length) { + final String key = Utils.getKeyFromArgs(args, i); + + i += 1; // try to find the value + + map.putIfAbsent(key, new ArrayList<>()); + if (i >= args.length) { + map.get(key).add(NO_VALUE_KEY); + } else if (NumberUtils.isNumber(args[i])) { + map.get(key).add(args[i]); + i += 1; + } else if (args[i].startsWith("--") || args[i].startsWith("-")) { + // the argument cannot be a negative number because we checked earlier + // -> the next argument is a parameter name + map.get(key).add(NO_VALUE_KEY); + } else { + map.get(key).add(args[i]); + i += 1; + } + } + + return fromMultiMap(map); + } + + /** + * Returns {@link MultipleParameterTool} for the given multi map. + * + * @param multiMap A map of arguments. Key is String and value is a Collection. + * @return A {@link MultipleParameterTool} + */ + public static MultipleParameterTool fromMultiMap(Map> multiMap) { + Preconditions.checkNotNull(multiMap, "Unable to initialize from empty map"); + return new MultipleParameterTool(multiMap); + } + + // ------------------ ParameterUtil ------------------------ + protected final Map> data; + + private MultipleParameterTool(Map> data) { + this.data = Collections.unmodifiableMap(new HashMap<>(data)); + + this.defaultData = new ConcurrentHashMap<>(data.size()); + + this.unrequestedParameters = Collections.newSetFromMap(new ConcurrentHashMap<>(data.size())); + + unrequestedParameters.addAll(data.keySet()); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MultipleParameterTool that = (MultipleParameterTool) o; + return Objects.equals(data, that.data) && + Objects.equals(defaultData, that.defaultData) && + Objects.equals(unrequestedParameters, that.unrequestedParameters); + } + + @Override + public int hashCode() { + return Objects.hash(data, defaultData, unrequestedParameters); + } + + // ------------------ Get data from the util ---------------- + + /** + * Returns number of parameters in {@link ParameterTool}. + */ + @Override + public int getNumberOfParameters() { + return data.size(); + } + + /** + * Returns the String value for the given key. The value should only have one item. + * Use {@link #getMultiParameter(String)} instead if want to get multiple values parameter. + * If the key does not exist it will return null. + */ + @Override + public String get(String key) { + addToDefaults(key, null); + unrequestedParameters.remove(key); + if (!data.containsKey(key)) { + return null; + } + Preconditions.checkState(data.get(key).size() == 1, + "Key %s should has only one value.", key); + return (String) data.get(key).toArray()[0]; + } + + /** + * Check if value is set. + */ + @Override + public boolean has(String value) { + addToDefaults(value, null); + unrequestedParameters.remove(value); + return data.containsKey(value); + } + + /** + * Returns the Collection of String values for the given key. + * If the key does not exist it will return null. + */ + public Collection getMultiParameter(String key) { + addToDefaults(key, null); + unrequestedParameters.remove(key); + return data.getOrDefault(key, null); + } + + /** + * Returns the Collection of String values for the given key. + * If the key does not exist it will throw a {@link RuntimeException}. + */ + public Collection getMultiParameterRequired(String key) { + addToDefaults(key, null); + Collection value = getMultiParameter(key); + if (value == null) { + throw new RuntimeException("No data for required key '" + key + "'"); + } + return value; + } + + // ------------------------- Export to different targets ------------------------- + + /** + * Return MultiMap of all the parameters processed by {@link MultipleParameterTool}. + * + * @return MultiMap of the {@link MultipleParameterTool}. Key is String and Value is a Collection of String. + */ + public Map> toMultiMap() { + return data; + } + + @Override + protected Object clone() throws CloneNotSupportedException { + return new MultipleParameterTool(this.data); + } + + // ------------------------- Interaction with other ParameterUtils ------------------------- + + /** + * Merges two {@link MultipleParameterTool}. + * + * @param other Other {@link MultipleParameterTool} object + * @return The Merged {@link MultipleParameterTool} + */ + public MultipleParameterTool mergeWith(MultipleParameterTool other) { + final Map> resultData = new HashMap<>(data.size() + other.data.size()); + resultData.putAll(data); + other.data.forEach((key, value) -> { + resultData.putIfAbsent(key, new ArrayList<>()); + resultData.get(key).addAll(value); + }); + + final MultipleParameterTool ret = new MultipleParameterTool(resultData); + + final HashSet requestedParametersLeft = new HashSet<>(data.keySet()); + requestedParametersLeft.removeAll(unrequestedParameters); + + final HashSet requestedParametersRight = new HashSet<>(other.data.keySet()); + requestedParametersRight.removeAll(other.unrequestedParameters); + + ret.unrequestedParameters.removeAll(requestedParametersLeft); + ret.unrequestedParameters.removeAll(requestedParametersRight); + + return ret; + } + + // ------------------------- ExecutionConfig.UserConfig interface ------------------------- + + @Override + public Map toMap() { + return getFlatMapOfData(data); + } + + /** + * Get the flat map of the multiple map data. If the key have multiple values, only the last one will be used. This + * is also the current behavior when multiple parameters is specified for {@link ParameterTool}. + * @param data multiple map of data. + * @return flat map of data. + */ + private static Map getFlatMapOfData(Map> data) { + return data.entrySet().stream().collect(Collectors.toMap( + Map.Entry::getKey, + e -> { + if (e.getValue().size() > 0) { + return (String) e.getValue().toArray()[e.getValue().size() - 1]; + } else { + return NO_VALUE_KEY; + } + })); + } + + // ------------------------- Serialization --------------------------------------------- + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + + defaultData = new ConcurrentHashMap<>(data.size()); + unrequestedParameters = Collections.newSetFromMap(new ConcurrentHashMap<>(data.size())); + } +} diff --git a/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java b/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java index 33b6ab9d3794..9d9ef3da56b5 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java @@ -19,8 +19,7 @@ package org.apache.flink.api.java.utils; import org.apache.flink.annotation.Public; -import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.java.Utils; import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Preconditions; @@ -34,7 +33,6 @@ import java.io.InputStream; import java.io.ObjectInputStream; import java.io.OutputStream; -import java.io.Serializable; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -42,19 +40,16 @@ import java.util.Map; import java.util.Objects; import java.util.Properties; -import java.util.Set; import java.util.concurrent.ConcurrentHashMap; /** * This class provides simple utility methods for reading and parsing program arguments from different sources. + * Only single value parameter could be supported in args. */ @Public -public class ParameterTool extends ExecutionConfig.GlobalJobParameters implements Serializable, Cloneable { +public class ParameterTool extends AbstractParameterTool { private static final long serialVersionUID = 1L; - protected static final String NO_VALUE_KEY = "__NO_VALUE_KEY"; - protected static final String DEFAULT_UNDEFINED = ""; - // ------------------ Constructors ------------------------ /** @@ -72,17 +67,7 @@ public static ParameterTool fromArgs(String[] args) { int i = 0; while (i < args.length) { - final String key; - - if (args[i].startsWith("--")) { - key = args[i].substring(2); - } else if (args[i].startsWith("-")) { - key = args[i].substring(1); - } else { - throw new IllegalArgumentException( - String.format("Error parsing arguments '%s' on '%s'. Please prefix keys with -- or -.", - Arrays.toString(args), args[i])); - } + final String key = Utils.getKeyFromArgs(args, i); if (key.isEmpty()) { throw new IllegalArgumentException( @@ -178,10 +163,6 @@ public static ParameterTool fromSystemProperties() { // ------------------ ParameterUtil ------------------------ protected final Map data; - // data which is only used on the client and does not need to be transmitted - protected transient Map defaultData; - protected transient Set unrequestedParameters; - private ParameterTool(Map data) { this.data = Collections.unmodifiableMap(new HashMap<>(data)); @@ -211,21 +192,12 @@ public int hashCode() { return Objects.hash(data, defaultData, unrequestedParameters); } - /** - * Returns the set of parameter names which have not been requested with - * {@link #has(String)} or one of the {@code get} methods. Access to the - * map returned by {@link #toMap()} is not tracked. - */ - @PublicEvolving - public Set getUnrequestedParameters() { - return Collections.unmodifiableSet(unrequestedParameters); - } - // ------------------ Get data from the util ---------------- /** * Returns number of parameters in {@link ParameterTool}. */ + @Override public int getNumberOfParameters() { return data.size(); } @@ -234,246 +206,23 @@ public int getNumberOfParameters() { * Returns the String value for the given key. * If the key does not exist it will return null. */ + @Override public String get(String key) { addToDefaults(key, null); unrequestedParameters.remove(key); return data.get(key); } - /** - * Returns the String value for the given key. - * If the key does not exist it will throw a {@link RuntimeException}. - */ - public String getRequired(String key) { - addToDefaults(key, null); - String value = get(key); - if (value == null) { - throw new RuntimeException("No data for required key '" + key + "'"); - } - return value; - } - - /** - * Returns the String value for the given key. - * If the key does not exist it will return the given default value. - */ - public String get(String key, String defaultValue) { - addToDefaults(key, defaultValue); - String value = get(key); - if (value == null) { - return defaultValue; - } else { - return value; - } - } - /** * Check if value is set. */ + @Override public boolean has(String value) { addToDefaults(value, null); unrequestedParameters.remove(value); return data.containsKey(value); } - // -------------- Integer - - /** - * Returns the Integer value for the given key. - * The method fails if the key does not exist or the value is not an Integer. - */ - public int getInt(String key) { - addToDefaults(key, null); - String value = getRequired(key); - return Integer.parseInt(value); - } - - /** - * Returns the Integer value for the given key. If the key does not exists it will return the default value given. - * The method fails if the value is not an Integer. - */ - public int getInt(String key, int defaultValue) { - addToDefaults(key, Integer.toString(defaultValue)); - String value = get(key); - if (value == null) { - return defaultValue; - } - return Integer.parseInt(value); - } - - // -------------- LONG - - /** - * Returns the Long value for the given key. - * The method fails if the key does not exist. - */ - public long getLong(String key) { - addToDefaults(key, null); - String value = getRequired(key); - return Long.parseLong(value); - } - - /** - * Returns the Long value for the given key. If the key does not exists it will return the default value given. - * The method fails if the value is not a Long. - */ - public long getLong(String key, long defaultValue) { - addToDefaults(key, Long.toString(defaultValue)); - String value = get(key); - if (value == null) { - return defaultValue; - } - return Long.parseLong(value); - } - - // -------------- FLOAT - - /** - * Returns the Float value for the given key. - * The method fails if the key does not exist. - */ - public float getFloat(String key) { - addToDefaults(key, null); - String value = getRequired(key); - return Float.valueOf(value); - } - - /** - * Returns the Float value for the given key. If the key does not exists it will return the default value given. - * The method fails if the value is not a Float. - */ - public float getFloat(String key, float defaultValue) { - addToDefaults(key, Float.toString(defaultValue)); - String value = get(key); - if (value == null) { - return defaultValue; - } else { - return Float.valueOf(value); - } - } - - // -------------- DOUBLE - - /** - * Returns the Double value for the given key. - * The method fails if the key does not exist. - */ - public double getDouble(String key) { - addToDefaults(key, null); - String value = getRequired(key); - return Double.valueOf(value); - } - - /** - * Returns the Double value for the given key. If the key does not exists it will return the default value given. - * The method fails if the value is not a Double. - */ - public double getDouble(String key, double defaultValue) { - addToDefaults(key, Double.toString(defaultValue)); - String value = get(key); - if (value == null) { - return defaultValue; - } else { - return Double.valueOf(value); - } - } - - // -------------- BOOLEAN - - /** - * Returns the Boolean value for the given key. - * The method fails if the key does not exist. - */ - public boolean getBoolean(String key) { - addToDefaults(key, null); - String value = getRequired(key); - return Boolean.valueOf(value); - } - - /** - * Returns the Boolean value for the given key. If the key does not exists it will return the default value given. - * The method returns whether the string of the value is "true" ignoring cases. - */ - public boolean getBoolean(String key, boolean defaultValue) { - addToDefaults(key, Boolean.toString(defaultValue)); - String value = get(key); - if (value == null) { - return defaultValue; - } else { - return Boolean.valueOf(value); - } - } - - // -------------- SHORT - - /** - * Returns the Short value for the given key. - * The method fails if the key does not exist. - */ - public short getShort(String key) { - addToDefaults(key, null); - String value = getRequired(key); - return Short.valueOf(value); - } - - /** - * Returns the Short value for the given key. If the key does not exists it will return the default value given. - * The method fails if the value is not a Short. - */ - public short getShort(String key, short defaultValue) { - addToDefaults(key, Short.toString(defaultValue)); - String value = get(key); - if (value == null) { - return defaultValue; - } else { - return Short.valueOf(value); - } - } - - // -------------- BYTE - - /** - * Returns the Byte value for the given key. - * The method fails if the key does not exist. - */ - public byte getByte(String key) { - addToDefaults(key, null); - String value = getRequired(key); - return Byte.valueOf(value); - } - - /** - * Returns the Byte value for the given key. If the key does not exists it will return the default value given. - * The method fails if the value is not a Byte. - */ - public byte getByte(String key, byte defaultValue) { - addToDefaults(key, Byte.toString(defaultValue)); - String value = get(key); - if (value == null) { - return defaultValue; - } else { - return Byte.valueOf(value); - } - } - - // --------------- Internals - - protected void addToDefaults(String key, String value) { - final String currentValue = defaultData.get(key); - if (currentValue == null) { - if (value == null) { - value = DEFAULT_UNDEFINED; - } - defaultData.put(key, value); - } else { - // there is already an entry for this key. Check if the value is the undefined - if (currentValue.equals(DEFAULT_UNDEFINED) && value != null) { - // update key with better default value - defaultData.put(key, value); - } - } - } - // ------------------------- Export to different targets ------------------------- /** diff --git a/flink-java/src/test/java/org/apache/flink/api/java/utils/AbstractParameterToolTest.java b/flink-java/src/test/java/org/apache/flink/api/java/utils/AbstractParameterToolTest.java index 40cf28e7b843..07e7c35f20fa 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/utils/AbstractParameterToolTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/utils/AbstractParameterToolTest.java @@ -24,11 +24,18 @@ import org.junit.Assert; import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; import java.io.FileInputStream; import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; import java.util.Properties; +import java.util.Set; /** * Base class for tests for {@link ParameterTool}. @@ -38,21 +45,511 @@ public abstract class AbstractParameterToolTest { @Rule public TemporaryFolder tmp = new TemporaryFolder(); - protected void validate(ParameterTool parameter) { + @Rule + public final ExpectedException exception = ExpectedException.none(); + + // Test parser + + @Test + public void testThrowExceptionIfParameterIsNotPrefixed() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("Error parsing arguments '[a]' on 'a'. Please prefix keys with -- or -."); + + createParameterToolFromArgs(new String[]{"a"}); + } + + @Test + public void testNoVal() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-berlin"}); + Assert.assertEquals(1, parameter.getNumberOfParameters()); + Assert.assertTrue(parameter.has("berlin")); + } + + @Test + public void testNoValDouble() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"--berlin"}); + Assert.assertEquals(1, parameter.getNumberOfParameters()); + Assert.assertTrue(parameter.has("berlin")); + } + + @Test + public void testMultipleNoVal() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"--a", "--b", "--c", "--d", "--e", "--f"}); + Assert.assertEquals(6, parameter.getNumberOfParameters()); + Assert.assertTrue(parameter.has("a")); + Assert.assertTrue(parameter.has("b")); + Assert.assertTrue(parameter.has("c")); + Assert.assertTrue(parameter.has("d")); + Assert.assertTrue(parameter.has("e")); + Assert.assertTrue(parameter.has("f")); + } + + @Test + public void testMultipleNoValMixed() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"--a", "-b", "-c", "-d", "--e", "--f"}); + Assert.assertEquals(6, parameter.getNumberOfParameters()); + Assert.assertTrue(parameter.has("a")); + Assert.assertTrue(parameter.has("b")); + Assert.assertTrue(parameter.has("c")); + Assert.assertTrue(parameter.has("d")); + Assert.assertTrue(parameter.has("e")); + Assert.assertTrue(parameter.has("f")); + } + + @Test + public void testEmptyVal() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("The input [--a, -b, --] contains an empty argument"); + + createParameterToolFromArgs(new String[]{"--a", "-b", "--"}); + } + + @Test + public void testEmptyValShort() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("The input [--a, -b, -] contains an empty argument"); + + createParameterToolFromArgs(new String[]{"--a", "-b", "-"}); + } + + // Test unrequested + // Boolean + + @Test + public void testUnrequestedBoolean() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-boolean", "true"}); + Assert.assertEquals(createHashSet("boolean"), parameter.getUnrequestedParameters()); + + // test parameter access + Assert.assertTrue(parameter.getBoolean("boolean")); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + + // test repeated access + Assert.assertTrue(parameter.getBoolean("boolean")); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + } + + @Test + public void testUnrequestedBooleanWithDefaultValue() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-boolean", "true"}); + Assert.assertEquals(createHashSet("boolean"), parameter.getUnrequestedParameters()); + + // test parameter access + Assert.assertTrue(parameter.getBoolean("boolean", false)); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + + // test repeated access + Assert.assertTrue(parameter.getBoolean("boolean", false)); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + } + + @Test + public void testUnrequestedBooleanWithMissingValue() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-boolean"}); + Assert.assertEquals(createHashSet("boolean"), parameter.getUnrequestedParameters()); + + parameter.getBoolean("boolean"); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + } + + // Byte + + @Test + public void testUnrequestedByte() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-byte", "1"}); + Assert.assertEquals(createHashSet("byte"), parameter.getUnrequestedParameters()); + + // test parameter access + Assert.assertEquals(1, parameter.getByte("byte")); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + + // test repeated access + Assert.assertEquals(1, parameter.getByte("byte")); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + } + + @Test + public void testUnrequestedByteWithDefaultValue() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-byte", "1"}); + Assert.assertEquals(createHashSet("byte"), parameter.getUnrequestedParameters()); + + // test parameter access + Assert.assertEquals(1, parameter.getByte("byte", (byte) 0)); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + + // test repeated access + Assert.assertEquals(1, parameter.getByte("byte", (byte) 0)); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + } + + @Test + public void testUnrequestedByteWithMissingValue() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-byte"}); + Assert.assertEquals(createHashSet("byte"), parameter.getUnrequestedParameters()); + + exception.expect(RuntimeException.class); + exception.expectMessage("For input string: \"__NO_VALUE_KEY\""); + + parameter.getByte("byte"); + } + + // Short + + @Test + public void testUnrequestedShort() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-short", "2"}); + Assert.assertEquals(createHashSet("short"), parameter.getUnrequestedParameters()); + + // test parameter access + Assert.assertEquals(2, parameter.getShort("short")); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + + // test repeated access + Assert.assertEquals(2, parameter.getShort("short")); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + } + + @Test + public void testUnrequestedShortWithDefaultValue() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-short", "2"}); + Assert.assertEquals(createHashSet("short"), parameter.getUnrequestedParameters()); + + // test parameter access + Assert.assertEquals(2, parameter.getShort("short", (short) 0)); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + + // test repeated access + Assert.assertEquals(2, parameter.getShort("short", (short) 0)); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + } + + @Test + public void testUnrequestedShortWithMissingValue() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-short"}); + Assert.assertEquals(createHashSet("short"), parameter.getUnrequestedParameters()); + + exception.expect(RuntimeException.class); + exception.expectMessage("For input string: \"__NO_VALUE_KEY\""); + + parameter.getShort("short"); + } + + // Int + + @Test + public void testUnrequestedInt() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-int", "4"}); + Assert.assertEquals(createHashSet("int"), parameter.getUnrequestedParameters()); + + // test parameter access + Assert.assertEquals(4, parameter.getInt("int")); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + + // test repeated access + Assert.assertEquals(4, parameter.getInt("int")); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + } + + @Test + public void testUnrequestedIntWithDefaultValue() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-int", "4"}); + Assert.assertEquals(createHashSet("int"), parameter.getUnrequestedParameters()); + + // test parameter access + Assert.assertEquals(4, parameter.getInt("int", 0)); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + + // test repeated access + Assert.assertEquals(4, parameter.getInt("int", 0)); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + } + + @Test + public void testUnrequestedIntWithMissingValue() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-int"}); + Assert.assertEquals(createHashSet("int"), parameter.getUnrequestedParameters()); + + exception.expect(RuntimeException.class); + exception.expectMessage("For input string: \"__NO_VALUE_KEY\""); + + parameter.getInt("int"); + } + + // Long + + @Test + public void testUnrequestedLong() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-long", "8"}); + Assert.assertEquals(createHashSet("long"), parameter.getUnrequestedParameters()); + + // test parameter access + Assert.assertEquals(8, parameter.getLong("long")); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + + // test repeated access + Assert.assertEquals(8, parameter.getLong("long")); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + } + + @Test + public void testUnrequestedLongWithDefaultValue() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-long", "8"}); + Assert.assertEquals(createHashSet("long"), parameter.getUnrequestedParameters()); + + // test parameter access + Assert.assertEquals(8, parameter.getLong("long", 0)); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + + // test repeated access + Assert.assertEquals(8, parameter.getLong("long", 0)); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + } + + @Test + public void testUnrequestedLongWithMissingValue() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-long"}); + Assert.assertEquals(createHashSet("long"), parameter.getUnrequestedParameters()); + + exception.expect(RuntimeException.class); + exception.expectMessage("For input string: \"__NO_VALUE_KEY\""); + + parameter.getLong("long"); + } + + // Float + + @Test + public void testUnrequestedFloat() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-float", "4"}); + Assert.assertEquals(createHashSet("float"), parameter.getUnrequestedParameters()); + + // test parameter access + Assert.assertEquals(4.0, parameter.getFloat("float"), 0.00001); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + + // test repeated access + Assert.assertEquals(4.0, parameter.getFloat("float"), 0.00001); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + } + + @Test + public void testUnrequestedFloatWithDefaultValue() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-float", "4"}); + Assert.assertEquals(createHashSet("float"), parameter.getUnrequestedParameters()); + + // test parameter access + Assert.assertEquals(4.0, parameter.getFloat("float", 0.0f), 0.00001); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + + // test repeated access + Assert.assertEquals(4.0, parameter.getFloat("float", 0.0f), 0.00001); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + } + + @Test + public void testUnrequestedFloatWithMissingValue() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-float"}); + Assert.assertEquals(createHashSet("float"), parameter.getUnrequestedParameters()); + + exception.expect(RuntimeException.class); + exception.expectMessage("For input string: \"__NO_VALUE_KEY\""); + + parameter.getFloat("float"); + } + + // Double + + @Test + public void testUnrequestedDouble() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-double", "8"}); + Assert.assertEquals(createHashSet("double"), parameter.getUnrequestedParameters()); + + // test parameter access + Assert.assertEquals(8.0, parameter.getDouble("double"), 0.00001); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + + // test repeated access + Assert.assertEquals(8.0, parameter.getDouble("double"), 0.00001); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + } + + @Test + public void testUnrequestedDoubleWithDefaultValue() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-double", "8"}); + Assert.assertEquals(createHashSet("double"), parameter.getUnrequestedParameters()); + + // test parameter access + Assert.assertEquals(8.0, parameter.getDouble("double", 0.0), 0.00001); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + + // test repeated access + Assert.assertEquals(8.0, parameter.getDouble("double", 0.0), 0.00001); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + } + + @Test + public void testUnrequestedDoubleWithMissingValue() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-double"}); + Assert.assertEquals(createHashSet("double"), parameter.getUnrequestedParameters()); + + exception.expect(RuntimeException.class); + exception.expectMessage("For input string: \"__NO_VALUE_KEY\""); + + parameter.getDouble("double"); + } + + // String + + @Test + public void testUnrequestedString() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-string", "∞"}); + Assert.assertEquals(createHashSet("string"), parameter.getUnrequestedParameters()); + + // test parameter access + Assert.assertEquals("∞", parameter.get("string")); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + + // test repeated access + Assert.assertEquals("∞", parameter.get("string")); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + } + + @Test + public void testUnrequestedStringWithDefaultValue() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-string", "∞"}); + Assert.assertEquals(createHashSet("string"), parameter.getUnrequestedParameters()); + + // test parameter access + Assert.assertEquals("∞", parameter.get("string", "0.0")); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + + // test repeated access + Assert.assertEquals("∞", parameter.get("string", "0.0")); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + } + + @Test + public void testUnrequestedStringWithMissingValue() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-string"}); + Assert.assertEquals(createHashSet("string"), parameter.getUnrequestedParameters()); + + parameter.get("string"); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + } + + // Additional methods + + @Test + public void testUnrequestedHas() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-boolean"}); + Assert.assertEquals(createHashSet("boolean"), parameter.getUnrequestedParameters()); + + // test parameter access + Assert.assertTrue(parameter.has("boolean")); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + + // test repeated access + Assert.assertTrue(parameter.has("boolean")); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + } + + @Test + public void testUnrequestedRequired() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-required", "∞"}); + Assert.assertEquals(createHashSet("required"), parameter.getUnrequestedParameters()); + + // test parameter access + Assert.assertEquals("∞", parameter.getRequired("required")); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + + // test repeated access + Assert.assertEquals("∞", parameter.getRequired("required")); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + } + + @Test + public void testUnrequestedMultiple() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{"-boolean", "true", "-byte", "1", + "-short", "2", "-int", "4", "-long", "8", "-float", "4.0", "-double", "8.0", "-string", "∞"}); + Assert.assertEquals(createHashSet("boolean", "byte", "short", "int", "long", "float", "double", "string"), + parameter.getUnrequestedParameters()); + + Assert.assertTrue(parameter.getBoolean("boolean")); + Assert.assertEquals(createHashSet("byte", "short", "int", "long", "float", "double", "string"), + parameter.getUnrequestedParameters()); + + Assert.assertEquals(1, parameter.getByte("byte")); + Assert.assertEquals(createHashSet("short", "int", "long", "float", "double", "string"), + parameter.getUnrequestedParameters()); + + Assert.assertEquals(2, parameter.getShort("short")); + Assert.assertEquals(createHashSet("int", "long", "float", "double", "string"), + parameter.getUnrequestedParameters()); + + Assert.assertEquals(4, parameter.getInt("int")); + Assert.assertEquals(createHashSet("long", "float", "double", "string"), + parameter.getUnrequestedParameters()); + + Assert.assertEquals(8, parameter.getLong("long")); + Assert.assertEquals(createHashSet("float", "double", "string"), + parameter.getUnrequestedParameters()); + + Assert.assertEquals(4.0, parameter.getFloat("float"), 0.00001); + Assert.assertEquals(createHashSet("double", "string"), + parameter.getUnrequestedParameters()); + + Assert.assertEquals(8.0, parameter.getDouble("double"), 0.00001); + Assert.assertEquals(createHashSet("string"), + parameter.getUnrequestedParameters()); + + Assert.assertEquals("∞", parameter.get("string")); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + } + + @Test + public void testUnrequestedUnknown() { + AbstractParameterTool parameter = createParameterToolFromArgs(new String[]{}); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + + Assert.assertTrue(parameter.getBoolean("boolean", true)); + Assert.assertEquals(0, parameter.getByte("byte", (byte) 0)); + Assert.assertEquals(0, parameter.getShort("short", (short) 0)); + Assert.assertEquals(0, parameter.getInt("int", 0)); + Assert.assertEquals(0, parameter.getLong("long", 0)); + Assert.assertEquals(0, parameter.getFloat("float", 0), 0.00001); + Assert.assertEquals(0, parameter.getDouble("double", 0), 0.00001); + Assert.assertEquals("0", parameter.get("string", "0")); + + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + } + + protected AbstractParameterTool createParameterToolFromArgs(String[] args) { + return ParameterTool.fromArgs(args); + } + + protected static Set createHashSet(T... elements) { + Set set = new HashSet<>(); + for (T element : elements) { + set.add(element); + } + return set; + } + + protected void validate(AbstractParameterTool parameter) { ClosureCleaner.ensureSerializable(parameter); internalValidate(parameter); // -------- test behaviour after serialization ------------ try { byte[] b = InstantiationUtil.serializeObject(parameter); - final ParameterTool copy = InstantiationUtil.deserializeObject(b, getClass().getClassLoader()); + final AbstractParameterTool copy = InstantiationUtil.deserializeObject(b, getClass().getClassLoader()); internalValidate(copy); } catch (IOException | ClassNotFoundException e) { throw new RuntimeException(e); } } - private void internalValidate(ParameterTool parameter) { + private void internalValidate(AbstractParameterTool parameter) { Assert.assertEquals("myInput", parameter.getRequired("input")); Assert.assertEquals("myDefaultValue", parameter.get("output", "myDefaultValue")); Assert.assertNull(parameter.get("whatever")); @@ -61,27 +558,39 @@ private void internalValidate(ParameterTool parameter) { Assert.assertEquals(42, parameter.getByte("myDefaultByte", (byte) 42)); Assert.assertEquals(42, parameter.getShort("myDefaultShort", (short) 42)); - final Configuration config = parameter.getConfiguration(); - Assert.assertEquals(15L, config.getLong("expectedCount", -1L)); + if (parameter instanceof ParameterTool) { + ParameterTool parameterTool = (ParameterTool) parameter; + final Configuration config = parameterTool.getConfiguration(); + Assert.assertEquals(15L, config.getLong("expectedCount", -1L)); - final Properties props = parameter.getProperties(); - Assert.assertEquals("myInput", props.getProperty("input")); + final Properties props = parameterTool.getProperties(); + Assert.assertEquals("myInput", props.getProperty("input")); - // -------- test the default file creation ------------ - try { - final String pathToFile = tmp.newFile().getAbsolutePath(); - parameter.createPropertiesFile(pathToFile); - final Properties defaultProps = new Properties(); - try (FileInputStream fis = new FileInputStream(pathToFile)) { - defaultProps.load(fis); + // -------- test the default file creation ------------ + try { + final String pathToFile = tmp.newFile().getAbsolutePath(); + parameterTool.createPropertiesFile(pathToFile); + final Properties defaultProps = new Properties(); + try (FileInputStream fis = new FileInputStream(pathToFile)) { + defaultProps.load(fis); + } + + Assert.assertEquals("myDefaultValue", defaultProps.get("output")); + Assert.assertEquals("-1", defaultProps.get("expectedCount")); + Assert.assertTrue(defaultProps.containsKey("input")); + + } catch (IOException e) { + throw new RuntimeException(e); } + } else if (parameter instanceof MultipleParameterTool) { + MultipleParameterTool multipleParameterTool = (MultipleParameterTool) parameter; + List multiValues = Arrays.asList("multiValue1", "multiValue2"); + Assert.assertEquals(multiValues, multipleParameterTool.getMultiParameter("multi")); - Assert.assertEquals("myDefaultValue", defaultProps.get("output")); - Assert.assertEquals("-1", defaultProps.get("expectedCount")); - Assert.assertTrue(defaultProps.containsKey("input")); + Assert.assertEquals(multiValues, multipleParameterTool.toMultiMap().get("multi")); - } catch (IOException e) { - throw new RuntimeException(e); + // The last value is used. + Assert.assertEquals("multiValue2", multipleParameterTool.toMap().get("multi")); } } } diff --git a/flink-java/src/test/java/org/apache/flink/api/java/utils/MultipleParameterToolTest.java b/flink-java/src/test/java/org/apache/flink/api/java/utils/MultipleParameterToolTest.java new file mode 100644 index 000000000000..b6c8fa31fa8b --- /dev/null +++ b/flink-java/src/test/java/org/apache/flink/api/java/utils/MultipleParameterToolTest.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.api.java.utils; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; + +/** + * Tests for {@link MultipleParameterTool}. + */ +public class MultipleParameterToolTest extends AbstractParameterToolTest { + + @Test + public void testFromCliArgsWithMultipleParameters() { + MultipleParameterTool parameter = (MultipleParameterTool) createParameterToolFromArgs(new String[]{ + "--input", "myInput", "-expectedCount", "15", "--multi", "multiValue1", "--multi", "multiValue2", + "--withoutValues", "--negativeFloat", "-0.58", "-isWorking", "true", "--maxByte", "127", "-negativeShort", "-1024"}); + + Assert.assertEquals(8, parameter.getNumberOfParameters()); + validate(parameter); + Assert.assertTrue(parameter.has("withoutValues")); + Assert.assertEquals(-0.58, parameter.getFloat("negativeFloat"), 0.1); + Assert.assertTrue(parameter.getBoolean("isWorking")); + Assert.assertEquals(127, parameter.getByte("maxByte")); + Assert.assertEquals(-1024, parameter.getShort("negativeShort")); + + exception.expect(IllegalStateException.class); + exception.expectMessage("Key multi should has only one value"); + parameter.get("multi"); + } + + @Test + public void testUnrequestedMultiParameter() { + MultipleParameterTool parameter = (MultipleParameterTool) createParameterToolFromArgs(new String[]{ + "--multi", "v1", "--multi", "v2", "--multi2", "vv1"}); + Assert.assertEquals(createHashSet("multi", "multi2"), parameter.getUnrequestedParameters()); + + Assert.assertEquals(Arrays.asList("v1", "v2"), parameter.getMultiParameter("multi")); + Assert.assertEquals(createHashSet("multi2"), parameter.getUnrequestedParameters()); + + Assert.assertEquals(Collections.singletonList("vv1"), parameter.getMultiParameterRequired("multi2")); + Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); + } + + @Test + public void testMerged() { + MultipleParameterTool parameter1 = (MultipleParameterTool) createParameterToolFromArgs(new String[]{ + "--input", "myInput", "--merge", "v1", "--merge", "v2"}); + MultipleParameterTool parameter2 = (MultipleParameterTool) createParameterToolFromArgs(new String[]{ + "--multi", "multiValue1", "--multi", "multiValue2", "-expectedCount", "15", "--merge", "v3"}); + MultipleParameterTool parameter = parameter1.mergeWith(parameter2); + validate(parameter); + Assert.assertEquals(Arrays.asList("v1", "v2", "v3"), parameter.getMultiParameter("merge")); + } + + @Override + protected AbstractParameterTool createParameterToolFromArgs(String[] args) { + return MultipleParameterTool.fromArgs(args); + } +} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/utils/ParameterToolTest.java b/flink-java/src/test/java/org/apache/flink/api/java/utils/ParameterToolTest.java index 69f95de0013a..fc2f1277112d 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/utils/ParameterToolTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/utils/ParameterToolTest.java @@ -19,9 +19,7 @@ package org.apache.flink.api.java.utils; import org.junit.Assert; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.ExpectedException; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -34,11 +32,8 @@ import java.io.OutputStream; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; import java.util.Map; import java.util.Properties; -import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; @@ -52,76 +47,11 @@ */ public class ParameterToolTest extends AbstractParameterToolTest { - @Rule - public final ExpectedException exception = ExpectedException.none(); - // ----- Parser tests ----------------- - @Test - public void testThrowExceptionIfParameterIsNotPrefixed() { - exception.expect(IllegalArgumentException.class); - exception.expectMessage("Error parsing arguments '[a]' on 'a'. Please prefix keys with -- or -."); - - ParameterTool.fromArgs(new String[]{"a"}); - } - - @Test - public void testNoVal() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-berlin"}); - Assert.assertEquals(1, parameter.getNumberOfParameters()); - Assert.assertTrue(parameter.has("berlin")); - } - - @Test - public void testNoValDouble() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"--berlin"}); - Assert.assertEquals(1, parameter.getNumberOfParameters()); - Assert.assertTrue(parameter.has("berlin")); - } - - @Test - public void testMultipleNoVal() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"--a", "--b", "--c", "--d", "--e", "--f"}); - Assert.assertEquals(6, parameter.getNumberOfParameters()); - Assert.assertTrue(parameter.has("a")); - Assert.assertTrue(parameter.has("b")); - Assert.assertTrue(parameter.has("c")); - Assert.assertTrue(parameter.has("d")); - Assert.assertTrue(parameter.has("e")); - Assert.assertTrue(parameter.has("f")); - } - - @Test - public void testMultipleNoValMixed() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"--a", "-b", "-c", "-d", "--e", "--f"}); - Assert.assertEquals(6, parameter.getNumberOfParameters()); - Assert.assertTrue(parameter.has("a")); - Assert.assertTrue(parameter.has("b")); - Assert.assertTrue(parameter.has("c")); - Assert.assertTrue(parameter.has("d")); - Assert.assertTrue(parameter.has("e")); - Assert.assertTrue(parameter.has("f")); - } - - @Test - public void testEmptyVal() { - exception.expect(IllegalArgumentException.class); - exception.expectMessage("The input [--a, -b, --] contains an empty argument"); - - ParameterTool.fromArgs(new String[]{"--a", "-b", "--"}); - } - - @Test - public void testEmptyValShort() { - exception.expect(IllegalArgumentException.class); - exception.expectMessage("The input [--a, -b, -] contains an empty argument"); - - ParameterTool.fromArgs(new String[]{"--a", "-b", "-"}); - } - @Test public void testFromCliArgs() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"--input", "myInput", "-expectedCount", "15", "--withoutValues", + ParameterTool parameter = (ParameterTool) createParameterToolFromArgs(new String[]{"--input", "myInput", "-expectedCount", "15", "--withoutValues", "--negativeFloat", "-0.58", "-isWorking", "true", "--maxByte", "127", "-negativeShort", "-1024"}); Assert.assertEquals(7, parameter.getNumberOfParameters()); validate(parameter); @@ -179,422 +109,14 @@ public void testSystemProperties() { @Test public void testMerged() { - ParameterTool parameter1 = ParameterTool.fromArgs(new String[]{"--input", "myInput"}); + ParameterTool parameter1 = (ParameterTool) createParameterToolFromArgs(new String[]{"--input", "myInput"}); System.setProperty("expectedCount", "15"); ParameterTool parameter2 = ParameterTool.fromSystemProperties(); ParameterTool parameter = parameter1.mergeWith(parameter2); validate(parameter); } - // Boolean - - @Test - public void testUnrequestedBoolean() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-boolean", "true"}); - Assert.assertEquals(createHashSet("boolean"), parameter.getUnrequestedParameters()); - - // test parameter access - Assert.assertTrue(parameter.getBoolean("boolean")); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - - // test repeated access - Assert.assertTrue(parameter.getBoolean("boolean")); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - } - - @Test - public void testUnrequestedBooleanWithDefaultValue() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-boolean", "true"}); - Assert.assertEquals(createHashSet("boolean"), parameter.getUnrequestedParameters()); - - // test parameter access - Assert.assertTrue(parameter.getBoolean("boolean", false)); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - - // test repeated access - Assert.assertTrue(parameter.getBoolean("boolean", false)); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - } - - @Test - public void testUnrequestedBooleanWithMissingValue() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-boolean"}); - Assert.assertEquals(createHashSet("boolean"), parameter.getUnrequestedParameters()); - - parameter.getBoolean("boolean"); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - } - - // Byte - - @Test - public void testUnrequestedByte() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-byte", "1"}); - Assert.assertEquals(createHashSet("byte"), parameter.getUnrequestedParameters()); - - // test parameter access - Assert.assertEquals(1, parameter.getByte("byte")); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - - // test repeated access - Assert.assertEquals(1, parameter.getByte("byte")); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - } - - @Test - public void testUnrequestedByteWithDefaultValue() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-byte", "1"}); - Assert.assertEquals(createHashSet("byte"), parameter.getUnrequestedParameters()); - - // test parameter access - Assert.assertEquals(1, parameter.getByte("byte", (byte) 0)); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - - // test repeated access - Assert.assertEquals(1, parameter.getByte("byte", (byte) 0)); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - } - - @Test - public void testUnrequestedByteWithMissingValue() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-byte"}); - Assert.assertEquals(createHashSet("byte"), parameter.getUnrequestedParameters()); - - exception.expect(RuntimeException.class); - exception.expectMessage("For input string: \"__NO_VALUE_KEY\""); - - parameter.getByte("byte"); - } - - // Short - - @Test - public void testUnrequestedShort() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-short", "2"}); - Assert.assertEquals(createHashSet("short"), parameter.getUnrequestedParameters()); - - // test parameter access - Assert.assertEquals(2, parameter.getShort("short")); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - - // test repeated access - Assert.assertEquals(2, parameter.getShort("short")); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - } - - @Test - public void testUnrequestedShortWithDefaultValue() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-short", "2"}); - Assert.assertEquals(createHashSet("short"), parameter.getUnrequestedParameters()); - - // test parameter access - Assert.assertEquals(2, parameter.getShort("short", (short) 0)); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - - // test repeated access - Assert.assertEquals(2, parameter.getShort("short", (short) 0)); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - } - - @Test - public void testUnrequestedShortWithMissingValue() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-short"}); - Assert.assertEquals(createHashSet("short"), parameter.getUnrequestedParameters()); - - exception.expect(RuntimeException.class); - exception.expectMessage("For input string: \"__NO_VALUE_KEY\""); - - parameter.getShort("short"); - } - - // Int - - @Test - public void testUnrequestedInt() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-int", "4"}); - Assert.assertEquals(createHashSet("int"), parameter.getUnrequestedParameters()); - - // test parameter access - Assert.assertEquals(4, parameter.getInt("int")); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - - // test repeated access - Assert.assertEquals(4, parameter.getInt("int")); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - } - - @Test - public void testUnrequestedIntWithDefaultValue() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-int", "4"}); - Assert.assertEquals(createHashSet("int"), parameter.getUnrequestedParameters()); - - // test parameter access - Assert.assertEquals(4, parameter.getInt("int", 0)); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - - // test repeated access - Assert.assertEquals(4, parameter.getInt("int", 0)); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - } - - @Test - public void testUnrequestedIntWithMissingValue() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-int"}); - Assert.assertEquals(createHashSet("int"), parameter.getUnrequestedParameters()); - - exception.expect(RuntimeException.class); - exception.expectMessage("For input string: \"__NO_VALUE_KEY\""); - - parameter.getInt("int"); - } - - // Long - - @Test - public void testUnrequestedLong() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-long", "8"}); - Assert.assertEquals(createHashSet("long"), parameter.getUnrequestedParameters()); - // test parameter access - Assert.assertEquals(8, parameter.getLong("long")); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - - // test repeated access - Assert.assertEquals(8, parameter.getLong("long")); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - } - - @Test - public void testUnrequestedLongWithDefaultValue() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-long", "8"}); - Assert.assertEquals(createHashSet("long"), parameter.getUnrequestedParameters()); - - // test parameter access - Assert.assertEquals(8, parameter.getLong("long", 0)); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - - // test repeated access - Assert.assertEquals(8, parameter.getLong("long", 0)); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - } - - @Test - public void testUnrequestedLongWithMissingValue() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-long"}); - Assert.assertEquals(createHashSet("long"), parameter.getUnrequestedParameters()); - - exception.expect(RuntimeException.class); - exception.expectMessage("For input string: \"__NO_VALUE_KEY\""); - - parameter.getLong("long"); - } - - // Float - - @Test - public void testUnrequestedFloat() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-float", "4"}); - Assert.assertEquals(createHashSet("float"), parameter.getUnrequestedParameters()); - - // test parameter access - Assert.assertEquals(4.0, parameter.getFloat("float"), 0.00001); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - - // test repeated access - Assert.assertEquals(4.0, parameter.getFloat("float"), 0.00001); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - } - - @Test - public void testUnrequestedFloatWithDefaultValue() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-float", "4"}); - Assert.assertEquals(createHashSet("float"), parameter.getUnrequestedParameters()); - - // test parameter access - Assert.assertEquals(4.0, parameter.getFloat("float", 0.0f), 0.00001); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - - // test repeated access - Assert.assertEquals(4.0, parameter.getFloat("float", 0.0f), 0.00001); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - } - - @Test - public void testUnrequestedFloatWithMissingValue() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-float"}); - Assert.assertEquals(createHashSet("float"), parameter.getUnrequestedParameters()); - - exception.expect(RuntimeException.class); - exception.expectMessage("For input string: \"__NO_VALUE_KEY\""); - - parameter.getFloat("float"); - } - - // Double - - @Test - public void testUnrequestedDouble() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-double", "8"}); - Assert.assertEquals(createHashSet("double"), parameter.getUnrequestedParameters()); - - // test parameter access - Assert.assertEquals(8.0, parameter.getDouble("double"), 0.00001); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - - // test repeated access - Assert.assertEquals(8.0, parameter.getDouble("double"), 0.00001); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - } - - @Test - public void testUnrequestedDoubleWithDefaultValue() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-double", "8"}); - Assert.assertEquals(createHashSet("double"), parameter.getUnrequestedParameters()); - - // test parameter access - Assert.assertEquals(8.0, parameter.getDouble("double", 0.0), 0.00001); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - - // test repeated access - Assert.assertEquals(8.0, parameter.getDouble("double", 0.0), 0.00001); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - } - - @Test - public void testUnrequestedDoubleWithMissingValue() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-double"}); - Assert.assertEquals(createHashSet("double"), parameter.getUnrequestedParameters()); - - exception.expect(RuntimeException.class); - exception.expectMessage("For input string: \"__NO_VALUE_KEY\""); - - parameter.getDouble("double"); - } - - // String - - @Test - public void testUnrequestedString() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-string", "∞"}); - Assert.assertEquals(createHashSet("string"), parameter.getUnrequestedParameters()); - - // test parameter access - Assert.assertEquals("∞", parameter.get("string")); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - - // test repeated access - Assert.assertEquals("∞", parameter.get("string")); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - } - - @Test - public void testUnrequestedStringWithDefaultValue() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-string", "∞"}); - Assert.assertEquals(createHashSet("string"), parameter.getUnrequestedParameters()); - - // test parameter access - Assert.assertEquals("∞", parameter.get("string", "0.0")); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - - // test repeated access - Assert.assertEquals("∞", parameter.get("string", "0.0")); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - } - - @Test - public void testUnrequestedStringWithMissingValue() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-string"}); - Assert.assertEquals(createHashSet("string"), parameter.getUnrequestedParameters()); - - parameter.get("string"); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - } - - // Additional methods - - @Test - public void testUnrequestedHas() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-boolean"}); - Assert.assertEquals(createHashSet("boolean"), parameter.getUnrequestedParameters()); - - // test parameter access - Assert.assertTrue(parameter.has("boolean")); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - - // test repeated access - Assert.assertTrue(parameter.has("boolean")); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - } - - @Test - public void testUnrequestedRequired() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-required", "∞"}); - Assert.assertEquals(createHashSet("required"), parameter.getUnrequestedParameters()); - - // test parameter access - Assert.assertEquals("∞", parameter.getRequired("required")); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - - // test repeated access - Assert.assertEquals("∞", parameter.getRequired("required")); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - } - - @Test - public void testUnrequestedMultiple() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{"-boolean", "true", "-byte", "1", - "-short", "2", "-int", "4", "-long", "8", "-float", "4.0", "-double", "8.0", "-string", "∞"}); - Assert.assertEquals(createHashSet("boolean", "byte", "short", "int", "long", "float", "double", "string"), - parameter.getUnrequestedParameters()); - - Assert.assertTrue(parameter.getBoolean("boolean")); - Assert.assertEquals(createHashSet("byte", "short", "int", "long", "float", "double", "string"), - parameter.getUnrequestedParameters()); - - Assert.assertEquals(1, parameter.getByte("byte")); - Assert.assertEquals(createHashSet("short", "int", "long", "float", "double", "string"), - parameter.getUnrequestedParameters()); - - Assert.assertEquals(2, parameter.getShort("short")); - Assert.assertEquals(createHashSet("int", "long", "float", "double", "string"), - parameter.getUnrequestedParameters()); - - Assert.assertEquals(4, parameter.getInt("int")); - Assert.assertEquals(createHashSet("long", "float", "double", "string"), - parameter.getUnrequestedParameters()); - - Assert.assertEquals(8, parameter.getLong("long")); - Assert.assertEquals(createHashSet("float", "double", "string"), - parameter.getUnrequestedParameters()); - - Assert.assertEquals(4.0, parameter.getFloat("float"), 0.00001); - Assert.assertEquals(createHashSet("double", "string"), - parameter.getUnrequestedParameters()); - - Assert.assertEquals(8.0, parameter.getDouble("double"), 0.00001); - Assert.assertEquals(createHashSet("string"), - parameter.getUnrequestedParameters()); - - Assert.assertEquals("∞", parameter.get("string")); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - } - - @Test - public void testUnrequestedUnknown() { - ParameterTool parameter = ParameterTool.fromArgs(new String[]{}); - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - - Assert.assertTrue(parameter.getBoolean("boolean", true)); - Assert.assertEquals(0, parameter.getByte("byte", (byte) 0)); - Assert.assertEquals(0, parameter.getShort("short", (short) 0)); - Assert.assertEquals(0, parameter.getInt("int", 0)); - Assert.assertEquals(0, parameter.getLong("long", 0)); - Assert.assertEquals(0, parameter.getFloat("float", 0), 0.00001); - Assert.assertEquals(0, parameter.getDouble("double", 0), 0.00001); - Assert.assertEquals("0", parameter.get("string", "0")); - - Assert.assertEquals(Collections.emptySet(), parameter.getUnrequestedParameters()); - } /** * Tests that we can concurrently serialize and access the ParameterTool. See FLINK-7943 @@ -612,7 +134,7 @@ public void testConcurrentExecutionConfigSerialization() throws ExecutionExcepti final String[] args = input.toArray(new String[0]); - final ParameterTool parameterTool = ParameterTool.fromArgs(args); + final ParameterTool parameterTool = (ParameterTool) createParameterToolFromArgs(args); final int numThreads = 5; final int numSerializations = 100; @@ -667,12 +189,4 @@ private void serializeDeserialize(ParameterTool parameterTool) throws IOExceptio ParameterTool deserializedParameterTool = ((ParameterTool) ois.readObject()); } } - - private static Set createHashSet(T... elements) { - Set set = new HashSet<>(); - for (T element : elements) { - set.add(element); - } - return set; - } } From 304736d7a49463364aaee1386c945f6a865009be Mon Sep 17 00:00:00 2001 From: "danrtsey.wy" Date: Mon, 11 Nov 2019 22:40:30 +0800 Subject: [PATCH 633/746] =?UTF-8?q?[FLINK-14382][examples]=20Adding=20mult?= =?UTF-8?q?iple=C2=A0--input=C2=A0support=20to=C2=A0WordCount?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../examples/java/wordcount/WordCount.java | 18 +++++++++++++----- .../examples/wordcount/WordCount.java | 18 +++++++++++++----- 2 files changed, 26 insertions(+), 10 deletions(-) diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/wordcount/WordCount.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/wordcount/WordCount.java index 2882fc7e908d..8cbcd15970e5 100644 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/wordcount/WordCount.java +++ b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/wordcount/WordCount.java @@ -22,9 +22,10 @@ import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.api.java.utils.MultipleParameterTool; import org.apache.flink.examples.java.wordcount.util.WordCountData; import org.apache.flink.util.Collector; +import org.apache.flink.util.Preconditions; /** * Implements the "WordCount" program that computes a simple word occurrence histogram @@ -51,7 +52,7 @@ public class WordCount { public static void main(String[] args) throws Exception { - final ParameterTool params = ParameterTool.fromArgs(args); + final MultipleParameterTool params = MultipleParameterTool.fromArgs(args); // set up the execution environment final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); @@ -60,10 +61,17 @@ public static void main(String[] args) throws Exception { env.getConfig().setGlobalJobParameters(params); // get input data - DataSet text; + DataSet text = null; if (params.has("input")) { - // read the text file from given input path - text = env.readTextFile(params.get("input")); + // union all the inputs from text files + for (String input : params.getMultiParameterRequired("input")) { + if (text == null) { + text = env.readTextFile(input); + } else { + text = text.union(env.readTextFile(input)); + } + } + Preconditions.checkNotNull(text, "Input DataSet should not be null."); } else { // get default test text data System.out.println("Executing WordCount example with default input data set."); diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java index 56eb9b110c67..4fc8fea884e3 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/wordcount/WordCount.java @@ -19,11 +19,12 @@ import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.api.java.utils.MultipleParameterTool; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.examples.wordcount.util.WordCountData; import org.apache.flink.util.Collector; +import org.apache.flink.util.Preconditions; /** * Implements the "WordCount" program that computes a simple word occurrence @@ -51,7 +52,7 @@ public class WordCount { public static void main(String[] args) throws Exception { // Checking input parameters - final ParameterTool params = ParameterTool.fromArgs(args); + final MultipleParameterTool params = MultipleParameterTool.fromArgs(args); // set up the execution environment final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); @@ -60,10 +61,17 @@ public static void main(String[] args) throws Exception { env.getConfig().setGlobalJobParameters(params); // get input data - DataStream text; + DataStream text = null; if (params.has("input")) { - // read the text file from given input path - text = env.readTextFile(params.get("input")); + // union all the inputs from text files + for (String input : params.getMultiParameterRequired("input")) { + if (text == null) { + text = env.readTextFile(input); + } else { + text = text.union(env.readTextFile(input)); + } + } + Preconditions.checkNotNull(text, "Input DataStream should not be null."); } else { System.out.println("Executing WordCount example with default input data set."); System.out.println("Use --input to specify file input."); From 8c217d729f70b89eb7487d5735cc2590e28e0412 Mon Sep 17 00:00:00 2001 From: "danrtsey.wy" Date: Mon, 11 Nov 2019 22:42:51 +0800 Subject: [PATCH 634/746] [FLINK-14382][e2e] Add another dummy fs for e2e test. It is same as dummy fs with different schema. --- .../another-dummy-fs/pom.xml | 74 ++++++++ .../anotherdummy/AnotherDummyFSFactory.java | 51 ++++++ .../AnotherDummyFSFileStatus.java | 67 ++++++++ .../AnotherDummyFSFileSystem.java | 162 ++++++++++++++++++ .../AnotherDummyFSInputStream.java | 51 ++++++ ...org.apache.flink.core.fs.FileSystemFactory | 16 ++ .../flink-plugins-test/dummy-fs/pom.xml | 74 ++++++++ .../apache/flink/fs/dummy/DummyFSFactory.java | 0 .../flink/fs/dummy/DummyFSFileStatus.java | 0 .../flink/fs/dummy/DummyFSFileSystem.java | 0 .../flink/fs/dummy/DummyFSInputStream.java | 0 ...org.apache.flink.core.fs.FileSystemFactory | 2 +- .../flink-plugins-test/pom.xml | 86 ++++------ 13 files changed, 528 insertions(+), 55 deletions(-) create mode 100644 flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/pom.xml create mode 100644 flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/src/main/java/org/apache/flink/fs/anotherdummy/AnotherDummyFSFactory.java create mode 100644 flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/src/main/java/org/apache/flink/fs/anotherdummy/AnotherDummyFSFileStatus.java create mode 100644 flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/src/main/java/org/apache/flink/fs/anotherdummy/AnotherDummyFSFileSystem.java create mode 100644 flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/src/main/java/org/apache/flink/fs/anotherdummy/AnotherDummyFSInputStream.java create mode 100644 flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/src/main/resources/META-INF/services/org.apache.flink.core.fs.FileSystemFactory create mode 100644 flink-end-to-end-tests/flink-plugins-test/dummy-fs/pom.xml rename flink-end-to-end-tests/flink-plugins-test/{ => dummy-fs}/src/main/java/org/apache/flink/fs/dummy/DummyFSFactory.java (100%) rename flink-end-to-end-tests/flink-plugins-test/{ => dummy-fs}/src/main/java/org/apache/flink/fs/dummy/DummyFSFileStatus.java (100%) rename flink-end-to-end-tests/flink-plugins-test/{ => dummy-fs}/src/main/java/org/apache/flink/fs/dummy/DummyFSFileSystem.java (100%) rename flink-end-to-end-tests/flink-plugins-test/{ => dummy-fs}/src/main/java/org/apache/flink/fs/dummy/DummyFSInputStream.java (100%) rename flink-end-to-end-tests/flink-plugins-test/{ => dummy-fs}/src/main/resources/META-INF/services/org.apache.flink.core.fs.FileSystemFactory (95%) diff --git a/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/pom.xml b/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/pom.xml new file mode 100644 index 000000000000..7db13054d1e0 --- /dev/null +++ b/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/pom.xml @@ -0,0 +1,74 @@ + + + + + flink-plugins-test + org.apache.flink + 1.10-SNAPSHOT + + 4.0.0 + + another-dummy-fs + + jar + + + + org.apache.flink + flink-core + ${project.version} + provided + + + + + + + + + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + flink-another-dummy-fs + + + + + + + diff --git a/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/src/main/java/org/apache/flink/fs/anotherdummy/AnotherDummyFSFactory.java b/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/src/main/java/org/apache/flink/fs/anotherdummy/AnotherDummyFSFactory.java new file mode 100644 index 000000000000..2c1cb1006851 --- /dev/null +++ b/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/src/main/java/org/apache/flink/fs/anotherdummy/AnotherDummyFSFactory.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.fs.anotherdummy; + +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.FileSystemFactory; + +import java.io.IOException; +import java.net.URI; +import java.util.HashMap; +import java.util.Map; + +/** + * Factory of dummy FileSystem. See documentation of {@link AnotherDummyFSFileSystem}. + */ +public class AnotherDummyFSFactory implements FileSystemFactory { + + private final FileSystem fileSystem = new AnotherDummyFSFileSystem(getData()); + + @Override + public String getScheme() { + return AnotherDummyFSFileSystem.FS_URI.getScheme(); + } + + @Override + public FileSystem create(URI fsUri) throws IOException { + return fileSystem; + } + + private static Map getData() { + Map data = new HashMap<>(); + data.put("/words", "Hello World how are you, my dear dear world\n"); + return data; + } +} diff --git a/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/src/main/java/org/apache/flink/fs/anotherdummy/AnotherDummyFSFileStatus.java b/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/src/main/java/org/apache/flink/fs/anotherdummy/AnotherDummyFSFileStatus.java new file mode 100644 index 000000000000..1b61e21e1d8f --- /dev/null +++ b/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/src/main/java/org/apache/flink/fs/anotherdummy/AnotherDummyFSFileStatus.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.fs.anotherdummy; + +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.Path; + +class AnotherDummyFSFileStatus implements FileStatus { + private final Path path; + private final int length; + + AnotherDummyFSFileStatus(Path path, int length) { + this.path = path; + this.length = length; + } + + @Override + public long getLen() { + return length; + } + + @Override + public long getBlockSize() { + return length; + } + + @Override + public short getReplication() { + return 0; + } + + @Override + public long getModificationTime() { + return 0; + } + + @Override + public long getAccessTime() { + return 0; + } + + @Override + public boolean isDir() { + return false; + } + + @Override + public Path getPath() { + return path; + } +} diff --git a/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/src/main/java/org/apache/flink/fs/anotherdummy/AnotherDummyFSFileSystem.java b/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/src/main/java/org/apache/flink/fs/anotherdummy/AnotherDummyFSFileSystem.java new file mode 100644 index 000000000000..99a5b47635b5 --- /dev/null +++ b/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/src/main/java/org/apache/flink/fs/anotherdummy/AnotherDummyFSFileSystem.java @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.fs.anotherdummy; + +import org.apache.flink.core.fs.BlockLocation; +import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.core.fs.FSDataOutputStream; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.FileSystemKind; +import org.apache.flink.core.fs.Path; +import org.apache.flink.core.fs.local.LocalBlockLocation; + +import javax.annotation.Nullable; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.net.URI; +import java.nio.charset.Charset; +import java.util.HashMap; +import java.util.Map; + +/** + * A FileSystem implementation for integration testing purposes. Supports and serves read-only content from static + * key value map. + */ +class AnotherDummyFSFileSystem extends FileSystem { + + static final URI FS_URI = URI.create("anotherDummy:///"); + + private static final String HOSTNAME = "localhost"; + + private final URI workingDir; + + private final URI homeDir; + + private final Map contents; + + AnotherDummyFSFileSystem(Map contents) { + this.workingDir = new File(System.getProperty("user.dir")).toURI(); + this.homeDir = new File(System.getProperty("user.home")).toURI(); + this.contents = convertToByteArrayMap(contents); + } + + // ------------------------------------------------------------------------ + + @Override + public URI getUri() { + return FS_URI; + } + + @Override + public Path getWorkingDirectory() { + return new Path(workingDir); + } + + @Override + public Path getHomeDirectory() { + return new Path(homeDir); + } + + @Override + public boolean exists(Path f) throws IOException { + return getDataByPath(f) != null; + } + + @Override + public FileStatus[] listStatus(final Path f) throws IOException { + byte[] data = getDataByPath(f); + if (data == null) { + return null; + } + return new FileStatus[] { new AnotherDummyFSFileStatus(f, data.length) }; + } + + @Override + public BlockLocation[] getFileBlockLocations(FileStatus file, long start, long len) throws IOException { + return new BlockLocation[] { + new LocalBlockLocation(HOSTNAME, file.getLen()) + }; + } + + @Override + public FileStatus getFileStatus(Path f) throws IOException { + byte[] data = getDataByPath(f); + if (data == null) { + throw new FileNotFoundException("File " + f + " does not exist."); + } + return new AnotherDummyFSFileStatus(f, data.length); + } + + @Override + public FSDataInputStream open(final Path f, final int bufferSize) throws IOException { + return open(f); + } + + @Override + public FSDataInputStream open(final Path f) throws IOException { + return AnotherDummyFSInputStream.create(getDataByPath(f)); + } + + @Override + public boolean delete(final Path path, final boolean recursive) throws IOException { + throw new UnsupportedOperationException("Dummy FS doesn't support delete operation"); + } + + @Override + public boolean mkdirs(final Path path) throws IOException { + throw new UnsupportedOperationException("Dummy FS doesn't support mkdirs operation"); + } + + @Override + public FSDataOutputStream create(final Path path, final WriteMode overwrite) throws IOException { + throw new UnsupportedOperationException("Dummy FS doesn't support create operation"); + } + + @Override + public boolean rename(final Path src, final Path dst) throws IOException { + throw new UnsupportedOperationException("Dummy FS doesn't support rename operation"); + } + + @Override + public boolean isDistributedFS() { + return true; + } + + @Override + public FileSystemKind getKind() { + return FileSystemKind.OBJECT_STORE; + } + + @Nullable + private byte[] getDataByPath(Path path) { + return contents.get(path.toUri().getPath()); + } + + private static Map convertToByteArrayMap(Map content) { + Map data = new HashMap<>(); + Charset utf8 = Charset.forName("UTF-8"); + content.entrySet().forEach( + entry -> data.put(entry.getKey(), entry.getValue().getBytes(utf8)) + ); + return data; + } +} diff --git a/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/src/main/java/org/apache/flink/fs/anotherdummy/AnotherDummyFSInputStream.java b/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/src/main/java/org/apache/flink/fs/anotherdummy/AnotherDummyFSInputStream.java new file mode 100644 index 000000000000..eaf0c01100fa --- /dev/null +++ b/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/src/main/java/org/apache/flink/fs/anotherdummy/AnotherDummyFSInputStream.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.fs.anotherdummy; + +import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.core.memory.ByteArrayInputStreamWithPos; + +import java.io.IOException; + +class AnotherDummyFSInputStream extends FSDataInputStream { + private final ByteArrayInputStreamWithPos stream; + + private AnotherDummyFSInputStream(ByteArrayInputStreamWithPos stream) { + this.stream = stream; + } + + static AnotherDummyFSInputStream create(byte[] buffer) { + return new AnotherDummyFSInputStream(new ByteArrayInputStreamWithPos(buffer)); + } + + @Override + public void seek(long desired) throws IOException { + stream.setPosition((int) desired); + } + + @Override + public long getPos() throws IOException { + return stream.getPosition(); + } + + @Override + public int read() throws IOException { + return stream.read(); + } +} diff --git a/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/src/main/resources/META-INF/services/org.apache.flink.core.fs.FileSystemFactory b/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/src/main/resources/META-INF/services/org.apache.flink.core.fs.FileSystemFactory new file mode 100644 index 000000000000..84639e5c18f6 --- /dev/null +++ b/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/src/main/resources/META-INF/services/org.apache.flink.core.fs.FileSystemFactory @@ -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.fs.anotherdummy.AnotherDummyFSFactory \ No newline at end of file diff --git a/flink-end-to-end-tests/flink-plugins-test/dummy-fs/pom.xml b/flink-end-to-end-tests/flink-plugins-test/dummy-fs/pom.xml new file mode 100644 index 000000000000..534507739763 --- /dev/null +++ b/flink-end-to-end-tests/flink-plugins-test/dummy-fs/pom.xml @@ -0,0 +1,74 @@ + + + + + flink-plugins-test + org.apache.flink + 1.10-SNAPSHOT + + 4.0.0 + + dummy-fs + + jar + + + + org.apache.flink + flink-core + ${project.version} + provided + + + + + + + + + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + flink-dummy-fs + + + + + + + diff --git a/flink-end-to-end-tests/flink-plugins-test/src/main/java/org/apache/flink/fs/dummy/DummyFSFactory.java b/flink-end-to-end-tests/flink-plugins-test/dummy-fs/src/main/java/org/apache/flink/fs/dummy/DummyFSFactory.java similarity index 100% rename from flink-end-to-end-tests/flink-plugins-test/src/main/java/org/apache/flink/fs/dummy/DummyFSFactory.java rename to flink-end-to-end-tests/flink-plugins-test/dummy-fs/src/main/java/org/apache/flink/fs/dummy/DummyFSFactory.java diff --git a/flink-end-to-end-tests/flink-plugins-test/src/main/java/org/apache/flink/fs/dummy/DummyFSFileStatus.java b/flink-end-to-end-tests/flink-plugins-test/dummy-fs/src/main/java/org/apache/flink/fs/dummy/DummyFSFileStatus.java similarity index 100% rename from flink-end-to-end-tests/flink-plugins-test/src/main/java/org/apache/flink/fs/dummy/DummyFSFileStatus.java rename to flink-end-to-end-tests/flink-plugins-test/dummy-fs/src/main/java/org/apache/flink/fs/dummy/DummyFSFileStatus.java diff --git a/flink-end-to-end-tests/flink-plugins-test/src/main/java/org/apache/flink/fs/dummy/DummyFSFileSystem.java b/flink-end-to-end-tests/flink-plugins-test/dummy-fs/src/main/java/org/apache/flink/fs/dummy/DummyFSFileSystem.java similarity index 100% rename from flink-end-to-end-tests/flink-plugins-test/src/main/java/org/apache/flink/fs/dummy/DummyFSFileSystem.java rename to flink-end-to-end-tests/flink-plugins-test/dummy-fs/src/main/java/org/apache/flink/fs/dummy/DummyFSFileSystem.java diff --git a/flink-end-to-end-tests/flink-plugins-test/src/main/java/org/apache/flink/fs/dummy/DummyFSInputStream.java b/flink-end-to-end-tests/flink-plugins-test/dummy-fs/src/main/java/org/apache/flink/fs/dummy/DummyFSInputStream.java similarity index 100% rename from flink-end-to-end-tests/flink-plugins-test/src/main/java/org/apache/flink/fs/dummy/DummyFSInputStream.java rename to flink-end-to-end-tests/flink-plugins-test/dummy-fs/src/main/java/org/apache/flink/fs/dummy/DummyFSInputStream.java diff --git a/flink-end-to-end-tests/flink-plugins-test/src/main/resources/META-INF/services/org.apache.flink.core.fs.FileSystemFactory b/flink-end-to-end-tests/flink-plugins-test/dummy-fs/src/main/resources/META-INF/services/org.apache.flink.core.fs.FileSystemFactory similarity index 95% rename from flink-end-to-end-tests/flink-plugins-test/src/main/resources/META-INF/services/org.apache.flink.core.fs.FileSystemFactory rename to flink-end-to-end-tests/flink-plugins-test/dummy-fs/src/main/resources/META-INF/services/org.apache.flink.core.fs.FileSystemFactory index 1c5ec8c05e92..d04458857770 100644 --- a/flink-end-to-end-tests/flink-plugins-test/src/main/resources/META-INF/services/org.apache.flink.core.fs.FileSystemFactory +++ b/flink-end-to-end-tests/flink-plugins-test/dummy-fs/src/main/resources/META-INF/services/org.apache.flink.core.fs.FileSystemFactory @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.fs.dummy.DummyFSFactory +org.apache.flink.fs.dummy.DummyFSFactory \ No newline at end of file diff --git a/flink-end-to-end-tests/flink-plugins-test/pom.xml b/flink-end-to-end-tests/flink-plugins-test/pom.xml index 456359b8292a..3c4f2161eb59 100644 --- a/flink-end-to-end-tests/flink-plugins-test/pom.xml +++ b/flink-end-to-end-tests/flink-plugins-test/pom.xml @@ -18,57 +18,35 @@ specific language governing permissions and limitations under the License. --> - - flink-end-to-end-tests - org.apache.flink - 1.10-SNAPSHOT - - 4.0.0 - - flink-plugins-test - - jar - - - - org.apache.flink - flink-core - ${project.version} - provided - - - - - - - - - - org.apache.maven.plugins - maven-deploy-plugin - - true - - - - - org.apache.maven.plugins - maven-shade-plugin - - - shade-flink - package - - shade - - - flink-dummy-fs - - - - - - - + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + 4.0.0 + + + flink-end-to-end-tests + org.apache.flink + 1.10-SNAPSHOT + + + pom + + flink-plugins-test + flink-plugins-test + + + dummy-fs + another-dummy-fs + + + + + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + + \ No newline at end of file From 67eaddb93fbec707ca7d0d5c45256bf3be7ecd0b Mon Sep 17 00:00:00 2001 From: "danrtsey.wy" Date: Fri, 8 Nov 2019 17:30:45 +0800 Subject: [PATCH 635/746] [hotfix] Refactor test_yarn_kerberos_docker.sh by moving common functions to common_yarn_docker.sh. --- .../test-scripts/common_yarn_docker.sh | 164 ++++++++++++++++++ .../test-scripts/test_yarn_kerberos_docker.sh | 135 +------------- 2 files changed, 167 insertions(+), 132 deletions(-) create mode 100755 flink-end-to-end-tests/test-scripts/common_yarn_docker.sh diff --git a/flink-end-to-end-tests/test-scripts/common_yarn_docker.sh b/flink-end-to-end-tests/test-scripts/common_yarn_docker.sh new file mode 100755 index 000000000000..fab81e55a527 --- /dev/null +++ b/flink-end-to-end-tests/test-scripts/common_yarn_docker.sh @@ -0,0 +1,164 @@ +#!/usr/bin/env bash +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ +set -o pipefail + +source "$(dirname "$0")"/common.sh + +FLINK_TARBALL_DIR=$TEST_DATA_DIR +FLINK_TARBALL=flink.tar.gz +FLINK_DIRNAME=$(basename $FLINK_DIR) + +MAX_RETRY_SECONDS=120 +CLUSTER_SETUP_RETRIES=3 + +echo "Flink Tarball directory $FLINK_TARBALL_DIR" +echo "Flink tarball filename $FLINK_TARBALL" +echo "Flink distribution directory name $FLINK_DIRNAME" +echo "End-to-end directory $END_TO_END_DIR" +docker --version +docker-compose --version + +start_time=$(date +%s) + +# make sure we stop our cluster at the end +function cluster_shutdown { + docker-compose -f $END_TO_END_DIR/test-scripts/docker-hadoop-secure-cluster/docker-compose.yml down + rm $FLINK_TARBALL_DIR/$FLINK_TARBALL +} +on_exit cluster_shutdown + +function start_hadoop_cluster() { + echo "Starting Hadoop cluster" + docker-compose -f $END_TO_END_DIR/test-scripts/docker-hadoop-secure-cluster/docker-compose.yml up -d + + # wait for kerberos to be set up + start_time=$(date +%s) + until docker logs master 2>&1 | grep -q "Finished master initialization"; do + current_time=$(date +%s) + time_diff=$((current_time - start_time)) + + if [ $time_diff -ge $MAX_RETRY_SECONDS ]; then + return 1 + else + echo "Waiting for hadoop cluster to come up. We have been trying for $time_diff seconds, retrying ..." + sleep 5 + fi + done + + # perform health checks + if ! { [ $(docker inspect -f '{{.State.Running}}' master 2>&1) = 'true' ] && + [ $(docker inspect -f '{{.State.Running}}' slave1 2>&1) = 'true' ] && + [ $(docker inspect -f '{{.State.Running}}' slave2 2>&1) = 'true' ] && + [ $(docker inspect -f '{{.State.Running}}' kdc 2>&1) = 'true' ]; }; + then + return 1 + fi + + # try and see if NodeManagers are up, otherwise the Flink job will not have enough resources + # to run + nm_running="0" + start_time=$(date +%s) + while [ "$nm_running" -lt "2" ]; do + current_time=$(date +%s) + time_diff=$((current_time - start_time)) + + if [ $time_diff -ge $MAX_RETRY_SECONDS ]; then + return 1 + else + echo "We only have $nm_running NodeManagers up. We have been trying for $time_diff seconds, retrying ..." + sleep 1 + fi + + docker exec -it master bash -c "kinit -kt /home/hadoop-user/hadoop-user.keytab hadoop-user" + nm_running=`docker exec -it master bash -c "yarn node -list" | grep RUNNING | wc -l` + docker exec -it master bash -c "kdestroy" + done + + return 0 +} + +function build_image() { + echo "Building Hadoop Docker container" + until docker build --build-arg HADOOP_VERSION=2.8.4 \ + -f $END_TO_END_DIR/test-scripts/docker-hadoop-secure-cluster/Dockerfile \ + -t flink/docker-hadoop-secure-cluster:latest \ + $END_TO_END_DIR/test-scripts/docker-hadoop-secure-cluster/; + do + # with all the downloading and ubuntu updating a lot of flakiness can happen, make sure + # we don't immediately fail + echo "Something went wrong while building the Docker image, retrying ..." + sleep 2 + done +} + +function start_hadoop_cluster_and_prepare_flink() { + build_image + if ! retry_times $CLUSTER_SETUP_RETRIES 0 start_hadoop_cluster; then + echo "ERROR: Could not start hadoop cluster. Aborting..." + exit 1 + fi + + mkdir -p $FLINK_TARBALL_DIR + tar czf $FLINK_TARBALL_DIR/$FLINK_TARBALL -C $(dirname $FLINK_DIR) . + + docker cp $FLINK_TARBALL_DIR/$FLINK_TARBALL master:/home/hadoop-user/ + + # now, at least the container is ready + docker exec -it master bash -c "tar xzf /home/hadoop-user/$FLINK_TARBALL --directory /home/hadoop-user/" + + # minimal Flink config, bebe + FLINK_CONFIG=$(cat << END +security.kerberos.login.keytab: /home/hadoop-user/hadoop-user.keytab +security.kerberos.login.principal: hadoop-user +slot.request.timeout: 120000 +containerized.heap-cutoff-min: 100 +END +) + docker exec -it master bash -c "echo \"$FLINK_CONFIG\" > /home/hadoop-user/$FLINK_DIRNAME/conf/flink-conf.yaml" + + echo "Flink config:" + docker exec -it master bash -c "cat /home/hadoop-user/$FLINK_DIRNAME/conf/flink-conf.yaml" +} + +function copy_and_show_logs { + mkdir -p $TEST_DATA_DIR/logs + echo "Hadoop logs:" + docker cp master:/var/log/hadoop/* $TEST_DATA_DIR/logs/ + for f in $TEST_DATA_DIR/logs/*; do + echo "$f:" + cat $f + done + echo "Docker logs:" + docker logs master + + echo "Flink logs:" + docker exec -it master bash -c "kinit -kt /home/hadoop-user/hadoop-user.keytab hadoop-user" + application_id=`docker exec -it master bash -c "yarn application -list -appStates ALL" | grep "Flink session cluster" | awk '{print \$1}'` + echo "Application ID: $application_id" + docker exec -it master bash -c "yarn logs -applicationId $application_id" + docker exec -it master bash -c "kdestroy" +} + +function get_output { + docker exec -it master bash -c "kinit -kt /home/hadoop-user/hadoop-user.keytab hadoop-user" + docker exec -it master bash -c "hdfs dfs -ls $1" + OUTPUT=$(docker exec -it master bash -c "hdfs dfs -cat $1") + docker exec -it master bash -c "kdestroy" + echo "$OUTPUT" +} diff --git a/flink-end-to-end-tests/test-scripts/test_yarn_kerberos_docker.sh b/flink-end-to-end-tests/test-scripts/test_yarn_kerberos_docker.sh index 1ece013aa834..a34961195631 100755 --- a/flink-end-to-end-tests/test-scripts/test_yarn_kerberos_docker.sh +++ b/flink-end-to-end-tests/test-scripts/test_yarn_kerberos_docker.sh @@ -16,23 +16,9 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ -set -o pipefail source "$(dirname "$0")"/common.sh - -FLINK_TARBALL_DIR=$TEST_DATA_DIR -FLINK_TARBALL=flink.tar.gz -FLINK_DIRNAME=$(basename $FLINK_DIR) - -MAX_RETRY_SECONDS=120 -CLUSTER_SETUP_RETRIES=3 - -echo "Flink Tarball directory $FLINK_TARBALL_DIR" -echo "Flink tarball filename $FLINK_TARBALL" -echo "Flink distribution directory name $FLINK_DIRNAME" -echo "End-to-end directory $END_TO_END_DIR" -docker --version -docker-compose --version +source "$(dirname "$0")"/common_yarn_docker.sh # Configure Flink dir before making tarball. INPUT_TYPE=${1:-default-input} @@ -54,134 +40,19 @@ case $INPUT_TYPE in ;; esac -# make sure we stop our cluster at the end -function cluster_shutdown { - docker-compose -f $END_TO_END_DIR/test-scripts/docker-hadoop-secure-cluster/docker-compose.yml down - rm $FLINK_TARBALL_DIR/$FLINK_TARBALL -} -on_exit cluster_shutdown - -function start_hadoop_cluster() { - echo "Starting Hadoop cluster" - docker-compose -f $END_TO_END_DIR/test-scripts/docker-hadoop-secure-cluster/docker-compose.yml up -d - - # wait for kerberos to be set up - start_time=$(date +%s) - until docker logs master 2>&1 | grep -q "Finished master initialization"; do - current_time=$(date +%s) - time_diff=$((current_time - start_time)) - - if [ $time_diff -ge $MAX_RETRY_SECONDS ]; then - return 1 - else - echo "Waiting for hadoop cluster to come up. We have been trying for $time_diff seconds, retrying ..." - sleep 5 - fi - done - - # perform health checks - if ! { [ $(docker inspect -f '{{.State.Running}}' master 2>&1) = 'true' ] && - [ $(docker inspect -f '{{.State.Running}}' slave1 2>&1) = 'true' ] && - [ $(docker inspect -f '{{.State.Running}}' slave2 2>&1) = 'true' ] && - [ $(docker inspect -f '{{.State.Running}}' kdc 2>&1) = 'true' ]; }; - then - return 1 - fi - - # try and see if NodeManagers are up, otherwise the Flink job will not have enough resources - # to run - nm_running="0" - start_time=$(date +%s) - while [ "$nm_running" -lt "2" ]; do - current_time=$(date +%s) - time_diff=$((current_time - start_time)) - - if [ $time_diff -ge $MAX_RETRY_SECONDS ]; then - return 1 - else - echo "We only have $nm_running NodeManagers up. We have been trying for $time_diff seconds, retrying ..." - sleep 1 - fi - - docker exec -it master bash -c "kinit -kt /home/hadoop-user/hadoop-user.keytab hadoop-user" - nm_running=`docker exec -it master bash -c "yarn node -list" | grep RUNNING | wc -l` - docker exec -it master bash -c "kdestroy" - done - - return 0 -} -echo "Building Hadoop Docker container" -until docker build --build-arg HADOOP_VERSION=2.8.4 \ - -f $END_TO_END_DIR/test-scripts/docker-hadoop-secure-cluster/Dockerfile \ - -t flink/docker-hadoop-secure-cluster:latest \ - $END_TO_END_DIR/test-scripts/docker-hadoop-secure-cluster/; -do - # with all the downloading and ubuntu updating a lot of flakiness can happen, make sure - # we don't immediately fail - echo "Something went wrong while building the Docker image, retrying ..." - sleep 2 -done - -if ! retry_times $CLUSTER_SETUP_RETRIES 0 start_hadoop_cluster; then - echo "ERROR: Could not start hadoop cluster. Aborting..." - exit 1 -fi - -mkdir -p $FLINK_TARBALL_DIR -tar czf $FLINK_TARBALL_DIR/$FLINK_TARBALL -C $(dirname $FLINK_DIR) . - -docker cp $FLINK_TARBALL_DIR/$FLINK_TARBALL master:/home/hadoop-user/ - -# now, at least the container is ready -docker exec -it master bash -c "tar xzf /home/hadoop-user/$FLINK_TARBALL --directory /home/hadoop-user/" - -# minimal Flink config, bebe -FLINK_CONFIG=$(cat << END -security.kerberos.login.keytab: /home/hadoop-user/hadoop-user.keytab -security.kerberos.login.principal: hadoop-user -slot.request.timeout: 120000 -containerized.heap-cutoff-min: 100 -END -) -docker exec -it master bash -c "echo \"$FLINK_CONFIG\" > /home/hadoop-user/$FLINK_DIRNAME/conf/flink-conf.yaml" - -echo "Flink config:" -docker exec -it master bash -c "cat /home/hadoop-user/$FLINK_DIRNAME/conf/flink-conf.yaml" +start_hadoop_cluster_and_prepare_flink # make the output path random, just in case it already exists, for example if we # had cached docker containers OUTPUT_PATH=hdfs:///user/hadoop-user/wc-out-$RANDOM -function copy_and_show_logs { - mkdir -p $TEST_DATA_DIR/logs - echo "Hadoop logs:" - docker cp master:/var/log/hadoop/* $TEST_DATA_DIR/logs/ - for f in $TEST_DATA_DIR/logs/*; do - echo "$f:" - cat $f - done - echo "Docker logs:" - docker logs master - - echo "Flink logs:" - docker exec -it master bash -c "kinit -kt /home/hadoop-user/hadoop-user.keytab hadoop-user" - application_id=`docker exec -it master bash -c "yarn application -list -appStates ALL" | grep "Flink session cluster" | awk '{print \$1}'` - echo "Application ID: $application_id" - docker exec -it master bash -c "yarn logs -applicationId $application_id" - docker exec -it master bash -c "kdestroy" -} - -start_time=$(date +%s) # it's important to run this with higher parallelism, otherwise we might risk that # JM and TM are on the same YARN node and that we therefore don't test the keytab shipping if docker exec -it master bash -c "export HADOOP_CLASSPATH=\`hadoop classpath\` && \ /home/hadoop-user/$FLINK_DIRNAME/bin/flink run -m yarn-cluster -ys 1 -ytm 1000 -yjm 1000 \ -p 3 /home/hadoop-user/$FLINK_DIRNAME/examples/streaming/WordCount.jar $INPUT_ARGS --output $OUTPUT_PATH"; then - docker exec -it master bash -c "kinit -kt /home/hadoop-user/hadoop-user.keytab hadoop-user" - docker exec -it master bash -c "hdfs dfs -ls $OUTPUT_PATH" - OUTPUT=$(docker exec -it master bash -c "hdfs dfs -cat $OUTPUT_PATH/*") - docker exec -it master bash -c "kdestroy" + OUTPUT=$(get_output "$OUTPUT_PATH/*") echo "$OUTPUT" else echo "Running the job failed." From 749965348170e4608ff2a23c9617f67b8c341df5 Mon Sep 17 00:00:00 2001 From: "danrtsey.wy" Date: Fri, 8 Nov 2019 17:35:51 +0800 Subject: [PATCH 636/746] [FLINK-14382][e2e] Update e2e tests to support two inputs from two dummy FS. --- .../test-scripts/common_dummy_fs.sh | 4 +++- .../test-scripts/test_batch_wordcount.sh | 16 +++++++++------- .../test-scripts/test_docker_embedded_job.sh | 10 ++++++---- .../test-scripts/test_yarn_kerberos_docker.sh | 4 ++-- 4 files changed, 20 insertions(+), 14 deletions(-) diff --git a/flink-end-to-end-tests/test-scripts/common_dummy_fs.sh b/flink-end-to-end-tests/test-scripts/common_dummy_fs.sh index bd6e75665f55..aba7e51a5df1 100644 --- a/flink-end-to-end-tests/test-scripts/common_dummy_fs.sh +++ b/flink-end-to-end-tests/test-scripts/common_dummy_fs.sh @@ -19,5 +19,7 @@ function dummy_fs_setup() { mkdir -p "$FLINK_DIR/plugins/dummy-fs" - cp "${END_TO_END_DIR}/flink-plugins-test/target/flink-dummy-fs.jar" "${FLINK_DIR}/plugins/dummy-fs/" + mkdir -p "$FLINK_DIR/plugins/another-dummy-fs" + cp "${END_TO_END_DIR}/flink-plugins-test/dummy-fs/target/flink-dummy-fs.jar" "${FLINK_DIR}/plugins/dummy-fs/" + cp "${END_TO_END_DIR}/flink-plugins-test/another-dummy-fs/target/flink-another-dummy-fs.jar" "${FLINK_DIR}/plugins/another-dummy-fs/" } diff --git a/flink-end-to-end-tests/test-scripts/test_batch_wordcount.sh b/flink-end-to-end-tests/test-scripts/test_batch_wordcount.sh index 2e90e0b85553..9f57a433afe6 100755 --- a/flink-end-to-end-tests/test-scripts/test_batch_wordcount.sh +++ b/flink-end-to-end-tests/test-scripts/test_batch_wordcount.sh @@ -20,29 +20,31 @@ source "$(dirname "$0")"/common.sh INPUT_TYPE=${1:-file} +RESULT_HASH="72a690412be8928ba239c2da967328a5" case $INPUT_TYPE in (file) - INPUT_LOCATION="${TEST_INFRA_DIR}/test-data/words" + INPUT_ARGS="--input ${TEST_INFRA_DIR}/test-data/words" ;; (hadoop) source "$(dirname "$0")"/common_s3.sh s3_setup hadoop - INPUT_LOCATION="${S3_TEST_DATA_WORDS_URI}" + INPUT_ARGS="--input ${S3_TEST_DATA_WORDS_URI}" ;; (hadoop_with_provider) source "$(dirname "$0")"/common_s3.sh s3_setup_with_provider hadoop "fs.s3a.aws.credentials.provider" - INPUT_LOCATION="${S3_TEST_DATA_WORDS_URI}" + INPUT_ARGS="--input ${S3_TEST_DATA_WORDS_URI}" ;; (presto) source "$(dirname "$0")"/common_s3.sh s3_setup presto - INPUT_LOCATION="${S3_TEST_DATA_WORDS_URI}" + INPUT_ARGS="--input ${S3_TEST_DATA_WORDS_URI}" ;; (dummy-fs) source "$(dirname "$0")"/common_dummy_fs.sh dummy_fs_setup - INPUT_LOCATION="dummy://localhost/words" + INPUT_ARGS="--input dummy://localhost/words --input anotherDummy://localhost/words" + RESULT_HASH="0e5bd0a3dd7d5a7110aa85ff70adb54b" ;; (*) echo "Unknown input type $INPUT_TYPE" @@ -58,5 +60,5 @@ start_cluster # The test may run against different source types. # But the sources should provide the same test data, so the checksum stays the same for all tests. -"${FLINK_DIR}/bin/flink" run -p 1 "${FLINK_DIR}/examples/batch/WordCount.jar" --input "${INPUT_LOCATION}" --output "${OUTPUT_LOCATION}" -check_result_hash "WordCount (${INPUT_TYPE})" "${OUTPUT_LOCATION}" "72a690412be8928ba239c2da967328a5" +eval "${FLINK_DIR}/bin/flink run -p 1 ${FLINK_DIR}/examples/batch/WordCount.jar ${INPUT_ARGS} --output ${OUTPUT_LOCATION}" +check_result_hash "WordCount (${INPUT_TYPE})" "${OUTPUT_LOCATION}" "${RESULT_HASH}" diff --git a/flink-end-to-end-tests/test-scripts/test_docker_embedded_job.sh b/flink-end-to-end-tests/test-scripts/test_docker_embedded_job.sh index c97259bcda5b..277781c2530b 100755 --- a/flink-end-to-end-tests/test-scripts/test_docker_embedded_job.sh +++ b/flink-end-to-end-tests/test-scripts/test_docker_embedded_job.sh @@ -32,14 +32,16 @@ export INPUT_PATH=/data/test/input export OUTPUT_PATH=/data/test/output INPUT_TYPE=${1:-file} +RESULT_HASH="72a690412be8928ba239c2da967328a5" case $INPUT_TYPE in (file) - INPUT_LOCATION=${INPUT_PATH}/words + INPUT_ARGS="--input ${INPUT_PATH}/words" ;; (dummy-fs) source "$(dirname "$0")"/common_dummy_fs.sh dummy_fs_setup - INPUT_LOCATION="dummy://localhost/words" + INPUT_ARGS="--input dummy://localhost/words --input anotherDummy://localhost/words" + RESULT_HASH="0e5bd0a3dd7d5a7110aa85ff70adb54b" ;; (*) echo "Unknown input type $INPUT_TYPE" @@ -47,7 +49,7 @@ case $INPUT_TYPE in ;; esac -export FLINK_JOB_ARGUMENTS="--input ${INPUT_LOCATION} --output ${OUTPUT_PATH}/docker_wc_out" +export FLINK_JOB_ARGUMENTS="${INPUT_ARGS} --output ${OUTPUT_PATH}/docker_wc_out" build_image() { ./build.sh --from-local-dist --job-artifacts ${FLINK_DIR}/examples/batch/WordCount.jar --image-name ${FLINK_DOCKER_IMAGE_NAME} @@ -68,4 +70,4 @@ docker-compose -f ${DOCKER_MODULE_DIR}/docker-compose.yml -f ${DOCKER_SCRIPTS}/d docker-compose -f ${DOCKER_MODULE_DIR}/docker-compose.yml -f ${DOCKER_SCRIPTS}/docker-compose.test.yml logs job-cluster > ${FLINK_DIR}/log/jobmanager.log docker-compose -f ${DOCKER_MODULE_DIR}/docker-compose.yml -f ${DOCKER_SCRIPTS}/docker-compose.test.yml logs taskmanager > ${FLINK_DIR}/log/taskmanager.log -check_result_hash "WordCount" $OUTPUT_VOLUME/docker_wc_out "72a690412be8928ba239c2da967328a5" +check_result_hash "WordCount" $OUTPUT_VOLUME/docker_wc_out "${RESULT_HASH}" diff --git a/flink-end-to-end-tests/test-scripts/test_yarn_kerberos_docker.sh b/flink-end-to-end-tests/test-scripts/test_yarn_kerberos_docker.sh index a34961195631..9c9b270ebfd5 100755 --- a/flink-end-to-end-tests/test-scripts/test_yarn_kerberos_docker.sh +++ b/flink-end-to-end-tests/test-scripts/test_yarn_kerberos_docker.sh @@ -31,8 +31,8 @@ case $INPUT_TYPE in (dummy-fs) source "$(dirname "$0")"/common_dummy_fs.sh dummy_fs_setup - INPUT_ARGS="--input dummy://localhost/words" - EXPECTED_RESULT_LOG_CONTAINS=("my,1" "dear,2" "world,2") + INPUT_ARGS="--input dummy://localhost/words --input anotherDummy://localhost/words" + EXPECTED_RESULT_LOG_CONTAINS=("my,2" "dear,4" "world,4") ;; (*) echo "Unknown input type $INPUT_TYPE" From bc4e2458fc2a9be11fd2defbdba74ca0dd7b1ddf Mon Sep 17 00:00:00 2001 From: "danrtsey.wy" Date: Tue, 12 Nov 2019 11:08:34 +0800 Subject: [PATCH 637/746] [FLINK-14382][yarn] Fixes the Flink plugin mechanism on yarn cluster by shipping only the plugin directory instead of adding it to the classpath. And add class isolation check in dummy FS. --- .../anotherdummy/AnotherDummyFSFactory.java | 11 +++++-- .../apache/flink/fs/dummy/DummyFSFactory.java | 11 +++++-- .../flink/yarn/YarnClusterDescriptor.java | 31 +++++++++++++------ .../flink/yarn/YarnClusterDescriptorTest.java | 16 ++++++---- 4 files changed, 50 insertions(+), 19 deletions(-) diff --git a/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/src/main/java/org/apache/flink/fs/anotherdummy/AnotherDummyFSFactory.java b/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/src/main/java/org/apache/flink/fs/anotherdummy/AnotherDummyFSFactory.java index 2c1cb1006851..fb91a67ef3c5 100644 --- a/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/src/main/java/org/apache/flink/fs/anotherdummy/AnotherDummyFSFactory.java +++ b/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/src/main/java/org/apache/flink/fs/anotherdummy/AnotherDummyFSFactory.java @@ -21,7 +21,6 @@ import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.FileSystemFactory; -import java.io.IOException; import java.net.URI; import java.util.HashMap; import java.util.Map; @@ -39,7 +38,15 @@ public String getScheme() { } @Override - public FileSystem create(URI fsUri) throws IOException { + public FileSystem create(URI fsUri) { + String dummyFileSystemClassName = "org.apache.flink.fs.dummy.DummyFSFileSystem"; + try { + this.getClassLoader().loadClass(dummyFileSystemClassName); + throw new RuntimeException(String.format("Class %s should not be visible for classloader of %s", + dummyFileSystemClassName, this.getClass().getCanonicalName())); + } catch (ClassNotFoundException e) { + // Expected exception. + } return fileSystem; } diff --git a/flink-end-to-end-tests/flink-plugins-test/dummy-fs/src/main/java/org/apache/flink/fs/dummy/DummyFSFactory.java b/flink-end-to-end-tests/flink-plugins-test/dummy-fs/src/main/java/org/apache/flink/fs/dummy/DummyFSFactory.java index 351879661b4d..a7107350360e 100644 --- a/flink-end-to-end-tests/flink-plugins-test/dummy-fs/src/main/java/org/apache/flink/fs/dummy/DummyFSFactory.java +++ b/flink-end-to-end-tests/flink-plugins-test/dummy-fs/src/main/java/org/apache/flink/fs/dummy/DummyFSFactory.java @@ -21,7 +21,6 @@ import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.FileSystemFactory; -import java.io.IOException; import java.net.URI; import java.util.HashMap; import java.util.Map; @@ -39,7 +38,15 @@ public String getScheme() { } @Override - public FileSystem create(URI fsUri) throws IOException { + public FileSystem create(URI fsUri) { + String anotherFileSystemClassName = "org.apache.flink.fs.anotherdummy.AnotherDummyFSFileSystem"; + try { + this.getClassLoader().loadClass(anotherFileSystemClassName); + throw new RuntimeException(String.format("Class %s should not be visible for classloader of %s", + anotherFileSystemClassName, this.getClass().getCanonicalName())); + } catch (ClassNotFoundException e) { + // Expected exception. + } return fileSystem; } 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 aad6d3600ddd..50c2fff63424 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 @@ -725,7 +725,10 @@ private ApplicationReport startAppMaster( } ApplicationSubmissionContext appContext = yarnApplication.getApplicationSubmissionContext(); + // The files need to be shipped and added to classpath. Set systemShipFiles = new HashSet<>(shipFiles.size()); + // The files only need to be shipped. + Set shipOnlyFiles = new HashSet<>(); for (File file : shipFiles) { systemShipFiles.add(file.getAbsoluteFile()); } @@ -735,7 +738,10 @@ private ApplicationReport startAppMaster( systemShipFiles.add(new File(logConfigFilePath)); } - addEnvironmentFoldersToShipFiles(systemShipFiles); + addLibFoldersToShipFiles(systemShipFiles); + + // Plugin files only need to be shipped and should not be added to classpath. + addPluginsFoldersToShipFiles(shipOnlyFiles); // Set-up ApplicationSubmissionContext for the application @@ -781,7 +787,7 @@ private ApplicationReport startAppMaster( // ship list that enables reuse of resources for task manager containers StringBuilder envShipFileList = new StringBuilder(); - // upload and register ship files + // upload and register ship files, these files will be added to classpath. List systemClassPaths = uploadAndRegisterFiles( systemShipFiles, fs, @@ -791,6 +797,16 @@ private ApplicationReport startAppMaster( localResources, envShipFileList); + // upload and register ship-only files + uploadAndRegisterFiles( + shipOnlyFiles, + fs, + homeDir, + appId, + paths, + localResources, + envShipFileList); + final List userClassPaths = uploadAndRegisterFiles( userJarFiles, fs, @@ -1559,12 +1575,8 @@ public void run() { } } - void addEnvironmentFoldersToShipFiles(Collection effectiveShipFiles) { - addLibFoldersToShipFiles(effectiveShipFiles); - addPluginsFoldersToShipFiles(effectiveShipFiles); - } - - private void addLibFoldersToShipFiles(Collection effectiveShipFiles) { + @VisibleForTesting + 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) @@ -1583,7 +1595,8 @@ private void addLibFoldersToShipFiles(Collection effectiveShipFiles) { } } - private void addPluginsFoldersToShipFiles(Collection effectiveShipFiles) { + @VisibleForTesting + void addPluginsFoldersToShipFiles(Collection effectiveShipFiles) { final Optional pluginsDir = PluginConfig.getPluginsDir(); pluginsDir.ifPresent(effectiveShipFiles::add); } 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 30ba407ecb51..b39fbf8825d1 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 @@ -440,7 +440,7 @@ public void testExplicitFileShipping() throws Exception { // only execute part of the deployment to test for shipped files Set effectiveShipFiles = new HashSet<>(); - descriptor.addEnvironmentFoldersToShipFiles(effectiveShipFiles); + descriptor.addLibFoldersToShipFiles(effectiveShipFiles); Assert.assertEquals(0, effectiveShipFiles.size()); Assert.assertEquals(2, descriptor.getShipFiles().size()); @@ -451,15 +451,15 @@ public void testExplicitFileShipping() throws Exception { @Test public void testEnvironmentLibShipping() throws Exception { - testEnvironmentDirectoryShipping(ConfigConstants.ENV_FLINK_LIB_DIR); + testEnvironmentDirectoryShipping(ConfigConstants.ENV_FLINK_LIB_DIR, false); } @Test public void testEnvironmentPluginsShipping() throws Exception { - testEnvironmentDirectoryShipping(ConfigConstants.ENV_FLINK_PLUGINS_DIR); + testEnvironmentDirectoryShipping(ConfigConstants.ENV_FLINK_PLUGINS_DIR, true); } - private void testEnvironmentDirectoryShipping(String environmentVariable) throws Exception { + private void testEnvironmentDirectoryShipping(String environmentVariable, boolean onlyShip) throws Exception { try (YarnClusterDescriptor descriptor = createYarnClusterDescriptor()) { File libFolder = temporaryFolder.newFolder().getAbsoluteFile(); File libFile = new File(libFolder, "libFile.jar"); @@ -473,7 +473,11 @@ private void testEnvironmentDirectoryShipping(String environmentVariable) throws env.put(environmentVariable, libFolder.getAbsolutePath()); CommonTestUtils.setEnv(env); // only execute part of the deployment to test for shipped files - descriptor.addEnvironmentFoldersToShipFiles(effectiveShipFiles); + if (onlyShip) { + descriptor.addPluginsFoldersToShipFiles(effectiveShipFiles); + } else { + descriptor.addLibFoldersToShipFiles(effectiveShipFiles); + } } finally { CommonTestUtils.setEnv(oldEnv); } @@ -498,7 +502,7 @@ public void testEnvironmentEmptyPluginsShipping() { env.put(ConfigConstants.ENV_FLINK_PLUGINS_DIR, pluginsFolder.getAbsolutePath()); CommonTestUtils.setEnv(env); // only execute part of the deployment to test for shipped files - descriptor.addEnvironmentFoldersToShipFiles(effectiveShipFiles); + descriptor.addPluginsFoldersToShipFiles(effectiveShipFiles); } finally { CommonTestUtils.setEnv(oldEnv); } From ae553ac1bc5697eba68fa98a2bb6c434287565b3 Mon Sep 17 00:00:00 2001 From: Paul Lam Date: Tue, 5 Nov 2019 22:03:32 +0800 Subject: [PATCH 638/746] [FLINK-13747] Make Client respect classloading policy --- .../org/apache/flink/client/ClientUtils.java | 16 ++++- .../apache/flink/client/RemoteExecutor.java | 3 +- .../apache/flink/client/cli/CliFrontend.java | 1 + .../flink/client/program/PackagedProgram.java | 25 ++++++- .../webmonitor/WebSubmissionExtension.java | 1 + .../webmonitor/handlers/JarListHandler.java | 11 ++- .../handlers/utils/JarHandlerUtils.java | 1 + .../handlers/JarSubmissionITCase.java | 1 + flink-tests/pom.xml | 19 +++++ .../test-classloading_policy-assembly.xml | 37 ++++++++++ .../test/classloading/ClassLoaderITCase.java | 71 +++++++++++++++++++ .../jar/ClassLoadingPolicyProgram.java | 48 +++++++++++++ flink-tests/src/test/resources/test-resource | 19 +++++ 13 files changed, 245 insertions(+), 8 deletions(-) create mode 100644 flink-tests/src/test/assembly/test-classloading_policy-assembly.xml create mode 100644 flink-tests/src/test/java/org/apache/flink/test/classloading/jar/ClassLoadingPolicyProgram.java create mode 100644 flink-tests/src/test/resources/test-resource 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 02b73a9248b6..7bfeba55c2eb 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 @@ -29,6 +29,7 @@ import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.client.program.ProgramMissingJobException; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -79,7 +80,11 @@ public static void checkJarFile(URL jar) throws IOException { } } - public static ClassLoader buildUserCodeClassLoader(List jars, List classpaths, ClassLoader parent) { + public static ClassLoader buildUserCodeClassLoader( + List jars, + List classpaths, + ClassLoader parent, + Configuration configuration) { URL[] urls = new URL[jars.size() + classpaths.size()]; for (int i = 0; i < jars.size(); i++) { urls[i] = jars.get(i); @@ -87,7 +92,12 @@ public static ClassLoader buildUserCodeClassLoader(List jars, List cla for (int i = 0; i < classpaths.size(); i++) { urls[i + jars.size()] = classpaths.get(i); } - return FlinkUserCodeClassLoaders.parentFirst(urls, parent); + final String[] alwaysParentFirstLoaderPatterns = CoreOptions.getParentFirstLoaderPatterns(configuration); + final String classLoaderResolveOrder = + configuration.getString(CoreOptions.CLASSLOADER_RESOLVE_ORDER); + FlinkUserCodeClassLoaders.ResolveOrder resolveOrder = + FlinkUserCodeClassLoaders.ResolveOrder.fromString(classLoaderResolveOrder); + return FlinkUserCodeClassLoaders.create(resolveOrder, urls, parent, alwaysParentFirstLoaderPatterns); } public static JobExecutionResult submitJob( @@ -149,7 +159,7 @@ public static JobSubmissionResult executeProgram( final boolean detached = executionConfigAccessor.getDetachedMode(); final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader(); - final ClassLoader userCodeClassLoader = ClientUtils.buildUserCodeClassLoader(jobJars, classpaths, contextClassLoader); + final ClassLoader userCodeClassLoader = ClientUtils.buildUserCodeClassLoader(jobJars, classpaths, contextClassLoader, configuration); try { Thread.currentThread().setContextClassLoader(userCodeClassLoader); 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 71fde6411247..c9a041de3612 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 @@ -114,7 +114,8 @@ public JobExecutionResult executePlan( ClassLoader userCodeClassLoader = ClientUtils.buildUserCodeClassLoader( jarFiles, globalClasspaths, - getClass().getClassLoader()); + getClass().getClassLoader(), + clientConfiguration); return executePlanWithJars(jobGraph, userCodeClassLoader); } 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 5d24aeeb8431..38243fc67d58 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 @@ -796,6 +796,7 @@ PackagedProgram buildProgram(final ProgramOptions runOptions) throws FileNotFoun .setJarFile(jarFile) .setUserClassPaths(classpaths) .setEntryPointClassName(entryPointClass) + .setConfiguration(configuration) .setArguments(programArgs) .build(); 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 1d966461be37..b2b990c8d774 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 @@ -21,6 +21,7 @@ import org.apache.flink.api.common.ProgramDescription; import org.apache.flink.client.ClientUtils; import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.util.InstantiationUtil; @@ -104,13 +105,19 @@ public class PackagedProgram { * @param jarFile The jar file which contains the plan. * @param classpaths Additional classpath URLs needed by the Program. * @param entryPointClassName Name of the class which generates the plan. Overrides the class defined - * in the jar file manifest + * in the jar file manifest. + * @param configuration Flink configuration which affects the classloading policy of the Program execution. * @param args Optional. The arguments used to create the pact plan, depend on * implementation of the pact plan. See getDescription(). * @throws ProgramInvocationException This invocation is thrown if the Program can't be properly loaded. Causes * may be a missing / wrong class or manifest files. */ - private PackagedProgram(@Nullable File jarFile, List classpaths, @Nullable String entryPointClassName, String... args) throws ProgramInvocationException { + private PackagedProgram( + @Nullable File jarFile, + List classpaths, + @Nullable String entryPointClassName, + Configuration configuration, + String... args) throws ProgramInvocationException { checkNotNull(classpaths); checkNotNull(args); checkArgument(jarFile != null || entryPointClassName != null, "Either the jarFile or the entryPointClassName needs to be non-null."); @@ -140,7 +147,11 @@ private PackagedProgram(@Nullable File jarFile, List classpaths, @Nullable // 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 = ClientUtils.buildUserCodeClassLoader(getJobJarAndDependencies(), classpaths, getClass().getClassLoader()); + this.userCodeClassLoader = ClientUtils.buildUserCodeClassLoader( + getJobJarAndDependencies(), + classpaths, + getClass().getClassLoader(), + configuration); // load the entry point class this.mainClass = loadMainClass(entryPointClassName, userCodeClassLoader); @@ -531,6 +542,8 @@ public static class Builder { private List userClassPaths = Collections.emptyList(); + private Configuration configuration = new Configuration(); + public Builder setJarFile(@Nullable File jarFile) { this.jarFile = jarFile; return this; @@ -551,6 +564,11 @@ public Builder setArguments(String... args) { return this; } + public Builder setConfiguration(Configuration configuration) { + this.configuration = configuration; + return this; + } + public PackagedProgram build() throws ProgramInvocationException { if (jarFile == null && entryPointClassName == null) { throw new IllegalArgumentException("The jarFile and entryPointClassName can not be null at the same time."); @@ -559,6 +577,7 @@ public PackagedProgram build() throws ProgramInvocationException { jarFile, userClassPaths, entryPointClassName, + configuration, args); } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebSubmissionExtension.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebSubmissionExtension.java index f145c8a0b5f1..efd3c6ab18d2 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebSubmissionExtension.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebSubmissionExtension.java @@ -78,6 +78,7 @@ public WebSubmissionExtension( JarListHeaders.getInstance(), localAddressFuture, jarDir.toFile(), + configuration, executor); final JarRunHandler jarRunHandler = new JarRunHandler( diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java index dc3b0beb02e9..d1e1d043998a 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.client.program.PackagedProgram; +import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.rest.handler.AbstractRestHandler; import org.apache.flink.runtime.rest.handler.HandlerRequest; @@ -59,6 +60,8 @@ public class JarListHandler extends AbstractRestHandler messageHeaders, CompletableFuture localAddressFuture, File jarDir, + Configuration configuration, Executor executor) { super(leaderRetriever, timeout, responseHeaders, messageHeaders); this.localAddressFuture = localAddressFuture; this.jarDir = requireNonNull(jarDir); + this.configuration = configuration; this.executor = requireNonNull(executor); } @@ -129,7 +134,11 @@ protected CompletableFuture handleRequest(@Nonnull HandlerRequest + + create-classloading_policy-jar + process-test-classes + + single + + + + + org.apache.flink.test.classloading.jar.ClassLoadingPolicyProgram + + + classloading_policy + false + + src/test/assembly/test-classloading_policy-assembly.xml + + + diff --git a/flink-tests/src/test/assembly/test-classloading_policy-assembly.xml b/flink-tests/src/test/assembly/test-classloading_policy-assembly.xml new file mode 100644 index 000000000000..db3cea6ff584 --- /dev/null +++ b/flink-tests/src/test/assembly/test-classloading_policy-assembly.xml @@ -0,0 +1,37 @@ + + + + test-jar + + jar + + false + + + ${project.build.testOutputDirectory} + / + + + org/apache/flink/test/classloading/jar/ClassLoadingPolicyProgram.class + + + + 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 23280f0d54a6..9392e976a6da 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 @@ -90,6 +90,9 @@ public class ClassLoaderITCase extends TestLogger { private static final String CHECKPOINTING_CUSTOM_KV_STATE_JAR_PATH = "checkpointing_custom_kv_state-test-jar.jar"; + private static final String CLASSLOADING_POLICY_JAR_PATH = "classloading_policy-test-jar.jar"; + + @ClassRule public static final TemporaryFolder FOLDER = new TemporaryFolder(); @@ -384,4 +387,72 @@ public void testDisposeSavepointWithCustomKvState() throws Exception { invokeThread.join(deadline.timeLeft().toMillis()); assertFalse("Program invoke thread still running", invokeThread.isAlive()); } + + @Test + public void testProgramWithChildFirstClassLoader() throws IOException, ProgramInvocationException { + // We have two files named test-resource in src/resource (parent classloader classpath) and + // tmp folders (child classloader classpath) respectively. + String childResourceDirName = "child0"; + String testResourceName = "test-resource"; + File childResourceDir = FOLDER.newFolder(childResourceDirName); + File childResource = new File(childResourceDir.getAbsolutePath() + File.separator + testResourceName); + assertTrue(childResource.createNewFile()); + + TestStreamEnvironment.setAsContext( + miniClusterResource.getMiniCluster(), + parallelism, + Collections.singleton(new Path(CLASSLOADING_POLICY_JAR_PATH)), + Collections.emptyList()); + + // default child first classloading + final PackagedProgram childFirstProgram = PackagedProgram.newBuilder() + .setJarFile(new File(CLASSLOADING_POLICY_JAR_PATH)) + .setUserClassPaths(Collections.singletonList(childResourceDir.toURI().toURL())) + .setArguments(testResourceName, childResourceDirName) + .build(); + + final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader(); + Thread.currentThread().setContextClassLoader(childFirstProgram.getUserCodeClassLoader()); + try { + childFirstProgram.invokeInteractiveModeForExecution(); + } finally { + Thread.currentThread().setContextClassLoader(contextClassLoader); + } + } + + @Test + public void testProgramWithParentFirstClassLoader() throws IOException, ProgramInvocationException { + // We have two files named test-resource in src/resource (parent classloader classpath) and + // tmp folders (child classloader classpath) respectively. + String childResourceDirName = "child1"; + String testResourceName = "test-resource"; + File childResourceDir = FOLDER.newFolder(childResourceDirName); + File childResource = new File(childResourceDir.getAbsolutePath() + File.separator + testResourceName); + assertTrue(childResource.createNewFile()); + + TestStreamEnvironment.setAsContext( + miniClusterResource.getMiniCluster(), + parallelism, + Collections.singleton(new Path(CLASSLOADING_POLICY_JAR_PATH)), + Collections.emptyList()); + final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader(); + + // parent-first classloading + Configuration parentFirstConf = new Configuration(); + parentFirstConf.setString("classloader.resolve-order", "parent-first"); + + final PackagedProgram parentFirstProgram = PackagedProgram.newBuilder() + .setJarFile(new File(CLASSLOADING_POLICY_JAR_PATH)) + .setUserClassPaths(Collections.singletonList(childResourceDir.toURI().toURL())) + .setConfiguration(parentFirstConf) + .setArguments(testResourceName, "test-classes") + .build(); + + Thread.currentThread().setContextClassLoader(parentFirstProgram.getUserCodeClassLoader()); + try { + parentFirstProgram.invokeInteractiveModeForExecution(); + } finally { + Thread.currentThread().setContextClassLoader(contextClassLoader); + } + } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/ClassLoadingPolicyProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/ClassLoadingPolicyProgram.java new file mode 100644 index 000000000000..a5619618c941 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/ClassLoadingPolicyProgram.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.test.classloading.jar; + +import java.io.File; +import java.net.URL; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A simple program that verifies the classloading policy by ensuring the resource loaded is under the specified + * directory. + **/ +public class ClassLoadingPolicyProgram { + + public static void main(String[] args) throws Exception { + if (args.length < 2) { + throw new IllegalArgumentException("Missing parameters. Expected: "); + } + String resourceName = args[0]; + String expectedResourceDir = args[1]; + URL url = Thread.currentThread().getContextClassLoader().getResource(resourceName); + checkNotNull(url, "Failed to find " + resourceName + " in the classpath"); + File file = new File(url.toURI()); + String actualResourceDir = file.getParentFile().getName(); + if (!actualResourceDir.equals(expectedResourceDir)) { + String msg = "Incorrect " + resourceName + " is loaded, which should be in " + expectedResourceDir + + ", but now is in " + actualResourceDir; + throw new RuntimeException(msg); + } + } +} diff --git a/flink-tests/src/test/resources/test-resource b/flink-tests/src/test/resources/test-resource new file mode 100644 index 000000000000..06d487a68909 --- /dev/null +++ b/flink-tests/src/test/resources/test-resource @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + empty file for testing classloading From a599b0088445d17db73dcea1abe9b844907be2b9 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 14 Nov 2019 14:11:05 +0100 Subject: [PATCH 639/746] [hotfix] Simplify tests in ClassLoaderITCase This also makes child-first classloading explicit in the test instead of relying on the default. --- .../flink/test/classloading/ClassLoaderITCase.java | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 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 9392e976a6da..36cbe24557d9 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 @@ -395,7 +395,7 @@ public void testProgramWithChildFirstClassLoader() throws IOException, ProgramIn String childResourceDirName = "child0"; String testResourceName = "test-resource"; File childResourceDir = FOLDER.newFolder(childResourceDirName); - File childResource = new File(childResourceDir.getAbsolutePath() + File.separator + testResourceName); + File childResource = new File(childResourceDir, testResourceName); assertTrue(childResource.createNewFile()); TestStreamEnvironment.setAsContext( @@ -404,10 +404,14 @@ public void testProgramWithChildFirstClassLoader() throws IOException, ProgramIn Collections.singleton(new Path(CLASSLOADING_POLICY_JAR_PATH)), Collections.emptyList()); - // default child first classloading + // child-first classloading + Configuration childFirstConf = new Configuration(); + childFirstConf.setString("classloader.resolve-order", "child-first"); + final PackagedProgram childFirstProgram = PackagedProgram.newBuilder() .setJarFile(new File(CLASSLOADING_POLICY_JAR_PATH)) .setUserClassPaths(Collections.singletonList(childResourceDir.toURI().toURL())) + .setConfiguration(childFirstConf) .setArguments(testResourceName, childResourceDirName) .build(); @@ -427,7 +431,7 @@ public void testProgramWithParentFirstClassLoader() throws IOException, ProgramI String childResourceDirName = "child1"; String testResourceName = "test-resource"; File childResourceDir = FOLDER.newFolder(childResourceDirName); - File childResource = new File(childResourceDir.getAbsolutePath() + File.separator + testResourceName); + File childResource = new File(childResourceDir, testResourceName); assertTrue(childResource.createNewFile()); TestStreamEnvironment.setAsContext( @@ -435,7 +439,6 @@ public void testProgramWithParentFirstClassLoader() throws IOException, ProgramI parallelism, Collections.singleton(new Path(CLASSLOADING_POLICY_JAR_PATH)), Collections.emptyList()); - final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader(); // parent-first classloading Configuration parentFirstConf = new Configuration(); @@ -448,6 +451,7 @@ public void testProgramWithParentFirstClassLoader() throws IOException, ProgramI .setArguments(testResourceName, "test-classes") .build(); + final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader(); Thread.currentThread().setContextClassLoader(parentFirstProgram.getUserCodeClassLoader()); try { parentFirstProgram.invokeInteractiveModeForExecution(); From 1f28a24263971c5e9591e90b825a5b8e4a65e99b Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Fri, 15 Nov 2019 12:49:49 +0100 Subject: [PATCH 640/746] [FLINK-14808] set correct Classloader in the CLI --- .../src/main/java/org/apache/flink/client/ClientUtils.java | 4 ++-- 1 file changed, 2 insertions(+), 2 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 7bfeba55c2eb..5e53bc316abd 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 @@ -158,9 +158,9 @@ public static JobSubmissionResult executeProgram( final int parallelism = executionConfigAccessor.getParallelism(); final boolean detached = executionConfigAccessor.getDetachedMode(); - final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader(); - final ClassLoader userCodeClassLoader = ClientUtils.buildUserCodeClassLoader(jobJars, classpaths, contextClassLoader, configuration); + final ClassLoader userCodeClassLoader = program.getUserCodeClassLoader(); + final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader(); try { Thread.currentThread().setContextClassLoader(userCodeClassLoader); From 749086daae433ec997a91b13aa1e50c4c931a012 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Thu, 14 Nov 2019 16:19:30 +0100 Subject: [PATCH 641/746] [FLINK-14794][runtime] Fixing KeyedStream#transform to properly relay type information if invoked with factories. --- .../flink/streaming/api/datastream/DataStream.java | 3 ++- .../flink/streaming/api/datastream/KeyedStream.java | 11 ++++++----- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java index b7bea55002cc..ba6c0d4d9852 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java @@ -1244,10 +1244,11 @@ public SingleOutputStreamOperator transform( String operatorName, TypeInformation outTypeInfo, OneInputStreamOperatorFactory operatorFactory) { + return doTransform(operatorName, outTypeInfo, operatorFactory); } - private SingleOutputStreamOperator doTransform( + protected SingleOutputStreamOperator doTransform( String operatorName, TypeInformation outTypeInfo, StreamOperatorFactory operatorFactory) { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java index 8c7937de1104..d2bcfdadae5e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java @@ -49,9 +49,9 @@ import org.apache.flink.streaming.api.graph.StreamGraphGenerator; import org.apache.flink.streaming.api.operators.KeyedProcessOperator; import org.apache.flink.streaming.api.operators.LegacyKeyedProcessOperator; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.StreamGroupedFold; import org.apache.flink.streaming.api.operators.StreamGroupedReduce; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; import org.apache.flink.streaming.api.operators.co.IntervalJoinOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; import org.apache.flink.streaming.api.transformations.PartitionTransformation; @@ -256,11 +256,12 @@ protected DataStream setConnectionType(StreamPartitioner partitioner) { // ------------------------------------------------------------------------ @Override - @PublicEvolving - public SingleOutputStreamOperator transform(String operatorName, - TypeInformation outTypeInfo, OneInputStreamOperator operator) { + protected SingleOutputStreamOperator doTransform( + final String operatorName, + final TypeInformation outTypeInfo, + final StreamOperatorFactory operatorFactory) { - SingleOutputStreamOperator returnStream = super.transform(operatorName, outTypeInfo, operator); + SingleOutputStreamOperator returnStream = super.doTransform(operatorName, outTypeInfo, operatorFactory); // inject the key selector and key type OneInputTransformation transform = (OneInputTransformation) returnStream.getTransformation(); From 6ae01fb21db623740d56cffabccb8369aa2df7d4 Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Tue, 12 Nov 2019 16:01:58 +0100 Subject: [PATCH 642/746] [hotfix][tests] Introduce PartitionDescriptorBuilder --- .../deployment/ShuffleDescriptorTest.java | 12 ++--- .../AbstractPartitionTrackerTest.java | 14 +++-- .../network/partition/PartitionTestUtils.java | 13 +++-- .../partition/ResultPartitionFactoryTest.java | 14 ++--- .../shuffle/PartitionDescriptorBuilder.java | 54 +++++++++++++++++++ .../TaskExecutorSubmissionTest.java | 11 ++-- .../flink/runtime/taskmanager/TaskTest.java | 9 +--- 7 files changed, 83 insertions(+), 44 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/shuffle/PartitionDescriptorBuilder.java diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/ShuffleDescriptorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/ShuffleDescriptorTest.java index f1c58cba3d46..38d906785d98 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/ShuffleDescriptorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/ShuffleDescriptorTest.java @@ -22,10 +22,10 @@ import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; -import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor; import org.apache.flink.runtime.shuffle.NettyShuffleMaster; import org.apache.flink.runtime.shuffle.PartitionDescriptor; +import org.apache.flink.runtime.shuffle.PartitionDescriptorBuilder; import org.apache.flink.runtime.shuffle.ProducerDescriptor; import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.shuffle.UnknownShuffleDescriptor; @@ -166,12 +166,10 @@ private static ResultPartitionDeploymentDescriptor createResultPartitionDeployme id.getProducerId(), STUB_CONNECTION_ID.getAddress().getAddress(), STUB_CONNECTION_ID.getAddress().getPort()); - PartitionDescriptor partitionDescriptor = new PartitionDescriptor( - new IntermediateDataSetID(), - id.getPartitionId(), - ResultPartitionType.PIPELINED, - 1, - 0); + PartitionDescriptor partitionDescriptor = PartitionDescriptorBuilder + .newBuilder() + .setPartitionId(id.getPartitionId()) + .build(); ShuffleDescriptor shuffleDescriptor = NettyShuffleMaster.INSTANCE.registerPartitionWithProducer( partitionDescriptor, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/AbstractPartitionTrackerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/AbstractPartitionTrackerTest.java index 3afabcd3a2af..60ccd7297042 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/AbstractPartitionTrackerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/AbstractPartitionTrackerTest.java @@ -19,8 +19,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; -import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; -import org.apache.flink.runtime.shuffle.PartitionDescriptor; +import org.apache.flink.runtime.shuffle.PartitionDescriptorBuilder; import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.util.TestLogger; @@ -69,12 +68,11 @@ static ResultPartitionDeploymentDescriptor createResultPartitionDeploymentDescri boolean hasLocalResources) { return new ResultPartitionDeploymentDescriptor( - new PartitionDescriptor( - new IntermediateDataSetID(), - resultPartitionId.getPartitionId(), - type, - 1, - 0), + PartitionDescriptorBuilder + .newBuilder() + .setPartitionId(resultPartitionId.getPartitionId()) + .setPartitionType(type) + .build(), new ShuffleDescriptor() { @Override public ResultPartitionID getResultPartitionID() { 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 a29f50f197b6..40a8d0cb5560 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 @@ -22,8 +22,8 @@ import org.apache.flink.runtime.io.disk.FileChannelManager; import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; -import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.shuffle.PartitionDescriptor; +import org.apache.flink.runtime.shuffle.PartitionDescriptorBuilder; import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.util.NettyShuffleDescriptorBuilder; @@ -99,12 +99,11 @@ static void verifyCreateSubpartitionViewThrowsException( public static ResultPartitionDeploymentDescriptor createPartitionDeploymentDescriptor( ResultPartitionType partitionType) { ShuffleDescriptor shuffleDescriptor = NettyShuffleDescriptorBuilder.newBuilder().buildLocal(); - PartitionDescriptor partitionDescriptor = new PartitionDescriptor( - new IntermediateDataSetID(), - shuffleDescriptor.getResultPartitionID().getPartitionId(), - partitionType, - 1, - 0); + PartitionDescriptor partitionDescriptor = PartitionDescriptorBuilder + .newBuilder() + .setPartitionId(shuffleDescriptor.getResultPartitionID().getPartitionId()) + .setPartitionType(partitionType) + .build(); return new ResultPartitionDeploymentDescriptor( partitionDescriptor, shuffleDescriptor, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactoryTest.java index 653c7f5a6583..fd1ad512f087 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactoryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactoryTest.java @@ -21,9 +21,7 @@ import org.apache.flink.runtime.io.disk.FileChannelManager; import org.apache.flink.runtime.io.disk.FileChannelManagerImpl; import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; -import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; -import org.apache.flink.runtime.shuffle.PartitionDescriptor; +import org.apache.flink.runtime.shuffle.PartitionDescriptorBuilder; import org.apache.flink.runtime.util.EnvironmentInformation; import org.apache.flink.runtime.util.NettyShuffleDescriptorBuilder; import org.apache.flink.util.TestLogger; @@ -103,12 +101,10 @@ private static ResultPartition createResultPartition( releasePartitionOnConsumption); final ResultPartitionDeploymentDescriptor descriptor = new ResultPartitionDeploymentDescriptor( - new PartitionDescriptor( - new IntermediateDataSetID(), - new IntermediateResultPartitionID(), - partitionType, - 1, - 0), + PartitionDescriptorBuilder + .newBuilder() + .setPartitionType(partitionType) + .build(), NettyShuffleDescriptorBuilder.newBuilder().buildLocal(), 1, true diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/shuffle/PartitionDescriptorBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/shuffle/PartitionDescriptorBuilder.java new file mode 100644 index 000000000000..54810c71f064 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/shuffle/PartitionDescriptorBuilder.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.shuffle; + +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; +import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; +import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; + +/** + * Builder for {@link PartitionDescriptor} in tests. + */ +public class PartitionDescriptorBuilder { + private IntermediateResultPartitionID partitionId; + private ResultPartitionType partitionType; + + private PartitionDescriptorBuilder() { + this.partitionId = new IntermediateResultPartitionID(); + this.partitionType = ResultPartitionType.PIPELINED; + } + + public PartitionDescriptorBuilder setPartitionId(IntermediateResultPartitionID partitionId) { + this.partitionId = partitionId; + return this; + } + + public PartitionDescriptorBuilder setPartitionType(ResultPartitionType partitionType) { + this.partitionType = partitionType; + return this; + } + + public PartitionDescriptor build() { + return new PartitionDescriptor(new IntermediateDataSetID(), partitionId, partitionType, 1, 0); + } + + public static PartitionDescriptorBuilder newBuilder() { + return new PartitionDescriptorBuilder(); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java index 68fd372db361..bd3dcc0b334a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java @@ -56,6 +56,7 @@ import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor; import org.apache.flink.runtime.shuffle.PartitionDescriptor; +import org.apache.flink.runtime.shuffle.PartitionDescriptorBuilder; import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable; import org.apache.flink.runtime.taskmanager.Task; @@ -690,12 +691,10 @@ private TaskDeploymentDescriptor createSender(NettyShuffleDescriptor shuffleDesc private TaskDeploymentDescriptor createSender( NettyShuffleDescriptor shuffleDescriptor, Class abstractInvokable) throws IOException { - PartitionDescriptor partitionDescriptor = new PartitionDescriptor( - new IntermediateDataSetID(), - shuffleDescriptor.getResultPartitionID().getPartitionId(), - ResultPartitionType.PIPELINED, - 1, - 0); + PartitionDescriptor partitionDescriptor = PartitionDescriptorBuilder + .newBuilder() + .setPartitionId(shuffleDescriptor.getResultPartitionID().getPartitionId()) + .build(); ResultPartitionDeploymentDescriptor resultPartitionDeploymentDescriptor = new ResultPartitionDeploymentDescriptor( partitionDescriptor, shuffleDescriptor, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java index b34a77367ceb..86696263fc38 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java @@ -43,10 +43,10 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.io.network.partition.consumer.RemoteChannelStateChecker; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobmanager.PartitionProducerDisposedException; import org.apache.flink.runtime.shuffle.PartitionDescriptor; +import org.apache.flink.runtime.shuffle.PartitionDescriptorBuilder; import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.taskexecutor.PartitionProducerStateChecker; @@ -245,12 +245,7 @@ public void testExecutionFailsInBlobsMissing() throws Exception { @Test public void testExecutionFailsInNetworkRegistrationForPartitions() throws Exception { - final PartitionDescriptor partitionDescriptor = new PartitionDescriptor( - new IntermediateDataSetID(), - new IntermediateResultPartitionID(), - ResultPartitionType.PIPELINED, - 1, - 1); + final PartitionDescriptor partitionDescriptor = PartitionDescriptorBuilder.newBuilder().build(); final ShuffleDescriptor shuffleDescriptor = NettyShuffleDescriptorBuilder.newBuilder().buildLocal(); final ResultPartitionDeploymentDescriptor dummyPartition = new ResultPartitionDeploymentDescriptor( partitionDescriptor, From cc02399c4f4c1076153a18d7928720f95725d400 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 8 Nov 2019 14:26:24 +0100 Subject: [PATCH 643/746] [FLINK-14679][shuffle] Store number of partitions in PartitionDescriptor --- .../ResultPartitionDeploymentDescriptor.java | 4 ++++ .../flink/runtime/shuffle/PartitionDescriptor.java | 11 +++++++++++ .../ResultPartitionDeploymentDescriptorTest.java | 3 +++ .../runtime/shuffle/PartitionDescriptorBuilder.java | 8 +++++++- 4 files changed, 25 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptor.java index 064c9bdff173..072019158a3a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptor.java @@ -72,6 +72,10 @@ public ResultPartitionType getPartitionType() { return partitionDescriptor.getPartitionType(); } + public int getTotalNumberOfPartitions() { + return partitionDescriptor.getTotalNumberOfPartitions(); + } + public int getNumberOfSubpartitions() { return partitionDescriptor.getNumberOfSubpartitions(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/PartitionDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/PartitionDescriptor.java index e63c97507088..63d17f28faad 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/PartitionDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/PartitionDescriptor.java @@ -42,6 +42,9 @@ public class PartitionDescriptor implements Serializable { /** The ID of the result this partition belongs to. */ private final IntermediateDataSetID resultId; + /** The total number of partitions for the result. */ + private final int totalNumberOfPartitions; + /** The ID of the partition. */ private final IntermediateResultPartitionID partitionId; @@ -57,11 +60,14 @@ public class PartitionDescriptor implements Serializable { @VisibleForTesting public PartitionDescriptor( IntermediateDataSetID resultId, + int totalNumberOfPartitions, IntermediateResultPartitionID partitionId, ResultPartitionType partitionType, int numberOfSubpartitions, int connectionIndex) { this.resultId = checkNotNull(resultId); + checkArgument(totalNumberOfPartitions >= 1); + this.totalNumberOfPartitions = totalNumberOfPartitions; this.partitionId = checkNotNull(partitionId); this.partitionType = checkNotNull(partitionType); checkArgument(numberOfSubpartitions >= 1); @@ -73,6 +79,10 @@ public IntermediateDataSetID getResultId() { return resultId; } + public int getTotalNumberOfPartitions() { + return totalNumberOfPartitions; + } + public IntermediateResultPartitionID getPartitionId() { return partitionId; } @@ -119,6 +129,7 @@ public static PartitionDescriptor from(IntermediateResultPartition partition) { IntermediateResult result = partition.getIntermediateResult(); return new PartitionDescriptor( result.getId(), + partition.getIntermediateResult().getNumberOfAssignedPartitions(), partition.getPartitionId(), result.getResultType(), numberOfSubpartitions, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptorTest.java index e76af09a53fd..040a055cec6f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/ResultPartitionDeploymentDescriptorTest.java @@ -47,6 +47,7 @@ */ public class ResultPartitionDeploymentDescriptorTest extends TestLogger { private static final IntermediateDataSetID resultId = new IntermediateDataSetID(); + private static final int numberOfPartitions = 5; private static final IntermediateResultPartitionID partitionId = new IntermediateResultPartitionID(); private static final ExecutionAttemptID producerExecutionId = new ExecutionAttemptID(); @@ -57,6 +58,7 @@ public class ResultPartitionDeploymentDescriptorTest extends TestLogger { private static final PartitionDescriptor partitionDescriptor = new PartitionDescriptor( resultId, + numberOfPartitions, partitionId, partitionType, numberOfSubpartitions, @@ -115,6 +117,7 @@ private static ResultPartitionDeploymentDescriptor createCopyAndVerifyResultPart private static void verifyResultPartitionDeploymentDescriptorCopy(ResultPartitionDeploymentDescriptor copy) { assertThat(copy.getResultId(), is(resultId)); + assertThat(copy.getTotalNumberOfPartitions(), is(numberOfPartitions)); assertThat(copy.getPartitionId(), is(partitionId)); assertThat(copy.getPartitionType(), is(partitionType)); assertThat(copy.getNumberOfSubpartitions(), is(numberOfSubpartitions)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/shuffle/PartitionDescriptorBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/shuffle/PartitionDescriptorBuilder.java index 54810c71f064..6b11a32c2bce 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/shuffle/PartitionDescriptorBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/shuffle/PartitionDescriptorBuilder.java @@ -28,6 +28,7 @@ public class PartitionDescriptorBuilder { private IntermediateResultPartitionID partitionId; private ResultPartitionType partitionType; + private int totalNumberOfPartitions = 1; private PartitionDescriptorBuilder() { this.partitionId = new IntermediateResultPartitionID(); @@ -44,8 +45,13 @@ public PartitionDescriptorBuilder setPartitionType(ResultPartitionType partition return this; } + public PartitionDescriptorBuilder setTotalNumberOfPartitions(int totalNumberOfPartitions) { + this.totalNumberOfPartitions = totalNumberOfPartitions; + return this; + } + public PartitionDescriptor build() { - return new PartitionDescriptor(new IntermediateDataSetID(), partitionId, partitionType, 1, 0); + return new PartitionDescriptor(new IntermediateDataSetID(), totalNumberOfPartitions, partitionId, partitionType, 1, 0); } public static PartitionDescriptorBuilder newBuilder() { From 22b2a8856307c310b4c75b32eeed33ba66c0206e Mon Sep 17 00:00:00 2001 From: Joao Boto Date: Thu, 14 Nov 2019 20:14:02 +0100 Subject: [PATCH 644/746] [FLINK-13850] refactor part file configurations into a single method Closes #9533 --- docs/dev/connectors/streamfile_sink.md | 14 +++ .../api/functions/sink/filesystem/Bucket.java | 24 ++--- .../sink/filesystem/BucketFactory.java | 4 +- .../functions/sink/filesystem/Buckets.java | 10 +- .../filesystem/DefaultBucketFactoryImpl.java | 8 +- .../sink/filesystem/OutputFileConfig.java | 98 +++++++++++++++++++ .../sink/filesystem/PartFileConfig.java | 53 ---------- .../sink/filesystem/StreamingFileSink.java | 58 ++++------- .../filesystem/BucketAssignerITCases.java | 2 +- .../functions/sink/filesystem/BucketTest.java | 32 +++++- .../sink/filesystem/BucketsTest.java | 6 +- .../sink/filesystem/BulkWriterTest.java | 3 +- .../sink/filesystem/RollingPolicyTest.java | 2 +- .../functions/sink/filesystem/TestUtils.java | 9 +- 14 files changed, 192 insertions(+), 131 deletions(-) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/OutputFileConfig.java delete mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/PartFileConfig.java diff --git a/docs/dev/connectors/streamfile_sink.md b/docs/dev/connectors/streamfile_sink.md index c59e646a3548..378a0f4d63cf 100644 --- a/docs/dev/connectors/streamfile_sink.md +++ b/docs/dev/connectors/streamfile_sink.md @@ -133,6 +133,20 @@ New buckets are created as dictated by the bucketing policy, and this doesn't af Old buckets can still receive new records as the bucketing policy is evaluated on a per-record basis. +### Part file configuration + +The filenames of the part files could be defined using `OutputFileConfig`, this configuration contain a part prefix and part suffix, +that will be used with the parallel subtask index of the sink and a rolling counter. +For example for a prefix "prefix" and a suffix ".ext" the file create: + +``` +└── 2019-08-25--12 + ├── prefix-0-0.ext + ├── prefix-0-1.ext.inprogress.bd053eb0-5ecf-4c85-8433-9eff486ac334 + ├── prefix-1-0.ext + └── prefix-1-1.ext.inprogress.bc279efe-b16f-47d8-b828-00ef6e2fbd11 +``` + ## File Formats The `StreamingFileSink` supports both row-wise and bulk encoding formats, such as [Apache Parquet](http://parquet.apache.org). diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java index 4a996e7884b9..7efd2bada644 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java @@ -70,7 +70,7 @@ public class Bucket { private final NavigableMap> pendingPartsPerCheckpoint; - private final PartFileConfig partFileConfig; + private final OutputFileConfig outputFileConfig; private long partCounter; @@ -90,7 +90,7 @@ private Bucket( final long initialPartCounter, final PartFileWriter.PartFileFactory partFileFactory, final RollingPolicy rollingPolicy, - final PartFileConfig partFileConfig) { + final OutputFileConfig outputFileConfig) { this.fsWriter = checkNotNull(fsWriter); this.subtaskIndex = subtaskIndex; this.bucketId = checkNotNull(bucketId); @@ -103,7 +103,7 @@ private Bucket( this.pendingPartsPerCheckpoint = new TreeMap<>(); this.resumablesPerCheckpoint = new TreeMap<>(); - this.partFileConfig = checkNotNull(partFileConfig); + this.outputFileConfig = checkNotNull(outputFileConfig); } /** @@ -116,7 +116,7 @@ private Bucket( final PartFileWriter.PartFileFactory partFileFactory, final RollingPolicy rollingPolicy, final BucketState bucketState, - final PartFileConfig partFileConfig) throws IOException { + final OutputFileConfig outputFileConfig) throws IOException { this( fsWriter, @@ -126,7 +126,7 @@ private Bucket( initialPartCounter, partFileFactory, rollingPolicy, - partFileConfig); + outputFileConfig); restoreInProgressFile(bucketState); commitRecoveredPendingFiles(bucketState); @@ -230,7 +230,7 @@ private void rollPartFile(final long currentTime) throws IOException { } private Path assembleNewPartPath() { - return new Path(bucketPath, partFileConfig.getPartPrefix() + '-' + subtaskIndex + '-' + partCounter + partFileConfig.getPartSuffix()); + return new Path(bucketPath, outputFileConfig.getPartPrefix() + '-' + subtaskIndex + '-' + partCounter + outputFileConfig.getPartSuffix()); } private CommitRecoverable closePartFile() throws IOException { @@ -369,7 +369,7 @@ List getPendingPartsForCurrentCheckpoint() { * @param partFileFactory the {@link PartFileWriter.PartFileFactory} the factory creating part file writers. * @param the type of input elements to the sink. * @param the type of the identifier of the bucket, as returned by the {@link BucketAssigner} - * @param partFileConfig the part file configuration. + * @param outputFileConfig the part file configuration. * @return The new Bucket. */ static Bucket getNew( @@ -380,8 +380,8 @@ static Bucket getNew( final long initialPartCounter, final PartFileWriter.PartFileFactory partFileFactory, final RollingPolicy rollingPolicy, - final PartFileConfig partFileConfig) { - return new Bucket<>(fsWriter, subtaskIndex, bucketId, bucketPath, initialPartCounter, partFileFactory, rollingPolicy, partFileConfig); + final OutputFileConfig outputFileConfig) { + return new Bucket<>(fsWriter, subtaskIndex, bucketId, bucketPath, initialPartCounter, partFileFactory, rollingPolicy, outputFileConfig); } /** @@ -393,7 +393,7 @@ static Bucket getNew( * @param bucketState the initial state of the restored bucket. * @param the type of input elements to the sink. * @param the type of the identifier of the bucket, as returned by the {@link BucketAssigner} - * @param partFileConfig the part file configuration. + * @param outputFileConfig the part file configuration. * @return The restored Bucket. */ static Bucket restore( @@ -403,7 +403,7 @@ static Bucket restore( final PartFileWriter.PartFileFactory partFileFactory, final RollingPolicy rollingPolicy, final BucketState bucketState, - final PartFileConfig partFileConfig) throws IOException { - return new Bucket<>(fsWriter, subtaskIndex, initialPartCounter, partFileFactory, rollingPolicy, bucketState, partFileConfig); + final OutputFileConfig outputFileConfig) throws IOException { + return new Bucket<>(fsWriter, subtaskIndex, initialPartCounter, partFileFactory, rollingPolicy, bucketState, outputFileConfig); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketFactory.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketFactory.java index c318ef213bb5..260e82c79601 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketFactory.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketFactory.java @@ -39,7 +39,7 @@ Bucket getNewBucket( final long initialPartCounter, final PartFileWriter.PartFileFactory partFileWriterFactory, final RollingPolicy rollingPolicy, - final PartFileConfig partFileConfig) throws IOException; + final OutputFileConfig outputFileConfig) throws IOException; Bucket restoreBucket( final RecoverableWriter fsWriter, @@ -48,5 +48,5 @@ Bucket restoreBucket( final PartFileWriter.PartFileFactory partFileWriterFactory, final RollingPolicy rollingPolicy, final BucketState bucketState, - final PartFileConfig partFileConfig) throws IOException; + final OutputFileConfig outputFileConfig) throws IOException; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java index fa7a72b7c572..eb61d18e0a58 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java @@ -77,7 +77,7 @@ public class Buckets { private final RecoverableWriter fsWriter; - private final PartFileConfig partFileConfig; + private final OutputFileConfig outputFileConfig; // --------------------------- State Related Fields ----------------------------- @@ -99,7 +99,7 @@ public class Buckets { final PartFileWriter.PartFileFactory partFileWriterFactory, final RollingPolicy rollingPolicy, final int subtaskIndex, - final PartFileConfig partFileConfig) throws IOException { + final OutputFileConfig outputFileConfig) throws IOException { this.basePath = Preconditions.checkNotNull(basePath); this.bucketAssigner = Preconditions.checkNotNull(bucketAssigner); @@ -108,7 +108,7 @@ public class Buckets { this.rollingPolicy = Preconditions.checkNotNull(rollingPolicy); this.subtaskIndex = subtaskIndex; - this.partFileConfig = Preconditions.checkNotNull(partFileConfig); + this.outputFileConfig = Preconditions.checkNotNull(outputFileConfig); this.activeBuckets = new HashMap<>(); this.bucketerContext = new Buckets.BucketerContext(); @@ -186,7 +186,7 @@ private void handleRestoredBucketState(final BucketState recoveredStat partFileWriterFactory, rollingPolicy, recoveredState, - partFileConfig + outputFileConfig ); updateActiveBucketId(bucketId, restoredBucket); @@ -293,7 +293,7 @@ private Bucket getOrCreateBucketForBucketId(final BucketID bucketI maxPartCounter, partFileWriterFactory, rollingPolicy, - partFileConfig); + outputFileConfig); activeBuckets.put(bucketId, bucket); } return bucket; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/DefaultBucketFactoryImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/DefaultBucketFactoryImpl.java index dac2a5af0de1..529b93afa8c5 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/DefaultBucketFactoryImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/DefaultBucketFactoryImpl.java @@ -41,7 +41,7 @@ public Bucket getNewBucket( final long initialPartCounter, final PartFileWriter.PartFileFactory partFileWriterFactory, final RollingPolicy rollingPolicy, - final PartFileConfig partFileConfig) { + final OutputFileConfig outputFileConfig) { return Bucket.getNew( fsWriter, @@ -51,7 +51,7 @@ public Bucket getNewBucket( initialPartCounter, partFileWriterFactory, rollingPolicy, - partFileConfig); + outputFileConfig); } @Override @@ -62,7 +62,7 @@ public Bucket restoreBucket( final PartFileWriter.PartFileFactory partFileWriterFactory, final RollingPolicy rollingPolicy, final BucketState bucketState, - final PartFileConfig partFileConfig) throws IOException { + final OutputFileConfig outputFileConfig) throws IOException { return Bucket.restore( fsWriter, @@ -71,6 +71,6 @@ public Bucket restoreBucket( partFileWriterFactory, rollingPolicy, bucketState, - partFileConfig); + outputFileConfig); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/OutputFileConfig.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/OutputFileConfig.java new file mode 100644 index 000000000000..98b84d178a27 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/OutputFileConfig.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.api.functions.sink.filesystem; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.util.Preconditions; + +import java.io.Serializable; + +/** + * Part file name configuration. + * This allow to define a prefix and a suffix to the part file name. + */ +public class OutputFileConfig implements Serializable { + + private final String partPrefix; + + private final String partSuffix; + + /** + * Initiates the {@code PartFileConfig} with values passed as parameters. + * + * @param partPrefix - the beginning of part file name + * @param partSuffix - the ending of part file name + */ + public OutputFileConfig(final String partPrefix, final String partSuffix) { + this.partPrefix = Preconditions.checkNotNull(partPrefix); + this.partSuffix = Preconditions.checkNotNull(partSuffix); + } + + /** + * The prefix for the part name. + */ + String getPartPrefix() { + return partPrefix; + } + + /** + * The suffix for the part name. + */ + String getPartSuffix() { + return partSuffix; + } + + public static OutputFileConfigBuilder builder() { + return new OutputFileConfigBuilder(); + } + + /** + * A builder to create the part file configuration. + */ + @PublicEvolving + public static class OutputFileConfigBuilder { + + private static final String DEFAULT_PART_PREFIX = "part"; + + private static final String DEFAULT_PART_SUFFIX = ""; + + private String partPrefix; + + private String partSuffix; + + private OutputFileConfigBuilder() { + this.partPrefix = DEFAULT_PART_PREFIX; + this.partSuffix = DEFAULT_PART_SUFFIX; + } + + public OutputFileConfigBuilder withPartPrefix(String prefix) { + this.partPrefix = prefix; + return this; + } + + public OutputFileConfigBuilder withPartSuffix(String suffix) { + this.partSuffix = suffix; + return this; + } + + public OutputFileConfig build() { + return new OutputFileConfig(partPrefix, partSuffix); + } + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/PartFileConfig.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/PartFileConfig.java deleted file mode 100644 index f3eaa59261cb..000000000000 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/PartFileConfig.java +++ /dev/null @@ -1,53 +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.api.functions.sink.filesystem; - -import org.apache.flink.util.Preconditions; - -/** - * Part file name configuration. - * This allow to define a prefix and a suffix to the part file name. - */ -class PartFileConfig { - - public static final String DEFAULT_PART_PREFIX = "part"; - - public static final String DEFAULT_PART_SUFFIX = ""; - - private final String partPrefix; - - private final String partSuffix; - - PartFileConfig() { - this(DEFAULT_PART_PREFIX, DEFAULT_PART_SUFFIX); - } - - PartFileConfig(final String partPrefix, final String partSuffix) { - this.partPrefix = Preconditions.checkNotNull(partPrefix); - this.partSuffix = Preconditions.checkNotNull(partSuffix); - } - - String getPartPrefix() { - return partPrefix; - } - - String getPartSuffix() { - return partSuffix; - } -} 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 ed7e9264249a..cd4afc2d8404 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 @@ -67,9 +67,11 @@ * {@link StreamingFileSink#forBulkFormat(Path, BulkWriter.Factory)}. * * - *

    The filenames of the part files contain the part prefix, "part-", the parallel subtask index of the sink - * and a rolling counter. For example the file {@code "part-1-17"} contains the data from - * {@code subtask 1} of the sink and is the {@code 17th} bucket created by that subtask. + *

    The filenames of the part files could be defined using {@link OutputFileConfig}, this configuration contain + * a part prefix and part suffix, that will be used with the parallel subtask index of the sink + * and a rolling counter. For example for a prefix "prefix" and a suffix ".ext" the file create will have a name + * {@code "prefix-1-17.ext"} containing the data from {@code subtask 1} of the sink and is the {@code 17th} bucket + * created by that subtask. * Part files roll based on the user-specified {@link RollingPolicy}. By default, a {@link DefaultRollingPolicy} * is used. * @@ -217,12 +219,10 @@ public static class RowFormatBuilder bucketFactory; - private String partFilePrefix; - - private String partFileSuffix; + private OutputFileConfig outputFileConfig; 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); + this(basePath, encoder, bucketAssigner, DefaultRollingPolicy.builder().build(), DEFAULT_BUCKET_CHECK_INTERVAL, new DefaultBucketFactoryImpl<>(), OutputFileConfig.builder().build()); } protected RowFormatBuilder( @@ -232,16 +232,14 @@ protected RowFormatBuilder( RollingPolicy policy, long bucketCheckInterval, BucketFactory bucketFactory, - String partFilePrefix, - String partFileSuffix) { + OutputFileConfig outputFileConfig) { this.basePath = Preconditions.checkNotNull(basePath); this.encoder = Preconditions.checkNotNull(encoder); this.bucketAssigner = Preconditions.checkNotNull(assigner); this.rollingPolicy = Preconditions.checkNotNull(policy); this.bucketCheckInterval = bucketCheckInterval; this.bucketFactory = Preconditions.checkNotNull(bucketFactory); - this.partFilePrefix = Preconditions.checkNotNull(partFilePrefix); - this.partFileSuffix = Preconditions.checkNotNull(partFileSuffix); + this.outputFileConfig = Preconditions.checkNotNull(outputFileConfig); } public long getBucketCheckInterval() { @@ -263,19 +261,14 @@ public T withRollingPolicy(final RollingPolicy policy) { return self(); } - public T withPartFilePrefix(final String partPrefix) { - this.partFilePrefix = partPrefix; - return self(); - } - - public T withPartFileSuffix(final String partSuffix) { - this.partFileSuffix = partSuffix; + public T withOutputFileConfig(final OutputFileConfig outputFileConfig) { + this.outputFileConfig = outputFileConfig; 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); + return new RowFormatBuilder<>(basePath, encoder, Preconditions.checkNotNull(assigner), Preconditions.checkNotNull(policy), bucketCheckInterval, new DefaultBucketFactoryImpl<>(), outputFileConfig); } /** Creates the actual sink. */ @@ -298,7 +291,7 @@ Buckets createBuckets(int subtaskIndex) throws IOException { new RowWisePartWriter.Factory<>(encoder), rollingPolicy, subtaskIndex, - new PartFileConfig(partFilePrefix, partFileSuffix)); + outputFileConfig); } } @@ -320,12 +313,10 @@ public static class BulkFormatBuilder bucketFactory; - private String partFilePrefix; - - private String partFileSuffix; + private OutputFileConfig outputFileConfig; 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); + this(basePath, writerFactory, assigner, DEFAULT_BUCKET_CHECK_INTERVAL, new DefaultBucketFactoryImpl<>(), OutputFileConfig.builder().build()); } protected BulkFormatBuilder( @@ -334,15 +325,13 @@ protected BulkFormatBuilder( BucketAssigner assigner, long bucketCheckInterval, BucketFactory bucketFactory, - String partFilePrefix, - String partFileSuffix) { + OutputFileConfig outputFileConfig) { this.basePath = Preconditions.checkNotNull(basePath); this.writerFactory = writerFactory; this.bucketAssigner = Preconditions.checkNotNull(assigner); this.bucketCheckInterval = bucketCheckInterval; this.bucketFactory = Preconditions.checkNotNull(bucketFactory); - this.partFilePrefix = Preconditions.checkNotNull(partFilePrefix); - this.partFileSuffix = Preconditions.checkNotNull(partFileSuffix); + this.outputFileConfig = Preconditions.checkNotNull(outputFileConfig); } public long getBucketCheckInterval() { @@ -363,21 +352,16 @@ public T withBucketAssigner(BucketAssigner assigner) { T withBucketFactory(final BucketFactory factory) { this.bucketFactory = Preconditions.checkNotNull(factory); return self(); - } - - public T withPartFilePrefix(final String partPrefix) { - this.partFilePrefix = partPrefix; - return self(); } - public T withPartFileSuffix(final String partSuffix) { - this.partFileSuffix = partSuffix; + public T withOutputFileConfig(final OutputFileConfig outputFileConfig) { + this.outputFileConfig = outputFileConfig; return self(); } 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); + return new BulkFormatBuilder<>(basePath, writerFactory, Preconditions.checkNotNull(assigner), bucketCheckInterval, new DefaultBucketFactoryImpl<>(), outputFileConfig); } /** Creates the actual sink. */ @@ -394,7 +378,7 @@ Buckets createBuckets(int subtaskIndex) throws IOException { new BulkPartWriter.Factory<>(writerFactory), OnCheckpointRollingPolicy.build(), subtaskIndex, - new PartFileConfig(partFilePrefix, partFileSuffix)); + outputFileConfig); } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketAssignerITCases.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketAssignerITCases.java index cc91a335b7fb..02c126411e83 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketAssignerITCases.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketAssignerITCases.java @@ -57,7 +57,7 @@ public void testAssembleBucketPath() throws Exception { new RowWisePartWriter.Factory<>(new SimpleStringEncoder<>()), rollingPolicy, 0, - new PartFileConfig() + PartFileConfig.builder().build() ); Bucket bucket = diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java index 583bacfc8155..2bcb881bfc1d 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java @@ -64,7 +64,7 @@ public void shouldNotCleanupResumablesThatArePartOfTheAckedCheckpoint() throws I final TestRecoverableWriter recoverableWriter = getRecoverableWriter(path); final Bucket bucketUnderTest = - createBucket(recoverableWriter, path, 0, 0, new PartFileConfig()); + createBucket(recoverableWriter, path, 0, 0, PartFileConfig.builder().build()); bucketUnderTest.write("test-element", 0L); @@ -82,7 +82,7 @@ public void shouldCleanupOutdatedResumablesOnCheckpointAck() throws IOException final TestRecoverableWriter recoverableWriter = getRecoverableWriter(path); final Bucket bucketUnderTest = - createBucket(recoverableWriter, path, 0, 0, new PartFileConfig()); + createBucket(recoverableWriter, path, 0, 0, PartFileConfig.builder().build()); bucketUnderTest.write("test-element", 0L); @@ -98,6 +98,28 @@ public void shouldCleanupOutdatedResumablesOnCheckpointAck() throws IOException assertThat(recoverableWriter, hasCalledDiscard(2)); // that is for checkpoints 0 and 1 } + @Test + public void shouldCleanupResumableAfterRestoring() throws Exception { + final File outDir = TEMP_FOLDER.newFolder(); + final Path path = new Path(outDir.toURI()); + + final TestRecoverableWriter recoverableWriter = getRecoverableWriter(path); + final Bucket bucketUnderTest = + createBucket(recoverableWriter, path, 0, 0, PartFileConfig.builder().build()); + + bucketUnderTest.write("test-element", 0L); + + final BucketState state = bucketUnderTest.onReceptionOfCheckpoint(0L); + assertThat(state, hasActiveInProgressFile()); + + bucketUnderTest.onSuccessfulCompletionOfCheckpoint(0L); + + final TestRecoverableWriter newRecoverableWriter = getRecoverableWriter(path); + restoreBucket(newRecoverableWriter, 0, 1, state, PartFileConfig.builder().build()); + + assertThat(newRecoverableWriter, hasCalledDiscard(1)); // that is for checkpoints 0 and 1 + } + @Test public void shouldNotCallCleanupWithoutInProgressPartFiles() throws Exception { final File outDir = TEMP_FOLDER.newFolder(); @@ -105,7 +127,7 @@ public void shouldNotCallCleanupWithoutInProgressPartFiles() throws Exception { final TestRecoverableWriter recoverableWriter = getRecoverableWriter(path); final Bucket bucketUnderTest = - createBucket(recoverableWriter, path, 0, 0, new PartFileConfig()); + createBucket(recoverableWriter, path, 0, 0, PartFileConfig.builder().build()); final BucketState state = bucketUnderTest.onReceptionOfCheckpoint(0L); assertThat(state, hasNoActiveInProgressFile()); @@ -403,7 +425,7 @@ private static TestRecoverableWriter getRecoverableWriter(Path path) { private Bucket getRestoredBucketWithOnlyInProgressPart(final BaseStubWriter writer) throws IOException { final BucketState stateWithOnlyInProgressFile = new BucketState<>("test", new Path(), 12345L, new NoOpRecoverable(), new HashMap<>()); - return Bucket.restore(writer, 0, 1L, partFileFactory, rollingPolicy, stateWithOnlyInProgressFile, new PartFileConfig()); + return Bucket.restore(writer, 0, 1L, partFileFactory, rollingPolicy, stateWithOnlyInProgressFile, PartFileConfig.builder().build()); } private Bucket getRestoredBucketWithOnlyPendingParts(final BaseStubWriter writer, final int numberOfPendingParts) throws IOException { @@ -412,7 +434,7 @@ private Bucket getRestoredBucketWithOnlyPendingParts(final BaseS final BucketState initStateWithOnlyInProgressFile = new BucketState<>("test", new Path(), 12345L, null, completePartsPerCheckpoint); - return Bucket.restore(writer, 0, 1L, partFileFactory, rollingPolicy, initStateWithOnlyInProgressFile, new PartFileConfig()); + return Bucket.restore(writer, 0, 1L, partFileFactory, rollingPolicy, initStateWithOnlyInProgressFile, PartFileConfig.builder().build()); } private Map> createPendingPartsPerCheckpoint(int noOfCheckpoints) { diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketsTest.java index ac7071d781ac..edbed9b9e9b0 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketsTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketsTest.java @@ -318,7 +318,7 @@ private void testCorrectTimestampPassingInContext(Long timestamp, long watermark new RowWisePartWriter.Factory<>(new SimpleStringEncoder<>()), DefaultRollingPolicy.builder().build(), 2, - new PartFileConfig() + PartFileConfig.builder().build() ); buckets.onElement( @@ -377,7 +377,7 @@ private static Buckets createBuckets( basePath, rollingPolicy, subtaskIdx, - new PartFileConfig()); + PartFileConfig.builder().build()); } private static Buckets createBuckets( @@ -408,7 +408,7 @@ private static Buckets restoreBuckets( subtaskIdx, bucketState, partCounterState, - new PartFileConfig()); + PartFileConfig.builder().build()); } private static Buckets restoreBuckets( 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 bab18aac3b05..f416d8af56ef 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 @@ -101,8 +101,7 @@ public void testCustomBulkWriterWithPartConfig() throws Exception { new TestUtils.TupleToStringBucketer(), new TestBulkWriterFactory(), new DefaultBucketFactoryImpl<>(), - "prefix", - ".ext") + PartFileConfig.builder().withPartPrefix("prefix").withPartSuffix(".ext").build()) ) { testPartFilesWithStringBucketer(testHarness, outDir, ".prefix-0-0.ext.inprogress", ".prefix-0-1.ext.inprogress"); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicyTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicyTest.java index be9db3e8bec3..2fe1943e6e51 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicyTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicyTest.java @@ -204,7 +204,7 @@ private static Buckets createBuckets( new RowWisePartWriter.Factory<>(new SimpleStringEncoder<>()), rollingPolicyToTest, 0, - new PartFileConfig() + PartFileConfig.builder().build() ); } 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 c17a2d6a24ab..8b3ab28edb30 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 @@ -140,8 +140,7 @@ static OneInputStreamOperatorTestHarness, Object> create bucketer, writer, bucketFactory, - PartFileConfig.DEFAULT_PART_PREFIX, - PartFileConfig.DEFAULT_PART_SUFFIX); + PartFileConfig.builder().build()); } static OneInputStreamOperatorTestHarness, Object> createTestSinkWithBulkEncoder( @@ -152,16 +151,14 @@ static OneInputStreamOperatorTestHarness, Object> create final BucketAssigner, String> bucketer, final BulkWriter.Factory> writer, final BucketFactory, String> bucketFactory, - final String partFilePrefix, - final String partFileSuffix) throws Exception { + final PartFileConfig partFileConfig) throws Exception { StreamingFileSink> sink = StreamingFileSink .forBulkFormat(new Path(outDir.toURI()), writer) .withBucketAssigner(bucketer) .withBucketCheckInterval(bucketCheckInterval) .withBucketFactory(bucketFactory) - .withPartFilePrefix(partFilePrefix) - .withPartFileSuffix(partFileSuffix) + .withPartFileConfig(partFileConfig) .build(); return new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink), MAX_PARALLELISM, totalParallelism, taskIdx); From fdc2555c7d417ddd62a700b2ea67b59193ea720e Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 15 Nov 2019 14:44:06 +0100 Subject: [PATCH 645/746] Revert "[FLINK-13850] refactor part file configurations into a single method" This reverts commit 22b2a8856307c310b4c75b32eeed33ba66c0206e. --- docs/dev/connectors/streamfile_sink.md | 14 --- .../api/functions/sink/filesystem/Bucket.java | 24 ++--- .../sink/filesystem/BucketFactory.java | 4 +- .../functions/sink/filesystem/Buckets.java | 10 +- .../filesystem/DefaultBucketFactoryImpl.java | 8 +- .../sink/filesystem/OutputFileConfig.java | 98 ------------------- .../sink/filesystem/PartFileConfig.java | 53 ++++++++++ .../sink/filesystem/StreamingFileSink.java | 58 +++++++---- .../filesystem/BucketAssignerITCases.java | 2 +- .../functions/sink/filesystem/BucketTest.java | 32 +----- .../sink/filesystem/BucketsTest.java | 6 +- .../sink/filesystem/BulkWriterTest.java | 3 +- .../sink/filesystem/RollingPolicyTest.java | 2 +- .../functions/sink/filesystem/TestUtils.java | 9 +- 14 files changed, 131 insertions(+), 192 deletions(-) delete mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/OutputFileConfig.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/PartFileConfig.java diff --git a/docs/dev/connectors/streamfile_sink.md b/docs/dev/connectors/streamfile_sink.md index 378a0f4d63cf..c59e646a3548 100644 --- a/docs/dev/connectors/streamfile_sink.md +++ b/docs/dev/connectors/streamfile_sink.md @@ -133,20 +133,6 @@ New buckets are created as dictated by the bucketing policy, and this doesn't af Old buckets can still receive new records as the bucketing policy is evaluated on a per-record basis. -### Part file configuration - -The filenames of the part files could be defined using `OutputFileConfig`, this configuration contain a part prefix and part suffix, -that will be used with the parallel subtask index of the sink and a rolling counter. -For example for a prefix "prefix" and a suffix ".ext" the file create: - -``` -└── 2019-08-25--12 - ├── prefix-0-0.ext - ├── prefix-0-1.ext.inprogress.bd053eb0-5ecf-4c85-8433-9eff486ac334 - ├── prefix-1-0.ext - └── prefix-1-1.ext.inprogress.bc279efe-b16f-47d8-b828-00ef6e2fbd11 -``` - ## File Formats The `StreamingFileSink` supports both row-wise and bulk encoding formats, such as [Apache Parquet](http://parquet.apache.org). diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java index 7efd2bada644..4a996e7884b9 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java @@ -70,7 +70,7 @@ public class Bucket { private final NavigableMap> pendingPartsPerCheckpoint; - private final OutputFileConfig outputFileConfig; + private final PartFileConfig partFileConfig; private long partCounter; @@ -90,7 +90,7 @@ private Bucket( final long initialPartCounter, final PartFileWriter.PartFileFactory partFileFactory, final RollingPolicy rollingPolicy, - final OutputFileConfig outputFileConfig) { + final PartFileConfig partFileConfig) { this.fsWriter = checkNotNull(fsWriter); this.subtaskIndex = subtaskIndex; this.bucketId = checkNotNull(bucketId); @@ -103,7 +103,7 @@ private Bucket( this.pendingPartsPerCheckpoint = new TreeMap<>(); this.resumablesPerCheckpoint = new TreeMap<>(); - this.outputFileConfig = checkNotNull(outputFileConfig); + this.partFileConfig = checkNotNull(partFileConfig); } /** @@ -116,7 +116,7 @@ private Bucket( final PartFileWriter.PartFileFactory partFileFactory, final RollingPolicy rollingPolicy, final BucketState bucketState, - final OutputFileConfig outputFileConfig) throws IOException { + final PartFileConfig partFileConfig) throws IOException { this( fsWriter, @@ -126,7 +126,7 @@ private Bucket( initialPartCounter, partFileFactory, rollingPolicy, - outputFileConfig); + partFileConfig); restoreInProgressFile(bucketState); commitRecoveredPendingFiles(bucketState); @@ -230,7 +230,7 @@ private void rollPartFile(final long currentTime) throws IOException { } private Path assembleNewPartPath() { - return new Path(bucketPath, outputFileConfig.getPartPrefix() + '-' + subtaskIndex + '-' + partCounter + outputFileConfig.getPartSuffix()); + return new Path(bucketPath, partFileConfig.getPartPrefix() + '-' + subtaskIndex + '-' + partCounter + partFileConfig.getPartSuffix()); } private CommitRecoverable closePartFile() throws IOException { @@ -369,7 +369,7 @@ List getPendingPartsForCurrentCheckpoint() { * @param partFileFactory the {@link PartFileWriter.PartFileFactory} the factory creating part file writers. * @param the type of input elements to the sink. * @param the type of the identifier of the bucket, as returned by the {@link BucketAssigner} - * @param outputFileConfig the part file configuration. + * @param partFileConfig the part file configuration. * @return The new Bucket. */ static Bucket getNew( @@ -380,8 +380,8 @@ static Bucket getNew( final long initialPartCounter, final PartFileWriter.PartFileFactory partFileFactory, final RollingPolicy rollingPolicy, - final OutputFileConfig outputFileConfig) { - return new Bucket<>(fsWriter, subtaskIndex, bucketId, bucketPath, initialPartCounter, partFileFactory, rollingPolicy, outputFileConfig); + final PartFileConfig partFileConfig) { + return new Bucket<>(fsWriter, subtaskIndex, bucketId, bucketPath, initialPartCounter, partFileFactory, rollingPolicy, partFileConfig); } /** @@ -393,7 +393,7 @@ static Bucket getNew( * @param bucketState the initial state of the restored bucket. * @param the type of input elements to the sink. * @param the type of the identifier of the bucket, as returned by the {@link BucketAssigner} - * @param outputFileConfig the part file configuration. + * @param partFileConfig the part file configuration. * @return The restored Bucket. */ static Bucket restore( @@ -403,7 +403,7 @@ static Bucket restore( final PartFileWriter.PartFileFactory partFileFactory, final RollingPolicy rollingPolicy, final BucketState bucketState, - final OutputFileConfig outputFileConfig) throws IOException { - return new Bucket<>(fsWriter, subtaskIndex, initialPartCounter, partFileFactory, rollingPolicy, bucketState, outputFileConfig); + final PartFileConfig partFileConfig) throws IOException { + return new Bucket<>(fsWriter, subtaskIndex, initialPartCounter, partFileFactory, rollingPolicy, bucketState, partFileConfig); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketFactory.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketFactory.java index 260e82c79601..c318ef213bb5 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketFactory.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketFactory.java @@ -39,7 +39,7 @@ Bucket getNewBucket( final long initialPartCounter, final PartFileWriter.PartFileFactory partFileWriterFactory, final RollingPolicy rollingPolicy, - final OutputFileConfig outputFileConfig) throws IOException; + final PartFileConfig partFileConfig) throws IOException; Bucket restoreBucket( final RecoverableWriter fsWriter, @@ -48,5 +48,5 @@ Bucket restoreBucket( final PartFileWriter.PartFileFactory partFileWriterFactory, final RollingPolicy rollingPolicy, final BucketState bucketState, - final OutputFileConfig outputFileConfig) throws IOException; + final PartFileConfig partFileConfig) throws IOException; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java index eb61d18e0a58..fa7a72b7c572 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java @@ -77,7 +77,7 @@ public class Buckets { private final RecoverableWriter fsWriter; - private final OutputFileConfig outputFileConfig; + private final PartFileConfig partFileConfig; // --------------------------- State Related Fields ----------------------------- @@ -99,7 +99,7 @@ public class Buckets { final PartFileWriter.PartFileFactory partFileWriterFactory, final RollingPolicy rollingPolicy, final int subtaskIndex, - final OutputFileConfig outputFileConfig) throws IOException { + final PartFileConfig partFileConfig) throws IOException { this.basePath = Preconditions.checkNotNull(basePath); this.bucketAssigner = Preconditions.checkNotNull(bucketAssigner); @@ -108,7 +108,7 @@ public class Buckets { this.rollingPolicy = Preconditions.checkNotNull(rollingPolicy); this.subtaskIndex = subtaskIndex; - this.outputFileConfig = Preconditions.checkNotNull(outputFileConfig); + this.partFileConfig = Preconditions.checkNotNull(partFileConfig); this.activeBuckets = new HashMap<>(); this.bucketerContext = new Buckets.BucketerContext(); @@ -186,7 +186,7 @@ private void handleRestoredBucketState(final BucketState recoveredStat partFileWriterFactory, rollingPolicy, recoveredState, - outputFileConfig + partFileConfig ); updateActiveBucketId(bucketId, restoredBucket); @@ -293,7 +293,7 @@ private Bucket getOrCreateBucketForBucketId(final BucketID bucketI maxPartCounter, partFileWriterFactory, rollingPolicy, - outputFileConfig); + partFileConfig); activeBuckets.put(bucketId, bucket); } return bucket; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/DefaultBucketFactoryImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/DefaultBucketFactoryImpl.java index 529b93afa8c5..dac2a5af0de1 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/DefaultBucketFactoryImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/DefaultBucketFactoryImpl.java @@ -41,7 +41,7 @@ public Bucket getNewBucket( final long initialPartCounter, final PartFileWriter.PartFileFactory partFileWriterFactory, final RollingPolicy rollingPolicy, - final OutputFileConfig outputFileConfig) { + final PartFileConfig partFileConfig) { return Bucket.getNew( fsWriter, @@ -51,7 +51,7 @@ public Bucket getNewBucket( initialPartCounter, partFileWriterFactory, rollingPolicy, - outputFileConfig); + partFileConfig); } @Override @@ -62,7 +62,7 @@ public Bucket restoreBucket( final PartFileWriter.PartFileFactory partFileWriterFactory, final RollingPolicy rollingPolicy, final BucketState bucketState, - final OutputFileConfig outputFileConfig) throws IOException { + final PartFileConfig partFileConfig) throws IOException { return Bucket.restore( fsWriter, @@ -71,6 +71,6 @@ public Bucket restoreBucket( partFileWriterFactory, rollingPolicy, bucketState, - outputFileConfig); + partFileConfig); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/OutputFileConfig.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/OutputFileConfig.java deleted file mode 100644 index 98b84d178a27..000000000000 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/OutputFileConfig.java +++ /dev/null @@ -1,98 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.streaming.api.functions.sink.filesystem; - -import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.util.Preconditions; - -import java.io.Serializable; - -/** - * Part file name configuration. - * This allow to define a prefix and a suffix to the part file name. - */ -public class OutputFileConfig implements Serializable { - - private final String partPrefix; - - private final String partSuffix; - - /** - * Initiates the {@code PartFileConfig} with values passed as parameters. - * - * @param partPrefix - the beginning of part file name - * @param partSuffix - the ending of part file name - */ - public OutputFileConfig(final String partPrefix, final String partSuffix) { - this.partPrefix = Preconditions.checkNotNull(partPrefix); - this.partSuffix = Preconditions.checkNotNull(partSuffix); - } - - /** - * The prefix for the part name. - */ - String getPartPrefix() { - return partPrefix; - } - - /** - * The suffix for the part name. - */ - String getPartSuffix() { - return partSuffix; - } - - public static OutputFileConfigBuilder builder() { - return new OutputFileConfigBuilder(); - } - - /** - * A builder to create the part file configuration. - */ - @PublicEvolving - public static class OutputFileConfigBuilder { - - private static final String DEFAULT_PART_PREFIX = "part"; - - private static final String DEFAULT_PART_SUFFIX = ""; - - private String partPrefix; - - private String partSuffix; - - private OutputFileConfigBuilder() { - this.partPrefix = DEFAULT_PART_PREFIX; - this.partSuffix = DEFAULT_PART_SUFFIX; - } - - public OutputFileConfigBuilder withPartPrefix(String prefix) { - this.partPrefix = prefix; - return this; - } - - public OutputFileConfigBuilder withPartSuffix(String suffix) { - this.partSuffix = suffix; - return this; - } - - public OutputFileConfig build() { - return new OutputFileConfig(partPrefix, partSuffix); - } - } -} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/PartFileConfig.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/PartFileConfig.java new file mode 100644 index 000000000000..f3eaa59261cb --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/PartFileConfig.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.api.functions.sink.filesystem; + +import org.apache.flink.util.Preconditions; + +/** + * Part file name configuration. + * This allow to define a prefix and a suffix to the part file name. + */ +class PartFileConfig { + + public static final String DEFAULT_PART_PREFIX = "part"; + + public static final String DEFAULT_PART_SUFFIX = ""; + + private final String partPrefix; + + private final String partSuffix; + + PartFileConfig() { + this(DEFAULT_PART_PREFIX, DEFAULT_PART_SUFFIX); + } + + PartFileConfig(final String partPrefix, final String partSuffix) { + this.partPrefix = Preconditions.checkNotNull(partPrefix); + this.partSuffix = Preconditions.checkNotNull(partSuffix); + } + + String getPartPrefix() { + return partPrefix; + } + + String getPartSuffix() { + return partSuffix; + } +} 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 cd4afc2d8404..ed7e9264249a 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 @@ -67,11 +67,9 @@ * {@link StreamingFileSink#forBulkFormat(Path, BulkWriter.Factory)}. * * - *

    The filenames of the part files could be defined using {@link OutputFileConfig}, this configuration contain - * a part prefix and part suffix, that will be used with the parallel subtask index of the sink - * and a rolling counter. For example for a prefix "prefix" and a suffix ".ext" the file create will have a name - * {@code "prefix-1-17.ext"} containing the data from {@code subtask 1} of the sink and is the {@code 17th} bucket - * created by that subtask. + *

    The filenames of the part files contain the part prefix, "part-", the parallel subtask index of the sink + * and a rolling counter. For example the file {@code "part-1-17"} contains the data from + * {@code subtask 1} of the sink and is the {@code 17th} bucket created by that subtask. * Part files roll based on the user-specified {@link RollingPolicy}. By default, a {@link DefaultRollingPolicy} * is used. * @@ -219,10 +217,12 @@ public static class RowFormatBuilder bucketFactory; - private OutputFileConfig outputFileConfig; + private String partFilePrefix; + + private String partFileSuffix; protected RowFormatBuilder(Path basePath, Encoder encoder, BucketAssigner bucketAssigner) { - this(basePath, encoder, bucketAssigner, DefaultRollingPolicy.builder().build(), DEFAULT_BUCKET_CHECK_INTERVAL, new DefaultBucketFactoryImpl<>(), OutputFileConfig.builder().build()); + this(basePath, encoder, bucketAssigner, DefaultRollingPolicy.builder().build(), DEFAULT_BUCKET_CHECK_INTERVAL, new DefaultBucketFactoryImpl<>(), PartFileConfig.DEFAULT_PART_PREFIX, PartFileConfig.DEFAULT_PART_SUFFIX); } protected RowFormatBuilder( @@ -232,14 +232,16 @@ protected RowFormatBuilder( RollingPolicy policy, long bucketCheckInterval, BucketFactory bucketFactory, - OutputFileConfig outputFileConfig) { + String partFilePrefix, + String partFileSuffix) { this.basePath = Preconditions.checkNotNull(basePath); this.encoder = Preconditions.checkNotNull(encoder); this.bucketAssigner = Preconditions.checkNotNull(assigner); this.rollingPolicy = Preconditions.checkNotNull(policy); this.bucketCheckInterval = bucketCheckInterval; this.bucketFactory = Preconditions.checkNotNull(bucketFactory); - this.outputFileConfig = Preconditions.checkNotNull(outputFileConfig); + this.partFilePrefix = Preconditions.checkNotNull(partFilePrefix); + this.partFileSuffix = Preconditions.checkNotNull(partFileSuffix); } public long getBucketCheckInterval() { @@ -261,14 +263,19 @@ public T withRollingPolicy(final RollingPolicy policy) { return self(); } - public T withOutputFileConfig(final OutputFileConfig outputFileConfig) { - this.outputFileConfig = outputFileConfig; + public T withPartFilePrefix(final String partPrefix) { + this.partFilePrefix = partPrefix; + return self(); + } + + 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<>(), outputFileConfig); + return new RowFormatBuilder<>(basePath, encoder, Preconditions.checkNotNull(assigner), Preconditions.checkNotNull(policy), bucketCheckInterval, new DefaultBucketFactoryImpl<>(), partFilePrefix, partFileSuffix); } /** Creates the actual sink. */ @@ -291,7 +298,7 @@ Buckets createBuckets(int subtaskIndex) throws IOException { new RowWisePartWriter.Factory<>(encoder), rollingPolicy, subtaskIndex, - outputFileConfig); + new PartFileConfig(partFilePrefix, partFileSuffix)); } } @@ -313,10 +320,12 @@ public static class BulkFormatBuilder bucketFactory; - private OutputFileConfig outputFileConfig; + private String partFilePrefix; + + private String partFileSuffix; protected BulkFormatBuilder(Path basePath, BulkWriter.Factory writerFactory, BucketAssigner assigner) { - this(basePath, writerFactory, assigner, DEFAULT_BUCKET_CHECK_INTERVAL, new DefaultBucketFactoryImpl<>(), OutputFileConfig.builder().build()); + this(basePath, writerFactory, assigner, DEFAULT_BUCKET_CHECK_INTERVAL, new DefaultBucketFactoryImpl<>(), PartFileConfig.DEFAULT_PART_PREFIX, PartFileConfig.DEFAULT_PART_SUFFIX); } protected BulkFormatBuilder( @@ -325,13 +334,15 @@ protected BulkFormatBuilder( BucketAssigner assigner, long bucketCheckInterval, BucketFactory bucketFactory, - OutputFileConfig outputFileConfig) { + String partFilePrefix, + String partFileSuffix) { this.basePath = Preconditions.checkNotNull(basePath); this.writerFactory = writerFactory; this.bucketAssigner = Preconditions.checkNotNull(assigner); this.bucketCheckInterval = bucketCheckInterval; this.bucketFactory = Preconditions.checkNotNull(bucketFactory); - this.outputFileConfig = Preconditions.checkNotNull(outputFileConfig); + this.partFilePrefix = Preconditions.checkNotNull(partFilePrefix); + this.partFileSuffix = Preconditions.checkNotNull(partFileSuffix); } public long getBucketCheckInterval() { @@ -352,16 +363,21 @@ public T withBucketAssigner(BucketAssigner assigner) { T withBucketFactory(final BucketFactory factory) { this.bucketFactory = Preconditions.checkNotNull(factory); return self(); + } + + public T withPartFilePrefix(final String partPrefix) { + this.partFilePrefix = partPrefix; + return self(); } - public T withOutputFileConfig(final OutputFileConfig outputFileConfig) { - this.outputFileConfig = outputFileConfig; + public T withPartFileSuffix(final String partSuffix) { + this.partFileSuffix = partSuffix; return self(); } 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<>(), outputFileConfig); + return new BulkFormatBuilder<>(basePath, writerFactory, Preconditions.checkNotNull(assigner), bucketCheckInterval, new DefaultBucketFactoryImpl<>(), partFilePrefix, partFileSuffix); } /** Creates the actual sink. */ @@ -378,7 +394,7 @@ Buckets createBuckets(int subtaskIndex) throws IOException { new BulkPartWriter.Factory<>(writerFactory), OnCheckpointRollingPolicy.build(), subtaskIndex, - outputFileConfig); + new PartFileConfig(partFilePrefix, partFileSuffix)); } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketAssignerITCases.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketAssignerITCases.java index 02c126411e83..cc91a335b7fb 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketAssignerITCases.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketAssignerITCases.java @@ -57,7 +57,7 @@ public void testAssembleBucketPath() throws Exception { new RowWisePartWriter.Factory<>(new SimpleStringEncoder<>()), rollingPolicy, 0, - PartFileConfig.builder().build() + new PartFileConfig() ); Bucket bucket = diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java index 2bcb881bfc1d..583bacfc8155 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java @@ -64,7 +64,7 @@ public void shouldNotCleanupResumablesThatArePartOfTheAckedCheckpoint() throws I final TestRecoverableWriter recoverableWriter = getRecoverableWriter(path); final Bucket bucketUnderTest = - createBucket(recoverableWriter, path, 0, 0, PartFileConfig.builder().build()); + createBucket(recoverableWriter, path, 0, 0, new PartFileConfig()); bucketUnderTest.write("test-element", 0L); @@ -82,7 +82,7 @@ public void shouldCleanupOutdatedResumablesOnCheckpointAck() throws IOException final TestRecoverableWriter recoverableWriter = getRecoverableWriter(path); final Bucket bucketUnderTest = - createBucket(recoverableWriter, path, 0, 0, PartFileConfig.builder().build()); + createBucket(recoverableWriter, path, 0, 0, new PartFileConfig()); bucketUnderTest.write("test-element", 0L); @@ -98,28 +98,6 @@ public void shouldCleanupOutdatedResumablesOnCheckpointAck() throws IOException assertThat(recoverableWriter, hasCalledDiscard(2)); // that is for checkpoints 0 and 1 } - @Test - public void shouldCleanupResumableAfterRestoring() throws Exception { - final File outDir = TEMP_FOLDER.newFolder(); - final Path path = new Path(outDir.toURI()); - - final TestRecoverableWriter recoverableWriter = getRecoverableWriter(path); - final Bucket bucketUnderTest = - createBucket(recoverableWriter, path, 0, 0, PartFileConfig.builder().build()); - - bucketUnderTest.write("test-element", 0L); - - final BucketState state = bucketUnderTest.onReceptionOfCheckpoint(0L); - assertThat(state, hasActiveInProgressFile()); - - bucketUnderTest.onSuccessfulCompletionOfCheckpoint(0L); - - final TestRecoverableWriter newRecoverableWriter = getRecoverableWriter(path); - restoreBucket(newRecoverableWriter, 0, 1, state, PartFileConfig.builder().build()); - - assertThat(newRecoverableWriter, hasCalledDiscard(1)); // that is for checkpoints 0 and 1 - } - @Test public void shouldNotCallCleanupWithoutInProgressPartFiles() throws Exception { final File outDir = TEMP_FOLDER.newFolder(); @@ -127,7 +105,7 @@ public void shouldNotCallCleanupWithoutInProgressPartFiles() throws Exception { final TestRecoverableWriter recoverableWriter = getRecoverableWriter(path); final Bucket bucketUnderTest = - createBucket(recoverableWriter, path, 0, 0, PartFileConfig.builder().build()); + createBucket(recoverableWriter, path, 0, 0, new PartFileConfig()); final BucketState state = bucketUnderTest.onReceptionOfCheckpoint(0L); assertThat(state, hasNoActiveInProgressFile()); @@ -425,7 +403,7 @@ private static TestRecoverableWriter getRecoverableWriter(Path path) { private Bucket getRestoredBucketWithOnlyInProgressPart(final BaseStubWriter writer) throws IOException { final BucketState stateWithOnlyInProgressFile = new BucketState<>("test", new Path(), 12345L, new NoOpRecoverable(), new HashMap<>()); - return Bucket.restore(writer, 0, 1L, partFileFactory, rollingPolicy, stateWithOnlyInProgressFile, PartFileConfig.builder().build()); + return Bucket.restore(writer, 0, 1L, partFileFactory, rollingPolicy, stateWithOnlyInProgressFile, new PartFileConfig()); } private Bucket getRestoredBucketWithOnlyPendingParts(final BaseStubWriter writer, final int numberOfPendingParts) throws IOException { @@ -434,7 +412,7 @@ private Bucket getRestoredBucketWithOnlyPendingParts(final BaseS final BucketState initStateWithOnlyInProgressFile = new BucketState<>("test", new Path(), 12345L, null, completePartsPerCheckpoint); - return Bucket.restore(writer, 0, 1L, partFileFactory, rollingPolicy, initStateWithOnlyInProgressFile, PartFileConfig.builder().build()); + return Bucket.restore(writer, 0, 1L, partFileFactory, rollingPolicy, initStateWithOnlyInProgressFile, new PartFileConfig()); } private Map> createPendingPartsPerCheckpoint(int noOfCheckpoints) { diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketsTest.java index edbed9b9e9b0..ac7071d781ac 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketsTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketsTest.java @@ -318,7 +318,7 @@ private void testCorrectTimestampPassingInContext(Long timestamp, long watermark new RowWisePartWriter.Factory<>(new SimpleStringEncoder<>()), DefaultRollingPolicy.builder().build(), 2, - PartFileConfig.builder().build() + new PartFileConfig() ); buckets.onElement( @@ -377,7 +377,7 @@ private static Buckets createBuckets( basePath, rollingPolicy, subtaskIdx, - PartFileConfig.builder().build()); + new PartFileConfig()); } private static Buckets createBuckets( @@ -408,7 +408,7 @@ private static Buckets restoreBuckets( subtaskIdx, bucketState, partCounterState, - PartFileConfig.builder().build()); + new PartFileConfig()); } private static Buckets restoreBuckets( 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 f416d8af56ef..bab18aac3b05 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 @@ -101,7 +101,8 @@ public void testCustomBulkWriterWithPartConfig() throws Exception { new TestUtils.TupleToStringBucketer(), new TestBulkWriterFactory(), new DefaultBucketFactoryImpl<>(), - PartFileConfig.builder().withPartPrefix("prefix").withPartSuffix(".ext").build()) + "prefix", + ".ext") ) { testPartFilesWithStringBucketer(testHarness, outDir, ".prefix-0-0.ext.inprogress", ".prefix-0-1.ext.inprogress"); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicyTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicyTest.java index 2fe1943e6e51..be9db3e8bec3 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicyTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicyTest.java @@ -204,7 +204,7 @@ private static Buckets createBuckets( new RowWisePartWriter.Factory<>(new SimpleStringEncoder<>()), rollingPolicyToTest, 0, - PartFileConfig.builder().build() + new PartFileConfig() ); } 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 8b3ab28edb30..c17a2d6a24ab 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 @@ -140,7 +140,8 @@ static OneInputStreamOperatorTestHarness, Object> create bucketer, writer, bucketFactory, - PartFileConfig.builder().build()); + PartFileConfig.DEFAULT_PART_PREFIX, + PartFileConfig.DEFAULT_PART_SUFFIX); } static OneInputStreamOperatorTestHarness, Object> createTestSinkWithBulkEncoder( @@ -151,14 +152,16 @@ static OneInputStreamOperatorTestHarness, Object> create final BucketAssigner, String> bucketer, final BulkWriter.Factory> writer, final BucketFactory, String> bucketFactory, - final PartFileConfig partFileConfig) throws Exception { + final String partFilePrefix, + final String partFileSuffix) throws Exception { StreamingFileSink> sink = StreamingFileSink .forBulkFormat(new Path(outDir.toURI()), writer) .withBucketAssigner(bucketer) .withBucketCheckInterval(bucketCheckInterval) .withBucketFactory(bucketFactory) - .withPartFileConfig(partFileConfig) + .withPartFilePrefix(partFilePrefix) + .withPartFileSuffix(partFileSuffix) .build(); return new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink), MAX_PARALLELISM, totalParallelism, taskIdx); From fa8d11b53fdac1dc6d40e5a15a79ad5dee6a8d26 Mon Sep 17 00:00:00 2001 From: tison Date: Fri, 15 Nov 2019 09:03:33 +0800 Subject: [PATCH 646/746] [FLINK-14699][type] Move ClosureCleaner to flink-core --- .../main/java/org/apache/flink/api/java/ClosureCleaner.java | 0 .../java/org/apache/flink/api/java}/ClosureCleanerTest.java | 3 +-- 2 files changed, 1 insertion(+), 2 deletions(-) rename {flink-java => flink-core}/src/main/java/org/apache/flink/api/java/ClosureCleaner.java (100%) rename {flink-java/src/test/java/org/apache/flink/api/java/functions => flink-core/src/test/java/org/apache/flink/api/java}/ClosureCleanerTest.java (99%) diff --git a/flink-java/src/main/java/org/apache/flink/api/java/ClosureCleaner.java b/flink-core/src/main/java/org/apache/flink/api/java/ClosureCleaner.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/ClosureCleaner.java rename to flink-core/src/main/java/org/apache/flink/api/java/ClosureCleaner.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/functions/ClosureCleanerTest.java b/flink-core/src/test/java/org/apache/flink/api/java/ClosureCleanerTest.java similarity index 99% rename from flink-java/src/test/java/org/apache/flink/api/java/functions/ClosureCleanerTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/ClosureCleanerTest.java index 7b93e8af9599..db1f812a7971 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/functions/ClosureCleanerTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/java/ClosureCleanerTest.java @@ -16,12 +16,11 @@ * limitations under the License. */ -package org.apache.flink.api.java.functions; +package org.apache.flink.api.java; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.java.ClosureCleaner; import org.apache.flink.api.java.tuple.Tuple1; import org.junit.Assert; From 1f5c5b78bd69e198f09a6bc222c501f783e24012 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Thu, 14 Nov 2019 11:13:30 +0100 Subject: [PATCH 647/746] [FLINK-14780][runtime] Register restart metrics only when start scheduling Defer registering restart metrics so that we avoid leaking an instance of the scheduler before the object is fully created. This closes #10192. --- .../flink/runtime/scheduler/SchedulerBase.java | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) 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 95f378e9288a..09854ca37953 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 @@ -140,8 +140,9 @@ public abstract class SchedulerBase implements SchedulerNG { private final BlobWriter blobWriter; - private final Time slotRequestTimeout; + private final JobManagerJobMetricGroup jobManagerJobMetricGroup; + private final Time slotRequestTimeout; private ComponentMainThreadExecutor mainThreadExecutor = new ComponentMainThreadExecutor.DummyComponentMainThreadExecutor( "SchedulerBase is not initialized with proper main thread executor. " + @@ -188,6 +189,7 @@ public SchedulerBase( log.info("Using restart strategy {} for {} ({}).", this.restartStrategy, jobGraph.getName(), jobGraph.getJobID()); this.blobWriter = checkNotNull(blobWriter); + this.jobManagerJobMetricGroup = checkNotNull(jobManagerJobMetricGroup); this.slotRequestTimeout = checkNotNull(slotRequestTimeout); this.executionGraph = createAndRestoreExecutionGraph(jobManagerJobMetricGroup, checkNotNull(shuffleMaster), checkNotNull(partitionTracker)); @@ -195,9 +197,6 @@ public SchedulerBase( this.failoverTopology = executionGraph.getFailoverTopology(); this.inputsLocationsRetriever = new ExecutionGraphToInputsLocationsRetrieverAdapter(executionGraph); - - jobManagerJobMetricGroup.gauge(MetricNames.NUM_RESTARTS, this::getNumberOfRestarts); - jobManagerJobMetricGroup.gauge(MetricNames.FULL_RESTARTS, this::getNumberOfRestarts); } private ExecutionGraph createAndRestoreExecutionGraph( @@ -396,9 +395,15 @@ public void registerJobStatusListener(final JobStatusListener jobStatusListener) @Override public final void startScheduling() { mainThreadExecutor.assertRunningInMainThread(); + registerJobMetrics(); startSchedulingInternal(); } + private void registerJobMetrics() { + jobManagerJobMetricGroup.gauge(MetricNames.NUM_RESTARTS, this::getNumberOfRestarts); + jobManagerJobMetricGroup.gauge(MetricNames.FULL_RESTARTS, this::getNumberOfRestarts); + } + protected abstract void startSchedulingInternal(); @Override From 71142816113a4f1d2160b73cdaed71c4bbfb575f Mon Sep 17 00:00:00 2001 From: "bowen.li" Date: Thu, 14 Nov 2019 13:21:41 -0800 Subject: [PATCH 648/746] [FLINK-14782][table] CoreModule#getFunctionDefinition should return empty optional when function does not exist CoreModule#getFunctionDefinition should return empty optional when function does not exist. this closes #14782. --- .../apache/flink/table/module/CoreModule.java | 6 ++-- .../flink/table/module/CoreModuleTest.java | 33 +++++++++++++++++++ 2 files changed, 36 insertions(+), 3 deletions(-) create mode 100644 flink-table/flink-table-common/src/test/java/org/apache/flink/table/module/CoreModuleTest.java diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/CoreModule.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/CoreModule.java index a4b371a9b2b2..213da5df74f1 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/CoreModule.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/module/CoreModule.java @@ -23,6 +23,7 @@ import java.util.Optional; import java.util.Set; +import java.util.function.Function; import java.util.stream.Collectors; /** @@ -44,10 +45,9 @@ public Set listFunctions() { @Override public Optional getFunctionDefinition(String name) { - return Optional.ofNullable( - BuiltInFunctionDefinitions.getDefinitions().stream() + return BuiltInFunctionDefinitions.getDefinitions().stream() .filter(f -> f.getName().equalsIgnoreCase(name)) .findFirst() - .get()); + .map(Function.identity()); } } diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/module/CoreModuleTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/module/CoreModuleTest.java new file mode 100644 index 000000000000..d9c79fa3a054 --- /dev/null +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/module/CoreModuleTest.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.module; + +import org.junit.Test; + +import static org.junit.Assert.assertFalse; + +/** + * Test for {@link CoreModule}. + */ +public class CoreModuleTest { + @Test + public void testGetNonExistFunction() { + assertFalse(CoreModule.INSTANCE.getFunctionDefinition("nonexist").isPresent()); + } +} From 3a2efb4472375169f39dc1000dd1ea0fff74fa19 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 7 Nov 2019 17:25:17 +0100 Subject: [PATCH 649/746] [hotfix][kinesis] Eliminate compliler warnings and apply simple inspection-based automated cleanups --- .../kinesis/internals/KinesisDataFetcher.java | 32 +++++++++---------- .../kinesis/proxy/KinesisProxy.java | 10 +----- .../kinesis/util/KinesisConfigUtil.java | 7 ++-- 3 files changed, 19 insertions(+), 30 deletions(-) diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java index 80b724b18536..afd61385151a 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java @@ -76,9 +76,9 @@ * and runs a single fetcher throughout the subtask's lifetime. The fetcher accomplishes the following: *

      *
    • 1. continuously poll Kinesis to discover shards that the subtask should subscribe to. The subscribed subset - * of shards, including future new shards, is non-overlapping across subtasks (no two subtasks will be - * subscribed to the same shard) and determinate across subtask restores (the subtask will always subscribe - * to the same subset of shards even after restoring)
    • + * of shards, including future new shards, is non-overlapping across subtasks (no two subtasks will be + * subscribed to the same shard) and determinate across subtask restores (the subtask will always subscribe + * to the same subset of shards even after restoring) *
    • 2. decide where in each discovered shard should the fetcher start subscribing to
    • *
    • 3. subscribe to shards by creating a single thread for each shard
    • *
    @@ -87,6 +87,7 @@ * and 2) last processed sequence numbers of each subscribed shard. Since operations on the second state will be performed * by multiple threads, these operations should only be done using the handler methods provided in this class. */ +@SuppressWarnings("unchecked") @Internal public class KinesisDataFetcher { @@ -195,8 +196,8 @@ public class KinesisDataFetcher { private final AssignerWithPeriodicWatermarks periodicWatermarkAssigner; private final WatermarkTracker watermarkTracker; - private final transient RecordEmitter recordEmitter; - private transient boolean isIdle; + private final RecordEmitter recordEmitter; + private boolean isIdle; /** * The watermark related state for each shard consumer. Entries in this map will be created when shards @@ -280,8 +281,8 @@ private class SyncKinesisRecordEmitter extends AsyncKinesisRecordEmitter { @Override public RecordQueue> getQueue(int producerIndex) { - return queues.computeIfAbsent(producerIndex, (key) -> { - return new RecordQueue>() { + return queues.computeIfAbsent(producerIndex, (key) -> + new RecordQueue>() { @Override public void put(RecordWrapper record) { emit(record, this); @@ -296,8 +297,6 @@ public int getSize() { public RecordWrapper peek() { return null; } - - }; }); } } @@ -621,6 +620,7 @@ public void shutdownFetcher() { } /** After calling {@link KinesisDataFetcher#shutdownFetcher()}, this can be called to await the fetcher shutdown. */ + @SuppressWarnings("StatementWithEmptyBody") public void awaitTermination() throws InterruptedException { while (!shardConsumersExecutor.awaitTermination(1, TimeUnit.MINUTES)) { // Keep waiting. @@ -757,8 +757,6 @@ protected void emitRecordAndUpdateState(T record, long recordTimestamp, int shar * *

    Responsible for tracking per shard watermarks and emit timestamps extracted from * the record, when a watermark assigner was configured. - * - * @param rw */ private void emitRecordAndUpdateState(RecordWrapper rw) { synchronized (checkpointLock) { @@ -952,22 +950,22 @@ public void onProcessingTime(long timestamp) { /** Timer task to update shared watermark state. */ private class WatermarkSyncCallback implements ProcessingTimeCallback { + private static final long LOG_INTERVAL_MILLIS = 60_000; + private final ProcessingTimeService timerService; private final long interval; - private final MetricGroup shardMetricsGroup; private long lastGlobalWatermark = Long.MIN_VALUE; private long propagatedLocalWatermark = Long.MIN_VALUE; - private long logIntervalMillis = 60_000; private int stalledWatermarkIntervalCount = 0; private long lastLogged; WatermarkSyncCallback(ProcessingTimeService timerService, long interval) { this.timerService = checkNotNull(timerService); this.interval = interval; - this.shardMetricsGroup = consumerMetricGroup.addGroup("subtaskId", + MetricGroup shardMetricsGroup = consumerMetricGroup.addGroup("subtaskId", String.valueOf(indexOfThisConsumerSubtask)); - this.shardMetricsGroup.gauge("localWatermark", () -> nextWatermark); - this.shardMetricsGroup.gauge("globalWatermark", () -> lastGlobalWatermark); + shardMetricsGroup.gauge("localWatermark", () -> nextWatermark); + shardMetricsGroup.gauge("globalWatermark", () -> lastGlobalWatermark); } public void start() { @@ -987,7 +985,7 @@ public void onProcessingTime(long timestamp) { LOG.info("WatermarkSyncCallback subtask: {} is idle", indexOfThisConsumerSubtask); } - if (timestamp - lastLogged > logIntervalMillis) { + if (timestamp - lastLogged > LOG_INTERVAL_MILLIS) { lastLogged = System.currentTimeMillis(); LOG.info("WatermarkSyncCallback subtask: {} local watermark: {}" + ", global watermark: {}, delta: {} timeouts: {}, emitter: {}", diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java index 2c21f11b236d..fe4ae32be3ab 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java @@ -52,7 +52,6 @@ import java.util.ArrayList; import java.util.Date; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Properties; @@ -213,8 +212,6 @@ protected KinesisProxy(Properties configProps) { /** * Create the Kinesis client, using the provided configuration properties and default {@link ClientConfiguration}. * Derived classes can override this method to customize the client configuration. - * @param configProps - * @return */ protected AmazonKinesis createKinesisClient(Properties configProps) { @@ -483,12 +480,7 @@ private ListShardsResult listShards(String streamName, @Nullable String startSha // https://github.com/lyft/kinesalite/pull/4 if (startShardId != null && listShardsResults != null) { List shards = listShardsResults.getShards(); - Iterator shardItr = shards.iterator(); - while (shardItr.hasNext()) { - if (StreamShardHandle.compareShardIds(shardItr.next().getShardId(), startShardId) <= 0) { - shardItr.remove(); - } - } + shards.removeIf(shard -> StreamShardHandle.compareShardIds(shard.getShardId(), startShardId) <= 0); } return listShardsResults; diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java index b277696bf1a5..5493774eecad 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java @@ -28,8 +28,6 @@ import com.amazonaws.regions.Regions; import com.amazonaws.services.kinesis.producer.KinesisProducerConfiguration; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.text.ParseException; import java.text.SimpleDateFormat; @@ -77,8 +75,6 @@ public class KinesisConfigUtil { /** Default values for ThreadPoolSize. **/ protected static final int DEFAULT_THREAD_POOL_SIZE = 10; - private static final Logger LOG = LoggerFactory.getLogger(KinesisConfigUtil.class); - /** * Validate configuration properties for {@link FlinkKinesisConsumer}. */ @@ -87,6 +83,7 @@ public static void validateConsumerConfiguration(Properties config) { validateAwsConfiguration(config); + //noinspection SimplifiableBooleanExpression - the current logic expression is actually easier to understand if (!(config.containsKey(AWSConfigConstants.AWS_REGION) ^ config.containsKey(ConsumerConfigConstants.AWS_ENDPOINT))) { // per validation in AwsClientBuilder throw new IllegalArgumentException(String.format("For FlinkKinesisConsumer either AWS region ('%s') or AWS endpoint ('%s') must be set in the config.", @@ -177,6 +174,7 @@ public static void validateConsumerConfiguration(Properties config) { * Replace deprecated configuration properties for {@link FlinkKinesisProducer}. * This should be remove along with deprecated keys */ + @SuppressWarnings("deprecation") public static Properties replaceDeprecatedProducerKeys(Properties configProps) { // Replace deprecated key if (configProps.containsKey(ProducerConfigConstants.COLLECTION_MAX_COUNT)) { @@ -207,6 +205,7 @@ public static Properties replaceDeprecatedProducerKeys(Properties configProps) { * @param configProps original config properties. * @return backfilled config properties. */ + @SuppressWarnings("UnusedReturnValue") public static Properties backfillConsumerKeys(Properties configProps) { HashMap oldKeyToNewKeys = new HashMap<>(); oldKeyToNewKeys.put(ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_BASE, ConsumerConfigConstants.LIST_SHARDS_BACKOFF_BASE); From 2b0a8ceeb131c938d2e41dfee66099bfa5f366ae Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Mon, 11 Nov 2019 10:04:49 +0100 Subject: [PATCH 650/746] [FLINK-14635][e2e tests] Use non-relocated imports for AWS SDK Kinesis classes in tests - Moving pubsub client testutil to main project's to test jar --- .../flink-connector-kinesis/pom.xml | 5 +++ .../testutils}/KinesisPubsubClient.java | 35 ++++++++++--------- .../flink-streaming-kinesis-test/pom.xml | 8 +++++ .../kinesis/test/KinesisExampleTest.java | 15 ++------ 4 files changed, 33 insertions(+), 30 deletions(-) rename {flink-end-to-end-tests/flink-streaming-kinesis-test/src/main/java/org/apache/flink/streaming/kinesis/test => flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils}/KinesisPubsubClient.java (78%) diff --git a/flink-connectors/flink-connector-kinesis/pom.xml b/flink-connectors/flink-connector-kinesis/pom.xml index 415fbb89ebaf..652a5106dd98 100644 --- a/flink-connectors/flink-connector-kinesis/pom.xml +++ b/flink-connectors/flink-connector-kinesis/pom.xml @@ -169,6 +169,11 @@ under the License. test-jar + + + **/org/apache/flink/streaming/connectors/kinesis/testutils/** + + diff --git a/flink-end-to-end-tests/flink-streaming-kinesis-test/src/main/java/org/apache/flink/streaming/kinesis/test/KinesisPubsubClient.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/KinesisPubsubClient.java similarity index 78% rename from flink-end-to-end-tests/flink-streaming-kinesis-test/src/main/java/org/apache/flink/streaming/kinesis/test/KinesisPubsubClient.java rename to flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/KinesisPubsubClient.java index 486b565a8b39..30eeb2cb59bc 100644 --- a/flink-end-to-end-tests/flink-streaming-kinesis-test/src/main/java/org/apache/flink/streaming/kinesis/test/KinesisPubsubClient.java +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/KinesisPubsubClient.java @@ -15,26 +15,26 @@ * limitations under the License. */ -package org.apache.flink.streaming.kinesis.test; +package org.apache.flink.streaming.connectors.kinesis.testutils; import org.apache.flink.api.common.time.Deadline; -import org.apache.flink.kinesis.shaded.com.amazonaws.AmazonClientException; -import org.apache.flink.kinesis.shaded.com.amazonaws.auth.AWSCredentialsProvider; -import org.apache.flink.kinesis.shaded.com.amazonaws.auth.EnvironmentVariableCredentialsProvider; -import org.apache.flink.kinesis.shaded.com.amazonaws.client.builder.AwsClientBuilder; -import org.apache.flink.kinesis.shaded.com.amazonaws.services.kinesis.AmazonKinesis; -import org.apache.flink.kinesis.shaded.com.amazonaws.services.kinesis.AmazonKinesisClientBuilder; -import org.apache.flink.kinesis.shaded.com.amazonaws.services.kinesis.model.GetRecordsResult; -import org.apache.flink.kinesis.shaded.com.amazonaws.services.kinesis.model.PutRecordRequest; -import org.apache.flink.kinesis.shaded.com.amazonaws.services.kinesis.model.PutRecordResult; -import org.apache.flink.kinesis.shaded.com.amazonaws.services.kinesis.model.Record; -import org.apache.flink.kinesis.shaded.com.amazonaws.services.kinesis.model.ResourceNotFoundException; import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants; import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle; import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult; import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy; import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface; +import com.amazonaws.AmazonClientException; +import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.EnvironmentVariableCredentialsProvider; +import com.amazonaws.client.builder.AwsClientBuilder; +import com.amazonaws.services.kinesis.AmazonKinesis; +import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder; +import com.amazonaws.services.kinesis.model.GetRecordsResult; +import com.amazonaws.services.kinesis.model.PutRecordRequest; +import com.amazonaws.services.kinesis.model.PutRecordResult; +import com.amazonaws.services.kinesis.model.Record; +import com.amazonaws.services.kinesis.model.ResourceNotFoundException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,18 +46,21 @@ import java.util.Map; import java.util.Properties; -class KinesisPubsubClient implements KinesisExampleTest.PubsubClient { +/** + * Simple client to publish and retrieve messages, using the AWS Kinesis SDK and the + * Flink Kinesis Connectos classes. + */ +public class KinesisPubsubClient { private static final Logger LOG = LoggerFactory.getLogger(KinesisPubsubClient.class); private final AmazonKinesis kinesisClient; private final Properties properties; - KinesisPubsubClient(Properties properties) { + public KinesisPubsubClient(Properties properties) { this.kinesisClient = createClientWithCredentials(properties); this.properties = properties; } - @Override public void createTopic(String stream, int shards, Properties props) throws Exception { try { kinesisClient.describeStream(stream); @@ -83,7 +86,6 @@ public void createTopic(String stream, int shards, Properties props) throws Exce } } - @Override public void sendMessage(String topic, String msg) { PutRecordRequest putRecordRequest = new PutRecordRequest(); putRecordRequest.setStreamName(topic); @@ -93,7 +95,6 @@ public void sendMessage(String topic, String msg) { LOG.info("added record: {}", putRecordResult.getSequenceNumber()); } - @Override public List readAllMessages(String streamName) throws Exception { KinesisProxyInterface kinesisProxy = KinesisProxy.create(properties); Map streamNamesWithLastSeenShardIds = new HashMap<>(); diff --git a/flink-end-to-end-tests/flink-streaming-kinesis-test/pom.xml b/flink-end-to-end-tests/flink-streaming-kinesis-test/pom.xml index b1b66e909df4..88b93541cbf4 100644 --- a/flink-end-to-end-tests/flink-streaming-kinesis-test/pom.xml +++ b/flink-end-to-end-tests/flink-streaming-kinesis-test/pom.xml @@ -51,6 +51,14 @@ under the License. ${project.version} + + org.apache.flink + flink-connector-kinesis_${scala.binary.version} + ${project.version} + test-jar + compile + + junit junit diff --git a/flink-end-to-end-tests/flink-streaming-kinesis-test/src/main/java/org/apache/flink/streaming/kinesis/test/KinesisExampleTest.java b/flink-end-to-end-tests/flink-streaming-kinesis-test/src/main/java/org/apache/flink/streaming/kinesis/test/KinesisExampleTest.java index 1a6d6d74808d..6f8f20a90b81 100644 --- a/flink-end-to-end-tests/flink-streaming-kinesis-test/src/main/java/org/apache/flink/streaming/kinesis/test/KinesisExampleTest.java +++ b/flink-end-to-end-tests/flink-streaming-kinesis-test/src/main/java/org/apache/flink/streaming/kinesis/test/KinesisExampleTest.java @@ -19,6 +19,7 @@ import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.streaming.connectors.kinesis.testutils.KinesisPubsubClient; import org.junit.Assert; import org.slf4j.Logger; @@ -26,7 +27,6 @@ import java.time.Duration; import java.util.List; -import java.util.Properties; import java.util.concurrent.atomic.AtomicReference; /** @@ -35,17 +35,6 @@ public class KinesisExampleTest { private static final Logger LOG = LoggerFactory.getLogger(KinesisExampleTest.class); - /** - * Interface to the pubsub system for this test. - */ - interface PubsubClient { - void createTopic(String topic, int partitions, Properties props) throws Exception; - - void sendMessage(String topic, String msg); - - List readAllMessages(String streamName) throws Exception; - } - public static void main(String[] args) throws Exception { LOG.info("System properties: {}", System.getProperties()); final ParameterTool parameterTool = ParameterTool.fromArgs(args); @@ -53,7 +42,7 @@ public static void main(String[] args) throws Exception { String inputStream = parameterTool.getRequired("input-stream"); String outputStream = parameterTool.getRequired("output-stream"); - PubsubClient pubsub = new KinesisPubsubClient(parameterTool.getProperties()); + KinesisPubsubClient pubsub = new KinesisPubsubClient(parameterTool.getProperties()); pubsub.createTopic(inputStream, 2, parameterTool.getProperties()); pubsub.createTopic(outputStream, 2, parameterTool.getProperties()); From 0d181e9ed3864e2b81ae645771f866acac283b87 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 29 Jan 2019 13:27:13 +0100 Subject: [PATCH 651/746] [FLINK-14663[tests] Support starting individual Job-/TaskManagers --- .../org/apache/flink/tests/util/FlinkDistribution.java | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java index 8236f4b9a779..b79361e9f99d 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java @@ -146,6 +146,16 @@ public void afterTestFailure() { afterTestSuccess(); } + public void startJobManager() throws IOException { + LOG.info("Starting Flink JobManager."); + AutoClosableProcess.runBlocking(bin.resolve("jobmanager.sh").toAbsolutePath().toString(), "start"); + } + + public void startTaskManager() throws IOException { + LOG.info("Starting Flink TaskManager."); + AutoClosableProcess.runBlocking(bin.resolve("taskmanager.sh").toAbsolutePath().toString(), "start"); + } + public void startFlinkCluster() throws IOException { LOG.info("Starting Flink cluster."); AutoClosableProcess.runBlocking(bin.resolve("start-cluster.sh").toAbsolutePath().toString()); From 44538e10dca767124e175496a58d9dc7c8710eec Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 29 Jan 2019 13:30:52 +0100 Subject: [PATCH 652/746] [FLINK-11463][tests] Support job submissions --- .../flink/tests/util/FlinkDistribution.java | 50 +++++++ .../flink/tests/util/flink/JobSubmission.java | 128 ++++++++++++++++++ 2 files changed, 178 insertions(+) create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JobSubmission.java diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java index b79361e9f99d..a04662ecafa1 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java @@ -18,9 +18,11 @@ package org.apache.flink.tests.util; +import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.configuration.UnmodifiableConfiguration; +import org.apache.flink.tests.util.flink.JobSubmission; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.ExternalResource; @@ -200,6 +202,54 @@ public void stopFlinkCluster() throws IOException { AutoClosableProcess.runBlocking(bin.resolve("stop-cluster.sh").toAbsolutePath().toString()); } + public JobID submitJob(final JobSubmission jobSubmission) throws IOException { + final List commands = new ArrayList<>(4); + commands.add(bin.resolve("flink").toString()); + commands.add("run"); + if (jobSubmission.isDetached()) { + commands.add("-d"); + } + if (jobSubmission.getParallelism() > 0) { + commands.add("-p"); + commands.add(String.valueOf(jobSubmission.getParallelism())); + } + commands.add(jobSubmission.getJar().toAbsolutePath().toString()); + commands.addAll(jobSubmission.getArguments()); + + LOG.info("Running {}.", commands.stream().collect(Collectors.joining(" "))); + + try (AutoClosableProcess flink = new AutoClosableProcess(new ProcessBuilder() + .command(commands) + .start())) { + + final Pattern pattern = jobSubmission.isDetached() + ? Pattern.compile("Job has been submitted with JobID (.*)") + : Pattern.compile("Job with JobID (.*) has finished."); + + if (jobSubmission.isDetached()) { + try { + flink.getProcess().waitFor(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + + try (BufferedReader bufferedReader = new BufferedReader(new InputStreamReader(flink.getProcess().getInputStream(), StandardCharsets.UTF_8))) { + final Optional jobId = bufferedReader.lines() + .peek(LOG::info) + .map(pattern::matcher) + .filter(Matcher::matches) + .map(matcher -> matcher.group(1)) + .findAny(); + if (!jobId.isPresent()) { + throw new IOException("Could not determine Job ID."); + } else { + return JobID.fromHexString(jobId.get()); + } + } + } + } + public void copyOptJarsToLib(String jarNamePrefix) throws FileNotFoundException, IOException { final Optional reporterJarOptional; try (Stream logFiles = Files.walk(opt)) { diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JobSubmission.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JobSubmission.java new file mode 100644 index 000000000000..61759c5e32ff --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JobSubmission.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.flink; + +import org.apache.flink.util.Preconditions; + +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** + * Programmatic definition of a job-submission. + */ +public class JobSubmission { + + private final Path jar; + private final int parallelism; + private final boolean detached; + private final List arguments; + + JobSubmission(final Path jar, final int parallelism, final boolean detached, final List arguments) { + this.jar = jar; + this.parallelism = parallelism; + this.detached = detached; + this.arguments = Collections.unmodifiableList(arguments); + } + + public List getArguments() { + return arguments; + } + + public boolean isDetached() { + return detached; + } + + public int getParallelism() { + return parallelism; + } + + public Path getJar() { + return jar; + } + + /** + * Builder for the {@link JobSubmission}. + */ + public static class JobSubmissionBuilder { + private final Path jar; + private int parallelism = 0; + private final List arguments = new ArrayList<>(2); + private boolean detached = false; + + public JobSubmissionBuilder(final Path jar) { + Preconditions.checkNotNull(jar); + Preconditions.checkArgument(jar.isAbsolute(), "Jar path must be absolute."); + this.jar = jar; + } + + /** + * Sets the parallelism for the job. + * + * @param parallelism parallelism for the job + * @return the modified builder + */ + public JobSubmissionBuilder setParallelism(final int parallelism) { + this.parallelism = parallelism; + return this; + } + + /** + * Sets whether the job should be submitted in a detached manner. + * + * @param detached whether to submit the job in a detached manner + * @return the modified builder + */ + public JobSubmissionBuilder setDetached(final boolean detached) { + this.detached = detached; + return this; + } + + /** + * Adds a program argument. + * + * @param argument argument argument + * @return the modified builder + */ + public JobSubmissionBuilder addArgument(final String argument) { + Preconditions.checkNotNull(argument); + this.arguments.add(argument); + return this; + } + + /** + * Convenience method for providing key-value program arguments. Invoking this method is equivalent to invoking + * {@link #addArgument(String)} twice. + * + * @param key argument key + * @param value argument value + * @return the modified builder + */ + public JobSubmissionBuilder addArgument(final String key, final String value) { + addArgument(key); + addArgument(value); + return this; + } + + public JobSubmission build() { + return new JobSubmission(jar, parallelism, detached, arguments); + } + } +} From 3f92bd428d5f5a367ed7070dc7a663ac125dbaff Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 29 Jan 2019 13:34:47 +0100 Subject: [PATCH 653/746] [FLINK-11463][tests] Add utilities - FactoryUtils for simplified ServiceLoader access - TestUtils for simplified resource jar access - ParameterProperty for defining parameters - OperatingSystemRestriction to restrict tests to operating systems --- .../apache/flink/tests/util/TestUtils.java | 114 ++++++++++++++++++ .../OperatingSystemRestriction.java | 62 ++++++++++ .../util/parameters/ParameterProperty.java | 64 ++++++++++ .../flink/tests/util/util/FactoryUtils.java | 66 ++++++++++ 4 files changed, 306 insertions(+) create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/TestUtils.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/activation/OperatingSystemRestriction.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/parameters/ParameterProperty.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/util/FactoryUtils.java diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/TestUtils.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/TestUtils.java new file mode 100644 index 000000000000..db289fdde338 --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/TestUtils.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util; + +import org.apache.flink.util.Preconditions; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.nio.file.FileAlreadyExistsException; +import java.nio.file.FileVisitResult; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.SimpleFileVisitor; +import java.nio.file.StandardCopyOption; +import java.nio.file.attribute.BasicFileAttributes; +import java.util.List; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * General test utilities. + */ +public enum TestUtils { + ; + + /** + * Searches for a jar matching the given regex in the given directory. This method is primarily intended to be used + * for the initialization of static {@link Path} fields for jars that reside in the modules {@code target} directory. + * + * @param jarNameRegex regex pattern to match against + * @return Path pointing to the matching jar + * @throws RuntimeException if none or multiple jars could be found + */ + public static Path getResourceJar(final String jarNameRegex) { + String moduleDirProp = System.getProperty("moduleDir"); + Preconditions.checkNotNull(moduleDirProp, "The moduleDir property was not set, You can set it when running maven via -DmoduleDir="); + + try (Stream dependencyJars = Files.walk(Paths.get(moduleDirProp))) { + final List matchingJars = dependencyJars + .filter(jar -> Pattern.compile(jarNameRegex).matcher(jar.toAbsolutePath().toString()).find()) + .collect(Collectors.toList()); + switch (matchingJars.size()) { + case 0: + throw new RuntimeException( + new FileNotFoundException( + String.format("No jar could be found that matches the pattern %s.", jarNameRegex) + ) + ); + case 1: + return matchingJars.get(0); + default: + throw new RuntimeException( + new IOException( + String.format("Multiple jars were found matching the pattern %s. Matches=%s", jarNameRegex, matchingJars) + ) + ); + } + } catch (final IOException ioe) { + throw new RuntimeException("Could not search for resource jars.", ioe); + } + } + + /** + * Copy all the files and sub-directories under source directory to destination directory recursively. + * + * @param source directory or file path to copy from. + * @param destination directory or file path to copy to. + * @return Path of the destination directory. + * @throws IOException if any IO error happen. + */ + public static Path copyDirectory(final Path source, final Path destination) throws IOException { + Files.walkFileTree(source, new SimpleFileVisitor() { + @Override + public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes ignored) + throws IOException { + final Path targetRir = destination.resolve(source.relativize(dir)); + try { + Files.copy(dir, targetRir, StandardCopyOption.COPY_ATTRIBUTES); + } catch (FileAlreadyExistsException e) { + if (!Files.isDirectory(targetRir)) { + throw e; + } + } + return FileVisitResult.CONTINUE; + } + + @Override + public FileVisitResult visitFile(Path file, BasicFileAttributes ignored) throws IOException { + Files.copy(file, destination.resolve(source.relativize(file)), StandardCopyOption.COPY_ATTRIBUTES); + return FileVisitResult.CONTINUE; + } + }); + + return destination; + } +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/activation/OperatingSystemRestriction.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/activation/OperatingSystemRestriction.java new file mode 100644 index 000000000000..e9bc2e12b75b --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/activation/OperatingSystemRestriction.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.activation; + +import org.apache.flink.util.OperatingSystem; + +import org.junit.Assume; +import org.junit.AssumptionViolatedException; + +import java.util.Arrays; +import java.util.EnumSet; + +/** + * Utility for tests/resources to restrict test execution to certain operating systems. + * + *

    Do not call these methods in a static initializer block as this marks the test as failed. + */ +public enum OperatingSystemRestriction { + ; + + /** + * Restricts the execution to the given set of operating systems. + * + * @param reason reason for the restriction + * @param operatingSystems allowed operating systems + * @throws AssumptionViolatedException if this method is called on a forbidden operating system + */ + public static void restrictTo(final String reason, final OperatingSystem... operatingSystems) throws AssumptionViolatedException { + final EnumSet allowed = EnumSet.copyOf(Arrays.asList(operatingSystems)); + Assume.assumeTrue(reason, allowed.contains(OperatingSystem.getCurrentOperatingSystem())); + } + + /** + * Forbids the execution on the given set of operating systems. + * + * @param reason reason for the restriction + * @param forbiddenSystems forbidden operating systems + * @throws AssumptionViolatedException if this method is called on a forbidden operating system + */ + public static void forbid(final String reason, final OperatingSystem... forbiddenSystems) throws AssumptionViolatedException { + final OperatingSystem os = OperatingSystem.getCurrentOperatingSystem(); + for (final OperatingSystem forbiddenSystem : forbiddenSystems) { + Assume.assumeTrue(reason, os != forbiddenSystem); + } + } +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/parameters/ParameterProperty.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/parameters/ParameterProperty.java new file mode 100644 index 000000000000..a64e6fd9afa8 --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/parameters/ParameterProperty.java @@ -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.tests.util.parameters; + +import java.util.Optional; +import java.util.function.Function; + +/** + * System-property based parameters for tests and resources. + */ +public class ParameterProperty { + + private final String propertyName; + private final Function converter; + + public ParameterProperty(final String propertyName, final Function converter) { + this.propertyName = propertyName; + this.converter = converter; + } + + public String getPropertyName() { + return propertyName; + } + + /** + * Retrieves the value of this property. + * + * @return Optional containing the value of this property + */ + public Optional get() { + final String value = System.getProperty(propertyName); + return value == null + ? Optional.empty() + : Optional.of(converter.apply(value)); + } + + /** + * Retrieves the value of this property, or the given default if no value was set. + * + * @return the value of this property, or the given default if no value was set + */ + public V get(final V defaultValue) { + final String value = System.getProperty(propertyName); + return value == null + ? defaultValue + : converter.apply(value); + } +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/util/FactoryUtils.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/util/FactoryUtils.java new file mode 100644 index 000000000000..079443c5bd77 --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/util/FactoryUtils.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.tests.util.util; + +import java.util.List; +import java.util.Optional; +import java.util.ServiceLoader; +import java.util.function.Function; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +/** + * Utilities for factories. + */ +public enum FactoryUtils { + ; + + /** + * Loads all factories for the given class using the {@link ServiceLoader} and attempts to create an instance. + * + * @param factoryInterface factory interface + * @param factoryInvoker factory invoker + * @param defaultProvider default factory provider + * @param resource type + * @param factory type + * @throws RuntimeException if no or multiple resources could be instantiated + * @return created instance + */ + public static R loadAndInvokeFactory(final Class factoryInterface, final Function> factoryInvoker, final Supplier defaultProvider) { + final ServiceLoader factories = ServiceLoader.load(factoryInterface); + + final List resources = StreamSupport.stream(factories.spliterator(), false) + .map(factoryInvoker) + .filter(Optional::isPresent) + .map(Optional::get) + .collect(Collectors.toList()); + + if (resources.size() == 1) { + return resources.get(0); + } + + if (resources.isEmpty()) { + return factoryInvoker.apply(defaultProvider.get()) + .orElseThrow(() -> new RuntimeException("Could not instantiate instance using default factory.")); + } + + throw new RuntimeException("Multiple instances were created: " + resources); + } +} From 04aabae4acdedc558a8fc6ed5ba0d2d5a03d84ce Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 18 Jul 2019 11:21:13 +0200 Subject: [PATCH 654/746] [FLINK-11463][tests] Always run in a fresh copy of the distribution --- .../flink/tests/util/FlinkDistribution.java | 64 ++++++++----------- 1 file changed, 26 insertions(+), 38 deletions(-) diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java index a04662ecafa1..b31ecf9a948b 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java @@ -34,6 +34,7 @@ import okhttp3.Response; import org.apache.commons.io.FileUtils; import org.junit.Assert; +import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,11 +47,11 @@ import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; -import java.nio.file.StandardCopyOption; import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.Optional; +import java.util.UUID; import java.util.function.Function; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -66,18 +67,16 @@ public final class FlinkDistribution implements ExternalResource { private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private static final Path FLINK_CONF_YAML = Paths.get("flink-conf.yaml"); - private static final Path FLINK_CONF_YAML_BACKUP = Paths.get("flink-conf.yaml.bak"); - - private final List filesToDelete = new ArrayList<>(4); - private final Optional logBackupDir; - private final Path opt; - private final Path lib; - private final Path conf; - private final Path log; - private final Path bin; + private final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private final Path originalFlinkDir; + private Path opt; + private Path lib; + private Path conf; + private Path log; + private Path bin; private Configuration defaultConfig; @@ -90,21 +89,25 @@ public FlinkDistribution() { logBackupDir = backupDirProperty == null ? Optional.empty() : Optional.of(Paths.get(backupDirProperty)); - final Path flinkDir = Paths.get(distDirProperty); + originalFlinkDir = Paths.get(distDirProperty); + } + + @Override + public void before() throws IOException { + temporaryFolder.create(); + + final Path flinkDir = temporaryFolder.newFolder().toPath(); + + LOG.info("Copying distribution to {}.", flinkDir); + TestUtils.copyDirectory(originalFlinkDir, flinkDir); + bin = flinkDir.resolve("bin"); opt = flinkDir.resolve("opt"); lib = flinkDir.resolve("lib"); conf = flinkDir.resolve("conf"); log = flinkDir.resolve("log"); - } - @Override - public void before() throws IOException { defaultConfig = new UnmodifiableConfiguration(GlobalConfiguration.loadConfiguration(conf.toAbsolutePath().toString())); - final Path originalConfig = conf.resolve(FLINK_CONF_YAML); - final Path backupConfig = conf.resolve(FLINK_CONF_YAML_BACKUP); - Files.copy(originalConfig, backupConfig); - filesToDelete.add(new AutoClosablePath(backupConfig)); } @Override @@ -115,31 +118,17 @@ public void afterTestSuccess() { LOG.error("Failure while shutting down Flink cluster.", e); } - final Path originalConfig = conf.resolve(FLINK_CONF_YAML); - final Path backupConfig = conf.resolve(FLINK_CONF_YAML_BACKUP); - - try { - Files.move(backupConfig, originalConfig, StandardCopyOption.REPLACE_EXISTING); - } catch (IOException e) { - LOG.error("Failed to restore flink-conf.yaml", e); - } - - for (AutoCloseable fileToDelete : filesToDelete) { - try { - fileToDelete.close(); - } catch (Exception e) { - LOG.error("Failure while cleaning up file.", e); - } - } + temporaryFolder.delete(); } @Override public void afterTestFailure() { logBackupDir.ifPresent(backupLocation -> { - LOG.info("Backing up logs to {}.", backupLocation); + final UUID id = UUID.randomUUID(); + LOG.info("Backing up logs to {}/{}.", backupLocation, id); try { Files.createDirectories(backupLocation); - FileUtils.copyDirectory(log.toFile(), backupLocation.toFile()); + FileUtils.copyDirectory(log.toFile(), backupLocation.resolve(id.toString()).toFile()); } catch (IOException e) { LOG.warn("An error occurred while backing up logs.", e); } @@ -261,7 +250,6 @@ public void copyOptJarsToLib(String jarNamePrefix) throws FileNotFoundException, final Path optReporterJar = reporterJarOptional.get(); final Path libReporterJar = lib.resolve(optReporterJar.getFileName()); Files.copy(optReporterJar, libReporterJar); - filesToDelete.add(new AutoClosablePath(libReporterJar)); } else { throw new FileNotFoundException("No jar could be found matching the pattern " + jarNamePrefix + "."); } From 4d033ce1b0a011235673e1cee4d0f02019cf80eb Mon Sep 17 00:00:00 2001 From: huzheng Date: Thu, 14 Nov 2019 14:07:04 +0100 Subject: [PATCH 655/746] [hotfix][tests] Remove optional usage --- .../flink/tests/util/FlinkDistribution.java | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java index b31ecf9a948b..914101ba7d0d 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java @@ -67,7 +67,7 @@ public final class FlinkDistribution implements ExternalResource { private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private final Optional logBackupDir; + private final Path logBackupDir; private final TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -86,9 +86,7 @@ public FlinkDistribution() { Assert.fail("The distDir property was not set. You can set it when running maven via -DdistDir= ."); } final String backupDirProperty = System.getProperty("logBackupDir"); - logBackupDir = backupDirProperty == null - ? Optional.empty() - : Optional.of(Paths.get(backupDirProperty)); + logBackupDir = backupDirProperty == null ? null : Paths.get(backupDirProperty); originalFlinkDir = Paths.get(distDirProperty); } @@ -123,16 +121,16 @@ public void afterTestSuccess() { @Override public void afterTestFailure() { - logBackupDir.ifPresent(backupLocation -> { + if (logBackupDir != null) { final UUID id = UUID.randomUUID(); - LOG.info("Backing up logs to {}/{}.", backupLocation, id); + LOG.info("Backing up logs to {}/{}.", logBackupDir, id); try { - Files.createDirectories(backupLocation); - FileUtils.copyDirectory(log.toFile(), backupLocation.resolve(id.toString()).toFile()); + Files.createDirectories(logBackupDir); + FileUtils.copyDirectory(log.toFile(), logBackupDir.resolve(id.toString()).toFile()); } catch (IOException e) { LOG.warn("An error occurred while backing up logs.", e); } - }); + } afterTestSuccess(); } From 8fb5f1b985c8b2216467f287ed8aaee51fb7491f Mon Sep 17 00:00:00 2001 From: huzheng Date: Thu, 14 Nov 2019 14:38:17 +0100 Subject: [PATCH 656/746] [hotfix][tests] Use TestUtils#copyDirectory --- .../java/org/apache/flink/tests/util/FlinkDistribution.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java index 914101ba7d0d..cab6d3669e9f 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/FlinkDistribution.java @@ -32,7 +32,6 @@ import okhttp3.OkHttpClient; import okhttp3.Request; import okhttp3.Response; -import org.apache.commons.io.FileUtils; import org.junit.Assert; import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; @@ -126,7 +125,7 @@ public void afterTestFailure() { LOG.info("Backing up logs to {}/{}.", logBackupDir, id); try { Files.createDirectories(logBackupDir); - FileUtils.copyDirectory(log.toFile(), logBackupDir.resolve(id.toString()).toFile()); + TestUtils.copyDirectory(log, logBackupDir.resolve(id.toString())); } catch (IOException e) { LOG.warn("An error occurred while backing up logs.", e); } From 751184d2b210d8ff60e26ff1fdebe6f4ff4d82c3 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 29 Jan 2019 13:32:11 +0100 Subject: [PATCH 657/746] [FLINK-11465][tests] Add FlinkResource --- .../tests/util/flink/ClusterController.java | 38 +++++ .../flink/tests/util/flink/FlinkResource.java | 66 ++++++++ .../util/flink/FlinkResourceFactory.java | 36 +++++ .../flink/tests/util/flink/JobController.java | 25 +++ .../flink/LocalStandaloneFlinkResource.java | 153 ++++++++++++++++++ .../LocalStandaloneFlinkResourceFactory.java | 37 +++++ 6 files changed, 355 insertions(+) create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/ClusterController.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResource.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResourceFactory.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JobController.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResource.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResourceFactory.java diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/ClusterController.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/ClusterController.java new file mode 100644 index 000000000000..2c187aa612e1 --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/ClusterController.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.flink; + +import org.apache.flink.util.AutoCloseableAsync; + +import java.io.IOException; + +/** + * Controller for interacting with a cluster. + */ +public interface ClusterController extends AutoCloseableAsync { + + /** + * Submits the given job to the cluster. + * + * @param job job to submit + * @return JobController for the submitted job + * @throws IOException + */ + JobController submitJob(JobSubmission job) throws IOException; +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResource.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResource.java new file mode 100644 index 000000000000..0d4f317f2ead --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResource.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.tests.util.flink; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.tests.util.util.FactoryUtils; +import org.apache.flink.util.ExternalResource; + +import java.io.IOException; + +/** + * Generic interface for interacting with Flink. + */ +public interface FlinkResource extends ExternalResource { + + /** + * Adds the given configuration to the existing configuration of this resource. Entries in the existing configuration + * will be overwritten. + * + * @param config config to add + * @throws IOException + */ + void addConfiguration(Configuration config) throws IOException; + + /** + * Starts a cluster. + * + *

    The exact constellation of the cluster is undefined. + * + *

    In the case of per-job clusters this method may not start any Flink processes, deferring this to + * {@link ClusterController#submitJob(JobSubmission)}. + * + * @return controller for interacting with the cluster + * @throws IOException + * @param numTaskManagers number of task managers + */ + ClusterController startCluster(int numTaskManagers) throws IOException; + + /** + * Returns the configured FlinkResource implementation, or a {@link LocalStandaloneFlinkResource} if none is configured. + * + * @return configured FlinkResource, or {@link LocalStandaloneFlinkResource} is none is configured + */ + static FlinkResource get() { + return FactoryUtils.loadAndInvokeFactory( + FlinkResourceFactory.class, + FlinkResourceFactory::create, + LocalStandaloneFlinkResourceFactory::new); + } +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResourceFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResourceFactory.java new file mode 100644 index 000000000000..ce89be71040a --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkResourceFactory.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.flink; + +import java.util.Optional; + +/** + * A factory for {@link FlinkResource} implementations. + */ +@FunctionalInterface +public interface FlinkResourceFactory { + + /** + * Returns a {@link FlinkResource} instance. If the instance could not be instantiated (for example, because a + * mandatory parameter was missing), then an empty {@link Optional} should be returned. + * + * @return FlinkResource instance, or an empty Optional if the instance could not be instantiated + */ + Optional create(); +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JobController.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JobController.java new file mode 100644 index 000000000000..9febd9884222 --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/JobController.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.flink; + +/** + * Controller for interacting with a job. + */ +public interface JobController { +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResource.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResource.java new file mode 100644 index 000000000000..e3a217176982 --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResource.java @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.flink; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.queryablestate.FutureUtils; +import org.apache.flink.runtime.concurrent.Executors; +import org.apache.flink.runtime.rest.RestClient; +import org.apache.flink.runtime.rest.RestClientConfiguration; +import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagersHeaders; +import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagersInfo; +import org.apache.flink.tests.util.FlinkDistribution; +import org.apache.flink.util.ConfigurationException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +/** + * Flink resource that start local standalone clusters. + */ +public class LocalStandaloneFlinkResource implements FlinkResource { + + private static final Logger LOG = LoggerFactory.getLogger(LocalStandaloneFlinkResource.class); + + private final FlinkDistribution distribution = new FlinkDistribution(); + + @Override + public void before() throws Exception { + distribution.before(); + } + + @Override + public void afterTestSuccess() { + distribution.afterTestSuccess(); + } + + @Override + public void afterTestFailure() { + distribution.afterTestFailure(); + } + + @Override + public void addConfiguration(final Configuration config) throws IOException { + distribution.appendConfiguration(config); + } + + @Override + public ClusterController startCluster(int numTaskManagers) throws IOException { + distribution.startJobManager(); + for (int x = 0; x < numTaskManagers; x++) { + distribution.startTaskManager(); + } + + try (final RestClient restClient = new RestClient(RestClientConfiguration.fromConfiguration(new Configuration()), Executors.directExecutor())) { + for (int retryAttempt = 0; retryAttempt < 30; retryAttempt++) { + final CompletableFuture localhost = restClient.sendRequest( + "localhost", + 8081, + TaskManagersHeaders.getInstance(), + EmptyMessageParameters.getInstance(), + EmptyRequestBody.getInstance()); + + try { + final TaskManagersInfo taskManagersInfo = localhost.get(1, TimeUnit.SECONDS); + + final int numRunningTaskManagers = taskManagersInfo.getTaskManagerInfos().size(); + if (numRunningTaskManagers == numTaskManagers) { + return new StandaloneClusterController(distribution); + } else { + LOG.info("Waiting for task managers to come up. {}/{} are currently running.", numRunningTaskManagers, numTaskManagers); + } + } catch (InterruptedException e) { + LOG.info("Waiting for dispatcher REST endpoint to come up..."); + Thread.currentThread().interrupt(); + } catch (TimeoutException | ExecutionException e) { + // ExecutionExceptions may occur if leader election is still going on + LOG.info("Waiting for dispatcher REST endpoint to come up..."); + } + + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } catch (ConfigurationException e) { + throw new RuntimeException("Could not create RestClient.", e); + } catch (Exception e) { + throw new RuntimeException(e); + } + + throw new RuntimeException("Cluster did not start in expected time-frame."); + } + + private static class StandaloneClusterController implements ClusterController { + + private final FlinkDistribution distribution; + + StandaloneClusterController(FlinkDistribution distribution) { + this.distribution = distribution; + } + + @Override + public JobController submitJob(JobSubmission job) throws IOException { + final JobID run = distribution.submitJob(job); + + return new StandaloneJobController(run); + } + + @Override + public CompletableFuture closeAsync() { + try { + distribution.stopFlinkCluster(); + return CompletableFuture.completedFuture(null); + } catch (IOException e) { + return FutureUtils.getFailedFuture(e); + } + } + } + + private static class StandaloneJobController implements JobController { + private final JobID jobId; + + StandaloneJobController(JobID jobId) { + this.jobId = jobId; + } + } +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResourceFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResourceFactory.java new file mode 100644 index 000000000000..af998121af7e --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/LocalStandaloneFlinkResourceFactory.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.tests.util.flink; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Optional; + +/** + * A {@link FlinkResourceFactory} for the {@link LocalStandaloneFlinkResource}. + */ +public final class LocalStandaloneFlinkResourceFactory implements FlinkResourceFactory { + private static final Logger LOG = LoggerFactory.getLogger(LocalStandaloneFlinkResourceFactory.class); + + @Override + public Optional create() { + LOG.info("Created {}.", LocalStandaloneFlinkResource.class.getSimpleName()); + return Optional.of(new LocalStandaloneFlinkResource()); + } +} From f0be1623fdc5fb45f8ef27ee9985842ea27da400 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 29 Jan 2019 13:31:53 +0100 Subject: [PATCH 658/746] [FLINK-11464][tests] Add DownloadCache --- .../util/cache/AbstractDownloadCache.java | 152 ++++++++++++++++++ .../flink/tests/util/cache/DownloadCache.java | 56 +++++++ .../util/cache/DownloadCacheFactory.java | 36 +++++ .../flink/tests/util/cache/LolCache.java | 70 ++++++++ .../tests/util/cache/LolCacheFactory.java | 45 ++++++ .../util/cache/PersistingDownloadCache.java | 81 ++++++++++ .../cache/PersistingDownloadCacheFactory.java | 53 ++++++ .../tests/util/cache/TravisDownloadCache.java | 78 +++++++++ .../cache/TravisDownloadCacheFactory.java | 60 +++++++ ...link.tests.util.cache.DownloadCacheFactory | 17 ++ 10 files changed, 648 insertions(+) create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/AbstractDownloadCache.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/DownloadCache.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/DownloadCacheFactory.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/LolCache.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/LolCacheFactory.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/PersistingDownloadCache.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/PersistingDownloadCacheFactory.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/TravisDownloadCache.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/TravisDownloadCacheFactory.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/resources/META-INF/services/org.apache.flink.tests.util.cache.DownloadCacheFactory diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/AbstractDownloadCache.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/AbstractDownloadCache.java new file mode 100644 index 000000000000..7d5e913630ff --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/AbstractDownloadCache.java @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.cache; + +import org.apache.flink.tests.util.AutoClosableProcess; +import org.apache.flink.tests.util.CommandLineWrapper; +import org.apache.flink.tests.util.TestUtils; + +import org.apache.commons.io.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.time.Duration; +import java.util.Iterator; +import java.util.Optional; +import java.util.regex.Matcher; +import java.util.stream.Stream; + +/** + * Base-class for {@link DownloadCache} implementations. This class handles the download and caching of files and + * provides hooks for encoding/decoding a time-to-live into the file name. + */ +abstract class AbstractDownloadCache implements DownloadCache { + + protected final Logger log = LoggerFactory.getLogger(getClass()); + + private final Path tmpDir; + private final Path downloadsDir; + private final Path cacheFilesDir; + + AbstractDownloadCache(final Path tmpDir) { + this.tmpDir = tmpDir; + this.downloadsDir = tmpDir.resolve("downloads"); + this.cacheFilesDir = tmpDir.resolve("cachefiles"); + } + + @Override + public void before() throws IOException { + Files.createDirectories(tmpDir); + Files.createDirectories(downloadsDir); + Files.createDirectories(cacheFilesDir); + + try (Stream cacheFiles = Files.list(cacheFilesDir)) { + final Iterator iterator = cacheFiles.iterator(); + while (iterator.hasNext()) { + final Path cacheFile = iterator.next(); + final String cacheFileName = cacheFile.getFileName().toString(); + + final Matcher matcher = createCacheFileMatcher(cacheFileName); + + if (matcher.matches()) { + if (exceedsTimeToLive(matcher)) { + log.info("Invalidating cache entry {}.", regenerateOriginalFileName(matcher)); + Files.delete(cacheFile); + } + } + } + } + } + + @Override + public void afterTestSuccess() { + } + + abstract Matcher createCacheFileMatcher(String cacheFileName); + + abstract String generateCacheFileName(String url, String fileName); + + abstract String regenerateOriginalFileName(Matcher matcher); + + abstract boolean exceedsTimeToLive(Matcher matcher); + + abstract boolean matchesCachedFile(Matcher matcher, String url); + + @Override + public Path getOrDownload(final String url, final Path targetDir) throws IOException { + final Optional cachedFile = getCachedFile(url); + + final Path cacheFile; + if (cachedFile.isPresent()) { + log.info("Using cached version of {}.", url); + cacheFile = cachedFile.get(); + } else { + final Path scopedDownloadDir = downloadsDir.resolve(String.valueOf(url.hashCode())); + Files.createDirectories(scopedDownloadDir); + log.info("Downloading {}.", url); + AutoClosableProcess + .create( + CommandLineWrapper.wget(url) + .targetDir(scopedDownloadDir) + .build()) + .runBlocking(Duration.ofMinutes(2)); + + final Path download; + try (Stream files = Files.list(scopedDownloadDir)) { + final Optional any = files.findAny(); + download = any.orElseThrow(() -> new IOException("Failed to download " + url + '.')); + } + + final String cacheFileName = generateCacheFileName(url, download.getFileName().toString()); + + cacheFile = cacheFilesDir.resolve(cacheFileName); + if (Files.isDirectory(download)) { + FileUtils.moveDirectory(download.toFile(), cacheFile.toFile()); + } else { + Files.move(download, cacheFile); + } + } + + final String cacheFileName = cacheFile.getFileName().toString(); + + final Matcher matcher = createCacheFileMatcher(cacheFileName); + if (!matcher.matches()) { + // this indicates an implementation error or corrupted cache + throw new RuntimeException("Cache file matcher did not accept file retrieved from cache."); + } + final String originalFileName = regenerateOriginalFileName(matcher); + + return TestUtils.copyDirectory(cacheFile, targetDir.resolve(originalFileName)); + } + + private Optional getCachedFile(final String url) throws IOException { + try (Stream cacheFiles = Files.list(cacheFilesDir)) { + return cacheFiles + .filter(cacheFile -> { + final String cacheFileName = cacheFile.getFileName().toString(); + final Matcher matcher = createCacheFileMatcher(cacheFileName); + return matcher.matches() && matchesCachedFile(matcher, url); + }) + .findAny(); + } + } +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/DownloadCache.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/DownloadCache.java new file mode 100644 index 000000000000..bac6fb4dc309 --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/DownloadCache.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.cache; + +import org.apache.flink.tests.util.util.FactoryUtils; +import org.apache.flink.util.ExternalResource; + +import java.io.IOException; +import java.nio.file.Path; + +/** + * A {@link DownloadCache} allows tests to download a files and/or directories and optionally caches them. + * + *

    Whether, how, and for how long files are cached is implementation-dependent. + */ +public interface DownloadCache extends ExternalResource { + + /** + * Returns either a cached or newly downloaded version of the given file. The returned file path is guaranteed to be + * located in the given target directory. + * + * @param url File/directory to download + * @param targetDir directory to place file in + * @return downloaded or cached file + * @throws IOException if any IO operation fails + */ + Path getOrDownload(String url, Path targetDir) throws IOException; + + /** + * Returns the configured DownloadCache implementation, or a {@link LolCache} if none is configured. + * + * @return configured DownloadCache, or {@link LolCache} is none is configured + */ + static DownloadCache get() { + return FactoryUtils.loadAndInvokeFactory( + DownloadCacheFactory.class, + DownloadCacheFactory::create, + LolCacheFactory::new); + } +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/DownloadCacheFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/DownloadCacheFactory.java new file mode 100644 index 000000000000..82ff48bdb1c9 --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/DownloadCacheFactory.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.cache; + +import java.util.Optional; + +/** + * A factory for {@link DownloadCache} implementations. + */ +@FunctionalInterface +public interface DownloadCacheFactory { + + /** + * Returns a {@link DownloadCache} instance. If the instance could not be instantiated (for example, because a + * mandatory parameter was missing), then an empty {@link Optional} should be returned. + * + * @return DownloadCache instance, or an empty Optional if the instance could not be instantiated + */ + Optional create(); +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/LolCache.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/LolCache.java new file mode 100644 index 000000000000..90aad7dab271 --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/LolCache.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.cache; + +import org.junit.rules.TemporaryFolder; + +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * {@link DownloadCache} implementation that does not cache anything. + * + * @see LolCacheFactory + */ +public final class LolCache extends AbstractDownloadCache { + + private static final Pattern CACHE_FILE_NAME_PATTERN = Pattern.compile(".*"); + private final TemporaryFolder folder; + + public LolCache(TemporaryFolder folder) { + super(folder.getRoot().toPath()); + this.folder = folder; + } + + @Override + public void afterTestSuccess() { + folder.delete(); + } + + @Override + Matcher createCacheFileMatcher(String cacheFileName) { + return CACHE_FILE_NAME_PATTERN.matcher(cacheFileName); + } + + @Override + String generateCacheFileName(String url, String fileName) { + return fileName; + } + + @Override + String regenerateOriginalFileName(Matcher matcher) { + return matcher.group(0); + } + + @Override + boolean exceedsTimeToLive(Matcher matcher) { + return true; + } + + @Override + boolean matchesCachedFile(Matcher matcher, String url) { + return false; + } +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/LolCacheFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/LolCacheFactory.java new file mode 100644 index 000000000000..68e4973c539e --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/LolCacheFactory.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.tests.util.cache; + +import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Optional; + +/** + * A {@link DownloadCacheFactory} for the {@link LolCache}. + */ +public final class LolCacheFactory implements DownloadCacheFactory { + private static final Logger LOG = LoggerFactory.getLogger(LolCacheFactory.class); + + @Override + public Optional create() { + final TemporaryFolder folder = new TemporaryFolder(); + try { + folder.create(); + } catch (IOException e) { + throw new RuntimeException("Could not initialize temporary directory.", e); + } + LOG.info("Created {}.", LolCache.class.getSimpleName()); + return Optional.of(new LolCache(folder)); + } +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/PersistingDownloadCache.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/PersistingDownloadCache.java new file mode 100644 index 000000000000..545631332561 --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/PersistingDownloadCache.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.tests.util.cache; + +import java.nio.file.Path; +import java.time.LocalDate; +import java.time.Period; +import java.time.format.DateTimeFormatter; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * {@link DownloadCache} implementation that caches downloaded files in a configured directory. Cached files that are + * older than the configured time-to-live {@link Period} will be removed. + * + * @see PersistingDownloadCacheFactory + * @see PersistingDownloadCacheFactory#TMP_DIR + * @see PersistingDownloadCacheFactory#TIME_TO_LIVE + */ +public final class PersistingDownloadCache extends AbstractDownloadCache { + + private static final DateTimeFormatter DATE_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd"); + private static final String CACHE_FILE_NAME_DELIMITER = "__"; + private static final Pattern CACHE_FILE_NAME_PATTERN = + Pattern.compile("(?.*)" + CACHE_FILE_NAME_DELIMITER + "(?.*)" + CACHE_FILE_NAME_DELIMITER + "(?.*)"); + + private final Period ttl; + + public PersistingDownloadCache(final Path path, final Period ttl) { + super(path); + this.ttl = ttl; + } + + @Override + Matcher createCacheFileMatcher(final String cacheFileName) { + return CACHE_FILE_NAME_PATTERN.matcher(cacheFileName); + } + + @Override + String generateCacheFileName(final String url, final String fileName) { + final String hash = String.valueOf(url.hashCode()); + final String datePrefix = LocalDate.now().format(DATE_FORMATTER); + + return hash + CACHE_FILE_NAME_DELIMITER + datePrefix + CACHE_FILE_NAME_DELIMITER + fileName; + } + + @Override + String regenerateOriginalFileName(final Matcher matcher) { + return matcher.group("name"); + } + + @Override + boolean exceedsTimeToLive(final Matcher matcher) { + final LocalDate cacheFileDate = LocalDate.parse(matcher.group("date"), DATE_FORMATTER); + + return ttl != Period.ZERO && Period.between(cacheFileDate, LocalDate.now()).getDays() > ttl.getDays(); + } + + @Override + boolean matchesCachedFile(final Matcher matcher, final String url) { + final String hash = matcher.group("hash"); + + return url.hashCode() == Integer.parseInt(hash); + } +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/PersistingDownloadCacheFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/PersistingDownloadCacheFactory.java new file mode 100644 index 000000000000..27013f2fabab --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/PersistingDownloadCacheFactory.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.tests.util.cache; + +import org.apache.flink.tests.util.parameters.ParameterProperty; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.file.Path; +import java.nio.file.Paths; +import java.time.Period; +import java.util.Optional; + +/** + * A {@link DownloadCacheFactory} for the {@link PersistingDownloadCache}. + */ +public final class PersistingDownloadCacheFactory implements DownloadCacheFactory { + private static final Logger LOG = LoggerFactory.getLogger(PersistingDownloadCacheFactory.class); + + private static final ParameterProperty TMP_DIR = new ParameterProperty<>("cache-dir", value -> Paths.get(value)); + private static final ParameterProperty TIME_TO_LIVE = new ParameterProperty<>("cache-ttl", Period::parse); + + private static final Period TIME_TO_LIVE_DEFAULT = Period.ZERO; + + @Override + public Optional create() { + final Optional tmpDir = TMP_DIR.get(); + final Period timeToLive = TIME_TO_LIVE.get(TIME_TO_LIVE_DEFAULT); + if (!tmpDir.isPresent()) { + LOG.debug("Not loading {} because {} was not set.", PersistingDownloadCache.class, TMP_DIR.getPropertyName()); + return Optional.empty(); + } + LOG.info("Created {}.", PersistingDownloadCache.class.getSimpleName()); + return Optional.of(new PersistingDownloadCache(tmpDir.get(), timeToLive)); + } +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/TravisDownloadCache.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/TravisDownloadCache.java new file mode 100644 index 000000000000..772f3b23218f --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/TravisDownloadCache.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.tests.util.cache; + +import java.nio.file.Path; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Travis-specific {@link DownloadCache} implementation that caches downloaded files in a configured directory. Cached + * files that are older than the configured number of builds will be removed. + * + * @see TravisDownloadCacheFactory + * @see TravisDownloadCacheFactory#TMP_DIR + * @see TravisDownloadCacheFactory#BUILDS_TO_LIVE + */ +public final class TravisDownloadCache extends AbstractDownloadCache { + + private static final String CACHE_FILE_NAME_DELIMITER = "__"; + private static final Pattern CACHE_FILE_NAME_PATTERN = + Pattern.compile("(?.*)" + CACHE_FILE_NAME_DELIMITER + "(?.*)" + CACHE_FILE_NAME_DELIMITER + "(?.*)"); + + private final int ttl; + private final int buildNumber; + + public TravisDownloadCache(final Path path, final int ttl, final int buildNumber) { + super(path); + this.ttl = ttl; + this.buildNumber = buildNumber; + } + + @Override + Matcher createCacheFileMatcher(final String cacheFileName) { + return CACHE_FILE_NAME_PATTERN.matcher(cacheFileName); + } + + @Override + String generateCacheFileName(final String url, final String fileName) { + final String hash = String.valueOf(url.hashCode()); + + return hash + CACHE_FILE_NAME_DELIMITER + buildNumber + CACHE_FILE_NAME_DELIMITER + fileName; + } + + @Override + String regenerateOriginalFileName(final Matcher matcher) { + return matcher.group("name"); + } + + @Override + boolean exceedsTimeToLive(final Matcher matcher) { + int cachedBuildNumber = Integer.parseInt(matcher.group("build")); + + return buildNumber - cachedBuildNumber > ttl; + } + + @Override + boolean matchesCachedFile(final Matcher matcher, final String url) { + final String hash = matcher.group("hash"); + + return url.hashCode() == Integer.parseInt(hash); + } +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/TravisDownloadCacheFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/TravisDownloadCacheFactory.java new file mode 100644 index 000000000000..fa1057d10c01 --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/cache/TravisDownloadCacheFactory.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.tests.util.cache; + +import org.apache.flink.tests.util.parameters.ParameterProperty; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Optional; + +/** + * A {@link DownloadCacheFactory} for the {@link TravisDownloadCache}. + */ +public final class TravisDownloadCacheFactory implements DownloadCacheFactory { + private static final Logger LOG = LoggerFactory.getLogger(TravisDownloadCacheFactory.class); + + private static final ParameterProperty TMP_DIR = new ParameterProperty<>("cache-dir", value -> Paths.get(value)); + private static final ParameterProperty BUILDS_TO_LIVE = new ParameterProperty<>("cache-btl", Integer::parseInt); + private static final ParameterProperty BUILD_NUMBER = new ParameterProperty<>("TRAVIS_BUILD_NUMBER", Integer::parseInt); + + @Override + public Optional create() { + final Optional tmpDir = TMP_DIR.get(); + final Optional timeToLive = BUILDS_TO_LIVE.get(); + final Optional buildNumber = BUILD_NUMBER.get(); + if (!tmpDir.isPresent()) { + LOG.debug("Not loading {} because {} was not set.", TravisDownloadCache.class, TMP_DIR.getPropertyName()); + return Optional.empty(); + } + if (!timeToLive.isPresent()) { + LOG.debug("Not loading {} because {} was not set.", TravisDownloadCache.class, BUILDS_TO_LIVE.getPropertyName()); + return Optional.empty(); + } + if (!buildNumber.isPresent()) { + LOG.debug("Not loading {} because {} was not set.", TravisDownloadCache.class, BUILD_NUMBER.getPropertyName()); + return Optional.empty(); + } + LOG.info("Created {}.", TravisDownloadCache.class.getSimpleName()); + return Optional.of(new TravisDownloadCache(tmpDir.get(), timeToLive.get(), buildNumber.get())); + } +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/resources/META-INF/services/org.apache.flink.tests.util.cache.DownloadCacheFactory b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/resources/META-INF/services/org.apache.flink.tests.util.cache.DownloadCacheFactory new file mode 100644 index 000000000000..2f0c45b2d57e --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/resources/META-INF/services/org.apache.flink.tests.util.cache.DownloadCacheFactory @@ -0,0 +1,17 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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.tests.util.cache.PersistingDownloadCacheFactory +org.apache.flink.tests.util.cache.TravisDownloadCacheFactory From 5221c26056a9d85b08d4be4e8ffd522926ecad97 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 14 Nov 2019 13:47:33 +0100 Subject: [PATCH 659/746] [FLINK-11466][tests] Add KafkaResource --- .../pom.xml | 49 +++ .../flink/tests/util/kafka/KafkaResource.java | 112 ++++++ .../util/kafka/KafkaResourceFactory.java | 36 ++ .../kafka/LocalStandaloneKafkaResource.java | 368 ++++++++++++++++++ .../LocalStandaloneKafkaResourceFactory.java | 37 ++ flink-end-to-end-tests/pom.xml | 1 + 6 files changed, 603 insertions(+) create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/pom.xml create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/KafkaResource.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/KafkaResourceFactory.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/LocalStandaloneKafkaResource.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/LocalStandaloneKafkaResourceFactory.java diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/pom.xml new file mode 100644 index 000000000000..320626764e2a --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/pom.xml @@ -0,0 +1,49 @@ + + + + + flink-end-to-end-tests + org.apache.flink + 1.10-SNAPSHOT + + 4.0.0 + + flink-end-to-end-tests-common-kafka + + + + org.apache.flink + flink-end-to-end-tests-common + ${project.version} + + + org.apache.flink + flink-test-utils-junit + ${project.version} + compile + + + junit + junit + + + diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/KafkaResource.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/KafkaResource.java new file mode 100644 index 000000000000..679d6c403f71 --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/KafkaResource.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.tests.util.kafka; + +import org.apache.flink.tests.util.util.FactoryUtils; +import org.apache.flink.util.ExternalResource; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.Collection; +import java.util.List; + +/** + * Generic interface for interacting with Kafka. + */ +public interface KafkaResource extends ExternalResource { + + /** + * Creates a topic with the given name, replication factor and number of partitions. + * + * @param replicationFactor replication factor + * @param numPartitions number of partitions + * @param topic desired topic name + * @throws IOException + */ + void createTopic(int replicationFactor, int numPartitions, String topic) throws IOException; + + /** + * Sends the given messages to the given topic. + * + * @param topic topic name + * @param messages messages to send + * @throws IOException + */ + void sendMessages(String topic, String ... messages) throws IOException; + + /** + * Returns the kafka bootstrap server addresses. + * @return kafka bootstrap server addresses + */ + Collection getBootstrapServerAddresses(); + + /** + * Returns the address of Zookeeper. + * @return zookeeper address + */ + InetSocketAddress getZookeeperAddress(); + + /** + * Reads up to {@code maxNumMessages} from the given topic. + * + * @param maxNumMessages maximum number of messages that should be read + * @param groupId group id to identify consumer + * @param topic topic name + * @return read messages + * @throws IOException + */ + List readMessage(int maxNumMessages, String groupId, String topic) throws IOException; + + /** + * Modifies the number of partitions for the given topic. + * @param numPartitions desired number of partitions + * @param topic topic to modify + * @throws IOException + */ + void setNumPartitions(int numPartitions, String topic) throws IOException; + + /** + * Returns the current number of partitions for the given topic. + * @param topic topic name + * @return number of partitions for the given topic + * @throws IOException + */ + int getNumPartitions(String topic) throws IOException; + + /** + * Returns the current partition offset for the given partition of the given topic. + * @param topic topic name + * @param partition topic partition + * @return partition offset for the given partition + * @throws IOException + */ + long getPartitionOffset(String topic, int partition) throws IOException; + + /** + * Returns the configured KafkaResource implementation, or a {@link LocalStandaloneKafkaResource} if none is configured. + * + * @return configured KafkaResource, or {@link LocalStandaloneKafkaResource} is none is configured + */ + static KafkaResource get(final String version) { + return FactoryUtils.loadAndInvokeFactory( + KafkaResourceFactory.class, + factory -> factory.create(version), + LocalStandaloneKafkaResourceFactory::new); + } +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/KafkaResourceFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/KafkaResourceFactory.java new file mode 100644 index 000000000000..3d08d64bb64e --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/KafkaResourceFactory.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.kafka; + +import java.util.Optional; + +/** + * A factory for {@link KafkaResource} implementations. + */ +@FunctionalInterface +public interface KafkaResourceFactory { + + /** + * Returns a {@link KafkaResource} instance. If the instance could not be instantiated (for example, because a + * mandatory parameter was missing), then an empty {@link Optional} should be returned. + * + * @return KafkaResource instance, or an empty Optional if the instance could not be instantiated + */ + Optional create(String kafkaVersion); +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/LocalStandaloneKafkaResource.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/LocalStandaloneKafkaResource.java new file mode 100644 index 000000000000..1b1a3395c73b --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/LocalStandaloneKafkaResource.java @@ -0,0 +1,368 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.kafka; + +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.tests.util.AutoClosableProcess; +import org.apache.flink.tests.util.CommandLineWrapper; +import org.apache.flink.tests.util.activation.OperatingSystemRestriction; +import org.apache.flink.tests.util.cache.DownloadCache; +import org.apache.flink.util.OperatingSystem; + +import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.io.PrintStream; +import java.io.PrintWriter; +import java.net.InetSocketAddress; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardOpenOption; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * {@link KafkaResource} that downloads kafka and sets up a local kafka cluster with the bundled zookeeper. + */ +public class LocalStandaloneKafkaResource implements KafkaResource { + + private static final Logger LOG = LoggerFactory.getLogger(LocalStandaloneKafkaResource.class); + private static final Pattern ZK_DATA_DIR_PATTERN = Pattern.compile(".*(dataDir=).*"); + private static final Pattern KAFKA_LOG_DIR_PATTERN = Pattern.compile(".*(log\\.dirs=).*"); + + private static final String ZOOKEEPER_HOST = "localhost"; + private static final int ZOOKEEPER_PORT = 2181; + private static final String ZOOKEEPER_ADDRESS = ZOOKEEPER_HOST + ':' + ZOOKEEPER_PORT; + private static final String KAFKA_HOST = "localhost"; + private static final int KAFKA_PORT = 9092; + private static final String KAFKA_ADDRESS = KAFKA_HOST + ':' + KAFKA_PORT; + + private final TemporaryFolder tmp = new TemporaryFolder(); + + private final DownloadCache downloadCache = DownloadCache.get(); + private final String kafkaVersion; + private Path kafkaDir; + + LocalStandaloneKafkaResource(final String kafkaVersion) { + OperatingSystemRestriction.forbid( + String.format("The %s relies on UNIX utils and shell scripts.", getClass().getSimpleName()), + OperatingSystem.WINDOWS); + this.kafkaVersion = kafkaVersion; + } + + private static String getKafkaDownloadUrl(final String kafkaVersion) { + return String.format("https://archive.apache.org/dist/kafka/%s/kafka_2.11-%s.tgz", kafkaVersion, kafkaVersion); + } + + @Override + public void before() throws Exception { + tmp.create(); + downloadCache.before(); + + this.kafkaDir = tmp.newFolder("kafka").toPath().toAbsolutePath(); + setupKafkaDist(); + setupKafkaCluster(); + } + + private void setupKafkaDist() throws IOException { + final Path downloadDirectory = tmp.newFolder("getOrDownload").toPath(); + final Path kafkaArchive = downloadCache.getOrDownload(getKafkaDownloadUrl(kafkaVersion), downloadDirectory); + + LOG.info("Kafka location: {}", kafkaDir.toAbsolutePath()); + AutoClosableProcess.runBlocking(CommandLineWrapper + .tar(kafkaArchive) + .extract() + .zipped() + .strip(1) + .targetDir(kafkaDir) + .build()); + + LOG.info("Updating ZooKeeper properties"); + final Path zookeeperPropertiesFile = kafkaDir.resolve(Paths.get("config", "zookeeper.properties")); + final List zookeeperPropertiesFileLines = Files.readAllLines(zookeeperPropertiesFile); + try (PrintWriter pw = new PrintWriter(new OutputStreamWriter(Files.newOutputStream(zookeeperPropertiesFile, StandardOpenOption.TRUNCATE_EXISTING), StandardCharsets.UTF_8.name()))) { + zookeeperPropertiesFileLines.stream() + .map(line -> ZK_DATA_DIR_PATTERN.matcher(line).replaceAll("$1" + kafkaDir.resolve("zookeeper").toAbsolutePath())) + .forEachOrdered(pw::println); + } + + LOG.info("Updating Kafka properties"); + final Path kafkaPropertiesFile = kafkaDir.resolve(Paths.get("config", "server.properties")); + final List kafkaPropertiesFileLines = Files.readAllLines(kafkaPropertiesFile); + try (PrintWriter pw = new PrintWriter(new OutputStreamWriter(Files.newOutputStream(kafkaPropertiesFile, StandardOpenOption.TRUNCATE_EXISTING), StandardCharsets.UTF_8.name()))) { + kafkaPropertiesFileLines.stream() + .map(line -> KAFKA_LOG_DIR_PATTERN.matcher(line).replaceAll("$1" + kafkaDir.resolve("kafka").toAbsolutePath())) + .forEachOrdered(pw::println); + } + } + + private void setupKafkaCluster() throws IOException { + LOG.info("Starting zookeeper"); + AutoClosableProcess.runBlocking( + kafkaDir.resolve(Paths.get("bin", "zookeeper-server-start.sh")).toString(), + "-daemon", + kafkaDir.resolve(Paths.get("config", "zookeeper.properties")).toString() + ); + LOG.info("Starting kafka"); + AutoClosableProcess.runBlocking( + kafkaDir.resolve(Paths.get("bin", "kafka-server-start.sh")).toString(), + "-daemon", + kafkaDir.resolve(Paths.get("config", "server.properties")).toString() + ); + + while (!isZookeeperRunning(kafkaDir) || !isKafkaRunning(kafkaDir)) { + try { + LOG.info("Waiting for kafka & zookeeper to start."); + Thread.sleep(500L); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + break; + } + } + } + + @Override + public void afterTestSuccess() { + try { + AutoClosableProcess.runBlocking( + kafkaDir.resolve(Paths.get("bin", "kafka-server-stop.sh")).toString() + ); + while (isKafkaRunning(kafkaDir)) { + try { + Thread.sleep(500L); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + break; + } + } + } catch (IOException ioe) { + LOG.warn("Error while shutting down kafka.", ioe); + } + try { + AutoClosableProcess.runBlocking( + kafkaDir.resolve(Paths.get("bin", "zookeeper-server-stop.sh")).toString() + ); + while (isZookeeperRunning(kafkaDir)) { + try { + Thread.sleep(500L); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + break; + } + } + } catch (IOException ioe) { + LOG.warn("Error while shutting down zookeeper.", ioe); + } + downloadCache.afterTestSuccess(); + tmp.delete(); + } + + private static boolean isZookeeperRunning(final Path kafkaDir) { + try { + queryBrokerStatus(kafkaDir, line -> {}); + return true; + } catch (final IOException ioe) { + // we get an exception if zookeeper isn't running + return false; + } + } + + private static boolean isKafkaRunning(final Path kafkaDir) throws IOException { + try { + final AtomicBoolean atomicBrokerStarted = new AtomicBoolean(false); + queryBrokerStatus(kafkaDir, line -> atomicBrokerStarted.compareAndSet(false, !line.contains("Node does not exist"))); + return atomicBrokerStarted.get(); + } catch (final IOException ioe) { + // we get an exception if zookeeper isn't running + return false; + } + } + + private static void queryBrokerStatus(final Path kafkaDir, final Consumer stderrProcessor) throws IOException { + AutoClosableProcess + .create( + kafkaDir.resolve(Paths.get("bin", "zookeeper-shell.sh")).toString(), + ZOOKEEPER_ADDRESS, + "get", + "/brokers/ids/0") + .setStderrProcessor(stderrProcessor) + .runBlocking(); + } + + @Override + public void createTopic(int replicationFactor, int numPartitions, String topic) throws IOException { + AutoClosableProcess.runBlocking( + kafkaDir.resolve(Paths.get("bin", "kafka-topics.sh")).toString(), + "--create", + "--zookeeper", + ZOOKEEPER_ADDRESS, + "--replication-factor", + String.valueOf(replicationFactor), + "--partitions", + String.valueOf(numPartitions), + "--topic", + topic); + } + + @Override + public void sendMessages(String topic, String... messages) throws IOException { + try (AutoClosableProcess autoClosableProcess = AutoClosableProcess.runNonBlocking( + kafkaDir.resolve(Paths.get("bin", "kafka-console-producer.sh")).toString(), + "--broker-list", + KAFKA_ADDRESS, + "--topic", + topic)) { + + try (PrintStream printStream = new PrintStream(autoClosableProcess.getProcess().getOutputStream(), true, StandardCharsets.UTF_8.name())) { + for (final String message : messages) { + printStream.println(message); + } + printStream.flush(); + } + + try { + // wait until the process shuts down on it's own + // this is the only reliable way to ensure the producer has actually processed our input + autoClosableProcess.getProcess().waitFor(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } + + @Override + public List readMessage(int maxNumMessages, String groupId, String topic) throws IOException { + final List messages = Collections.synchronizedList(new ArrayList<>(maxNumMessages)); + + try (final AutoClosableProcess kafka = AutoClosableProcess + .create(kafkaDir.resolve(Paths.get("bin", "kafka-console-consumer.sh")).toString(), + "--bootstrap-server", + KAFKA_ADDRESS, + "--from-beginning", + "--max-messages", + String.valueOf(maxNumMessages), + "--topic", + topic, + "--consumer-property", + "group.id=" + groupId) + .setStdoutProcessor(messages::add) + .runNonBlocking()) { + + final Deadline deadline = Deadline.fromNow(Duration.ofSeconds(30)); + while (deadline.hasTimeLeft() && messages.size() < maxNumMessages) { + try { + LOG.info("Waiting for messages. Received {}/{}.", messages.size(), maxNumMessages); + Thread.sleep(500); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + break; + } + } + return messages; + } + } + + @Override + public void setNumPartitions(int numPartitions, String topic) throws IOException { + AutoClosableProcess.runBlocking( + kafkaDir.resolve(Paths.get("bin", "kafka-topics.sh")).toString(), + "--alter", + "--topic", + topic, + "--partitions", + String.valueOf(numPartitions), + "--zookeeper", + ZOOKEEPER_ADDRESS + ); + } + + @Override + public int getNumPartitions(String topic) throws IOException { + final Pattern partitionCountPattern = Pattern.compile(".*PartitionCount:([0-9]+).*"); + final AtomicReference partitionCountFound = new AtomicReference<>(-1); + AutoClosableProcess + .create(kafkaDir.resolve(Paths.get("bin", "kafka-topics.sh")).toString(), + "--describe", + "--topic", + topic, + "--zookeeper", + ZOOKEEPER_ADDRESS) + .setStdoutProcessor(line -> { + final Matcher matcher = partitionCountPattern.matcher(line); + if (matcher.matches()) { + partitionCountFound.compareAndSet(-1, Integer.parseInt(matcher.group(1))); + } + }) + .runBlocking(); + return partitionCountFound.get(); + } + + @Override + public long getPartitionOffset(String topic, int partition) throws IOException { + final Pattern partitionOffsetPattern = Pattern.compile(".*:.*:([0-9]+)"); + final AtomicReference partitionOffsetFound = new AtomicReference<>(-1); + AutoClosableProcess + .create(kafkaDir.resolve(Paths.get("bin", "kafka-run-class.sh")).toString(), + "kafka.tools.GetOffsetShell", + "--broker-list", + KAFKA_ADDRESS, + "--topic", + topic, + "--partitions", + String.valueOf(partition), + "--time", + "-1") + .setStdoutProcessor(line -> { + final Matcher matcher = partitionOffsetPattern.matcher(line); + if (matcher.matches()) { + partitionOffsetFound.compareAndSet(-1, Integer.parseInt(matcher.group(1))); + } + }) + .runBlocking(); + + final int partitionOffset = partitionOffsetFound.get(); + if (partitionOffset == -1) { + throw new IOException("Could not determine partition offset."); + } + return partitionOffset; + } + + @Override + public Collection getBootstrapServerAddresses() { + return Collections.singletonList(InetSocketAddress.createUnresolved(KAFKA_HOST, KAFKA_PORT)); + } + + @Override + public InetSocketAddress getZookeeperAddress() { + return InetSocketAddress.createUnresolved(KAFKA_HOST, KAFKA_PORT); + } +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/LocalStandaloneKafkaResourceFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/LocalStandaloneKafkaResourceFactory.java new file mode 100644 index 000000000000..0d61119155c8 --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/main/java/org/apache/flink/tests/util/kafka/LocalStandaloneKafkaResourceFactory.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.tests.util.kafka; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Optional; + +/** + * A {@link KafkaResourceFactory} for the {@link LocalStandaloneKafkaResourceFactory}. + */ +public final class LocalStandaloneKafkaResourceFactory implements KafkaResourceFactory { + private static final Logger LOG = LoggerFactory.getLogger(LocalStandaloneKafkaResourceFactory.class); + + @Override + public Optional create(final String kafkaVersion) { + LOG.info("Created {}.", LocalStandaloneKafkaResource.class.getSimpleName()); + return Optional.of(new LocalStandaloneKafkaResource(kafkaVersion)); + } +} diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml index bdf04b3139c3..43975b66a95a 100644 --- a/flink-end-to-end-tests/pom.xml +++ b/flink-end-to-end-tests/pom.xml @@ -72,6 +72,7 @@ under the License. flink-tpch-test flink-streaming-kinesis-test flink-elasticsearch7-test + flink-end-to-end-tests-common-kafka From 499fe56c7eab9510258c1d19b1d062fe291c95ae Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 14 Nov 2019 13:49:35 +0100 Subject: [PATCH 660/746] [FLINK-11468][tests] Setup surefire execution --- .travis.yml | 49 ++++++++++--------- .../flink/tests/util/categories/Dummy.java | 25 ++++++++++ .../flink/tests/util/categories/Hadoop.java | 25 ++++++++++ .../tests/util/categories/PreCommit.java | 25 ++++++++++ .../tests/util/categories/TravisGroup1.java | 25 ++++++++++ .../tests/util/categories/TravisGroup2.java | 25 ++++++++++ .../tests/util/categories/TravisGroup3.java | 25 ++++++++++ .../tests/util/categories/TravisGroup4.java | 25 ++++++++++ .../tests/util/categories/TravisGroup5.java | 25 ++++++++++ .../tests/util/categories/TravisGroup6.java | 25 ++++++++++ .../flink-metrics-availability-test/pom.xml | 33 ------------- .../tests/MetricsAvailabilityITCase.java | 3 ++ .../pom.xml | 33 ------------- .../PrometheusReporterEndToEndITCase.java | 3 ++ flink-end-to-end-tests/pom.xml | 28 ++++++++++- flink-end-to-end-tests/run-nightly-tests.sh | 3 +- tools/travis_watchdog.sh | 19 ++++++- 17 files changed, 302 insertions(+), 94 deletions(-) create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/Dummy.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/Hadoop.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/PreCommit.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/TravisGroup1.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/TravisGroup2.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/TravisGroup3.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/TravisGroup4.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/TravisGroup5.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/TravisGroup6.java diff --git a/.travis.yml b/.travis.yml index 109533f34338..65cec31a6f7a 100644 --- a/.travis.yml +++ b/.travis.yml @@ -27,6 +27,7 @@ cache: - $HOME/maven_cache # keep in sync with tools/travis/docs.sh - $HOME/gem_cache + - $HOME/flink_download_cache # do not cache our own artifacts before_cache: @@ -41,6 +42,8 @@ git: env: global: + - cache-dir: $HOME/flink_download_cache + - cache-btl: 30 # Global variable to avoid hanging travis builds when downloading cache archives. - MALLOC_ARENA_MAX=2 - DOCKER_COMPOSE_VERSION=1.22.0 @@ -273,99 +276,99 @@ jobs: # E2E profiles - Hadoop 2.8 - if: type = cron stage: test - env: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -De2e-metrics" + env: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -De2e-metrics -DincludeE2E=org.apache.flink.tests.util.categories.TravisGroup1" script: ./tools/travis/nightly.sh split_misc.sh name: e2e - misc - hadoop 2.8 - if: type = cron - env: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3" + env: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -DincludeE2E=org.apache.flink.tests.util.categories.TravisGroup2" script: ./tools/travis/nightly.sh split_ha.sh name: e2e - ha - hadoop 2.8 - if: type = cron - env: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3" + env: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -DincludeE2E=org.apache.flink.tests.util.categories.TravisGroup3" script: ./tools/travis/nightly.sh split_sticky.sh name: e2e - sticky - hadoop 2.8 - if: type = cron - env: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3" + env: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -DincludeE2E=org.apache.flink.tests.util.categories.TravisGroup4" script: ./tools/travis/nightly.sh split_checkpoints.sh name: e2e - checkpoints - hadoop 2.8 - if: type = cron - env: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3" + env: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -DincludeE2E=org.apache.flink.tests.util.categories.TravisGroup5" script: ./tools/travis/nightly.sh split_container.sh name: e2e - container - hadoop 2.8 - if: type = cron - env: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3" + env: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -DincludeE2E=org.apache.flink.tests.util.categories.TravisGroup6" script: ./tools/travis/nightly.sh split_heavy.sh name: e2e - heavy - hadoop 2.8 # E2E profiles - Scala 2.12 - if: type = cron - env: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dscala-2.12 -De2e-metrics" + env: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dscala-2.12 -De2e-metrics -DincludeE2E=org.apache.flink.tests.util.categories.TravisGroup1" script: ./tools/travis/nightly.sh split_misc.sh name: e2e - misc - scala 2.12 - if: type = cron - env: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dscala-2.12" + env: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dscala-2.12 -DincludeE2E=org.apache.flink.tests.util.categories.TravisGroup2" script: ./tools/travis/nightly.sh split_ha.sh name: e2e - ha - scala 2.12 - if: type = cron - env: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dscala-2.12" + env: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dscala-2.12 -DincludeE2E=org.apache.flink.tests.util.categories.TravisGroup3" script: ./tools/travis/nightly.sh split_sticky.sh name: e2e - sticky - scala 2.12 - if: type = cron - env: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dscala-2.12" + env: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dscala-2.12 -DincludeE2E=org.apache.flink.tests.util.categories.TravisGroup4" script: ./tools/travis/nightly.sh split_checkpoints.sh name: e2e - checkpoints - scala 2.12 - if: type = cron - env: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dscala-2.12" + env: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dscala-2.12 -DincludeE2E=org.apache.flink.tests.util.categories.TravisGroup5" script: ./tools/travis/nightly.sh split_container.sh name: e2e - container - scala 2.12 - if: type = cron - env: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dscala-2.12" + env: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dscala-2.12 -DincludeE2E=org.apache.flink.tests.util.categories.TravisGroup6" script: ./tools/travis/nightly.sh split_heavy.sh name: e2e - heavy - scala 2.12 # E2E profiles - Hadoop-free - if: type = cron - env: PROFILE="-De2e-metrics" + env: PROFILE="-De2e-metrics -DincludeE2E=org.apache.flink.tests.util.categories.TravisGroup1" script: ./tools/travis/nightly.sh split_misc_hadoopfree.sh name: e2e - misc - if: type = cron - env: PROFILE="" + env: PROFILE="-DincludeE2E=org.apache.flink.tests.util.categories.TravisGroup2" script: ./tools/travis/nightly.sh split_ha.sh name: e2e - ha - if: type = cron - env: PROFILE="" + env: PROFILE="-DincludeE2E=org.apache.flink.tests.util.categories.TravisGroup3" script: ./tools/travis/nightly.sh split_sticky.sh name: e2e - sticky - if: type = cron - env: PROFILE="" + env: PROFILE="-DincludeE2E=org.apache.flink.tests.util.categories.TravisGroup4" script: ./tools/travis/nightly.sh split_checkpoints.sh name: e2e - checkpoints - if: type = cron - env: PROFILE="" + env: PROFILE="-DincludeE2E=org.apache.flink.tests.util.categories.TravisGroup5" script: ./tools/travis/nightly.sh split_container.sh name: e2e - container - if: type = cron - env: PROFILE="" + env: PROFILE="-DincludeE2E=org.apache.flink.tests.util.categories.TravisGroup6" script: ./tools/travis/nightly.sh split_heavy.sh name: e2e - heavy # E2E profiles - Java 11 - if: type = cron stage: test jdk: "openjdk11" - env: PROFILE="-Djdk11 -Dinclude-hadoop -Dhadoop.version=2.8.3 -De2e-metrics" + env: PROFILE="-Djdk11 -Dinclude-hadoop -Dhadoop.version=2.8.3 -De2e-metrics -DincludeE2E=org.apache.flink.tests.util.categories.TravisGroup1" script: ./tools/travis/nightly.sh split_misc.sh name: e2e - misc - jdk11 - if: type = cron - env: PROFILE="-Djdk11 -Dinclude-hadoop -Dhadoop.version=2.8.3" + env: PROFILE="-Djdk11 -Dinclude-hadoop -Dhadoop.version=2.8.3 -DincludeE2E=org.apache.flink.tests.util.categories.TravisGroup2" script: ./tools/travis/nightly.sh split_ha.sh name: e2e - ha - jdk11 - if: type = cron - env: PROFILE="-Djdk11 -Dinclude-hadoop -Dhadoop.version=2.8.3" + env: PROFILE="-Djdk11 -Dinclude-hadoop -Dhadoop.version=2.8.3 -DincludeE2E=org.apache.flink.tests.util.categories.TravisGroup3" script: ./tools/travis/nightly.sh split_sticky.sh name: e2e - sticky - jdk 11 - if: type = cron - env: PROFILE="-Djdk11 -Dinclude-hadoop -Dhadoop.version=2.8.3" + env: PROFILE="-Djdk11 -Dinclude-hadoop -Dhadoop.version=2.8.3 -DincludeE2E=org.apache.flink.tests.util.categories.TravisGroup4" script: ./tools/travis/nightly.sh split_checkpoints.sh name: e2e - checkpoints - jdk 11 - if: type = cron - env: PROFILE="-Djdk11 -Dinclude-hadoop -Dhadoop.version=2.8.3" + env: PROFILE="-Djdk11 -Dinclude-hadoop -Dhadoop.version=2.8.3 -DincludeE2E=org.apache.flink.tests.util.categories.TravisGroup6" script: ./tools/travis/nightly.sh split_heavy.sh name: e2e - heavy - jdk 11 diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/Dummy.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/Dummy.java new file mode 100644 index 000000000000..ef0d656ad0dd --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/Dummy.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.categories; + +/** + * Dummy marker interface used for excluding all other groups. + */ +public interface Dummy { +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/Hadoop.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/Hadoop.java new file mode 100644 index 000000000000..b28ad8701bae --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/Hadoop.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.categories; + +/** + * Marker interface for tests that require Hadoop. + */ +public interface Hadoop { +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/PreCommit.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/PreCommit.java new file mode 100644 index 000000000000..4477608783e4 --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/PreCommit.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.categories; + +/** + * Marker interface for tests that should be run on every commit. + */ +public interface PreCommit { +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/TravisGroup1.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/TravisGroup1.java new file mode 100644 index 000000000000..d8468d85579d --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/TravisGroup1.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.categories; + +/** + * Marker interface for tests that should be run in the first build on Travis. + */ +public interface TravisGroup1 { +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/TravisGroup2.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/TravisGroup2.java new file mode 100644 index 000000000000..1f05974513f8 --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/TravisGroup2.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.categories; + +/** + * Marker interface for tests that should be run in the second build on Travis. + */ +public interface TravisGroup2 { +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/TravisGroup3.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/TravisGroup3.java new file mode 100644 index 000000000000..a71398c2e72f --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/TravisGroup3.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.categories; + +/** + * Marker interface for tests that should be run in the third build on Travis. + */ +public interface TravisGroup3 { +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/TravisGroup4.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/TravisGroup4.java new file mode 100644 index 000000000000..5b56bffb6191 --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/TravisGroup4.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.categories; + +/** + * Marker interface for tests that should be run in the fourth build on Travis. + */ +public interface TravisGroup4 { +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/TravisGroup5.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/TravisGroup5.java new file mode 100644 index 000000000000..9f103fc26f23 --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/TravisGroup5.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.categories; + +/** + * Marker interface for tests that should be run in the fifth build on Travis. + */ +public interface TravisGroup5 { +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/TravisGroup6.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/TravisGroup6.java new file mode 100644 index 000000000000..a5a1dd62484a --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/TravisGroup6.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.categories; + +/** + * Marker interface for tests that should be run in the sixth build on Travis. + */ +public interface TravisGroup6 { +} diff --git a/flink-end-to-end-tests/flink-metrics-availability-test/pom.xml b/flink-end-to-end-tests/flink-metrics-availability-test/pom.xml index 92e5f08c2ce5..3e5db28c29f0 100644 --- a/flink-end-to-end-tests/flink-metrics-availability-test/pom.xml +++ b/flink-end-to-end-tests/flink-metrics-availability-test/pom.xml @@ -57,37 +57,4 @@ under the License. - - - e2e-metric-availability - - - e2e-metrics - - - - - - org.apache.maven.plugins - maven-surefire-plugin - - - e2e-metric-availability - integration-test - - test - - - - **/*ITCase.* - - - - - - - - - - diff --git a/flink-end-to-end-tests/flink-metrics-availability-test/src/test/java/org/pache/flink/metrics/tests/MetricsAvailabilityITCase.java b/flink-end-to-end-tests/flink-metrics-availability-test/src/test/java/org/pache/flink/metrics/tests/MetricsAvailabilityITCase.java index 28a21354e5d2..49d151e9f1bd 100644 --- a/flink-end-to-end-tests/flink-metrics-availability-test/src/test/java/org/pache/flink/metrics/tests/MetricsAvailabilityITCase.java +++ b/flink-end-to-end-tests/flink-metrics-availability-test/src/test/java/org/pache/flink/metrics/tests/MetricsAvailabilityITCase.java @@ -37,6 +37,7 @@ import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagersHeaders; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagersInfo; import org.apache.flink.tests.util.FlinkDistribution; +import org.apache.flink.tests.util.categories.TravisGroup1; import org.apache.flink.util.TestLogger; import org.apache.flink.util.function.SupplierWithException; @@ -44,6 +45,7 @@ import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; +import org.junit.experimental.categories.Category; import javax.annotation.Nullable; @@ -63,6 +65,7 @@ /** * End-to-end test for the availability of metrics. */ +@Category(TravisGroup1.class) public class MetricsAvailabilityITCase extends TestLogger { private static final String HOST = "localhost"; diff --git a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml index ab8a91588e45..db6c25883ded 100644 --- a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml +++ b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml @@ -68,37 +68,4 @@ under the License. - - - e2e-prometheus - - - e2e-metrics - - - - - - org.apache.maven.plugins - maven-surefire-plugin - - - e2e-prometheus - integration-test - - test - - - - **/*ITCase.* - - - - - - - - - - diff --git a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java index 258d293bb20d..9c15bf7a65b2 100644 --- a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java +++ b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/src/test/java/org/apache/flink/metrics/prometheus/tests/PrometheusReporterEndToEndITCase.java @@ -24,6 +24,7 @@ import org.apache.flink.tests.util.AutoClosableProcess; import org.apache.flink.tests.util.CommandLineWrapper; import org.apache.flink.tests.util.FlinkDistribution; +import org.apache.flink.tests.util.categories.TravisGroup1; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.OperatingSystem; import org.apache.flink.util.TestLogger; @@ -37,6 +38,7 @@ import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -54,6 +56,7 @@ /** * End-to-end test for the PrometheusReporter. */ +@Category(TravisGroup1.class) public class PrometheusReporterEndToEndITCase extends TestLogger { private static final Logger LOG = LoggerFactory.getLogger(PrometheusReporterEndToEndITCase.class); diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml index 43975b66a95a..b9f80fb12eff 100644 --- a/flink-end-to-end-tests/pom.xml +++ b/flink-end-to-end-tests/pom.xml @@ -34,6 +34,11 @@ under the License. flink-end-to-end-tests flink-end-to-end-tests + + org.apache.flink.tests.util.categories.Dummy + + + flink-cli-test flink-parent-child-classloading-test-program @@ -88,8 +93,6 @@ under the License. org.apache.maven.plugins maven-surefire-plugin - default-test none @@ -98,6 +101,27 @@ under the License. integration-tests none + + end-to-end-tests + integration-test + + test + + + + **/*ITCase.* + + + ${includeE2E} + ${excludeE2E} + + 1 + + ${project.basedir} + + + diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh index 9307188a531b..d067a655f978 100755 --- a/flink-end-to-end-tests/run-nightly-tests.sh +++ b/flink-end-to-end-tests/run-nightly-tests.sh @@ -175,5 +175,6 @@ run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 10 rocks false true" "skip_check_exceptions" run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 10 rocks true true" "skip_check_exceptions" -printf "\n[PASS] All tests passed\n" +printf "\n[PASS] All bash e2e-tests passed\n" +printf "\n Please remember to run java e2e-tests by running mvn verify -Dcategories=\"\" -DdistDir= in flink-end-to-end-tests\n" exit 0 diff --git a/tools/travis_watchdog.sh b/tools/travis_watchdog.sh index 4e82e567e9be..a0ee6ff40500 100755 --- a/tools/travis_watchdog.sh +++ b/tools/travis_watchdog.sh @@ -65,8 +65,11 @@ MVN_COMMON_OPTIONS="-nsu -Dflink.forkCount=2 -Dflink.forkCountTestPackage=2 -Dfa MVN_COMPILE_OPTIONS="-DskipTests" MVN_TEST_OPTIONS="-Dflink.tests.with-openssl" +e2e_modules=$(find flink-end-to-end-tests -mindepth 2 -maxdepth 5 -name 'pom.xml' -printf '%h\n' | sort -u | tr '\n' ',') + MVN_COMPILE="mvn $MVN_COMMON_OPTIONS $MVN_COMPILE_OPTIONS $PROFILE $MVN_COMPILE_MODULES install" MVN_TEST="mvn $MVN_COMMON_OPTIONS $MVN_TEST_OPTIONS $PROFILE $MVN_TEST_MODULES verify" +MVN_E2E="mvn $MVN_COMMON_OPTIONS $MVN_TEST_OPTIONS $PROFILE -DincludeE2E="org.apache.flink.tests.util.categories.PreCommit" -pl ${e2e_modules},flink-dist verify" MVN_PID="${ARTIFACTS_DIR}/watchdog.mvn.pid" MVN_EXIT="${ARTIFACTS_DIR}/watchdog.mvn.exit" @@ -264,7 +267,7 @@ case $TEST in (misc) if [ $EXIT_CODE == 0 ]; then printf "\n\n==============================================================================\n" - printf "Running end-to-end tests\n" + printf "Running bash end-to-end tests\n" printf "==============================================================================\n" FLINK_DIR=build-target flink-end-to-end-tests/run-pre-commit-tests.sh @@ -272,8 +275,20 @@ case $TEST in EXIT_CODE=$? else printf "\n==============================================================================\n" - printf "Previous build failure detected, skipping end-to-end tests.\n" + printf "Previous build failure detected, skipping bash end-to-end tests.\n" + printf "==============================================================================\n" + fi + if [ $EXIT_CODE == 0 ]; then + printf "\n\n==============================================================================\n" + printf "Running java end-to-end tests\n" printf "==============================================================================\n" + + run_with_watchdog "$MVN_E2E" + + EXIT_CODE=$? + else + printf "\n==============================================================================\n" + printf "Previous build failure detected, skipping java end-to-end tests.\n" fi ;; esac From 33b5663e43fe9f59445d51ec5b27c3283e27142f Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 14 Nov 2019 13:50:41 +0100 Subject: [PATCH 661/746] [FLINK-11467][kafka][tests] Port Kafka Streaming E2E test --- .../pom.xml | 91 +++++++++ .../util/kafka/StreamingKafkaITCase.java | 173 ++++++++++++++++++ .../src/test/resources/log4j-test.properties | 27 +++ .../flink-streaming-kafka-test/pom.xml | 6 +- .../flink-streaming-kafka010-test/pom.xml | 6 +- .../flink-streaming-kafka011-test/pom.xml | 6 +- .../run-pre-commit-tests.sh | 6 +- .../test-scripts/test_streaming_kafka.sh | 25 --- .../test-scripts/test_streaming_kafka010.sh | 26 --- .../test-scripts/test_streaming_kafka011.sh | 24 --- .../test_streaming_kafka_common.sh | 103 ----------- 11 files changed, 307 insertions(+), 186 deletions(-) create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/StreamingKafkaITCase.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/resources/log4j-test.properties delete mode 100755 flink-end-to-end-tests/test-scripts/test_streaming_kafka.sh delete mode 100755 flink-end-to-end-tests/test-scripts/test_streaming_kafka010.sh delete mode 100755 flink-end-to-end-tests/test-scripts/test_streaming_kafka011.sh delete mode 100644 flink-end-to-end-tests/test-scripts/test_streaming_kafka_common.sh diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/pom.xml index 320626764e2a..baf1b7ab01b2 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/pom.xml +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/pom.xml @@ -45,5 +45,96 @@ under the License. junit junit + + + + org.apache.flink + flink-streaming-kafka010-test_${scala.binary.version} + ${project.version} + + provided + + + + org.apache.flink + flink-connector-kafka-0.10_${scala.binary.version} + + + + + org.apache.flink + flink-streaming-kafka011-test_${scala.binary.version} + ${project.version} + + provided + + + + org.apache.flink + flink-connector-kafka-0.11_${scala.binary.version} + + + + + org.apache.flink + flink-streaming-kafka-test_${scala.binary.version} + ${project.version} + + provided + + + + org.apache.flink + flink-connector-kafka-_${scala.binary.version} + + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + 3.0.1 + + + copy + process-test-resources + + copy + + + + + + + org.apache.flink + flink-streaming-kafka010-test_${scala.binary.version} + ${project.version} + jar + true + ${project.build.directory}/dependencies + + + org.apache.flink + flink-streaming-kafka011-test_${scala.binary.version} + ${project.version} + jar + true + ${project.build.directory}/dependencies + + + org.apache.flink + flink-streaming-kafka-test_${scala.binary.version} + ${project.version} + jar + true + ${project.build.directory}/dependencies + + + + + + diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/StreamingKafkaITCase.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/StreamingKafkaITCase.java new file mode 100644 index 000000000000..e6843363925a --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/StreamingKafkaITCase.java @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.kafka; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.tests.util.TestUtils; +import org.apache.flink.tests.util.categories.PreCommit; +import org.apache.flink.tests.util.categories.TravisGroup1; +import org.apache.flink.tests.util.flink.ClusterController; +import org.apache.flink.tests.util.flink.FlinkResource; +import org.apache.flink.tests.util.flink.JobSubmission; +import org.apache.flink.testutils.junit.FailsOnJava11; +import org.apache.flink.util.TestLogger; + +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.file.Path; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.stream.Collectors; + +/** + * End-to-end test for the kafka connectors. + */ +@RunWith(Parameterized.class) +@Category(value = {TravisGroup1.class, PreCommit.class, FailsOnJava11.class}) +public class StreamingKafkaITCase extends TestLogger { + + private static final Logger LOG = LoggerFactory.getLogger(StreamingKafkaITCase.class); + + @Parameterized.Parameters(name = "{index}: kafka-version:{1}") + public static Collection data() { + return Arrays.asList(new Object[][]{ + {"flink-streaming-kafka010-test.*", "0.10.2.0"}, + {"flink-streaming-kafka011-test.*", "0.11.0.2"}, + {"flink-streaming-kafka-test.*", "2.2.0"} + }); + } + + private final Path kafkaExampleJar; + + @Rule + public final KafkaResource kafka; + + @Rule + public final FlinkResource flink = FlinkResource.get(); + + public StreamingKafkaITCase(final String kafkaExampleJarPattern, final String kafkaVersion) { + this.kafkaExampleJar = TestUtils.getResourceJar(kafkaExampleJarPattern); + this.kafka = KafkaResource.get(kafkaVersion); + } + + @Test + public void testKafka() throws Exception { + // modify configuration to have enough slots + final Configuration flinkConfig = new Configuration(); + flinkConfig.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, 3); + flink.addConfiguration(flinkConfig); + + try (final ClusterController clusterController = flink.startCluster(1)) { + + final String inputTopic = "test-input"; + final String outputTopic = "test-output"; + + // create the required topics + kafka.createTopic(1, 1, inputTopic); + kafka.createTopic(1, 1, outputTopic); + + // run the Flink job (detached mode) + clusterController.submitJob(new JobSubmission.JobSubmissionBuilder(kafkaExampleJar) + .setDetached(true) + .addArgument("--input-topic", inputTopic) + .addArgument("--output-topic", outputTopic) + .addArgument("--prefix", "PREFIX") + .addArgument("--bootstrap.servers", kafka.getBootstrapServerAddresses().stream().map(address -> address.getHostString() + ':' + address.getPort()).collect(Collectors.joining(","))) + .addArgument("--zookeeper.connect ", kafka.getZookeeperAddress().getHostString() + ':' + kafka.getZookeeperAddress().getPort()) + .addArgument("--group.id", "myconsumer") + .addArgument("--auto.offset.reset", "earliest") + .addArgument("--transaction.timeout.ms", "900000") + .addArgument("--flink.partition-discovery.interval-millis", "1000") + .build()); + + LOG.info("Sending messages to Kafka topic [{}] ...", inputTopic); + // send some data to Kafka + kafka.sendMessages(inputTopic, + "elephant,5,45218", + "squirrel,12,46213", + "bee,3,51348", + "squirrel,22,52444", + "bee,10,53412", + "elephant,9,54867"); + + LOG.info("Verifying messages from Kafka topic [{}] ...", outputTopic); + { + final List messages = kafka.readMessage(6, "kafka-e2e-driver", outputTopic); + + final List elephants = filterMessages(messages, "elephant"); + final List squirrels = filterMessages(messages, "squirrel"); + final List bees = filterMessages(messages, "bee"); + + // check all keys + Assert.assertEquals(Arrays.asList("elephant,5,45218", "elephant,14,54867"), elephants); + Assert.assertEquals(Arrays.asList("squirrel,12,46213", "squirrel,34,52444"), squirrels); + Assert.assertEquals(Arrays.asList("bee,3,51348", "bee,13,53412"), bees); + } + + // now, we add a new partition to the topic + LOG.info("Repartitioning Kafka topic [{}] ...", inputTopic); + kafka.setNumPartitions(2, inputTopic); + Assert.assertEquals("Failed adding a partition to input topic.", 2, kafka.getNumPartitions(inputTopic)); + + // send some more messages to Kafka + LOG.info("Sending more messages to Kafka topic [{}] ...", inputTopic); + kafka.sendMessages(inputTopic, + "elephant,13,64213", + "giraffe,9,65555", + "bee,5,65647", + "squirrel,18,66413"); + + // verify that our assumption that the new partition actually has written messages is correct + Assert.assertNotEquals( + "The newly created partition does not have any new messages, and therefore partition discovery cannot be verified.", + 0L, + kafka.getPartitionOffset(inputTopic, 1)); + + LOG.info("Verifying messages from Kafka topic [{}] ...", outputTopic); + { + final List messages = kafka.readMessage(4, "kafka-e2e-driver", outputTopic); + + final List elephants = filterMessages(messages, "elephant"); + final List squirrels = filterMessages(messages, "squirrel"); + final List bees = filterMessages(messages, "bee"); + final List giraffes = filterMessages(messages, "giraffe"); + + Assert.assertEquals(Arrays.asList("elephant,27,64213"), elephants); + Assert.assertEquals(Arrays.asList("squirrel,52,66413"), squirrels); + Assert.assertEquals(Arrays.asList("bee,18,65647"), bees); + Assert.assertEquals(Arrays.asList("giraffe,9,65555"), giraffes); + } + } + } + + private static List filterMessages(final List messages, final String keyword) { + return messages.stream() + .filter(msg -> msg.contains(keyword)) + .collect(Collectors.toList()); + } +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/resources/log4j-test.properties b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/resources/log4j-test.properties new file mode 100644 index 000000000000..ca8abe7bec74 --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/resources/log4j-test.properties @@ -0,0 +1,27 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +# Set root logger level to OFF to not flood build logs +# set manually to INFO for debugging purposes +log4j.rootLogger=OFF, testlogger + +# A1 is set to be a ConsoleAppender. +log4j.appender.testlogger=org.apache.log4j.ConsoleAppender +log4j.appender.testlogger.target = System.err +log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout +log4j.appender.testlogger.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2}: %.1000m%n diff --git a/flink-end-to-end-tests/flink-streaming-kafka-test/pom.xml b/flink-end-to-end-tests/flink-streaming-kafka-test/pom.xml index ab77fdfefc90..c1d2c639b241 100644 --- a/flink-end-to-end-tests/flink-streaming-kafka-test/pom.xml +++ b/flink-end-to-end-tests/flink-streaming-kafka-test/pom.xml @@ -52,12 +52,17 @@ under the License. + KafkaExample org.apache.maven.plugins maven-shade-plugin + + shade-flink + none + fat-jar-kafka-example package @@ -73,7 +78,6 @@ under the License. org.apache.flink.streaming.kafka.test.KafkaExample - KafkaExample diff --git a/flink-end-to-end-tests/flink-streaming-kafka010-test/pom.xml b/flink-end-to-end-tests/flink-streaming-kafka010-test/pom.xml index ed70d1a80962..b71cc2384c04 100644 --- a/flink-end-to-end-tests/flink-streaming-kafka010-test/pom.xml +++ b/flink-end-to-end-tests/flink-streaming-kafka010-test/pom.xml @@ -52,12 +52,17 @@ under the License. + Kafka010Example org.apache.maven.plugins maven-shade-plugin + + shade-flink + none + fat-jar-kafka-example package @@ -73,7 +78,6 @@ under the License. org.apache.flink.streaming.kafka.test.Kafka010Example - Kafka010Example diff --git a/flink-end-to-end-tests/flink-streaming-kafka011-test/pom.xml b/flink-end-to-end-tests/flink-streaming-kafka011-test/pom.xml index ac73b162b7c3..83f5afb2137a 100644 --- a/flink-end-to-end-tests/flink-streaming-kafka011-test/pom.xml +++ b/flink-end-to-end-tests/flink-streaming-kafka011-test/pom.xml @@ -53,12 +53,17 @@ under the License. + Kafka010Example org.apache.maven.plugins maven-shade-plugin + + shade-flink + none + fat-jar-kafka-example package @@ -74,7 +79,6 @@ under the License. org.apache.flink.streaming.kafka.test.Kafka011Example - Kafka011Example diff --git a/flink-end-to-end-tests/run-pre-commit-tests.sh b/flink-end-to-end-tests/run-pre-commit-tests.sh index 599d634d8941..a34001fa606b 100755 --- a/flink-end-to-end-tests/run-pre-commit-tests.sh +++ b/flink-end-to-end-tests/run-pre-commit-tests.sh @@ -54,11 +54,7 @@ run_test "Wordcount end-to-end test" "$END_TO_END_DIR/test-scripts/test_batch_wo run_test "Shaded Hadoop S3A end-to-end test" "$END_TO_END_DIR/test-scripts/test_batch_wordcount.sh hadoop" run_test "Shaded Presto S3 end-to-end test" "$END_TO_END_DIR/test-scripts/test_batch_wordcount.sh presto" run_test "Custom FS plugin end-to-end test" "$END_TO_END_DIR/test-scripts/test_batch_wordcount.sh dummy-fs" -if [[ ${PROFILE} != *"jdk11"* ]]; then - run_test "Kafka 0.10 end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_kafka010.sh" - run_test "Kafka 0.11 end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_kafka011.sh" - run_test "Modern Kafka end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_kafka.sh" -fi + run_test "Kinesis end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_kinesis.sh" run_test "class loading end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_classloader.sh" run_test "Distributed cache end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_distributed_cache_via_blob.sh" diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_kafka.sh b/flink-end-to-end-tests/test-scripts/test_streaming_kafka.sh deleted file mode 100755 index 162bdc04f482..000000000000 --- a/flink-end-to-end-tests/test-scripts/test_streaming_kafka.sh +++ /dev/null @@ -1,25 +0,0 @@ -#!/usr/bin/env bash -################################################################################ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -################################################################################ - -set -Eeuo pipefail - -source "$(dirname "$0")"/common.sh -source "$(dirname "$0")"/kafka-common.sh 2.2.0 5.0.0 5.0 - -source "$(dirname "$0")"/test_streaming_kafka_common.sh ${END_TO_END_DIR}/flink-streaming-kafka-test/target/KafkaExample.jar diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_kafka010.sh b/flink-end-to-end-tests/test-scripts/test_streaming_kafka010.sh deleted file mode 100755 index 97a71f900526..000000000000 --- a/flink-end-to-end-tests/test-scripts/test_streaming_kafka010.sh +++ /dev/null @@ -1,26 +0,0 @@ -#!/usr/bin/env bash -################################################################################ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -################################################################################ - -set -Eeuo pipefail - -source "$(dirname "$0")"/common.sh -source "$(dirname "$0")"/kafka-common.sh 0.10.2.0 3.2.0 3.2 - -source "$(dirname "$0")"/test_streaming_kafka_common.sh ${END_TO_END_DIR}/flink-streaming-kafka010-test/target/Kafka010Example.jar - diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_kafka011.sh b/flink-end-to-end-tests/test-scripts/test_streaming_kafka011.sh deleted file mode 100755 index 84fa6edc90ec..000000000000 --- a/flink-end-to-end-tests/test-scripts/test_streaming_kafka011.sh +++ /dev/null @@ -1,24 +0,0 @@ -#!/usr/bin/env bash -################################################################################ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -################################################################################ - -source "$(dirname "$0")"/common.sh -source "$(dirname "$0")"/kafka-common.sh 0.11.0.2 3.2.0 3.2 - -source "$(dirname "$0")"/test_streaming_kafka_common.sh ${END_TO_END_DIR}/flink-streaming-kafka011-test/target/Kafka011Example.jar - diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_kafka_common.sh b/flink-end-to-end-tests/test-scripts/test_streaming_kafka_common.sh deleted file mode 100644 index cfa6648af058..000000000000 --- a/flink-end-to-end-tests/test-scripts/test_streaming_kafka_common.sh +++ /dev/null @@ -1,103 +0,0 @@ -#!/usr/bin/env bash -################################################################################ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -################################################################################ - -KAFKA_EXAMPLE_JAR="$1" - -setup_kafka_dist -start_kafka_cluster - -# modify configuration to have enough slots -set_config_key "taskmanager.numberOfTaskSlots" "3" - -start_cluster - -function test_cleanup { - stop_kafka_cluster -} -on_exit test_cleanup - -# create the required topics -create_kafka_topic 1 1 test-input -create_kafka_topic 1 1 test-output - -# run the Flink job (detached mode) -$FLINK_DIR/bin/flink run -d $KAFKA_EXAMPLE_JAR \ - --input-topic test-input --output-topic test-output \ - --prefix=PREFIX \ - --bootstrap.servers localhost:9092 --zookeeper.connect localhost:2181 --group.id myconsumer --auto.offset.reset earliest \ - --transaction.timeout.ms 900000 \ - --flink.partition-discovery.interval-millis 1000 - -function verify_output { - local expected=$(printf $1) - - if [[ "$2" != "$expected" ]]; then - echo "Output from Flink program does not match expected output." - echo -e "EXPECTED FOR KEY: --$expected--" - echo -e "ACTUAL: --$2--" - exit 1 - fi -} - -echo "Sending messages to Kafka topic [test-input] ..." -# send some data to Kafka -send_messages_to_kafka "elephant,5,45218\nsquirrel,12,46213\nbee,3,51348\nsquirrel,22,52444\nbee,10,53412\nelephant,9,54867" test-input - -echo "Verifying messages from Kafka topic [test-output] ..." - -KEY_1_MSGS=$(read_messages_from_kafka 6 test-output elephant_consumer | grep elephant) -KEY_2_MSGS=$(read_messages_from_kafka 6 test-output squirrel_consumer | grep squirrel) -KEY_3_MSGS=$(read_messages_from_kafka 6 test-output bee_consumer | grep bee) - -# check all keys; make sure we have actual newlines in the string, not "\n" -verify_output "elephant,5,45218\nelephant,14,54867" "$KEY_1_MSGS" -verify_output "squirrel,12,46213\nsquirrel,34,52444" "$KEY_2_MSGS" -verify_output "bee,3,51348\nbee,13,53412" "$KEY_3_MSGS" - -# now, we add a new partition to the topic -echo "Repartitioning Kafka topic [test-input] ..." -modify_num_partitions test-input 2 - -if (( $(get_num_partitions test-input) != 2 )); then - echo "Failed adding a partition to test-input topic." - exit 1 -fi - -# send some more messages to Kafka -echo "Sending more messages to Kafka topic [test-input] ..." -send_messages_to_kafka "elephant,13,64213\ngiraffe,9,65555\nbee,5,65647\nsquirrel,18,66413" test-input - -# verify that our assumption that the new partition actually has written messages is correct -if (( $(get_partition_end_offset test-input 1) == 0 )); then - echo "The newly created partition does not have any new messages, and therefore partition discovery cannot be verified." - exit 1 -fi - -# all new messages should have been consumed, and has produced correct output -echo "Verifying messages from Kafka topic [test-output] ..." - -KEY_1_MSGS=$(read_messages_from_kafka 4 test-output elephant_consumer | grep elephant) -KEY_2_MSGS=$(read_messages_from_kafka 4 test-output squirrel_consumer | grep squirrel) -KEY_3_MSGS=$(read_messages_from_kafka 4 test-output bee_consumer | grep bee) -KEY_4_MSGS=$(read_messages_from_kafka 10 test-output giraffe_consumer | grep giraffe) - -verify_output "elephant,27,64213" "$KEY_1_MSGS" -verify_output "squirrel,52,66413" "$KEY_2_MSGS" -verify_output "bee,18,65647" "$KEY_3_MSGS" -verify_output "giraffe,9,65555" "$KEY_4_MSGS" From 671029d203318f23b64b199e8f6565be02f8ba6a Mon Sep 17 00:00:00 2001 From: huzheng Date: Thu, 14 Nov 2019 13:57:15 +0100 Subject: [PATCH 662/746] [FLINK-11463][tests] Modify logging - log process output on debug by default - log created processes with their arguments --- .../org/apache/flink/tests/util/AutoClosableProcess.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosableProcess.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosableProcess.java index 533ffd01bc25..2d08bd85f906 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosableProcess.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/AutoClosableProcess.java @@ -31,6 +31,7 @@ import java.io.StringWriter; import java.nio.charset.StandardCharsets; import java.time.Duration; +import java.util.Arrays; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.function.Consumer; @@ -70,10 +71,8 @@ public static AutoClosableProcessBuilder create(String... commands) { */ public static final class AutoClosableProcessBuilder { private final String[] commands; - private Consumer stdoutProcessor = line -> { - }; - private Consumer stderrProcessor = line -> { - }; + private Consumer stdoutProcessor = LOG::debug; + private Consumer stderrProcessor = LOG::debug; AutoClosableProcessBuilder(final String... commands) { this.commands = commands; @@ -123,6 +122,7 @@ public AutoClosableProcess runNonBlocking() throws IOException { private static Process createProcess(final String[] commands, Consumer stdoutProcessor, Consumer stderrProcessor) throws IOException { final ProcessBuilder processBuilder = new ProcessBuilder(); + LOG.debug("Creating process: {}", Arrays.toString(commands)); processBuilder.command(commands); final Process process = processBuilder.start(); From e6fc1c6b98be89ded846f42f65a946330f40e54e Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sat, 16 Nov 2019 14:36:48 +0800 Subject: [PATCH 663/746] [FLINK-14801][sql-parser] Improve the local variable name in SqlCreateTable#unparser() Modify `withFrame` to `partitionedByFrame` to make the field name more specifically. This closes #10229 --- .../java/org/apache/flink/sql/parser/ddl/SqlCreateTable.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 f542af1f92ab..c4c7f1199b80 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 @@ -291,9 +291,9 @@ public void unparse( if (this.partitionKeyList.size() > 0) { writer.newlineAndIndent(); writer.keyword("PARTITIONED BY"); - SqlWriter.Frame withFrame = writer.startList("(", ")"); + SqlWriter.Frame partitionedByFrame = writer.startList("(", ")"); this.partitionKeyList.unparse(writer, leftPrec, rightPrec); - writer.endList(withFrame); + writer.endList(partitionedByFrame); writer.newlineAndIndent(); } From b390e216b6b78ed02961ef10e5b5ab1b3f5fc235 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Sun, 17 Nov 2019 11:58:19 +0800 Subject: [PATCH 664/746] [docs-sync] Synchronize the latest documentation changes (commits to fdc2555c) into Chinese documents --- .../stream/operators/process_function.zh.md | 3 +- .../dev/table/streaming/temporal_tables.zh.md | 4 +-- docs/ops/deployment/kubernetes.zh.md | 28 +++++++++++++------ docs/ops/state/state_backends.zh.md | 7 ++++- docs/tutorials/python_table_api.zh.md | 10 +++---- 5 files changed, 34 insertions(+), 18 deletions(-) diff --git a/docs/dev/stream/operators/process_function.zh.md b/docs/dev/stream/operators/process_function.zh.md index 41c339c30fe6..e68e8fbbba5c 100644 --- a/docs/dev/stream/operators/process_function.zh.md +++ b/docs/dev/stream/operators/process_function.zh.md @@ -44,8 +44,7 @@ For fault-tolerant state, the `ProcessFunction` gives access to Flink's [keyed s The timers allow applications to react to changes in processing time and in [event time]({{ site.baseurl }}/dev/event_time.html). Every call to the function `processElement(...)` gets a `Context` object which gives access to the element's event time timestamp, and to the *TimerService*. The `TimerService` can be used to register callbacks for future -event-/processing-time instants. When a timer's particular time is reached, the `onTimer(...)` method is -called. During that call, all states are again scoped to the key with which the timer was created, allowing +event-/processing-time instants. With event-time timers, the `onTimer(...)` method is called when the current watermark is advanced up to or beyond the timestamp of the timer, while with processing-time timers, `onTimer(...)` is called when wall clock time reaches the specified time. During that call, all states are again scoped to the key with which the timer was created, allowing timers to manipulate keyed state. Note If you want to access keyed state and timers you have diff --git a/docs/dev/table/streaming/temporal_tables.zh.md b/docs/dev/table/streaming/temporal_tables.zh.md index baecbd07d553..bc7addd345cc 100644 --- a/docs/dev/table/streaming/temporal_tables.zh.md +++ b/docs/dev/table/streaming/temporal_tables.zh.md @@ -178,7 +178,7 @@ ratesHistoryData.add(Tuple2.of("Euro", 119L)); DataStream> ratesHistoryStream = env.fromCollection(ratesHistoryData); Table ratesHistory = tEnv.fromDataStream(ratesHistoryStream, "r_currency, r_rate, r_proctime.proctime"); -tEnv.registerTable("RatesHistory", ratesHistory); +tEnv.createTemporaryView("RatesHistory", ratesHistory); // Create and register a temporal table function. // Define "r_proctime" as the time attribute and "r_currency" as the primary key. @@ -206,7 +206,7 @@ val ratesHistory = env .fromCollection(ratesHistoryData) .toTable(tEnv, 'r_currency, 'r_rate, 'r_proctime.proctime) -tEnv.registerTable("RatesHistory", ratesHistory) +tEnv.createTemporaryView("RatesHistory", ratesHistory) // Create and register TemporalTableFunction. // Define "r_proctime" as the time attribute and "r_currency" as the primary key. diff --git a/docs/ops/deployment/kubernetes.zh.md b/docs/ops/deployment/kubernetes.zh.md index e86466bcc1ec..2cc18f844245 100644 --- a/docs/ops/deployment/kubernetes.zh.md +++ b/docs/ops/deployment/kubernetes.zh.md @@ -34,13 +34,13 @@ Please follow [Kubernetes' setup guide](https://kubernetes.io/docs/setup/) in or If you want to run Kubernetes locally, we recommend using [MiniKube](https://kubernetes.io/docs/setup/minikube/).

    - Note: If using MiniKube please make sure to execute `minikube ssh 'sudo ip link set docker0 promisc on'` before deploying a Flink cluster. - Otherwise Flink components are not able to self reference themselves through a Kubernetes service. + Note: If using MiniKube please make sure to execute `minikube ssh 'sudo ip link set docker0 promisc on'` before deploying a Flink cluster. + Otherwise Flink components are not able to self reference themselves through a Kubernetes service.
    ## Flink session cluster on Kubernetes -A Flink session cluster is executed as a long-running Kubernetes Deployment. +A Flink session cluster is executed as a long-running Kubernetes Deployment. Note that you can run multiple Flink jobs on a session cluster. Each job needs to be submitted to the cluster after the cluster has been deployed. @@ -89,15 +89,15 @@ In order to terminate the Flink session cluster, use `kubectl`: ## Flink job cluster on Kubernetes -A Flink job cluster is a dedicated cluster which runs a single job. -The job is part of the image and, thus, there is no extra job submission needed. +A Flink job cluster is a dedicated cluster which runs a single job. +The job is part of the image and, thus, there is no extra job submission needed. ### Creating the job-specific image The Flink job cluster image needs to contain the user code jars of the job for which the cluster is started. Therefore, one needs to build a dedicated container image for every job. Please follow these [instructions](https://github.com/apache/flink/blob/{{ site.github_branch }}/flink-container/docker/README.md) to build the Docker image. - + ### Deploy Flink job cluster on Kubernetes In order to deploy the a job cluster on Kubernetes please follow these [instructions](https://github.com/apache/flink/blob/{{ site.github_branch }}/flink-container/kubernetes/README.md#deploy-flink-job-cluster). @@ -145,12 +145,16 @@ data: `jobmanager-deployment.yaml` {% highlight yaml %} -apiVersion: extensions/v1beta1 +apiVersion: apps/v1 kind: Deployment metadata: name: flink-jobmanager spec: replicas: 1 + selector: + matchLabels: + app: flink + component: jobmanager template: metadata: labels: @@ -183,6 +187,8 @@ spec: volumeMounts: - name: flink-config-volume mountPath: /opt/flink/conf + securityContext: + runAsUser: 9999 # refers to user _flink_ from official flink image, change if necessary volumes: - name: flink-config-volume configMap: @@ -196,12 +202,16 @@ spec: `taskmanager-deployment.yaml` {% highlight yaml %} -apiVersion: extensions/v1beta1 +apiVersion: apps/v1 kind: Deployment metadata: name: flink-taskmanager spec: replicas: 2 + selector: + matchLabels: + app: flink + component: taskmanager template: metadata: labels: @@ -230,6 +240,8 @@ spec: volumeMounts: - name: flink-config-volume mountPath: /opt/flink/conf/ + securityContext: + runAsUser: 9999 # refers to user _flink_ from official flink image, change if necessary volumes: - name: flink-config-volume configMap: diff --git a/docs/ops/state/state_backends.zh.md b/docs/ops/state/state_backends.zh.md index 0dc155d70f30..40c1472de2cc 100644 --- a/docs/ops/state/state_backends.zh.md +++ b/docs/ops/state/state_backends.zh.md @@ -143,16 +143,21 @@ env.setStateBackend(new FsStateBackend("hdfs://namenode:40010/flink/checkpoints"
    -如果想使用 `RocksDBStateBackend`,必须添加以下依赖到 Flink 项目中。 +如果你想在 IDE 中使用`RocksDBStateBackend`,或者需要在作业中编程方式动态配置`RocksDBStateBackend`,必须添加以下依赖到 Flink 项目中。 {% highlight xml %} org.apache.flink flink-statebackend-rocksdb{{ site.scala_version_suffix }} {{ site.version }} + provided {% endhighlight %} +
    + 注意: 由于 RocksDB 是 Flink 默认分发包的一部分,所以如果你没在代码中使用 RocksDB,那就不需要依赖它。而且可以在你的`flink-conf.yaml`文件中通过`state.backend`配置 state backend,以及更多的 [checkpointing]({{ site.baseurl }}/ops/config.html#checkpointing) 和 [RocksDB 特定的]({{ site.baseurl }}/ops/config.html#rocksdb-state-backend) 参数。 +
    + ### 设置默认的(全局的) State Backend diff --git a/docs/tutorials/python_table_api.zh.md b/docs/tutorials/python_table_api.zh.md index 6ba3b37e2106..62bdc03c4d34 100644 --- a/docs/tutorials/python_table_api.zh.md +++ b/docs/tutorials/python_table_api.zh.md @@ -59,7 +59,7 @@ t_env.connect(FileSystem().path('/tmp/input')) \ .field('word', DataTypes.STRING())) \ .with_schema(Schema() .field('word', DataTypes.STRING())) \ - .register_table_source('mySource') + .create_temporary_table('mySource') t_env.connect(FileSystem().path('/tmp/output')) \ .with_format(OldCsv() @@ -69,7 +69,7 @@ t_env.connect(FileSystem().path('/tmp/output')) \ .with_schema(Schema() .field('word', DataTypes.STRING()) .field('count', DataTypes.BIGINT())) \ - .register_table_sink('mySink') + .create_temporary_table('mySink') {% endhighlight %} 上面的程序展示了如何创建及在`ExecutionEnvironment`中注册表名分别为`mySource`和`mySink`的表。 @@ -111,7 +111,7 @@ t_env.connect(FileSystem().path('/tmp/input')) \ .field('word', DataTypes.STRING())) \ .with_schema(Schema() .field('word', DataTypes.STRING())) \ - .register_table_source('mySource') + .create_temporary_table('mySource') t_env.connect(FileSystem().path('/tmp/output')) \ .with_format(OldCsv() @@ -121,9 +121,9 @@ t_env.connect(FileSystem().path('/tmp/output')) \ .with_schema(Schema() .field('word', DataTypes.STRING()) .field('count', DataTypes.BIGINT())) \ - .register_table_sink('mySink') + .create_temporary_table('mySink') -t_env.scan('mySource') \ +t_env.from_path('mySource') \ .group_by('word') \ .select('word, count(1)') \ .insert_into('mySink') From e0824a0e4b7ad724ac9784b541c67fa2278570f3 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Mon, 28 Oct 2019 11:12:56 +0100 Subject: [PATCH 665/746] [hotfix][runtime] Refactor the method of AvailabilityProvider#isAvailable There are two main changes for this refactor: 1. Rename AvailabilityProvider#isAvailable() to AvailabilityProvider#getAvailableFuture() for more suitable with the return type of CompleteFuture. 2. Provide default implementations of #isAvailable() and #isApproximatelyAvailable() for AvailabilityProvider based on two considerations: - Concentrate on the concerns of performance and visibility in these two methods to provide proper usages and implementations. - Avoid reimplementing these logics in existing/new usages, because it is not easy to handle them properly. And if we want to improve the way of CompleteFuture in future, it is easy to only focus on these concentrated methods. --- .../runtime/io/AvailabilityProvider.java | 67 +++++++++++-------- .../runtime/io/PullingAsyncDataInput.java | 4 +- .../io/network/api/writer/RecordWriter.java | 4 +- .../api/writer/ResultPartitionWriter.java | 3 +- .../io/network/buffer/LocalBufferPool.java | 24 +++---- .../io/network/buffer/NetworkBufferPool.java | 4 +- .../io/network/partition/ResultPartition.java | 4 +- .../network/partition/consumer/InputGate.java | 4 +- .../partition/consumer/UnionInputGate.java | 4 +- ...tifyingResultPartitionWriterDecorator.java | 4 +- .../taskmanager/InputGateWithMetrics.java | 4 +- .../flink/runtime/taskmanager/Task.java | 2 +- ...stractCollectingResultPartitionWriter.java | 2 +- .../network/api/writer/RecordWriterTest.java | 12 ++-- .../network/buffer/LocalBufferPoolTest.java | 14 ++-- .../network/buffer/NetworkBufferPoolTest.java | 40 +++++------ .../io/network/buffer/NoOpBufferPool.java | 2 +- .../partition/consumer/InputGateTestBase.java | 20 +++--- .../util/TestPooledBufferProvider.java | 2 +- .../runtime/io/CheckpointedInputGate.java | 4 +- .../runtime/io/StreamInputProcessor.java | 2 +- .../runtime/io/StreamOneInputProcessor.java | 4 +- .../runtime/io/StreamTaskNetworkInput.java | 6 +- .../runtime/io/StreamTaskSourceInput.java | 4 +- .../runtime/io/StreamTwoInputProcessor.java | 23 +++---- .../streaming/runtime/tasks/StreamTask.java | 2 +- .../io/StreamTaskNetworkInputTest.java | 2 +- .../tasks/SourceReaderStreamTaskTest.java | 4 +- .../runtime/tasks/StreamTaskTest.java | 2 +- 29 files changed, 136 insertions(+), 137 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 8ab59b3e5ef6..4783da9ec591 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 @@ -29,30 +29,42 @@ public interface AvailabilityProvider { /** * Constant that allows to avoid volatile checks {@link CompletableFuture#isDone()}. Check - * {@link #isAvailable()} for more explanation. + * {@link #isAvailable()} and {@link #isApproximatelyAvailable()} for more explanation. */ CompletableFuture AVAILABLE = CompletableFuture.completedFuture(null); /** - * Check if this instance is available for further processing. + * @return a future that is completed if the respective provider is available. + */ + CompletableFuture getAvailableFuture(); + + /** + * In order to best-effort avoid volatile access in {@link CompletableFuture#isDone()}, we check the condition + * of future == AVAILABLE firstly for getting probable performance benefits while hot looping. + * + *

    It is always safe to use this method in performance nonsensitive scenarios to get the precise state. + * + * @return true if this instance is available for further processing. + */ + default boolean isAvailable() { + CompletableFuture future = getAvailableFuture(); + return future == AVAILABLE || future.isDone(); + } + + /** + * Checks whether this instance is available only via constant {@link #AVAILABLE} to avoid + * performance concern caused by volatile access in {@link CompletableFuture#isDone()}. So it is + * mainly used in the performance sensitive scenarios which do not always need the precise state. * - *

    When hot looping to avoid volatile access in {@link CompletableFuture#isDone()} user of - * this method should do the following check: - *

    -	 * {@code
    -	 *	AvailabilityProvider input = ...;
    -	 *	if (input.isAvailable() == AvailabilityProvider.AVAILABLE || input.isAvailable().isDone()) {
    -	 *		// do something;
    -	 *	}
    -	 * }
    -	 * 
    + *

    This method is still safe to get the precise state if {@link #getAvailableFuture()} + * was touched via (.get(), .wait(), .isDone(), ...) before, which also has a "happen-before" + * relationship with this call. * - * @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 is finished. + * @return true if this instance is available for further processing. */ - CompletableFuture isAvailable(); + default boolean isApproximatelyAvailable() { + return getAvailableFuture() == AVAILABLE; + } /** * A availability implementation for providing the helpful functions of resetting the @@ -60,15 +72,14 @@ public interface AvailabilityProvider { */ final class AvailabilityHelper implements AvailabilityProvider { - private CompletableFuture isAvailable = new CompletableFuture<>(); + private CompletableFuture availableFuture = new CompletableFuture<>(); /** * Judges to reset the current available state as unavailable. */ public void resetUnavailable() { - // try to avoid volatile access in isDone()} - if (isAvailable == AVAILABLE || isAvailable.isDone()) { - isAvailable = new CompletableFuture<>(); + if (isAvailable()) { + availableFuture = new CompletableFuture<>(); } } @@ -76,7 +87,7 @@ public void resetUnavailable() { * Resets the constant completed {@link #AVAILABLE} as the current state. */ public void resetAvailable() { - isAvailable = AVAILABLE; + availableFuture = AVAILABLE; } /** @@ -84,8 +95,8 @@ public void resetAvailable() { * {@link #AVAILABLE} as the current state. */ public CompletableFuture getUnavailableToResetAvailable() { - CompletableFuture toNotify = isAvailable; - isAvailable = AVAILABLE; + CompletableFuture toNotify = availableFuture; + availableFuture = AVAILABLE; return toNotify; } @@ -94,8 +105,8 @@ public CompletableFuture getUnavailableToResetAvailable() { * previous uncompleted one. */ public CompletableFuture getUnavailableToResetUnavailable() { - CompletableFuture toNotify = isAvailable; - isAvailable = new CompletableFuture<>(); + CompletableFuture toNotify = availableFuture; + availableFuture = new CompletableFuture<>(); return toNotify; } @@ -103,8 +114,8 @@ public CompletableFuture getUnavailableToResetUnavailable() { * @return a future that is completed if the respective provider is available. */ @Override - public CompletableFuture isAvailable() { - return isAvailable; + public CompletableFuture getAvailableFuture() { + return availableFuture; } } } 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 17109ccebb0d..c5c4ef958a29 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 @@ -27,7 +27,7 @@ * *

    For the most efficient usage, user of this class is supposed to call {@link #pollNext()} * until it returns that no more elements are available. If that happens, he should check if - * input {@link #isFinished()}. If not, he should wait for {@link #isAvailable()} + * input {@link #isFinished()}. If not, he should wait for {@link #getAvailableFuture()} * {@link CompletableFuture} to be completed. For example: * *

    @@ -44,7 +44,7 @@
      *			// do something with next
      *		}
      *
    - *		input.isAvailable().get();
    + *		input.getAvailableFuture().get();
      *	}
      * }
      * 
    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 9264f0fd24df..6c680f5981c9 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 @@ -190,8 +190,8 @@ protected void finishBufferBuilder(BufferBuilder bufferBuilder) { } @Override - public CompletableFuture isAvailable() { - return targetPartition.isAvailable(); + public CompletableFuture getAvailableFuture() { + return targetPartition.getAvailableFuture(); } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java index cedf4de3487c..b7325abf81ae 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java @@ -99,6 +99,5 @@ public interface ResultPartitionWriter extends AutoCloseable { * * @return a future that is completed if it is available for output. */ - - CompletableFuture isAvailable(); + CompletableFuture getAvailableFuture(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java index 72f98c0959e5..d7aef4c30c32 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java @@ -228,7 +228,7 @@ private MemorySegment requestMemorySegmentBlocking() throws InterruptedException while ((segment = requestMemorySegment()) == null) { try { // wait until available - isAvailable().get(); + getAvailableFuture().get(); } catch (ExecutionException e) { LOG.error("The available future is completed exceptionally.", e); ExceptionUtils.rethrow(e); @@ -345,9 +345,8 @@ public void lazyDestroy() { listener.notifyBufferDestroyed(); } - final CompletableFuture isAvailable = availabilityHelper.isAvailable(); - if (!isAvailable(isAvailable)) { - toNotify = isAvailable; + if (!isAvailable()) { + toNotify = availabilityHelper.getAvailableFuture(); } isDestroyed = true; @@ -393,10 +392,7 @@ public void setNumBuffers(int numBuffers) throws IOException { returnExcessMemorySegments(); numExcessBuffers = numberOfRequestedMemorySegments - currentPoolSize; - - if (numExcessBuffers < 0 - && availableMemorySegments.isEmpty() - && isAvailable(networkBufferPool.isAvailable())) { + if (numExcessBuffers < 0 && availableMemorySegments.isEmpty() && networkBufferPool.isAvailable()) { toNotify = availabilityHelper.getUnavailableToResetUnavailable(); } } @@ -411,13 +407,13 @@ && isAvailable(networkBufferPool.isAvailable())) { } @Override - public CompletableFuture isAvailable() { + public CompletableFuture getAvailableFuture() { if (numberOfRequestedMemorySegments >= currentPoolSize) { - return availabilityHelper.isAvailable(); - } else if (availabilityHelper.isAvailable() == AVAILABLE || networkBufferPool.isAvailable() == AVAILABLE) { + return availabilityHelper.getAvailableFuture(); + } else if (availabilityHelper.isApproximatelyAvailable() || networkBufferPool.isApproximatelyAvailable()) { return AVAILABLE; } else { - return CompletableFuture.anyOf(availabilityHelper.isAvailable(), networkBufferPool.isAvailable()); + return CompletableFuture.anyOf(availabilityHelper.getAvailableFuture(), networkBufferPool.getAvailableFuture()); } } @@ -443,10 +439,6 @@ private void mayNotifyAvailable(@Nullable CompletableFuture toNotify) { } } - private boolean isAvailable(CompletableFuture isAvailable) { - return isAvailable == AVAILABLE || isAvailable.isDone(); - } - private void returnMemorySegment(MemorySegment segment) { assert Thread.holdsLock(availableMemorySegments); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java index 19a312539c10..85a326293f71 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java @@ -304,8 +304,8 @@ public int countBuffers() { * in this pool. */ @Override - public CompletableFuture isAvailable() { - return availabilityHelper.isAvailable(); + public CompletableFuture getAvailableFuture() { + return availabilityHelper.getAvailableFuture(); } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java index 26238b93dc16..d2ff92441a40 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java @@ -324,8 +324,8 @@ public boolean isReleased() { } @Override - public CompletableFuture isAvailable() { - return bufferPool.isAvailable(); + public CompletableFuture getAvailableFuture() { + return bufferPool.getAvailableFuture(); } @Override 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 070c848cf12f..0b637f294f17 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 @@ -99,8 +99,8 @@ public abstract class InputGate implements PullingAsyncDataInput, * not completed futures should become completed once there are more records available. */ @Override - public CompletableFuture isAvailable() { - return availabilityHelper.isAvailable(); + public CompletableFuture getAvailableFuture() { + return availabilityHelper.getAvailableFuture(); } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java index 9a023eaeb5a4..02a32e8bb647 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java @@ -108,7 +108,7 @@ public UnionInputGate(InputGate... inputGates) { currentNumberOfInputChannels += inputGate.getNumberOfInputChannels(); - CompletableFuture available = inputGate.isAvailable(); + CompletableFuture available = inputGate.getAvailableFuture(); if (available.isDone()) { inputGatesWithData.add(inputGate); @@ -185,7 +185,7 @@ private Optional> waitAndGetNextData(boo // enqueue the inputGate at the end to avoid starvation inputGatesWithData.add(inputGate.get()); } else if (!inputGate.get().isFinished()) { - inputGate.get().isAvailable().thenRun(() -> queueInputGate(inputGate.get())); + inputGate.get().getAvailableFuture().thenRun(() -> queueInputGate(inputGate.get())); } if (inputGatesWithData.isEmpty()) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ConsumableNotifyingResultPartitionWriterDecorator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ConsumableNotifyingResultPartitionWriterDecorator.java index d606a10d0ef4..8b1d97d7c912 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ConsumableNotifyingResultPartitionWriterDecorator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ConsumableNotifyingResultPartitionWriterDecorator.java @@ -121,8 +121,8 @@ public void fail(Throwable throwable) { } @Override - public CompletableFuture isAvailable() { - return partitionWriter.isAvailable(); + public CompletableFuture getAvailableFuture() { + return partitionWriter.getAvailableFuture(); } @Override 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 26ba5de036cb..337f90494864 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 @@ -46,8 +46,8 @@ public InputGateWithMetrics(InputGate inputGate, Counter numBytesIn) { } @Override - public CompletableFuture isAvailable() { - return inputGate.isAvailable(); + public CompletableFuture getAvailableFuture() { + return inputGate.getAvailableFuture(); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index 7848171ab15e..b32ab854f473 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -469,7 +469,7 @@ public boolean isBackPressured() { } final CompletableFuture[] outputFutures = new CompletableFuture[consumableNotifyingPartitionWriters.length]; for (int i = 0; i < outputFutures.length; ++i) { - outputFutures[i] = consumableNotifyingPartitionWriters[i].isAvailable(); + outputFutures[i] = consumableNotifyingPartitionWriters[i].getAvailableFuture(); } return !CompletableFuture.allOf(outputFutures).isDone(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/AbstractCollectingResultPartitionWriter.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/AbstractCollectingResultPartitionWriter.java index f4e6f95bf731..f83b6d79e166 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/AbstractCollectingResultPartitionWriter.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/AbstractCollectingResultPartitionWriter.java @@ -125,7 +125,7 @@ public void finish() { } @Override - public CompletableFuture isAvailable() { + public CompletableFuture getAvailableFuture() { return AvailabilityProvider.AVAILABLE; } 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 0b3dfdfb7d6d..cb29fb004ff6 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 @@ -450,18 +450,18 @@ public void testIsAvailableOrNot() throws Exception { try { // record writer is available because of initial available global pool - assertTrue(recordWriter.isAvailable().isDone()); + assertTrue(recordWriter.getAvailableFuture().isDone()); // request one buffer from the local pool to make it unavailable afterwards final BufferBuilder bufferBuilder = resultPartition.getBufferBuilder(); assertNotNull(bufferBuilder); - assertFalse(recordWriter.isAvailable().isDone()); + assertFalse(recordWriter.getAvailableFuture().isDone()); // recycle the buffer to make the local pool available again final Buffer buffer = BufferBuilderTestUtils.buildSingleBuffer(bufferBuilder); buffer.recycleBuffer(); - assertTrue(recordWriter.isAvailable().isDone()); - assertEquals(recordWriter.AVAILABLE, recordWriter.isAvailable()); + assertTrue(recordWriter.getAvailableFuture().isDone()); + assertEquals(recordWriter.AVAILABLE, recordWriter.getAvailableFuture()); } finally { localPool.lazyDestroy(); globalPool.destroy(); @@ -595,7 +595,7 @@ public void finish() { } @Override - public CompletableFuture isAvailable() { + public CompletableFuture getAvailableFuture() { return AVAILABLE; } @@ -676,7 +676,7 @@ public void finish() { } @Override - public CompletableFuture isAvailable() { + public CompletableFuture getAvailableFuture() { return AVAILABLE; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java index f22913b707d7..a0d1612f1376 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java @@ -410,39 +410,39 @@ public void testBoundedBuffer() throws Exception { public void testIsAvailableOrNot() throws Exception { // the local buffer pool should be in available state initially - assertTrue(localBufferPool.isAvailable().isDone()); + assertTrue(localBufferPool.getAvailableFuture().isDone()); // request one buffer final BufferBuilder bufferBuilder = checkNotNull(localBufferPool.requestBufferBuilderBlocking()); - CompletableFuture availableFuture = localBufferPool.isAvailable(); + CompletableFuture availableFuture = localBufferPool.getAvailableFuture(); assertFalse(availableFuture.isDone()); // set the pool size localBufferPool.setNumBuffers(2); assertTrue(availableFuture.isDone()); - assertTrue(localBufferPool.isAvailable().isDone()); + assertTrue(localBufferPool.getAvailableFuture().isDone()); // drain the global buffer pool final List segments = new ArrayList<>(numBuffers); while (networkBufferPool.getNumberOfAvailableMemorySegments() > 0) { segments.add(checkNotNull(networkBufferPool.requestMemorySegment())); } - assertFalse(localBufferPool.isAvailable().isDone()); + assertFalse(localBufferPool.getAvailableFuture().isDone()); // recycle the requested segments to global buffer pool for (final MemorySegment segment: segments) { networkBufferPool.recycle(segment); } - assertTrue(localBufferPool.isAvailable().isDone()); + assertTrue(localBufferPool.getAvailableFuture().isDone()); // reset the pool size localBufferPool.setNumBuffers(1); - availableFuture = localBufferPool.isAvailable(); + availableFuture = localBufferPool.getAvailableFuture(); assertFalse(availableFuture.isDone()); // recycle the requested buffer bufferBuilder.createBufferConsumer().close(); - assertTrue(localBufferPool.isAvailable().isDone()); + assertTrue(localBufferPool.getAvailableFuture().isDone()); assertTrue(availableFuture.isDone()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java index bd5339a1721e..7bdb88418c05 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java @@ -541,26 +541,26 @@ public void testIsAvailableOrNotAfterRequestAndRecycleSingleSegment() throws Exc try { // the global pool should be in available state initially - assertTrue(globalPool.isAvailable().isDone()); + assertTrue(globalPool.getAvailableFuture().isDone()); // request the first segment final MemorySegment segment1 = checkNotNull(globalPool.requestMemorySegment()); - assertTrue(globalPool.isAvailable().isDone()); + assertTrue(globalPool.getAvailableFuture().isDone()); // request the second segment final MemorySegment segment2 = checkNotNull(globalPool.requestMemorySegment()); - assertFalse(globalPool.isAvailable().isDone()); + assertFalse(globalPool.getAvailableFuture().isDone()); - final CompletableFuture availableFuture = globalPool.isAvailable(); + final CompletableFuture availableFuture = globalPool.getAvailableFuture(); // recycle the first segment globalPool.recycle(segment1); assertTrue(availableFuture.isDone()); - assertTrue(globalPool.isAvailable().isDone()); + assertTrue(globalPool.getAvailableFuture().isDone()); // recycle the second segment globalPool.recycle(segment2); - assertTrue(globalPool.isAvailable().isDone()); + assertTrue(globalPool.getAvailableFuture().isDone()); } finally { globalPool.destroy(); @@ -581,16 +581,16 @@ public void testIsAvailableOrNotAfterRequestAndRecycleMultiSegments() throws Exc try { // the global pool should be in available state initially - assertTrue(globalPool.isAvailable().isDone()); + assertTrue(globalPool.getAvailableFuture().isDone()); // request 5 segments List segments1 = globalPool.requestMemorySegments(); - assertTrue(globalPool.isAvailable().isDone()); + assertTrue(globalPool.getAvailableFuture().isDone()); assertEquals(numberOfSegmentsToRequest, segments1.size()); // request another 5 segments List segments2 = globalPool.requestMemorySegments(); - assertFalse(globalPool.isAvailable().isDone()); + assertFalse(globalPool.getAvailableFuture().isDone()); assertEquals(numberOfSegmentsToRequest, segments2.size()); // request another 5 segments @@ -607,22 +607,22 @@ public void go() throws Exception { asyncRequest.start(); // recycle 5 segments - CompletableFuture availableFuture = globalPool.isAvailable(); + CompletableFuture availableFuture = globalPool.getAvailableFuture(); globalPool.recycleMemorySegments(segments1); assertTrue(availableFuture.isDone()); // wait util the third request is fulfilled latch.await(); - assertFalse(globalPool.isAvailable().isDone()); + assertFalse(globalPool.getAvailableFuture().isDone()); assertEquals(numberOfSegmentsToRequest, segments3.size()); // recycle another 5 segments globalPool.recycleMemorySegments(segments2); - assertTrue(globalPool.isAvailable().isDone()); + assertTrue(globalPool.getAvailableFuture().isDone()); // recycle the last 5 segments globalPool.recycleMemorySegments(segments3); - assertTrue(globalPool.isAvailable().isDone()); + assertTrue(globalPool.getAvailableFuture().isDone()); } finally { globalPool.destroy(); @@ -650,7 +650,7 @@ public void testBlockingRequestFromMultiLocalBufferPool() throws Exception { for (int i = 0; i < numLocalBufferPool; ++i) { final BufferPool localPool = globalPool.createBufferPool(localPoolRequiredSize, localPoolMaxSize); localBufferPools.add(localPool); - assertTrue(localPool.isAvailable().isDone()); + assertTrue(localPool.getAvailableFuture().isDone()); } // request some segments from the global pool in two different ways @@ -659,9 +659,9 @@ public void testBlockingRequestFromMultiLocalBufferPool() throws Exception { segments.add(globalPool.requestMemorySegment()); } final List exclusiveSegments = globalPool.requestMemorySegments(); - assertTrue(globalPool.isAvailable().isDone()); + assertTrue(globalPool.getAvailableFuture().isDone()); for (final BufferPool localPool: localBufferPools) { - assertTrue(localPool.isAvailable().isDone()); + assertTrue(localPool.getAvailableFuture().isDone()); } // blocking request buffers form local buffer pools @@ -688,12 +688,12 @@ public void testBlockingRequestFromMultiLocalBufferPool() throws Exception { assertNull(cause.get()); } - final CompletableFuture globalPoolAvailableFuture = globalPool.isAvailable(); + final CompletableFuture globalPoolAvailableFuture = globalPool.getAvailableFuture(); assertFalse(globalPoolAvailableFuture.isDone()); final List> localPoolAvailableFutures = new ArrayList<>(numLocalBufferPool); for (BufferPool localPool: localBufferPools) { - CompletableFuture localPoolAvailableFuture = localPool.isAvailable(); + CompletableFuture localPoolAvailableFuture = localPool.getAvailableFuture(); localPoolAvailableFutures.add(localPoolAvailableFuture); assertFalse(localPoolAvailableFuture.isDone()); } @@ -714,9 +714,9 @@ public void testBlockingRequestFromMultiLocalBufferPool() throws Exception { assertNull(cause.get()); assertEquals(0, globalPool.getNumberOfAvailableMemorySegments()); - assertFalse(globalPool.isAvailable().isDone()); + assertFalse(globalPool.getAvailableFuture().isDone()); for (BufferPool localPool: localBufferPools) { - assertFalse(localPool.isAvailable().isDone()); + assertFalse(localPool.getAvailableFuture().isDone()); assertEquals(localPoolMaxSize, localPool.bestEffortGetNumOfUsedBuffers()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NoOpBufferPool.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NoOpBufferPool.java index 05d56e193975..b8a866c4accd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NoOpBufferPool.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NoOpBufferPool.java @@ -90,7 +90,7 @@ public void recycle(MemorySegment memorySegment) { } @Override - public CompletableFuture isAvailable() { + public CompletableFuture getAvailableFuture() { return AVAILABLE; } } 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 4069ec5faf8f..f4abf5a56a8b 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 @@ -54,29 +54,29 @@ protected void testIsAvailable( SingleInputGate inputGateToNotify, TestInputChannel inputChannelWithNewData) throws Exception { - assertFalse(inputGateToTest.isAvailable().isDone()); + assertFalse(inputGateToTest.getAvailableFuture().isDone()); assertFalse(inputGateToTest.pollNext().isPresent()); - CompletableFuture isAvailable = inputGateToTest.isAvailable(); + CompletableFuture future = inputGateToTest.getAvailableFuture(); - assertFalse(inputGateToTest.isAvailable().isDone()); + assertFalse(inputGateToTest.getAvailableFuture().isDone()); assertFalse(inputGateToTest.pollNext().isPresent()); - assertEquals(isAvailable, inputGateToTest.isAvailable()); + assertEquals(future, inputGateToTest.getAvailableFuture()); inputChannelWithNewData.readBuffer(); inputGateToNotify.notifyChannelNonEmpty(inputChannelWithNewData); - assertTrue(isAvailable.isDone()); - assertTrue(inputGateToTest.isAvailable().isDone()); - assertEquals(PullingAsyncDataInput.AVAILABLE, inputGateToTest.isAvailable()); + assertTrue(future.isDone()); + assertTrue(inputGateToTest.getAvailableFuture().isDone()); + assertEquals(PullingAsyncDataInput.AVAILABLE, inputGateToTest.getAvailableFuture()); } protected void testIsAvailableAfterFinished( InputGate inputGateToTest, Runnable endOfPartitionEvent) throws Exception { - CompletableFuture available = inputGateToTest.isAvailable(); + CompletableFuture available = inputGateToTest.getAvailableFuture(); assertFalse(available.isDone()); assertFalse(inputGateToTest.pollNext().isPresent()); @@ -85,8 +85,8 @@ protected void testIsAvailableAfterFinished( assertTrue(inputGateToTest.pollNext().isPresent()); // EndOfPartitionEvent assertTrue(available.isDone()); - assertTrue(inputGateToTest.isAvailable().isDone()); - assertEquals(PullingAsyncDataInput.AVAILABLE, inputGateToTest.isAvailable()); + assertTrue(inputGateToTest.getAvailableFuture().isDone()); + assertEquals(PullingAsyncDataInput.AVAILABLE, inputGateToTest.getAvailableFuture()); } protected SingleInputGate createInputGate() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPooledBufferProvider.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPooledBufferProvider.java index 4bb26f96e86f..d7c94398ef26 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPooledBufferProvider.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPooledBufferProvider.java @@ -97,7 +97,7 @@ public boolean isDestroyed() { } @Override - public CompletableFuture isAvailable() { + public CompletableFuture getAvailableFuture() { return AVAILABLE; } 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 63dd16b26753..317cd84b94c9 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 @@ -109,9 +109,9 @@ public CheckpointedInputGate( } @Override - public CompletableFuture isAvailable() { + public CompletableFuture getAvailableFuture() { if (bufferStorage.isEmpty()) { - return inputGate.isAvailable(); + return inputGate.getAvailableFuture(); } return AVAILABLE; } 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 3c070e28eff5..3c0623929578 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 @@ -31,7 +31,7 @@ public interface StreamInputProcessor extends AvailabilityProvider, Closeable { /** * @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()}. + * state and/or {@link #getAvailableFuture()}. */ 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 396355e28671..e14a09aa76dd 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 @@ -60,8 +60,8 @@ public StreamOneInputProcessor( } @Override - public CompletableFuture isAvailable() { - return input.isAvailable(); + public CompletableFuture getAvailableFuture() { + return input.getAvailableFuture(); } @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 ecb26ac061c2..196f993936c4 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 @@ -135,7 +135,7 @@ public InputStatus emitNext(DataOutput output) throws Exception { processBufferOrEvent(bufferOrEvent.get()); } else { if (checkpointedInputGate.isFinished()) { - checkState(checkpointedInputGate.isAvailable().isDone(), "Finished BarrierHandler should be available"); + checkState(checkpointedInputGate.getAvailableFuture().isDone(), "Finished BarrierHandler should be available"); if (!checkpointedInputGate.isEmpty()) { throw new IllegalStateException("Trailing data in checkpoint barrier handler."); } @@ -190,11 +190,11 @@ public int getInputIndex() { } @Override - public CompletableFuture isAvailable() { + public CompletableFuture getAvailableFuture() { if (currentRecordDeserializer != null) { return AVAILABLE; } - return checkpointedInputGate.isAvailable(); + return checkpointedInputGate.getAvailableFuture(); } @Override 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 index 46878248e9c1..a213558296f1 100644 --- 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 @@ -46,8 +46,8 @@ public InputStatus emitNext(DataOutput output) throws Exception { } @Override - public CompletableFuture isAvailable() { - return operator.isAvailable(); + public CompletableFuture getAvailableFuture() { + return operator.getAvailableFuture(); } /** 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 51344637de36..4bf9d9f91f3b 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 @@ -152,12 +152,12 @@ private void postProcessRecord(Counter numRecordsIn) { } @Override - public CompletableFuture isAvailable() { + public CompletableFuture getAvailableFuture() { if (inputSelectionHandler.areAllInputsSelected()) { return isAnyInputAvailable(); } else { StreamTaskInput input = (inputSelectionHandler.isFirstInputSelected()) ? input1 : input2; - return input.isAvailable(); + return input.getAvailableFuture(); } } @@ -283,7 +283,7 @@ private void updateAvailability() { } private void updateAvailability(InputStatus status, StreamTaskInput input) { - if (status == InputStatus.MORE_AVAILABLE || (status != InputStatus.END_OF_INPUT && input.isAvailable() == AVAILABLE)) { + if (status == InputStatus.MORE_AVAILABLE || (status != InputStatus.END_OF_INPUT && input.isApproximatelyAvailable())) { inputSelectionHandler.setAvailableInput(input.getInputIndex()); } else { inputSelectionHandler.setUnavailableInput(input.getInputIndex()); @@ -295,29 +295,26 @@ private void checkAndSetAvailable(int inputIndex) { if (status == InputStatus.END_OF_INPUT) { return; } - CompletableFuture inputAvailable = getInput(inputIndex).isAvailable(); - // TODO: inputAvailable.isDone() can be a costly operation (checking volatile). If one of + + // TODO: isAvailable() can be a costly operation (checking volatile). If one of // the input is constantly available and another is not, we will be checking this volatile // once per every record. This might be optimized to only check once per processed NetworkBuffer - if (inputAvailable == AVAILABLE || inputAvailable.isDone()) { + if (getInput(inputIndex).isAvailable()) { inputSelectionHandler.setAvailableInput(inputIndex); } } private CompletableFuture isAnyInputAvailable() { if (firstInputStatus == InputStatus.END_OF_INPUT) { - return input2.isAvailable(); + return input2.getAvailableFuture(); } if (secondInputStatus == InputStatus.END_OF_INPUT) { - return input1.isAvailable(); + return input1.getAvailableFuture(); } - CompletableFuture input1Available = input1.isAvailable(); - CompletableFuture input2Available = input2.isAvailable(); - - return (input1Available == AVAILABLE || input2Available == AVAILABLE) ? - AVAILABLE : CompletableFuture.anyOf(input1Available, input2Available); + return (input1.isApproximatelyAvailable() || input2.isApproximatelyAvailable()) ? + AVAILABLE : CompletableFuture.anyOf(input1.getAvailableFuture(), input2.getAvailableFuture()); } private StreamTaskInput getInput(int inputIndex) { 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 62c56170715f..6c8aaaa5594c 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 @@ -282,7 +282,7 @@ protected void processInput(MailboxDefaultAction.Controller controller) throws E } else if (status == InputStatus.NOTHING_AVAILABLE) { MailboxDefaultAction.Suspension suspendedDefaultAction = controller.suspendDefaultAction(); - inputProcessor.isAvailable().thenRun(suspendedDefaultAction::resume); + inputProcessor.getAvailableFuture().thenRun(suspendedDefaultAction::resume); } } 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 d4ba08ff93da..3b2606e39654 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 @@ -143,7 +143,7 @@ private void serializeRecord(long value, BufferBuilder bufferBuilder) throws IOE } private static void assertHasNextElement(StreamTaskNetworkInput input, DataOutput output) throws Exception { - assertTrue(input.isAvailable().isDone()); + assertTrue(input.getAvailableFuture().isDone()); InputStatus status = input.emitNext(output); assertThat(status, is(InputStatus.MORE_AVAILABLE)); } 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 index e740e80c33bb..a8e0190eba49 100644 --- 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 @@ -106,7 +106,7 @@ private TaskStateSnapshot executeAndWaitForCheckpoint( CheckpointMetaData checkpointMetaData = new CheckpointMetaData(checkpointId, checkpointId); // wait with triggering the checkpoint until we emit all of the data - while (testHarness.getTask().inputProcessor.isAvailable().isDone()) { + while (testHarness.getTask().inputProcessor.getAvailableFuture().isDone()) { Thread.sleep(1); } @@ -195,7 +195,7 @@ public void snapshotState(StateSnapshotContext context) throws Exception { } @Override - public CompletableFuture isAvailable() { + public CompletableFuture getAvailableFuture() { if (hasEmittedEverything()) { return new CompletableFuture<>(); } 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 e88b3a20ea5a..22d047357be3 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 @@ -1213,7 +1213,7 @@ public void close() throws IOException { } @Override - public CompletableFuture isAvailable() { + public CompletableFuture getAvailableFuture() { return AVAILABLE; } From 7374cccbb09fbfa66e98d227211a43ddd8591ad6 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Thu, 31 Oct 2019 09:06:43 +0100 Subject: [PATCH 666/746] [hotfix][network] Make ResultPartitionWriter extend AvailabilityProvider to avoid defining duplicate method ResultPartitionWriter#getAvailableFuture() was already defined by AvailabilityProvider interface, so we can make ResultPartitionWriter extend it to remove this duplicate method. --- .../io/network/api/writer/ResultPartitionWriter.java | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java index b7325abf81ae..75cd5fb9d360 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.io.network.api.writer; +import org.apache.flink.runtime.io.AvailabilityProvider; 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.ResultPartitionID; @@ -25,7 +26,6 @@ import javax.annotation.Nullable; import java.io.IOException; -import java.util.concurrent.CompletableFuture; /** * A buffer-oriented runtime result writer API for producing results. @@ -35,7 +35,7 @@ * In this case {@link ResultPartitionWriter#fail(Throwable)} still needs to be called afterwards to fully release * all resources associated the the partition and propagate failure cause to the consumer if possible. */ -public interface ResultPartitionWriter extends AutoCloseable { +public interface ResultPartitionWriter extends AutoCloseable, AvailabilityProvider { /** * Setup partition, potentially heavy-weight, blocking operation comparing to just creation. @@ -93,11 +93,4 @@ public interface ResultPartitionWriter extends AutoCloseable { *

    Closing of partition is still needed afterwards. */ void finish() throws IOException; - - /** - * Check whether the writer is available for output or not. - * - * @return a future that is completed if it is available for output. - */ - CompletableFuture getAvailableFuture(); } From 2c8b4ef572f05bf4740b7e204af1e5e709cd945c Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Fri, 1 Nov 2019 11:52:15 +0100 Subject: [PATCH 667/746] [FLINK-14553][runtime] Respect non-blocking output in StreamTask#processInput The non-blocking output was introduced in FLINK-14396 and FLINK-14498 to solve the problem of handling the checkpoint barrier in the case of backpressure. In order to make the whole process through, StreamInputProcessor should be allowed to process input elements if the output is also available. The default core size of LocalBufferPool for ResultPartition should also be increased by 1 in order not to cause stuck problem and impact the performance, and this tiny memory overhead could be ignored in practice. --- .../api/writer/MultipleRecordWriters.java | 96 +++++++++ .../network/api/writer/NonRecordWriter.java | 57 ++++++ .../api/writer/RecordWriterDelegate.java | 46 +++++ .../api/writer/SingleRecordWriter.java | 67 +++++++ .../partition/ResultPartitionFactory.java | 13 +- .../network/NettyShuffleEnvironmentTest.java | 8 +- ...AvailabilityTestResultPartitionWriter.java | 105 ++++++++++ .../api/writer/RecordWriterDelegateTest.java | 183 ++++++++++++++++++ .../network/api/writer/RecordWriterTest.java | 2 +- .../partition/ResultPartitionBuilder.java | 4 +- .../partition/ResultPartitionTest.java | 44 +++++ .../operators/testutils/MockEnvironment.java | 4 + .../runtime/tasks/OperatorChain.java | 5 +- .../streaming/runtime/tasks/StreamTask.java | 73 ++++--- .../tasks/mailbox/MailboxProcessor.java | 4 +- .../StreamNetworkThroughputBenchmarkTest.java | 2 +- .../operators/StreamOperatorChainingTest.java | 2 +- ...treamSourceOperatorLatencyMetricsTest.java | 2 +- .../runtime/tasks/StreamTaskTest.java | 98 ++++++++++ .../streaming/util/MockStreamConfig.java | 39 +++- .../streaming/runtime/BackPressureITCase.java | 2 +- 21 files changed, 813 insertions(+), 43 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/MultipleRecordWriters.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/NonRecordWriter.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterDelegate.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/SingleRecordWriter.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/AvailabilityTestResultPartitionWriter.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterDelegateTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/MultipleRecordWriters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/MultipleRecordWriters.java new file mode 100644 index 000000000000..45f53ece6422 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/MultipleRecordWriters.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.event.AbstractEvent; +import org.apache.flink.util.ExceptionUtils; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * The specific delegate implementation for the multiple outputs case. + */ +public class MultipleRecordWriters implements RecordWriterDelegate { + + /** The real record writer instances for this delegate. */ + private final List> recordWriters; + + /** + * Maintains the respective record writer futures to avoid allocating new arrays every time + * in {@link #getAvailableFuture()}. + */ + private final CompletableFuture[] futures; + + public MultipleRecordWriters(List> recordWriters) { + this.recordWriters = checkNotNull(recordWriters); + this.futures = new CompletableFuture[recordWriters.size()]; + } + + @Override + public void broadcastEvent(AbstractEvent event) throws IOException { + IOException exception = null; + for (RecordWriter recordWriter : recordWriters) { + try { + recordWriter.broadcastEvent(event); + } catch (IOException e) { + exception = ExceptionUtils.firstOrSuppressed( + new IOException("Could not send event to downstream tasks.", e), exception); + } + } + + if (exception != null) { + throw exception; + } + } + + @Override + public RecordWriter getRecordWriter(int outputIndex) { + return recordWriters.get(outputIndex); + } + + @Override + public CompletableFuture getAvailableFuture() { + for (int i = 0; i < futures.length; i++) { + futures[i] = recordWriters.get(i).getAvailableFuture(); + } + return CompletableFuture.allOf(futures); + } + + @Override + public boolean isAvailable() { + for (RecordWriter recordWriter : recordWriters) { + if (!recordWriter.isAvailable()) { + return false; + } + } + return true; + } + + @Override + public void close() { + for (RecordWriter recordWriter : recordWriters) { + recordWriter.close(); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/NonRecordWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/NonRecordWriter.java new file mode 100644 index 000000000000..e4c9f3351385 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/NonRecordWriter.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.api.writer; + +import org.apache.flink.core.io.IOReadableWritable; +import org.apache.flink.runtime.event.AbstractEvent; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; + +/** + * The specific delegate implementation for the non output case like sink task. + */ +public class NonRecordWriter implements RecordWriterDelegate { + + public NonRecordWriter() { + } + + @Override + public void broadcastEvent(AbstractEvent event) throws IOException { + } + + @Override + public RecordWriter getRecordWriter(int outputIndex) { + throw new UnsupportedOperationException("No record writer instance."); + } + + @Override + public CompletableFuture getAvailableFuture() { + throw new UnsupportedOperationException("No record writer instance."); + } + + @Override + public boolean isAvailable() { + return true; + } + + @Override + public void close() { + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterDelegate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterDelegate.java new file mode 100644 index 000000000000..a211672da30c --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterDelegate.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.api.writer; + +import org.apache.flink.core.io.IOReadableWritable; +import org.apache.flink.runtime.event.AbstractEvent; +import org.apache.flink.runtime.io.AvailabilityProvider; + +import java.io.IOException; + +/** + * The record writer delegate provides the availability function for task processor, and it might represent + * a single {@link RecordWriter} or multiple {@link RecordWriter} instances in specific implementations. + */ +public interface RecordWriterDelegate extends AvailabilityProvider, AutoCloseable { + + /** + * Broadcasts the provided event to all the internal record writer instances. + * + * @param event the event to be emitted to all the output channels. + */ + void broadcastEvent(AbstractEvent event) throws IOException; + + /** + * Returns the internal actual record writer instance based on the output index. + * + * @param outputIndex the index respective to the record writer instance. + */ + RecordWriter getRecordWriter(int outputIndex); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/SingleRecordWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/SingleRecordWriter.java new file mode 100644 index 000000000000..81327e8cf25c --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/SingleRecordWriter.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.api.writer; + +import org.apache.flink.core.io.IOReadableWritable; +import org.apache.flink.runtime.event.AbstractEvent; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * The specific delegate implementation for the single output case. + */ +public class SingleRecordWriter implements RecordWriterDelegate { + + private final RecordWriter recordWriter; + + public SingleRecordWriter(RecordWriter recordWriter) { + this.recordWriter = checkNotNull(recordWriter); + } + + @Override + public void broadcastEvent(AbstractEvent event) throws IOException { + recordWriter.broadcastEvent(event); + } + + @Override + public RecordWriter getRecordWriter(int outputIndex) { + checkArgument(outputIndex == 0, "The index should always be 0 for the single record writer delegate."); + + return recordWriter; + } + + @Override + public CompletableFuture getAvailableFuture() { + return recordWriter.getAvailableFuture(); + } + + @Override + public boolean isAvailable() { + return recordWriter.isAvailable(); + } + + @Override + public void close() { + recordWriter.close(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactory.java index e61653bd9f28..c06c5c7bad2c 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactory.java @@ -178,6 +178,16 @@ private static void releasePartitionsQuietly(ResultSubpartition[] partitions, in } } + /** + * The minimum pool size should be numberOfSubpartitions + 1 for two considerations: + * + *

    1. StreamTask can only process input if there is at-least one available buffer on output side, so it might cause + * stuck problem if the minimum pool size is exactly equal to the number of subpartitions, because every subpartition + * might maintain a partial unfilled buffer. + * + *

    2. Increases one more buffer for every output LocalBufferPool to void performance regression if processing input is + * based on at-least one buffer available on output side. + */ @VisibleForTesting FunctionWithException createBufferPoolFactory( int numberOfSubpartitions, @@ -187,7 +197,8 @@ FunctionWithException createBufferPool numberOfSubpartitions * networkBuffersPerChannel + floatingNetworkBuffersPerGate : Integer.MAX_VALUE; // If the partition type is back pressure-free, we register with the buffer pool for // callbacks to release memory. - return bufferPoolFactory.createBufferPool(numberOfSubpartitions, + return bufferPoolFactory.createBufferPool( + numberOfSubpartitions + 1, maxNumberOfMemorySegments, type.hasBackPressure() ? null : bufferPoolOwner); }; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironmentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironmentTest.java index 51f1c047c91a..30c7c91716bb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironmentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironmentTest.java @@ -94,10 +94,10 @@ public void testRegisterTaskWithLimitedBuffers() throws Exception { // outgoing: 1 buffer per channel (always) if (!enableCreditBasedFlowControl) { // incoming: 1 buffer per channel - bufferCount = 20; + bufferCount = 24; } else { // incoming: 2 exclusive buffers per channel - bufferCount = 10 + 10 * NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_PER_CHANNEL.defaultValue(); + bufferCount = 14 + 10 * NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_PER_CHANNEL.defaultValue(); } testRegisterTaskWithLimitedBuffers(bufferCount); @@ -178,8 +178,8 @@ private void testRegisterTaskWithLimitedBuffers(int bufferPoolSize) throws Excep assertEquals(expectedRp4Buffers, rp4.getBufferPool().getMaxNumberOfMemorySegments()); for (ResultPartition rp : resultPartitions) { - assertEquals(rp.getNumberOfSubpartitions(), rp.getBufferPool().getNumberOfRequiredMemorySegments()); - assertEquals(rp.getNumberOfSubpartitions(), rp.getBufferPool().getNumBuffers()); + assertEquals(rp.getNumberOfSubpartitions() + 1, rp.getBufferPool().getNumberOfRequiredMemorySegments()); + assertEquals(rp.getNumberOfSubpartitions() + 1, rp.getBufferPool().getNumBuffers()); } // verify buffer pools for the input gates (NOTE: credit-based uses minimum required buffers diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/AvailabilityTestResultPartitionWriter.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/AvailabilityTestResultPartitionWriter.java new file mode 100644 index 000000000000..1c96d6ae97dd --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/AvailabilityTestResultPartitionWriter.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.runtime.io.network.buffer.BufferBuilder; +import org.apache.flink.runtime.io.network.buffer.BufferConsumer; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; + +import javax.annotation.Nullable; + +import java.util.concurrent.CompletableFuture; + +/** + * A specific result partition writer implementation only used to control the output + * availability state in tests. + */ +public class AvailabilityTestResultPartitionWriter implements ResultPartitionWriter { + + /** This state is only valid in the first call of {@link #isAvailable()}. */ + private final boolean isAvailable; + + private final CompletableFuture future = new CompletableFuture(); + + /** The counter used to record how many calls of {@link #isAvailable()}. */ + private int counter; + + public AvailabilityTestResultPartitionWriter(boolean isAvailable) { + this.isAvailable = isAvailable; + } + + @Override + public void setup() { + } + + @Override + public ResultPartitionID getPartitionId() { + return new ResultPartitionID(); + } + + @Override + public int getNumberOfSubpartitions() { + return 1; + } + + @Override + public int getNumTargetKeyGroups() { + return 1; + } + + @Override + public BufferBuilder getBufferBuilder() { + throw new UnsupportedOperationException(); + } + + @Override + public boolean addBufferConsumer(BufferConsumer bufferConsumer, int targetChannel) { + return true; + } + + @Override + public void flushAll() { + } + + @Override + public void flush(int subpartitionIndex) { + } + + @Override + public void close() { + } + + @Override + public void fail(@Nullable Throwable throwable) { + } + + @Override + public void finish() { + } + + @Override + public CompletableFuture getAvailableFuture() { + return isAvailable ? AVAILABLE : future; + } + + @Override + public boolean isAvailable() { + return counter++ == 0 ? isAvailable : true; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterDelegateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterDelegateTest.java new file mode 100644 index 000000000000..4f6f7f84b5e1 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterDelegateTest.java @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.io.network.api.writer; + +import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker; +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.buffer.BufferPool; +import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; +import org.apache.flink.runtime.io.network.partition.ResultPartitionBuilder; +import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; +import org.apache.flink.runtime.io.network.util.TestPooledBufferProvider; +import org.apache.flink.util.TestLogger; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * Tests for the {@link SingleRecordWriter} and {@link MultipleRecordWriters}. + */ +public class RecordWriterDelegateTest extends TestLogger { + + private static final int numberOfBuffers = 10; + + private static final int memorySegmentSize = 128; + + private static final int numberOfSegmentsToRequest = 2; + + private NetworkBufferPool globalPool; + + @Before + public void setup() { + globalPool = new NetworkBufferPool(numberOfBuffers, memorySegmentSize, numberOfSegmentsToRequest); + } + + @After + public void teardown() { + globalPool.destroyAllBufferPools(); + globalPool.destroy(); + } + + @Test + @SuppressWarnings("unchecked") + public void testSingleRecordWriterAvailability() throws Exception { + final RecordWriter recordWriter = createRecordWriter(globalPool); + final RecordWriterDelegate writerDelegate = new SingleRecordWriter(recordWriter); + + assertEquals(recordWriter, writerDelegate.getRecordWriter(0)); + verifyAvailability(writerDelegate); + } + + @Test + @SuppressWarnings("unchecked") + public void testMultipleRecordWritersAvailability() throws Exception { + // setup + final int numRecordWriters = 2; + final List recordWriters = new ArrayList<>(numRecordWriters); + + for (int i = 0; i < numRecordWriters; i++) { + recordWriters.add(createRecordWriter(globalPool)); + } + + RecordWriterDelegate writerDelegate = new MultipleRecordWriters(recordWriters); + for (int i = 0; i < numRecordWriters; i++) { + assertEquals(recordWriters.get(i), writerDelegate.getRecordWriter(i)); + } + + verifyAvailability(writerDelegate); + } + + @Test + @SuppressWarnings("unchecked") + public void testSingleRecordWriterBroadcastEvent() throws Exception { + // setup + final ArrayDeque[] queues = new ArrayDeque[] { new ArrayDeque(), new ArrayDeque() }; + final RecordWriter recordWriter = createRecordWriter(queues); + final RecordWriterDelegate writerDelegate = new SingleRecordWriter(recordWriter); + + verifyBroadcastEvent(writerDelegate, queues, 1); + } + + @Test + @SuppressWarnings("unchecked") + public void testMultipleRecordWritersBroadcastEvent() throws Exception { + // setup + final int numRecordWriters = 2; + final List recordWriters = new ArrayList<>(numRecordWriters); + final ArrayDeque[] queues = new ArrayDeque[] { new ArrayDeque(), new ArrayDeque() }; + + for (int i = 0; i < numRecordWriters; i++) { + recordWriters.add(createRecordWriter(queues)); + } + final RecordWriterDelegate writerDelegate = new MultipleRecordWriters(recordWriters); + + verifyBroadcastEvent(writerDelegate, queues, numRecordWriters); + } + + private RecordWriter createRecordWriter(NetworkBufferPool globalPool) throws Exception { + final BufferPool localPool = globalPool.createBufferPool(1, 1); + final ResultPartitionWriter partition = new ResultPartitionBuilder() + .setBufferPoolFactory(p -> localPool) + .build(); + partition.setup(); + + return new RecordWriterBuilder().build(partition); + } + + private RecordWriter createRecordWriter(ArrayDeque[] queues) { + final ResultPartitionWriter partition = new RecordWriterTest.CollectingPartitionWriter( + queues, + new TestPooledBufferProvider(1)); + + return new RecordWriterBuilder().build(partition); + } + + private void verifyAvailability(RecordWriterDelegate writerDelegate) throws Exception { + // writer is available at the beginning + assertTrue(writerDelegate.isAvailable()); + assertTrue(writerDelegate.getAvailableFuture().isDone()); + + // request one buffer from the local pool to make it unavailable + final BufferBuilder bufferBuilder = checkNotNull(writerDelegate.getRecordWriter(0).getBufferBuilder(0)); + assertFalse(writerDelegate.isAvailable()); + CompletableFuture future = writerDelegate.getAvailableFuture(); + assertFalse(future.isDone()); + + // recycle the buffer to make the local pool available again + final Buffer buffer = BufferBuilderTestUtils.buildSingleBuffer(bufferBuilder); + buffer.recycleBuffer(); + assertTrue(future.isDone()); + assertTrue(writerDelegate.isAvailable()); + assertTrue(writerDelegate.getAvailableFuture().isDone()); + } + + private void verifyBroadcastEvent( + RecordWriterDelegate writerDelegate, + ArrayDeque[] queues, + int numRecordWriters) throws Exception { + + final CancelCheckpointMarker message = new CancelCheckpointMarker(1); + writerDelegate.broadcastEvent(message); + + // verify the added messages in all the queues + for (int i = 0; i < queues.length; i++) { + assertEquals(numRecordWriters, queues[i].size()); + + for (int j = 0; j < numRecordWriters; j++) { + BufferOrEvent boe = RecordWriterTest.parseBuffer(queues[i].remove(), i); + assertTrue(boe.isEvent()); + assertEquals(message, boe.getEvent()); + } + } + } +} 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 cb29fb004ff6..1cbd8fe0c801 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 @@ -604,7 +604,7 @@ public void close() { } } - private static BufferOrEvent parseBuffer(BufferConsumer bufferConsumer, int targetChannel) throws IOException { + static BufferOrEvent parseBuffer(BufferConsumer bufferConsumer, int targetChannel) throws IOException { Buffer buffer = buildSingleBuffer(bufferConsumer); if (buffer.isBuffer()) { return new BufferOrEvent(buffer, targetChannel); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionBuilder.java index 4bb1a85b0569..0028cf1a7d8b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionBuilder.java @@ -103,12 +103,12 @@ public ResultPartitionBuilder setNetworkBufferPool(NetworkBufferPool networkBuff return this; } - private ResultPartitionBuilder setNetworkBuffersPerChannel(int networkBuffersPerChannel) { + public ResultPartitionBuilder setNetworkBuffersPerChannel(int networkBuffersPerChannel) { this.networkBuffersPerChannel = networkBuffersPerChannel; return this; } - private ResultPartitionBuilder setFloatingNetworkBuffersPerGate(int floatingNetworkBuffersPerGate) { + public ResultPartitionBuilder setFloatingNetworkBuffersPerGate(int floatingNetworkBuffersPerGate) { this.floatingNetworkBuffersPerGate = floatingNetworkBuffersPerGate; return this; } 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 0b1e9a0c98c1..02b6eed2125a 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 @@ -27,6 +27,8 @@ 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.buffer.BufferPool; +import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; import org.apache.flink.runtime.taskmanager.ConsumableNotifyingResultPartitionWriterDecorator; import org.apache.flink.runtime.taskmanager.NoOpTaskActions; import org.apache.flink.runtime.taskmanager.TaskActions; @@ -327,6 +329,48 @@ private void testReleaseMemory(final ResultPartitionType resultPartitionType) th } } + @Test + public void testPipelinedPartitionBufferPool() throws Exception { + testPartitionBufferPool(ResultPartitionType.PIPELINED_BOUNDED); + } + + @Test + public void testBlockingPartitionBufferPool() throws Exception { + testPartitionBufferPool(ResultPartitionType.BLOCKING); + } + + private void testPartitionBufferPool(ResultPartitionType type) throws Exception { + //setup + final int networkBuffersPerChannel = 2; + final int floatingNetworkBuffersPerGate = 8; + final NetworkBufferPool globalPool = new NetworkBufferPool(20, 1, 1); + final ResultPartition partition = new ResultPartitionBuilder() + .setResultPartitionType(type) + .setFileChannelManager(fileChannelManager) + .setNetworkBuffersPerChannel(networkBuffersPerChannel) + .setFloatingNetworkBuffersPerGate(floatingNetworkBuffersPerGate) + .setNetworkBufferPool(globalPool) + .build(); + + try { + partition.setup(); + BufferPool bufferPool = partition.getBufferPool(); + // verify the amount of buffers in created local pool + assertEquals(partition.getNumberOfSubpartitions() + 1, bufferPool.getNumberOfRequiredMemorySegments()); + if (type.isBounded()) { + final int maxNumBuffers = networkBuffersPerChannel * partition.getNumberOfSubpartitions() + floatingNetworkBuffersPerGate; + assertEquals(maxNumBuffers, bufferPool.getMaxNumberOfMemorySegments()); + } else { + assertEquals(Integer.MAX_VALUE, bufferPool.getMaxNumberOfMemorySegments()); + } + + } finally { + // cleanup + globalPool.destroyAllBufferPools(); + globalPool.destroy(); + } + } + private ResultPartitionWriter createConsumableNotifyingResultPartitionWriter( ResultPartitionType partitionType, TaskActions taskActions, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java index 8ad9da085bcf..dce01213a58e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java @@ -188,6 +188,10 @@ public void addOutput(final List outputList) { } } + public void addOutputs(List writers) { + outputs.addAll(writers); + } + @Override public Configuration getTaskConfiguration() { return this.taskConfiguration; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java index 9acf8cd8e4c3..f273fef59e69 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java @@ -29,6 +29,7 @@ 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.writer.RecordWriter; +import org.apache.flink.runtime.io.network.api.writer.RecordWriterDelegate; import org.apache.flink.runtime.metrics.MetricNames; import org.apache.flink.runtime.metrics.groups.OperatorIOMetricGroup; import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup; @@ -106,7 +107,7 @@ public class OperatorChain> implements Strea public OperatorChain( StreamTask containingTask, - List>>> recordWriters) { + RecordWriterDelegate>> recordWriterDelegate) { final ClassLoader userCodeClassloader = containingTask.getUserCodeClassLoader(); final StreamConfig configuration = containingTask.getConfiguration(); @@ -129,7 +130,7 @@ public OperatorChain( StreamEdge outEdge = outEdgesInOrder.get(i); RecordWriterOutput streamOutput = createStreamOutput( - recordWriters.get(i), + recordWriterDelegate.getRecordWriter(i), outEdge, chainedConfigs.get(outEdge.getSourceId()), containingTask.getEnvironment()); 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 6c8aaaa5594c..6a842c4e70f2 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 @@ -35,9 +35,13 @@ import org.apache.flink.runtime.execution.CancelTaskException; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker; +import org.apache.flink.runtime.io.network.api.writer.MultipleRecordWriters; +import org.apache.flink.runtime.io.network.api.writer.NonRecordWriter; 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.RecordWriterDelegate; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; +import org.apache.flink.runtime.io.network.api.writer.SingleRecordWriter; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup; @@ -84,6 +88,7 @@ import java.util.List; import java.util.Map; import java.util.OptionalLong; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; @@ -199,7 +204,7 @@ public abstract class StreamTask> /** Thread pool for async snapshot workers. */ private ExecutorService asyncOperationsThreadPool; - private final List>>> recordWriters; + private final RecordWriterDelegate>> recordWriter; protected final MailboxProcessor mailboxProcessor; @@ -248,7 +253,7 @@ protected StreamTask( this.uncaughtExceptionHandler = Preconditions.checkNotNull(uncaughtExceptionHandler); this.configuration = new StreamConfig(getTaskConfiguration()); this.accumulatorMap = getEnvironment().getAccumulatorRegistry().getUserMap(); - this.recordWriters = createRecordWriters(configuration, environment); + this.recordWriter = createRecordWriterDelegate(configuration, environment); this.mailboxProcessor = new MailboxProcessor(this::processInput); this.asyncExceptionHandler = new StreamTaskAsyncExceptionHandler(environment); } @@ -277,12 +282,31 @@ protected void cleanup() throws Exception { */ protected void processInput(MailboxDefaultAction.Controller controller) throws Exception { InputStatus status = inputProcessor.processInput(); + if (status == InputStatus.MORE_AVAILABLE && recordWriter.isAvailable()) { + return; + } if (status == InputStatus.END_OF_INPUT) { controller.allActionsCompleted(); + return; } - else if (status == InputStatus.NOTHING_AVAILABLE) { - MailboxDefaultAction.Suspension suspendedDefaultAction = controller.suspendDefaultAction(); - inputProcessor.getAvailableFuture().thenRun(suspendedDefaultAction::resume); + CompletableFuture jointFuture = getInputOutputJointFuture(status); + MailboxDefaultAction.Suspension suspendedDefaultAction = controller.suspendDefaultAction(); + jointFuture.thenRun(suspendedDefaultAction::resume); + } + + /** + * Considers three scenarios to combine input and output futures: + * 1. Both input and output are unavailable. + * 2. Only input is unavailable. + * 3. Only output is unavailable. + */ + private CompletableFuture getInputOutputJointFuture(InputStatus status) { + if (status == InputStatus.NOTHING_AVAILABLE && !recordWriter.isAvailable()) { + return CompletableFuture.allOf(inputProcessor.getAvailableFuture(), recordWriter.getAvailableFuture()); + } else if (status == InputStatus.NOTHING_AVAILABLE) { + return inputProcessor.getAvailableFuture(); + } else { + return recordWriter.getAvailableFuture(); } } @@ -383,7 +407,7 @@ public final void invoke() throws Exception { timerThreadFactory); } - operatorChain = new OperatorChain<>(this, recordWriters); + operatorChain = new OperatorChain<>(this, recordWriter); headOperator = operatorChain.getHeadOperator(); // check environment for selective reading @@ -514,9 +538,7 @@ public final void invoke() throws Exception { } } else { // failed to allocate operatorChain, clean up record writers - for (RecordWriter>> writer: recordWriters) { - writer.close(); - } + recordWriter.close(); } mailboxProcessor.close(); @@ -852,21 +874,7 @@ private boolean performCheckpoint( // we cannot broadcast the cancellation markers on the 'operator chain', because it may not // yet be created final CancelCheckpointMarker message = new CancelCheckpointMarker(checkpointMetaData.getCheckpointId()); - Exception exception = null; - - for (RecordWriter>> recordWriter : recordWriters) { - try { - recordWriter.broadcastEvent(message); - } catch (Exception e) { - exception = ExceptionUtils.firstOrSuppressed( - new Exception("Could not send cancel checkpoint marker to downstream tasks.", e), - exception); - } - } - - if (exception != null) { - throw exception; - } + recordWriter.broadcastEvent(message); return false; } @@ -1396,7 +1404,22 @@ private enum AsyncCheckpointState { } @VisibleForTesting - public static List>>> createRecordWriters( + public static RecordWriterDelegate>> createRecordWriterDelegate( + StreamConfig configuration, + Environment environment) { + List>>> recordWrites = createRecordWriters( + configuration, + environment); + if (recordWrites.size() == 1) { + return new SingleRecordWriter<>(recordWrites.get(0)); + } else if (recordWrites.size() == 0) { + return new NonRecordWriter<>(); + } else { + return new MultipleRecordWriters<>(recordWrites); + } + } + + private static List>>> createRecordWriters( StreamConfig configuration, Environment environment) { List>>> recordWriters = new ArrayList<>(); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxProcessor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxProcessor.java index def6cfc10a36..4ae15c966137 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxProcessor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxProcessor.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.runtime.tasks.mailbox; import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.streaming.api.operators.MailboxExecutor; import org.apache.flink.util.Preconditions; @@ -231,7 +232,8 @@ private MailboxDefaultAction.Suspension suspendDefaultAction() { return suspendedDefaultAction; } - private boolean isDefaultActionUnavailable() { + @VisibleForTesting + public boolean isDefaultActionUnavailable() { return suspendedDefaultAction != null; } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmarkTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmarkTest.java index b31106071831..15ed1173dc4d 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmarkTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmarkTest.java @@ -103,7 +103,7 @@ public void remoteModeMinimumBuffers() throws Exception { int writers = 2; int channels = 2; - env.setUp(writers, channels, 100, false, writers * channels, writers * channels * + env.setUp(writers, channels, 100, false, writers * channels + writers, writers * channels * NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_PER_CHANNEL.defaultValue()); env.executeBenchmark(10_000); env.tearDown(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamOperatorChainingTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamOperatorChainingTest.java index 86ef070c64f8..70f96a16f1b3 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamOperatorChainingTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamOperatorChainingTest.java @@ -274,7 +274,7 @@ private > OperatorChain createOperator StreamConfig streamConfig, Environment environment, StreamTask task) { - return new OperatorChain<>(task, StreamTask.createRecordWriters(streamConfig, environment)); + return new OperatorChain<>(task, StreamTask.createRecordWriterDelegate(streamConfig, environment)); } private > StreamTask createMockTask( diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorLatencyMetricsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorLatencyMetricsTest.java index 6cbb846a60f7..cb0eccdbf581 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorLatencyMetricsTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamSourceOperatorLatencyMetricsTest.java @@ -167,7 +167,7 @@ private void testLatencyMarkEmission(int numberLatencyMarkers, OperatorSetupOper // run and wait to be stopped OperatorChain operatorChain = new OperatorChain<>( operator.getContainingTask(), - StreamTask.createRecordWriters(operator.getOperatorConfig(), new MockEnvironmentBuilder().build())); + StreamTask.createRecordWriterDelegate(operator.getOperatorConfig(), new MockEnvironmentBuilder().build())); try { operator.run(new Object(), mock(StreamStatusMaintainer.class), new CollectorOutput<>(output), operatorChain); operator.close(); 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 22d047357be3..65710e30767d 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 @@ -54,7 +54,9 @@ 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.AvailabilityTestResultPartitionWriter; import org.apache.flink.runtime.io.network.api.writer.RecordWriter; +import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; 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; @@ -106,6 +108,7 @@ 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.MailboxExecutor; import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.StreamOperator; @@ -118,6 +121,7 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; +import org.apache.flink.streaming.util.MockStreamConfig; import org.apache.flink.streaming.util.TestSequentialReadingStreamOperator; import org.apache.flink.util.CloseableIterable; import org.apache.flink.util.ExceptionUtils; @@ -889,6 +893,57 @@ public void testRecordWriterClosedOnStreamOperatorFactoryDeserializationError() .noneMatch(thread -> thread.getName().startsWith(RecordWriter.DEFAULT_OUTPUT_FLUSH_THREAD_NAME))); } + @Test + public void testProcessWithAvailableOutput() throws Exception { + try (final MockEnvironment environment = setupEnvironment(new boolean[] {true, true})) { + final int numberOfProcessCalls = 10; + final AvailabilityTestInputProcessor inputProcessor = new AvailabilityTestInputProcessor(numberOfProcessCalls); + final AvailabilityTestStreamTask task = new AvailabilityTestStreamTask<>(environment, inputProcessor); + + task.invoke(); + assertEquals(numberOfProcessCalls, inputProcessor.currentNumProcessCalls); + } + } + + @Test + public void testProcessWithUnAvailableOutput() throws Exception { + try (final MockEnvironment environment = setupEnvironment(new boolean[] {true, false})) { + final int numberOfProcessCalls = 10; + final AvailabilityTestInputProcessor inputProcessor = new AvailabilityTestInputProcessor(numberOfProcessCalls); + final AvailabilityTestStreamTask task = new AvailabilityTestStreamTask<>(environment, inputProcessor); + final MailboxExecutor executor = task.mailboxProcessor.getMainMailboxExecutor(); + + final Runnable completeFutureTask = () -> { + assertEquals(1, inputProcessor.currentNumProcessCalls); + assertTrue(task.mailboxProcessor.isDefaultActionUnavailable()); + environment.getWriter(1).getAvailableFuture().complete(null); + }; + + executor.submit(() -> { + executor.submit(completeFutureTask, "This task will complete the future to resume process input action."); }, + "This task will submit another task to execute after processing input once."); + + task.invoke(); + assertEquals(numberOfProcessCalls, inputProcessor.currentNumProcessCalls); + } + } + + private MockEnvironment setupEnvironment(boolean[] outputAvailabilities) { + final Configuration configuration = new Configuration(); + new MockStreamConfig(configuration, outputAvailabilities.length); + + final List writers = new ArrayList<>(outputAvailabilities.length); + for (int i = 0; i < outputAvailabilities.length; i++) { + writers.add(new AvailabilityTestResultPartitionWriter(outputAvailabilities[i])); + } + + final MockEnvironment environment = new MockEnvironmentBuilder() + .setTaskConfiguration(configuration) + .build(); + environment.addOutputs(writers); + return environment; + } + // ------------------------------------------------------------------------ // Test Utilities // ------------------------------------------------------------------------ @@ -988,6 +1043,49 @@ protected void init() throws Exception { protected void cleanup() throws Exception {} } + /** + * A stream task implementation used to construct a specific {@link StreamInputProcessor} for tests. + */ + private static class AvailabilityTestStreamTask> extends StreamTask { + + AvailabilityTestStreamTask(Environment environment, StreamInputProcessor inputProcessor) { + super(environment); + + this.inputProcessor = inputProcessor; + } + + @Override + protected void init() { + } + } + + /** + * A stream input processor implementation used to control the returned input status based on + * the total number of processing calls. + */ + private static class AvailabilityTestInputProcessor implements StreamInputProcessor { + private final int totalProcessCalls; + private int currentNumProcessCalls; + + AvailabilityTestInputProcessor(int totalProcessCalls) { + this.totalProcessCalls = totalProcessCalls; + } + + @Override + public InputStatus processInput() { + return ++currentNumProcessCalls < totalProcessCalls ? InputStatus.MORE_AVAILABLE : InputStatus.END_OF_INPUT; + } + + @Override + public void close() throws IOException { + } + + @Override + public CompletableFuture getAvailableFuture() { + return AVAILABLE; + } + } + private static class BlockingCloseStreamOperator extends AbstractStreamOperator { private static final long serialVersionUID = -9042150529568008847L; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamConfig.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamConfig.java index 598da48db27d..ec31592ca601 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamConfig.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/MockStreamConfig.java @@ -17,17 +17,50 @@ package org.apache.flink.streaming.util; +import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.OperatorID; 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.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner; +import org.apache.flink.streaming.runtime.tasks.SourceStreamTask; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; /** - * Handy mock for {@link StreamConfig}. + * A dummy stream config implementation for specifying the number of outputs in tests. */ public class MockStreamConfig extends StreamConfig { - public MockStreamConfig() { - super(new Configuration()); + public MockStreamConfig(Configuration configuration, int numberOfOutputs) { + super(configuration); + + setChainStart(); + setOutputSelectors(Collections.emptyList()); + setNumberOfOutputs(numberOfOutputs); + setTypeSerializerOut(new StringSerializer()); + setVertexID(0); + setStreamOperator(new TestSequentialReadingStreamOperator("test operator")); setOperatorID(new OperatorID()); + + StreamOperator dummyOperator = new AbstractStreamOperator() { + private static final long serialVersionUID = 1L; + }; + + StreamNode sourceVertex = new StreamNode(0, null, null, dummyOperator, "source", new ArrayList<>(), SourceStreamTask.class); + StreamNode targetVertex = new StreamNode(1, null, null, dummyOperator, "target", new ArrayList<>(), SourceStreamTask.class); + + List outEdgesInOrder = new ArrayList<>(numberOfOutputs); + for (int i = 0; i < numberOfOutputs; i++) { + outEdgesInOrder.add( + new StreamEdge(sourceVertex, targetVertex, numberOfOutputs, new ArrayList<>(), new BroadcastPartitioner<>(), null)); + } + setOutEdgesInOrder(outEdgesInOrder); + setNonChainedOutputs(outEdgesInOrder); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/BackPressureITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/BackPressureITCase.java index e7c2072219a8..f96b30de95f4 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/BackPressureITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/BackPressureITCase.java @@ -94,7 +94,7 @@ private static Configuration createNetworkBufferConfiguration() { final Configuration configuration = new Configuration(); final int memorySegmentSizeKb = 32; - final String networkBuffersMemory = (memorySegmentSizeKb * NUM_TASKS) + "kb"; + final String networkBuffersMemory = memorySegmentSizeKb * (NUM_TASKS + 2) + "kb"; configuration.setString(TaskManagerOptions.MEMORY_SEGMENT_SIZE, memorySegmentSizeKb + "kb"); configuration.setString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN, networkBuffersMemory); From 901438b65bfbf3874cec60ea17b0c3b05bdd8477 Mon Sep 17 00:00:00 2001 From: eskabetxe Date: Fri, 15 Nov 2019 15:44:16 +0100 Subject: [PATCH 668/746] [FLINK-13850] refactor part file configurations into a single method Closes #10225 --- docs/dev/connectors/streamfile_sink.md | 14 +++ .../api/functions/sink/filesystem/Bucket.java | 24 ++--- .../sink/filesystem/BucketFactory.java | 4 +- .../functions/sink/filesystem/Buckets.java | 10 +- .../filesystem/DefaultBucketFactoryImpl.java | 8 +- .../sink/filesystem/OutputFileConfig.java | 98 +++++++++++++++++++ .../sink/filesystem/PartFileConfig.java | 53 ---------- .../sink/filesystem/StreamingFileSink.java | 58 ++++------- .../filesystem/BucketAssignerITCases.java | 2 +- .../functions/sink/filesystem/BucketTest.java | 18 ++-- .../sink/filesystem/BucketsTest.java | 14 +-- .../sink/filesystem/BulkWriterTest.java | 3 +- .../sink/filesystem/RollingPolicyTest.java | 2 +- .../functions/sink/filesystem/TestUtils.java | 18 ++-- 14 files changed, 181 insertions(+), 145 deletions(-) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/OutputFileConfig.java delete mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/PartFileConfig.java diff --git a/docs/dev/connectors/streamfile_sink.md b/docs/dev/connectors/streamfile_sink.md index c59e646a3548..378a0f4d63cf 100644 --- a/docs/dev/connectors/streamfile_sink.md +++ b/docs/dev/connectors/streamfile_sink.md @@ -133,6 +133,20 @@ New buckets are created as dictated by the bucketing policy, and this doesn't af Old buckets can still receive new records as the bucketing policy is evaluated on a per-record basis. +### Part file configuration + +The filenames of the part files could be defined using `OutputFileConfig`, this configuration contain a part prefix and part suffix, +that will be used with the parallel subtask index of the sink and a rolling counter. +For example for a prefix "prefix" and a suffix ".ext" the file create: + +``` +└── 2019-08-25--12 + ├── prefix-0-0.ext + ├── prefix-0-1.ext.inprogress.bd053eb0-5ecf-4c85-8433-9eff486ac334 + ├── prefix-1-0.ext + └── prefix-1-1.ext.inprogress.bc279efe-b16f-47d8-b828-00ef6e2fbd11 +``` + ## File Formats The `StreamingFileSink` supports both row-wise and bulk encoding formats, such as [Apache Parquet](http://parquet.apache.org). diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java index 4a996e7884b9..7efd2bada644 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java @@ -70,7 +70,7 @@ public class Bucket { private final NavigableMap> pendingPartsPerCheckpoint; - private final PartFileConfig partFileConfig; + private final OutputFileConfig outputFileConfig; private long partCounter; @@ -90,7 +90,7 @@ private Bucket( final long initialPartCounter, final PartFileWriter.PartFileFactory partFileFactory, final RollingPolicy rollingPolicy, - final PartFileConfig partFileConfig) { + final OutputFileConfig outputFileConfig) { this.fsWriter = checkNotNull(fsWriter); this.subtaskIndex = subtaskIndex; this.bucketId = checkNotNull(bucketId); @@ -103,7 +103,7 @@ private Bucket( this.pendingPartsPerCheckpoint = new TreeMap<>(); this.resumablesPerCheckpoint = new TreeMap<>(); - this.partFileConfig = checkNotNull(partFileConfig); + this.outputFileConfig = checkNotNull(outputFileConfig); } /** @@ -116,7 +116,7 @@ private Bucket( final PartFileWriter.PartFileFactory partFileFactory, final RollingPolicy rollingPolicy, final BucketState bucketState, - final PartFileConfig partFileConfig) throws IOException { + final OutputFileConfig outputFileConfig) throws IOException { this( fsWriter, @@ -126,7 +126,7 @@ private Bucket( initialPartCounter, partFileFactory, rollingPolicy, - partFileConfig); + outputFileConfig); restoreInProgressFile(bucketState); commitRecoveredPendingFiles(bucketState); @@ -230,7 +230,7 @@ private void rollPartFile(final long currentTime) throws IOException { } private Path assembleNewPartPath() { - return new Path(bucketPath, partFileConfig.getPartPrefix() + '-' + subtaskIndex + '-' + partCounter + partFileConfig.getPartSuffix()); + return new Path(bucketPath, outputFileConfig.getPartPrefix() + '-' + subtaskIndex + '-' + partCounter + outputFileConfig.getPartSuffix()); } private CommitRecoverable closePartFile() throws IOException { @@ -369,7 +369,7 @@ List getPendingPartsForCurrentCheckpoint() { * @param partFileFactory the {@link PartFileWriter.PartFileFactory} the factory creating part file writers. * @param the type of input elements to the sink. * @param the type of the identifier of the bucket, as returned by the {@link BucketAssigner} - * @param partFileConfig the part file configuration. + * @param outputFileConfig the part file configuration. * @return The new Bucket. */ static Bucket getNew( @@ -380,8 +380,8 @@ static Bucket getNew( final long initialPartCounter, final PartFileWriter.PartFileFactory partFileFactory, final RollingPolicy rollingPolicy, - final PartFileConfig partFileConfig) { - return new Bucket<>(fsWriter, subtaskIndex, bucketId, bucketPath, initialPartCounter, partFileFactory, rollingPolicy, partFileConfig); + final OutputFileConfig outputFileConfig) { + return new Bucket<>(fsWriter, subtaskIndex, bucketId, bucketPath, initialPartCounter, partFileFactory, rollingPolicy, outputFileConfig); } /** @@ -393,7 +393,7 @@ static Bucket getNew( * @param bucketState the initial state of the restored bucket. * @param the type of input elements to the sink. * @param the type of the identifier of the bucket, as returned by the {@link BucketAssigner} - * @param partFileConfig the part file configuration. + * @param outputFileConfig the part file configuration. * @return The restored Bucket. */ static Bucket restore( @@ -403,7 +403,7 @@ static Bucket restore( final PartFileWriter.PartFileFactory partFileFactory, final RollingPolicy rollingPolicy, final BucketState bucketState, - final PartFileConfig partFileConfig) throws IOException { - return new Bucket<>(fsWriter, subtaskIndex, initialPartCounter, partFileFactory, rollingPolicy, bucketState, partFileConfig); + final OutputFileConfig outputFileConfig) throws IOException { + return new Bucket<>(fsWriter, subtaskIndex, initialPartCounter, partFileFactory, rollingPolicy, bucketState, outputFileConfig); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketFactory.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketFactory.java index c318ef213bb5..260e82c79601 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketFactory.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketFactory.java @@ -39,7 +39,7 @@ Bucket getNewBucket( final long initialPartCounter, final PartFileWriter.PartFileFactory partFileWriterFactory, final RollingPolicy rollingPolicy, - final PartFileConfig partFileConfig) throws IOException; + final OutputFileConfig outputFileConfig) throws IOException; Bucket restoreBucket( final RecoverableWriter fsWriter, @@ -48,5 +48,5 @@ Bucket restoreBucket( final PartFileWriter.PartFileFactory partFileWriterFactory, final RollingPolicy rollingPolicy, final BucketState bucketState, - final PartFileConfig partFileConfig) throws IOException; + final OutputFileConfig outputFileConfig) throws IOException; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java index fa7a72b7c572..eb61d18e0a58 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Buckets.java @@ -77,7 +77,7 @@ public class Buckets { private final RecoverableWriter fsWriter; - private final PartFileConfig partFileConfig; + private final OutputFileConfig outputFileConfig; // --------------------------- State Related Fields ----------------------------- @@ -99,7 +99,7 @@ public class Buckets { final PartFileWriter.PartFileFactory partFileWriterFactory, final RollingPolicy rollingPolicy, final int subtaskIndex, - final PartFileConfig partFileConfig) throws IOException { + final OutputFileConfig outputFileConfig) throws IOException { this.basePath = Preconditions.checkNotNull(basePath); this.bucketAssigner = Preconditions.checkNotNull(bucketAssigner); @@ -108,7 +108,7 @@ public class Buckets { this.rollingPolicy = Preconditions.checkNotNull(rollingPolicy); this.subtaskIndex = subtaskIndex; - this.partFileConfig = Preconditions.checkNotNull(partFileConfig); + this.outputFileConfig = Preconditions.checkNotNull(outputFileConfig); this.activeBuckets = new HashMap<>(); this.bucketerContext = new Buckets.BucketerContext(); @@ -186,7 +186,7 @@ private void handleRestoredBucketState(final BucketState recoveredStat partFileWriterFactory, rollingPolicy, recoveredState, - partFileConfig + outputFileConfig ); updateActiveBucketId(bucketId, restoredBucket); @@ -293,7 +293,7 @@ private Bucket getOrCreateBucketForBucketId(final BucketID bucketI maxPartCounter, partFileWriterFactory, rollingPolicy, - partFileConfig); + outputFileConfig); activeBuckets.put(bucketId, bucket); } return bucket; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/DefaultBucketFactoryImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/DefaultBucketFactoryImpl.java index dac2a5af0de1..529b93afa8c5 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/DefaultBucketFactoryImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/DefaultBucketFactoryImpl.java @@ -41,7 +41,7 @@ public Bucket getNewBucket( final long initialPartCounter, final PartFileWriter.PartFileFactory partFileWriterFactory, final RollingPolicy rollingPolicy, - final PartFileConfig partFileConfig) { + final OutputFileConfig outputFileConfig) { return Bucket.getNew( fsWriter, @@ -51,7 +51,7 @@ public Bucket getNewBucket( initialPartCounter, partFileWriterFactory, rollingPolicy, - partFileConfig); + outputFileConfig); } @Override @@ -62,7 +62,7 @@ public Bucket restoreBucket( final PartFileWriter.PartFileFactory partFileWriterFactory, final RollingPolicy rollingPolicy, final BucketState bucketState, - final PartFileConfig partFileConfig) throws IOException { + final OutputFileConfig outputFileConfig) throws IOException { return Bucket.restore( fsWriter, @@ -71,6 +71,6 @@ public Bucket restoreBucket( partFileWriterFactory, rollingPolicy, bucketState, - partFileConfig); + outputFileConfig); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/OutputFileConfig.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/OutputFileConfig.java new file mode 100644 index 000000000000..98b84d178a27 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/OutputFileConfig.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.api.functions.sink.filesystem; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.util.Preconditions; + +import java.io.Serializable; + +/** + * Part file name configuration. + * This allow to define a prefix and a suffix to the part file name. + */ +public class OutputFileConfig implements Serializable { + + private final String partPrefix; + + private final String partSuffix; + + /** + * Initiates the {@code PartFileConfig} with values passed as parameters. + * + * @param partPrefix - the beginning of part file name + * @param partSuffix - the ending of part file name + */ + public OutputFileConfig(final String partPrefix, final String partSuffix) { + this.partPrefix = Preconditions.checkNotNull(partPrefix); + this.partSuffix = Preconditions.checkNotNull(partSuffix); + } + + /** + * The prefix for the part name. + */ + String getPartPrefix() { + return partPrefix; + } + + /** + * The suffix for the part name. + */ + String getPartSuffix() { + return partSuffix; + } + + public static OutputFileConfigBuilder builder() { + return new OutputFileConfigBuilder(); + } + + /** + * A builder to create the part file configuration. + */ + @PublicEvolving + public static class OutputFileConfigBuilder { + + private static final String DEFAULT_PART_PREFIX = "part"; + + private static final String DEFAULT_PART_SUFFIX = ""; + + private String partPrefix; + + private String partSuffix; + + private OutputFileConfigBuilder() { + this.partPrefix = DEFAULT_PART_PREFIX; + this.partSuffix = DEFAULT_PART_SUFFIX; + } + + public OutputFileConfigBuilder withPartPrefix(String prefix) { + this.partPrefix = prefix; + return this; + } + + public OutputFileConfigBuilder withPartSuffix(String suffix) { + this.partSuffix = suffix; + return this; + } + + public OutputFileConfig build() { + return new OutputFileConfig(partPrefix, partSuffix); + } + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/PartFileConfig.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/PartFileConfig.java deleted file mode 100644 index f3eaa59261cb..000000000000 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/PartFileConfig.java +++ /dev/null @@ -1,53 +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.api.functions.sink.filesystem; - -import org.apache.flink.util.Preconditions; - -/** - * Part file name configuration. - * This allow to define a prefix and a suffix to the part file name. - */ -class PartFileConfig { - - public static final String DEFAULT_PART_PREFIX = "part"; - - public static final String DEFAULT_PART_SUFFIX = ""; - - private final String partPrefix; - - private final String partSuffix; - - PartFileConfig() { - this(DEFAULT_PART_PREFIX, DEFAULT_PART_SUFFIX); - } - - PartFileConfig(final String partPrefix, final String partSuffix) { - this.partPrefix = Preconditions.checkNotNull(partPrefix); - this.partSuffix = Preconditions.checkNotNull(partSuffix); - } - - String getPartPrefix() { - return partPrefix; - } - - String getPartSuffix() { - return partSuffix; - } -} 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 ed7e9264249a..cd4afc2d8404 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 @@ -67,9 +67,11 @@ * {@link StreamingFileSink#forBulkFormat(Path, BulkWriter.Factory)}. * * - *

    The filenames of the part files contain the part prefix, "part-", the parallel subtask index of the sink - * and a rolling counter. For example the file {@code "part-1-17"} contains the data from - * {@code subtask 1} of the sink and is the {@code 17th} bucket created by that subtask. + *

    The filenames of the part files could be defined using {@link OutputFileConfig}, this configuration contain + * a part prefix and part suffix, that will be used with the parallel subtask index of the sink + * and a rolling counter. For example for a prefix "prefix" and a suffix ".ext" the file create will have a name + * {@code "prefix-1-17.ext"} containing the data from {@code subtask 1} of the sink and is the {@code 17th} bucket + * created by that subtask. * Part files roll based on the user-specified {@link RollingPolicy}. By default, a {@link DefaultRollingPolicy} * is used. * @@ -217,12 +219,10 @@ public static class RowFormatBuilder bucketFactory; - private String partFilePrefix; - - private String partFileSuffix; + private OutputFileConfig outputFileConfig; 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); + this(basePath, encoder, bucketAssigner, DefaultRollingPolicy.builder().build(), DEFAULT_BUCKET_CHECK_INTERVAL, new DefaultBucketFactoryImpl<>(), OutputFileConfig.builder().build()); } protected RowFormatBuilder( @@ -232,16 +232,14 @@ protected RowFormatBuilder( RollingPolicy policy, long bucketCheckInterval, BucketFactory bucketFactory, - String partFilePrefix, - String partFileSuffix) { + OutputFileConfig outputFileConfig) { this.basePath = Preconditions.checkNotNull(basePath); this.encoder = Preconditions.checkNotNull(encoder); this.bucketAssigner = Preconditions.checkNotNull(assigner); this.rollingPolicy = Preconditions.checkNotNull(policy); this.bucketCheckInterval = bucketCheckInterval; this.bucketFactory = Preconditions.checkNotNull(bucketFactory); - this.partFilePrefix = Preconditions.checkNotNull(partFilePrefix); - this.partFileSuffix = Preconditions.checkNotNull(partFileSuffix); + this.outputFileConfig = Preconditions.checkNotNull(outputFileConfig); } public long getBucketCheckInterval() { @@ -263,19 +261,14 @@ public T withRollingPolicy(final RollingPolicy policy) { return self(); } - public T withPartFilePrefix(final String partPrefix) { - this.partFilePrefix = partPrefix; - return self(); - } - - public T withPartFileSuffix(final String partSuffix) { - this.partFileSuffix = partSuffix; + public T withOutputFileConfig(final OutputFileConfig outputFileConfig) { + this.outputFileConfig = outputFileConfig; 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); + return new RowFormatBuilder<>(basePath, encoder, Preconditions.checkNotNull(assigner), Preconditions.checkNotNull(policy), bucketCheckInterval, new DefaultBucketFactoryImpl<>(), outputFileConfig); } /** Creates the actual sink. */ @@ -298,7 +291,7 @@ Buckets createBuckets(int subtaskIndex) throws IOException { new RowWisePartWriter.Factory<>(encoder), rollingPolicy, subtaskIndex, - new PartFileConfig(partFilePrefix, partFileSuffix)); + outputFileConfig); } } @@ -320,12 +313,10 @@ public static class BulkFormatBuilder bucketFactory; - private String partFilePrefix; - - private String partFileSuffix; + private OutputFileConfig outputFileConfig; 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); + this(basePath, writerFactory, assigner, DEFAULT_BUCKET_CHECK_INTERVAL, new DefaultBucketFactoryImpl<>(), OutputFileConfig.builder().build()); } protected BulkFormatBuilder( @@ -334,15 +325,13 @@ protected BulkFormatBuilder( BucketAssigner assigner, long bucketCheckInterval, BucketFactory bucketFactory, - String partFilePrefix, - String partFileSuffix) { + OutputFileConfig outputFileConfig) { this.basePath = Preconditions.checkNotNull(basePath); this.writerFactory = writerFactory; this.bucketAssigner = Preconditions.checkNotNull(assigner); this.bucketCheckInterval = bucketCheckInterval; this.bucketFactory = Preconditions.checkNotNull(bucketFactory); - this.partFilePrefix = Preconditions.checkNotNull(partFilePrefix); - this.partFileSuffix = Preconditions.checkNotNull(partFileSuffix); + this.outputFileConfig = Preconditions.checkNotNull(outputFileConfig); } public long getBucketCheckInterval() { @@ -363,21 +352,16 @@ public T withBucketAssigner(BucketAssigner assigner) { T withBucketFactory(final BucketFactory factory) { this.bucketFactory = Preconditions.checkNotNull(factory); return self(); - } - - public T withPartFilePrefix(final String partPrefix) { - this.partFilePrefix = partPrefix; - return self(); } - public T withPartFileSuffix(final String partSuffix) { - this.partFileSuffix = partSuffix; + public T withOutputFileConfig(final OutputFileConfig outputFileConfig) { + this.outputFileConfig = outputFileConfig; return self(); } 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); + return new BulkFormatBuilder<>(basePath, writerFactory, Preconditions.checkNotNull(assigner), bucketCheckInterval, new DefaultBucketFactoryImpl<>(), outputFileConfig); } /** Creates the actual sink. */ @@ -394,7 +378,7 @@ Buckets createBuckets(int subtaskIndex) throws IOException { new BulkPartWriter.Factory<>(writerFactory), OnCheckpointRollingPolicy.build(), subtaskIndex, - new PartFileConfig(partFilePrefix, partFileSuffix)); + outputFileConfig); } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketAssignerITCases.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketAssignerITCases.java index cc91a335b7fb..8f65171b5448 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketAssignerITCases.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketAssignerITCases.java @@ -57,7 +57,7 @@ public void testAssembleBucketPath() throws Exception { new RowWisePartWriter.Factory<>(new SimpleStringEncoder<>()), rollingPolicy, 0, - new PartFileConfig() + OutputFileConfig.builder().build() ); Bucket bucket = diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java index 583bacfc8155..ee85e556f535 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java @@ -64,7 +64,7 @@ public void shouldNotCleanupResumablesThatArePartOfTheAckedCheckpoint() throws I final TestRecoverableWriter recoverableWriter = getRecoverableWriter(path); final Bucket bucketUnderTest = - createBucket(recoverableWriter, path, 0, 0, new PartFileConfig()); + createBucket(recoverableWriter, path, 0, 0, OutputFileConfig.builder().build()); bucketUnderTest.write("test-element", 0L); @@ -82,7 +82,7 @@ public void shouldCleanupOutdatedResumablesOnCheckpointAck() throws IOException final TestRecoverableWriter recoverableWriter = getRecoverableWriter(path); final Bucket bucketUnderTest = - createBucket(recoverableWriter, path, 0, 0, new PartFileConfig()); + createBucket(recoverableWriter, path, 0, 0, OutputFileConfig.builder().build()); bucketUnderTest.write("test-element", 0L); @@ -105,7 +105,7 @@ public void shouldNotCallCleanupWithoutInProgressPartFiles() throws Exception { final TestRecoverableWriter recoverableWriter = getRecoverableWriter(path); final Bucket bucketUnderTest = - createBucket(recoverableWriter, path, 0, 0, new PartFileConfig()); + createBucket(recoverableWriter, path, 0, 0, OutputFileConfig.builder().build()); final BucketState state = bucketUnderTest.onReceptionOfCheckpoint(0L); assertThat(state, hasNoActiveInProgressFile()); @@ -357,7 +357,7 @@ private static Bucket createBucket( final Path bucketPath, final int subtaskIdx, final int initialPartCounter, - final PartFileConfig partFileConfig) { + final OutputFileConfig outputFileConfig) { return Bucket.getNew( writer, @@ -367,7 +367,7 @@ private static Bucket createBucket( initialPartCounter, partFileFactory, rollingPolicy, - partFileConfig); + outputFileConfig); } private static Bucket restoreBucket( @@ -375,7 +375,7 @@ private static Bucket restoreBucket( final int subtaskIndex, final long initialPartCounter, final BucketState bucketState, - final PartFileConfig partFileConfig) throws Exception { + final OutputFileConfig outputFileConfig) throws Exception { return Bucket.restore( writer, @@ -384,7 +384,7 @@ private static Bucket restoreBucket( partFileFactory, rollingPolicy, bucketState, - partFileConfig); + outputFileConfig); } private static TestRecoverableWriter getRecoverableWriter(Path path) { @@ -403,7 +403,7 @@ private static TestRecoverableWriter getRecoverableWriter(Path path) { private Bucket getRestoredBucketWithOnlyInProgressPart(final BaseStubWriter writer) throws IOException { final BucketState stateWithOnlyInProgressFile = new BucketState<>("test", new Path(), 12345L, new NoOpRecoverable(), new HashMap<>()); - return Bucket.restore(writer, 0, 1L, partFileFactory, rollingPolicy, stateWithOnlyInProgressFile, new PartFileConfig()); + return Bucket.restore(writer, 0, 1L, partFileFactory, rollingPolicy, stateWithOnlyInProgressFile, OutputFileConfig.builder().build()); } private Bucket getRestoredBucketWithOnlyPendingParts(final BaseStubWriter writer, final int numberOfPendingParts) throws IOException { @@ -412,7 +412,7 @@ private Bucket getRestoredBucketWithOnlyPendingParts(final BaseS final BucketState initStateWithOnlyInProgressFile = new BucketState<>("test", new Path(), 12345L, null, completePartsPerCheckpoint); - return Bucket.restore(writer, 0, 1L, partFileFactory, rollingPolicy, initStateWithOnlyInProgressFile, new PartFileConfig()); + return Bucket.restore(writer, 0, 1L, partFileFactory, rollingPolicy, initStateWithOnlyInProgressFile, OutputFileConfig.builder().build()); } private Map> createPendingPartsPerCheckpoint(int noOfCheckpoints) { diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketsTest.java index ac7071d781ac..8695e5a41be1 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketsTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketsTest.java @@ -318,7 +318,7 @@ private void testCorrectTimestampPassingInContext(Long timestamp, long watermark new RowWisePartWriter.Factory<>(new SimpleStringEncoder<>()), DefaultRollingPolicy.builder().build(), 2, - new PartFileConfig() + OutputFileConfig.builder().build() ); buckets.onElement( @@ -377,14 +377,14 @@ private static Buckets createBuckets( basePath, rollingPolicy, subtaskIdx, - new PartFileConfig()); + OutputFileConfig.builder().build()); } private static Buckets createBuckets( final Path basePath, final RollingPolicy rollingPolicy, final int subtaskIdx, - final PartFileConfig partFileConfig) throws IOException { + final OutputFileConfig outputFileConfig) throws IOException { return new Buckets<>( basePath, new TestUtils.StringIdentityBucketAssigner(), @@ -392,7 +392,7 @@ private static Buckets createBuckets( new RowWisePartWriter.Factory<>(new SimpleStringEncoder<>()), rollingPolicy, subtaskIdx, - partFileConfig + outputFileConfig ); } @@ -408,7 +408,7 @@ private static Buckets restoreBuckets( subtaskIdx, bucketState, partCounterState, - new PartFileConfig()); + OutputFileConfig.builder().build()); } private static Buckets restoreBuckets( @@ -417,8 +417,8 @@ private static Buckets restoreBuckets( final int subtaskIdx, final ListState bucketState, final ListState partCounterState, - final PartFileConfig partFileConfig) throws Exception { - final Buckets restoredBuckets = createBuckets(basePath, rollingPolicy, subtaskIdx, partFileConfig); + final OutputFileConfig outputFileConfig) throws Exception { + final Buckets restoredBuckets = createBuckets(basePath, rollingPolicy, subtaskIdx, outputFileConfig); restoredBuckets.initializeState(bucketState, partCounterState); return restoredBuckets; } 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 bab18aac3b05..0449e1d247a5 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 @@ -101,8 +101,7 @@ public void testCustomBulkWriterWithPartConfig() throws Exception { new TestUtils.TupleToStringBucketer(), new TestBulkWriterFactory(), new DefaultBucketFactoryImpl<>(), - "prefix", - ".ext") + OutputFileConfig.builder().withPartPrefix("prefix").withPartSuffix(".ext").build()) ) { testPartFilesWithStringBucketer(testHarness, outDir, ".prefix-0-0.ext.inprogress", ".prefix-0-1.ext.inprogress"); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicyTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicyTest.java index be9db3e8bec3..ab94689db86d 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicyTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/RollingPolicyTest.java @@ -204,7 +204,7 @@ private static Buckets createBuckets( new RowWisePartWriter.Factory<>(new SimpleStringEncoder<>()), rollingPolicyToTest, 0, - new PartFileConfig() + OutputFileConfig.builder().build() ); } 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 c17a2d6a24ab..45460fccbc5d 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 @@ -140,8 +140,7 @@ static OneInputStreamOperatorTestHarness, Object> create bucketer, writer, bucketFactory, - PartFileConfig.DEFAULT_PART_PREFIX, - PartFileConfig.DEFAULT_PART_SUFFIX); + OutputFileConfig.builder().build()); } static OneInputStreamOperatorTestHarness, Object> createTestSinkWithBulkEncoder( @@ -152,16 +151,14 @@ static OneInputStreamOperatorTestHarness, Object> create final BucketAssigner, String> bucketer, final BulkWriter.Factory> writer, final BucketFactory, String> bucketFactory, - final String partFilePrefix, - final String partFileSuffix) throws Exception { + final OutputFileConfig outputFileConfig) throws Exception { StreamingFileSink> sink = StreamingFileSink .forBulkFormat(new Path(outDir.toURI()), writer) .withBucketAssigner(bucketer) .withBucketCheckInterval(bucketCheckInterval) .withBucketFactory(bucketFactory) - .withPartFilePrefix(partFilePrefix) - .withPartFileSuffix(partFileSuffix) + .withOutputFileConfig(outputFileConfig) .build(); return new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink), MAX_PARALLELISM, totalParallelism, taskIdx); @@ -184,8 +181,7 @@ static OneInputStreamOperatorTestHarness, Object> c bucketer, writer, bucketFactory, - PartFileConfig.DEFAULT_PART_PREFIX, - PartFileConfig.DEFAULT_PART_SUFFIX); + OutputFileConfig.builder().build()); } static OneInputStreamOperatorTestHarness, Object> createTestSinkWithCustomizedBulkEncoder( @@ -196,16 +192,14 @@ static OneInputStreamOperatorTestHarness, Object> c final BucketAssigner, ID> bucketer, final BulkWriter.Factory> writer, final BucketFactory, ID> bucketFactory, - final String partFilePrefix, - final String partFileSuffix) throws Exception { + final OutputFileConfig outputFileConfig) throws Exception { StreamingFileSink> sink = StreamingFileSink .forBulkFormat(new Path(outDir.toURI()), writer) .withNewBucketAssigner(bucketer) .withBucketCheckInterval(bucketCheckInterval) .withBucketFactory(bucketFactory) - .withPartFilePrefix(partFilePrefix) - .withPartFileSuffix(partFileSuffix) + .withOutputFileConfig(outputFileConfig) .build(); return new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink), MAX_PARALLELISM, totalParallelism, taskIdx); From 1b7a404beb97572116fd7fec821b7e805a8cd51b Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Fri, 15 Nov 2019 12:18:08 +0100 Subject: [PATCH 669/746] [hotfix][network] Reduce the synchronisation overhead in CreditBasedSequenceNumberingViewReader Previously CreditBasedSequenceNumberingViewReader#isAvailable() was synchronising twice on the lock subpartitionView if there are no credits: first in subpartitionView.isAvailable() and second in subpartitionView.nextBufferIsEvent(). The first one `subpartitionView.isAvailable()` check is redundant (always true) if `subpartitionView.nextBufferIsEvent()` is also true so it should be safe to simplify the code. --- .../CreditBasedSequenceNumberingViewReader.java | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java index b88248251af8..ff7eba9224e5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java @@ -115,8 +115,12 @@ public boolean isRegisteredAsAvailable() { @Override public boolean isAvailable() { // BEWARE: this must be in sync with #isAvailable(BufferAndBacklog)! - return hasBuffersAvailable() && - (numCreditsAvailable > 0 || subpartitionView.nextBufferIsEvent()); + if (numCreditsAvailable > 0) { + return subpartitionView.isAvailable(); + } + else { + return subpartitionView.nextBufferIsEvent(); + } } /** @@ -131,8 +135,12 @@ public boolean isAvailable() { */ private boolean isAvailable(BufferAndBacklog bufferAndBacklog) { // BEWARE: this must be in sync with #isAvailable()! - return bufferAndBacklog.isMoreAvailable() && - (numCreditsAvailable > 0 || bufferAndBacklog.nextBufferIsEvent()); + if (numCreditsAvailable > 0) { + return bufferAndBacklog.isMoreAvailable(); + } + else { + return bufferAndBacklog.nextBufferIsEvent(); + } } @Override From f6e39aa60d1ce3c020fcb49955912e559d5202e7 Mon Sep 17 00:00:00 2001 From: Congxian Qiu Date: Mon, 18 Nov 2019 10:46:48 +0800 Subject: [PATCH 670/746] [FLINK-14830][docs-zh] Correct the links in stream_checkpointing.zh.md page (#10233) The links in Chinese version pages should link to Chinese pages. --- docs/internals/stream_checkpointing.zh.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/internals/stream_checkpointing.zh.md b/docs/internals/stream_checkpointing.zh.md index 0774022326d3..8ebfc986df5f 100644 --- a/docs/internals/stream_checkpointing.zh.md +++ b/docs/internals/stream_checkpointing.zh.md @@ -45,11 +45,11 @@ The system then restarts the operators and resets them to the latest successful point of the state snapshot. Any records that are processed as part of the restarted parallel dataflow are guaranteed to not have been part of the previously checkpointed state. -*Note:* By default, checkpointing is disabled. See [Checkpointing]({{ site.baseurl }}/dev/stream/state/checkpointing.html) for details on how to enable and configure checkpointing. +*Note:* By default, checkpointing is disabled. See [Checkpointing]({{ site.baseurl }}/zh/dev/stream/state/checkpointing.html) for details on how to enable and configure checkpointing. *Note:* For this mechanism to realize its full guarantees, the data stream source (such as message queue or broker) needs to be able to rewind the stream to a defined recent point. [Apache Kafka](http://kafka.apache.org) has this ability and Flink's connector to -Kafka exploits this ability. See [Fault Tolerance Guarantees of Data Sources and Sinks]({{ site.baseurl }}/dev/connectors/guarantees.html) for +Kafka exploits this ability. See [Fault Tolerance Guarantees of Data Sources and Sinks]({{ site.baseurl }}/zh/dev/connectors/guarantees.html) for more information about the guarantees provided by Flink's connectors. *Note:* Because Flink's checkpoints are realized through distributed snapshots, we use the words *snapshot* and *checkpoint* interchangeably. @@ -106,10 +106,10 @@ the barrier *n* from the other inputs as well. Otherwise, it would mix records t When operators contain any form of *state*, this state must be part of the snapshots as well. Operator state comes in different forms: - - *User-defined state*: This is state that is created and modified directly by the transformation functions (like `map()` or `filter()`). See [State in Streaming Applications]({{ site.baseurl }}/dev/stream/state/index.html) for details. + - *User-defined state*: This is state that is created and modified directly by the transformation functions (like `map()` or `filter()`). See [State in Streaming Applications]({{ site.baseurl }}/zh/dev/stream/state/index.html) for details. - *System state*: This state refers to data buffers that are part of the operator's computation. A typical example for this state are the *window buffers*, inside which the system collects (and aggregates) records for windows until the window is evaluated and evicted. -Operators snapshot their state at the point in time when they have received all snapshot barriers from their input streams, and before emitting the barriers to their output streams. At that point, all updates to the state from records before the barriers will have been made, and no updates that depend on records from after the barriers have been applied. Because the state of a snapshot may be large, it is stored in a configurable *[state backend]({{ site.baseurl }}/ops/state/state_backends.html)*. By default, this is the JobManager's memory, but for production use a distributed reliable storage should be configured (such as HDFS). After the state has been stored, the operator acknowledges the checkpoint, emits the snapshot barrier into the output streams, and proceeds. +Operators snapshot their state at the point in time when they have received all snapshot barriers from their input streams, and before emitting the barriers to their output streams. At that point, all updates to the state from records before the barriers will have been made, and no updates that depend on records from after the barriers have been applied. Because the state of a snapshot may be large, it is stored in a configurable *[state backend]({{ site.baseurl }}/zh/ops/state/state_backends.html)*. By default, this is the JobManager's memory, but for production use a distributed reliable storage should be configured (such as HDFS). After the state has been stored, the operator acknowledges the checkpoint, emits the snapshot barrier into the output streams, and proceeds. The resulting snapshot now contains: @@ -145,7 +145,7 @@ It is possible to let an operator continue processing while it stores its state After receiving the checkpoint barriers on its inputs, the operator starts the asynchronous snapshot copying of its state. It immediately emits the barrier to its outputs and continues with the regular stream processing. Once the background copy process has completed, it acknowledges the checkpoint to the checkpoint coordinator (the JobManager). The checkpoint is now only complete after all sinks have received the barriers and all stateful operators have acknowledged their completed backup (which may be after the barriers reach the sinks). -See [State Backends]({{ site.baseurl }}/ops/state/state_backends.html) for details on the state snapshots. +See [State Backends]({{ site.baseurl }}/zh/ops/state/state_backends.html) for details on the state snapshots. ## Recovery @@ -156,7 +156,7 @@ stream from position Sk. For example in Apache Kafka, that mea If state was snapshotted incrementally, the operators start with the state of the latest full snapshot and then apply a series of incremental snapshot updates to that state. -See [Restart Strategies]({{ site.baseurl }}/dev/restart_strategies.html) for more information. +See [Restart Strategies]({{ site.baseurl }}/zh/dev/restart_strategies.html) for more information. ## Operator Snapshot Implementation From 7f3471c58da81f21b4f5400fff447c6f3b0db21a Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Fri, 8 Nov 2019 20:50:22 +0800 Subject: [PATCH 671/746] [hotfix] Add shortcuts for getting jvm heap / direct memory size. --- .../TaskExecutorResourceSpec.java | 22 +++++++++++++++++++ .../TaskExecutorResourceUtils.java | 7 ++---- 2 files changed, 24 insertions(+), 5 deletions(-) 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 index 0813aead4cb1..3f6f4fee39b0 100644 --- 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 @@ -155,4 +155,26 @@ public MemorySize getTotalFlinkMemorySize() { public MemorySize getTotalProcessMemorySize() { return getTotalFlinkMemorySize().add(jvmMetaspaceSize).add(jvmOverheadSize); } + + public MemorySize getJvmHeapMemorySize() { + return frameworkHeapSize.add(taskHeapSize).add(onHeapManagedMemorySize); + } + + public MemorySize getJvmDirectMemorySize() { + return taskOffHeapSize.add(shuffleMemSize); + } + + @Override + public String toString() { + return "TaskExecutorResourceSpec {" + + "frameworkHeapSize=" + frameworkHeapSize.toString() + + ", taskHeapSize=" + taskHeapSize.toString() + + ", taskOffHeapSize=" + taskOffHeapSize.toString() + + ", shuffleMemSize=" + shuffleMemSize.toString() + + ", onHeapManagedMemorySize=" + onHeapManagedMemorySize.toString() + + ", offHeapManagedMemorySize=" + offHeapManagedMemorySize.toString() + + ", jvmMetaspaceSize=" + jvmMetaspaceSize.toString() + + ", jvmOverheadSize=" + jvmOverheadSize.toString() + + "}"; + } } 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 082d4b7d7356..b7fab2ba6537 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 @@ -45,11 +45,8 @@ private TaskExecutorResourceUtils() {} // ------------------------------------------------------------------------ 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 jvmHeapSize = taskExecutorResourceSpec.getJvmHeapMemorySize(); + final MemorySize jvmDirectSize = taskExecutorResourceSpec.getJvmDirectMemorySize(); final MemorySize jvmMetaspaceSize = taskExecutorResourceSpec.getJvmMetaspaceSize(); return "-Xmx" + jvmHeapSize.getBytes() From 306b83b2f6c64b37162a0aa3b6ce290c4b34bc4f Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Fri, 8 Nov 2019 09:59:24 +0800 Subject: [PATCH 672/746] [FLINK-14637][core][runtime] Introduce config option for framework off-heap memory. At the moment after FLINK-13982, when we do not account for adhoc direct memory allocations for Flink framework (except network buffers) or done by some libraries used in Flink. In general, we expect this allocations to stay under a certain reasonably low limit but we have to have some margin for them so that JVM direct memory limit is not exactly equal to network buffers and does not fail. We can address it by introducing framework off heap memory config option. --- .../task_manager_memory_configuration.html | 5 ++++ .../configuration/TaskManagerOptions.java | 10 ++++++++ .../TaskExecutorResourceSpec.java | 13 ++++++++-- .../TaskExecutorResourceUtils.java | 25 ++++++++++++++++--- .../TaskExecutorResourceUtilsTest.java | 18 ++++++++++--- 5 files changed, 62 insertions(+), 9 deletions(-) diff --git a/docs/_includes/generated/task_manager_memory_configuration.html b/docs/_includes/generated/task_manager_memory_configuration.html index ca65770a6ec1..0f4ea879f7e6 100644 --- a/docs/_includes/generated/task_manager_memory_configuration.html +++ b/docs/_includes/generated/task_manager_memory_configuration.html @@ -14,6 +14,11 @@ String 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.framework.off-heap.size
    + "64m" + Framework Off-Heap Memory size for TaskExecutors. This is the size of off-heap memory (JVM direct memory or native memory) reserved for TaskExecutor framework, which will not be allocated to task slots. It will be accounted as part of the JVM max direct memory size limit. +
    taskmanager.memory.jvm-metaspace.size
    "192m" 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 b190ee8d1b8a..6d89c2c5b988 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 @@ -278,6 +278,16 @@ public class TaskManagerOptions { .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."); + /** + * Framework Off-Heap Memory size for TaskExecutors. + */ + public static final ConfigOption FRAMEWORK_OFF_HEAP_MEMORY = + key("taskmanager.memory.framework.off-heap.size") + .defaultValue("64m") + .withDescription("Framework Off-Heap Memory size for TaskExecutors. This is the size of off-heap memory" + + " (JVM direct memory or native memory) reserved for TaskExecutor framework, which will not be" + + " allocated to task slots. It will be accounted as part of the JVM max direct memory size limit."); + /** * Task Heap Memory size for TaskExecutors. */ 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 index 3f6f4fee39b0..4e42eca3ef52 100644 --- 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 @@ -78,6 +78,8 @@ public class TaskExecutorResourceSpec { private final MemorySize frameworkHeapSize; + private final MemorySize frameworkOffHeapMemorySize; + private final MemorySize taskHeapSize; private final MemorySize taskOffHeapSize; @@ -94,6 +96,7 @@ public class TaskExecutorResourceSpec { public TaskExecutorResourceSpec( MemorySize frameworkHeapSize, + MemorySize frameworkOffHeapSize, MemorySize taskHeapSize, MemorySize taskOffHeapSize, MemorySize shuffleMemSize, @@ -103,6 +106,7 @@ public TaskExecutorResourceSpec( MemorySize jvmOverheadSize) { this.frameworkHeapSize = frameworkHeapSize; + this.frameworkOffHeapMemorySize = frameworkOffHeapSize; this.taskHeapSize = taskHeapSize; this.taskOffHeapSize = taskOffHeapSize; this.shuffleMemSize = shuffleMemSize; @@ -116,6 +120,10 @@ public MemorySize getFrameworkHeapSize() { return frameworkHeapSize; } + public MemorySize getFrameworkOffHeapMemorySize() { + return frameworkOffHeapMemorySize; + } + public MemorySize getTaskHeapSize() { return taskHeapSize; } @@ -149,7 +157,7 @@ public MemorySize getJvmOverheadSize() { } public MemorySize getTotalFlinkMemorySize() { - return frameworkHeapSize.add(taskHeapSize).add(taskOffHeapSize).add(shuffleMemSize).add(getManagedMemorySize()); + return frameworkHeapSize.add(frameworkOffHeapMemorySize).add(taskHeapSize).add(taskOffHeapSize).add(shuffleMemSize).add(getManagedMemorySize()); } public MemorySize getTotalProcessMemorySize() { @@ -161,13 +169,14 @@ public MemorySize getJvmHeapMemorySize() { } public MemorySize getJvmDirectMemorySize() { - return taskOffHeapSize.add(shuffleMemSize); + return frameworkOffHeapMemorySize.add(taskOffHeapSize).add(shuffleMemSize); } @Override public String toString() { return "TaskExecutorResourceSpec {" + "frameworkHeapSize=" + frameworkHeapSize.toString() + + ", frameworkOffHeapSize=" + frameworkOffHeapMemorySize.toString() + ", taskHeapSize=" + taskHeapSize.toString() + ", taskOffHeapSize=" + taskOffHeapSize.toString() + ", shuffleMemSize=" + shuffleMemSize.toString() 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 b7fab2ba6537..cca3b19f4765 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 @@ -62,6 +62,7 @@ public static String generateJvmParametersStr(final TaskExecutorResourceSpec tas 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.FRAMEWORK_OFF_HEAP_MEMORY.key(), taskExecutorResourceSpec.getFrameworkOffHeapMemorySize().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"); @@ -108,13 +109,14 @@ private static TaskExecutorResourceSpec deriveResourceSpecWithExplicitTaskAndMan final MemorySize managedMemorySize = getManagedMemorySize(config); final MemorySize frameworkHeapMemorySize = getFrameworkHeapMemorySize(config); + final MemorySize frameworkOffHeapMemorySize = getFrameworkOffHeapMemorySize(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); + frameworkHeapMemorySize.add(frameworkOffHeapMemorySize).add(taskHeapMemorySize).add(taskOffHeapMemorySize).add(managedMemorySize); if (isTotalFlinkMemorySizeExplicitlyConfigured(config)) { // derive shuffle memory from total flink memory, and check against shuffle min/max @@ -122,6 +124,7 @@ private static TaskExecutorResourceSpec deriveResourceSpecWithExplicitTaskAndMan if (totalFlinkExcludeShuffleMemorySize.getBytes() > totalFlinkMemorySize.getBytes()) { throw new IllegalConfigurationException( "Sum of configured Framework Heap Memory (" + frameworkHeapMemorySize.toString() + + "), Framework Off-Heap Memory (" + frameworkOffHeapMemorySize.toString() + "), Task Heap Memory (" + taskHeapMemorySize.toString() + "), Task Off-Heap Memory (" + taskOffHeapMemorySize.toString() + ") and Managed Memory (" + managedMemorySize.toString() @@ -140,6 +143,7 @@ private static TaskExecutorResourceSpec deriveResourceSpecWithExplicitTaskAndMan final FlinkInternalMemory flinkInternalMemory = new FlinkInternalMemory( frameworkHeapMemorySize, + frameworkOffHeapMemorySize, taskHeapMemorySize, taskOffHeapMemorySize, shuffleMemorySize, @@ -203,6 +207,7 @@ private static JvmMetaspaceAndOverhead deriveJvmMetaspaceAndOverheadFromTotalFli private static FlinkInternalMemory deriveInternalMemoryFromTotalFlinkMemory( final Configuration config, final MemorySize totalFlinkMemorySize) { final MemorySize frameworkHeapMemorySize = getFrameworkHeapMemorySize(config); + final MemorySize frameworkOffHeapMemorySize = getFrameworkOffHeapMemorySize(config); final MemorySize taskOffHeapMemorySize = getTaskOffHeapMemorySize(config); final MemorySize taskHeapMemorySize; @@ -215,10 +220,11 @@ private static FlinkInternalMemory deriveInternalMemoryFromTotalFlinkMemory( taskHeapMemorySize = getTaskHeapMemorySize(config); managedMemorySize = deriveManagedMemoryAbsoluteOrWithFraction(config, totalFlinkMemorySize); final MemorySize totalFlinkExcludeShuffleMemorySize = - frameworkHeapMemorySize.add(taskHeapMemorySize).add(taskOffHeapMemorySize).add(managedMemorySize); + frameworkHeapMemorySize.add(frameworkOffHeapMemorySize).add(taskHeapMemorySize).add(taskOffHeapMemorySize).add(managedMemorySize); if (totalFlinkExcludeShuffleMemorySize.getBytes() > totalFlinkMemorySize.getBytes()) { throw new IllegalConfigurationException( "Sum of configured Framework Heap Memory (" + frameworkHeapMemorySize.toString() + + "), Framework Off-Heap Memory (" + frameworkOffHeapMemorySize.toString() + "), Task Heap Memory (" + taskHeapMemorySize.toString() + "), Task Off-Heap Memory (" + taskOffHeapMemorySize.toString() + ") and Managed Memory (" + managedMemorySize.toString() @@ -236,10 +242,11 @@ private static FlinkInternalMemory deriveInternalMemoryFromTotalFlinkMemory( } shuffleMemorySize = deriveShuffleMemoryWithFraction(config, totalFlinkMemorySize); final MemorySize totalFlinkExcludeTaskHeapMemorySize = - frameworkHeapMemorySize.add(taskOffHeapMemorySize).add(managedMemorySize).add(shuffleMemorySize); + frameworkHeapMemorySize.add(frameworkOffHeapMemorySize).add(taskOffHeapMemorySize).add(managedMemorySize).add(shuffleMemorySize); if (totalFlinkExcludeTaskHeapMemorySize.getBytes() > totalFlinkMemorySize.getBytes()) { throw new IllegalConfigurationException( "Sum of configured Framework Heap Memory (" + frameworkHeapMemorySize.toString() + + "), Framework Off-Heap Memory (" + frameworkOffHeapMemorySize.toString() + "), Task Off-Heap Memory (" + taskOffHeapMemorySize.toString() + "), Managed Memory (" + managedMemorySize.toString() + ") and Shuffle Memory (" + shuffleMemorySize.toString() @@ -251,6 +258,7 @@ private static FlinkInternalMemory deriveInternalMemoryFromTotalFlinkMemory( final OnHeapAndOffHeapManagedMemory onHeapAndOffHeapManagedMemory = deriveOnHeapAndOffHeapMemoryFromManagedMemory(config, managedMemorySize); final FlinkInternalMemory flinkInternalMemory = new FlinkInternalMemory( frameworkHeapMemorySize, + frameworkOffHeapMemorySize, taskHeapMemorySize, taskOffHeapMemorySize, shuffleMemorySize, @@ -328,6 +336,10 @@ private static MemorySize getFrameworkHeapMemorySize(final Configuration config) return MemorySize.parse(config.getString(TaskManagerOptions.FRAMEWORK_HEAP_MEMORY)); } + private static MemorySize getFrameworkOffHeapMemorySize(final Configuration config) { + return MemorySize.parse(config.getString(TaskManagerOptions.FRAMEWORK_OFF_HEAP_MEMORY)); + } + private static MemorySize getTaskHeapMemorySize(final Configuration config) { checkArgument(isTaskHeapMemorySizeExplicitlyConfigured(config)); return MemorySize.parse(config.getString(TaskManagerOptions.TASK_HEAP_MEMORY)); @@ -470,6 +482,7 @@ private static void sanityCheckTotalFlinkMemory(final Configuration config, fina + ") 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() + + "), Framework Off-Heap Memory (" + flinkInternalMemory.frameworkOffHeap.toString() + "), Task Heap Memory (" + flinkInternalMemory.taskHeap.toString() + "), Task Off-Heap Memory (" + flinkInternalMemory.taskOffHeap.toString() + "), Shuffle Memory (" + flinkInternalMemory.shuffle.toString() @@ -525,6 +538,7 @@ private static TaskExecutorResourceSpec createTaskExecutorResourceSpec( final FlinkInternalMemory flinkInternalMemory, final JvmMetaspaceAndOverhead jvmMetaspaceAndOverhead) { return new TaskExecutorResourceSpec( flinkInternalMemory.frameworkHeap, + flinkInternalMemory.frameworkOffHeap, flinkInternalMemory.taskHeap, flinkInternalMemory.taskOffHeap, flinkInternalMemory.shuffle, @@ -550,6 +564,7 @@ private static class RangeFraction { private static class FlinkInternalMemory { final MemorySize frameworkHeap; + final MemorySize frameworkOffHeap; final MemorySize taskHeap; final MemorySize taskOffHeap; final MemorySize shuffle; @@ -558,6 +573,7 @@ private static class FlinkInternalMemory { FlinkInternalMemory( final MemorySize frameworkHeap, + final MemorySize frameworkOffHeap, final MemorySize taskHeap, final MemorySize taskOffHeap, final MemorySize shuffle, @@ -565,6 +581,7 @@ private static class FlinkInternalMemory { final MemorySize offHeapManaged) { this.frameworkHeap = checkNotNull(frameworkHeap); + this.frameworkOffHeap = checkNotNull(frameworkOffHeap); this.taskHeap = checkNotNull(taskHeap); this.taskOffHeap = checkNotNull(taskOffHeap); this.shuffle = checkNotNull(shuffle); @@ -573,7 +590,7 @@ private static class FlinkInternalMemory { } MemorySize getTotalFlinkMemorySize() { - return frameworkHeap.add(taskHeap).add(taskOffHeap).add(shuffle).add(getManagedMemorySize()); + return frameworkHeap.add(frameworkOffHeap).add(taskHeap).add(taskOffHeap).add(shuffle).add(getManagedMemorySize()); } MemorySize getManagedMemorySize() { 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 29abfc2687a5..ebc8f392665c 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 @@ -44,7 +44,7 @@ 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_FLINK_MEM_SIZE = MemorySize.parse("900m"); private static final MemorySize TOTAL_PROCESS_MEM_SIZE = MemorySize.parse("1g"); private static final TaskExecutorResourceSpec TM_RESOURCE_SPEC = new TaskExecutorResourceSpec( @@ -55,7 +55,8 @@ public class TaskExecutorResourceUtilsTest extends TestLogger { MemorySize.parse("5m"), MemorySize.parse("6m"), MemorySize.parse("7m"), - MemorySize.parse("8m")); + MemorySize.parse("8m"), + MemorySize.parse("9m")); @Test public void testGenerateDynamicConfigurations() { @@ -75,6 +76,7 @@ public void testGenerateDynamicConfigurations() { } assertThat(MemorySize.parse(configs.get(TaskManagerOptions.FRAMEWORK_HEAP_MEMORY.key())), is(TM_RESOURCE_SPEC.getFrameworkHeapSize())); + assertThat(MemorySize.parse(configs.get(TaskManagerOptions.FRAMEWORK_OFF_HEAP_MEMORY.key())), is(TM_RESOURCE_SPEC.getFrameworkOffHeapMemorySize())); 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())); @@ -106,7 +108,7 @@ public void testGenerateJvmParameters() throws Exception { 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(directSize, is(TM_RESOURCE_SPEC.getFrameworkOffHeapMemorySize().add(TM_RESOURCE_SPEC.getTaskOffHeapSize()).add(TM_RESOURCE_SPEC.getShuffleMemSize()))); assertThat(metaspaceSize, is(TM_RESOURCE_SPEC.getJvmMetaspaceSize())); } @@ -119,6 +121,16 @@ public void testGenerateJvmParameters() throws Exception { validateInAllConfigurations(conf, taskExecutorResourceSpec -> assertThat(taskExecutorResourceSpec.getFrameworkHeapSize(), is(frameworkHeapSize))); } + @Test + public void testConfigFrameworkOffHeapMemory() { + final MemorySize frameworkOffHeapSize = MemorySize.parse("10m"); + + Configuration conf = new Configuration(); + conf.setString(TaskManagerOptions.FRAMEWORK_OFF_HEAP_MEMORY, frameworkOffHeapSize.getMebiBytes() + "m"); + + validateInAllConfigurations(conf, taskExecutorResourceSpec -> assertThat(taskExecutorResourceSpec.getFrameworkOffHeapMemorySize(), is(frameworkOffHeapSize))); + } + @Test public void testConfigTaskHeapMemory() { final MemorySize taskHeapSize = MemorySize.parse("50m"); From 6c4dfcf8a0ba9f53338ac62feacf9444fec72415 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Thu, 14 Nov 2019 17:14:30 +0100 Subject: [PATCH 673/746] [FLINK-14750][configuration] Migrate duration and memory size ConfigOptions in RestartStrategyOptions This closes #10216 --- .../restartstrategy/RestartStrategies.java | 5 ++++ .../configuration/RestartStrategyOptions.java | 18 +++++++++----- ...FailureRateRestartBackoffTimeStrategy.java | 24 +++---------------- .../FixedDelayRestartBackoffTimeStrategy.java | 12 +--------- .../restart/FailureRateRestartStrategy.java | 17 +++++++------ .../restart/FixedDelayRestartStrategy.java | 14 +---------- .../restart/RestartStrategyFactoryTest.java | 2 +- .../runtime/jobmaster/JobMasterTest.java | 2 +- ...mpleRecoveryFailureRateStrategyITBase.java | 6 +++-- ...coveryFixedDelayRestartStrategyITBase.java | 4 +++- 10 files changed, 39 insertions(+), 65 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/restartstrategy/RestartStrategies.java b/flink-core/src/main/java/org/apache/flink/api/common/restartstrategy/RestartStrategies.java index 4f67290ac36d..b1d1b53f5e93 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/restartstrategy/RestartStrategies.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/restartstrategy/RestartStrategies.java @@ -95,6 +95,11 @@ private RestartStrategyConfiguration() {} * @return Description of the restart strategy */ public abstract String getDescription(); + + @Override + public String toString() { + return getDescription(); + } } /** 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 8ff258178115..adc82fb0c6ff 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 @@ -23,6 +23,8 @@ import org.apache.flink.annotation.docs.ConfigGroups; import org.apache.flink.configuration.description.Description; +import java.time.Duration; + import static org.apache.flink.configuration.description.LinkElement.link; import static org.apache.flink.configuration.description.TextElement.code; import static org.apache.flink.configuration.description.TextElement.text; @@ -69,6 +71,7 @@ public class RestartStrategyOptions { public static final ConfigOption RESTART_STRATEGY_FIXED_DELAY_ATTEMPTS = ConfigOptions .key("restart-strategy.fixed-delay.attempts") + .intType() .defaultValue(1) .withDescription( Description.builder() @@ -78,9 +81,10 @@ public class RestartStrategyOptions { code("fixed-delay")) .build()); - public static final ConfigOption RESTART_STRATEGY_FIXED_DELAY_DELAY = ConfigOptions + public static final ConfigOption RESTART_STRATEGY_FIXED_DELAY_DELAY = ConfigOptions .key("restart-strategy.fixed-delay.delay") - .defaultValue("1 s") + .durationType() + .defaultValue(Duration.ofSeconds(1)) .withDescription( Description.builder() .text( @@ -103,9 +107,10 @@ public class RestartStrategyOptions { code("failure-rate")) .build()); - public static final ConfigOption RESTART_STRATEGY_FAILURE_RATE_FAILURE_RATE_INTERVAL = ConfigOptions + public static final ConfigOption RESTART_STRATEGY_FAILURE_RATE_FAILURE_RATE_INTERVAL = ConfigOptions .key("restart-strategy.failure-rate.failure-rate-interval") - .defaultValue("1 min") + .durationType() + .defaultValue(Duration.ofMinutes(1)) .withDescription( Description.builder() .text( @@ -115,9 +120,10 @@ public class RestartStrategyOptions { code("failure-rate")) .build()); - public static final ConfigOption RESTART_STRATEGY_FAILURE_RATE_DELAY = ConfigOptions + public static final ConfigOption RESTART_STRATEGY_FAILURE_RATE_DELAY = ConfigOptions .key("restart-strategy.failure-rate.delay") - .defaultValue("1 s") + .durationType() + .defaultValue(Duration.ofSeconds(1)) .withDescription( Description.builder() .text( 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 2d589112e8a2..b1895b5bee8f 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 @@ -22,7 +22,6 @@ 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; @@ -110,26 +109,9 @@ private String generateStrategyString() { public static FailureRateRestartBackoffTimeStrategyFactory createFactory(final Configuration configuration) { 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); - } + long failuresInterval = configuration.get(RestartStrategyOptions.RESTART_STRATEGY_FAILURE_RATE_FAILURE_RATE_INTERVAL) + .toMillis(); + long delay = configuration.get(RestartStrategyOptions.RESTART_STRATEGY_FAILURE_RATE_DELAY).toMillis(); 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 ca786d3fb5d7..00c9315d0a52 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 @@ -20,7 +20,6 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.RestartStrategyOptions; -import org.apache.flink.util.TimeUtils; import static org.apache.flink.util.Preconditions.checkArgument; @@ -84,16 +83,7 @@ private String generateStrategyString() { public static FixedDelayRestartBackoffTimeStrategyFactory createFactory(final Configuration configuration) { 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); - } + long delay = configuration.get(RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_DELAY).toMillis(); return new FixedDelayRestartBackoffTimeStrategyFactory(maxAttempts, delay); } 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 69c6350baf12..9979d89e3b04 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,9 +25,7 @@ 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; @@ -91,13 +89,14 @@ public String toString() { public static FailureRateRestartStrategyFactory createFactory(Configuration configuration) throws Exception { 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); - - Duration failuresInterval = TimeUtils.parseDuration(failuresIntervalString); - Duration delay = TimeUtils.parseDuration(delayString); - - return new FailureRateRestartStrategyFactory(maxFailuresPerInterval, Time.milliseconds(failuresInterval.toMillis()), Time.milliseconds(delay.toMillis())); + long failuresInterval = configuration.get(RestartStrategyOptions.RESTART_STRATEGY_FAILURE_RATE_FAILURE_RATE_INTERVAL) + .toMillis(); + long delay = configuration.get(RestartStrategyOptions.RESTART_STRATEGY_FAILURE_RATE_DELAY).toMillis(); + + return new FailureRateRestartStrategyFactory( + maxFailuresPerInterval, + Time.milliseconds(failuresInterval), + Time.milliseconds(delay)); } public static class FailureRateRestartStrategyFactory extends RestartStrategyFactory { 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 95b84cb49f08..28a7335a2e1e 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,7 +25,6 @@ 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; @@ -75,18 +74,7 @@ public CompletableFuture restart(final RestartCallback restarter, Schedule */ public static FixedDelayRestartStrategyFactory createFactory(Configuration configuration) throws Exception { 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 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')", ex); - } + long delay = configuration.get(RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_DELAY).toMillis(); return new FixedDelayRestartStrategyFactory(maxAttempts, delay); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/restart/RestartStrategyFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/restart/RestartStrategyFactoryTest.java index c26c13cec7b6..24b8634aa954 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/restart/RestartStrategyFactoryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/restart/RestartStrategyFactoryTest.java @@ -61,7 +61,7 @@ public void createRestartStrategyFactory_fixedDelayRestartStrategyConfigured_ret final Configuration configuration = new Configuration(); configuration.setString(RestartStrategyOptions.RESTART_STRATEGY, "fixed-delay"); configuration.setInteger(RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_ATTEMPTS, attempts); - configuration.setString(RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_DELAY, delayBetweenRestartAttempts.getSeconds() + "s"); + configuration.set(RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_DELAY, delayBetweenRestartAttempts); final RestartStrategyFactory restartStrategyFactory = RestartStrategyFactory.createRestartStrategyFactory(configuration); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java index 39685f2a3b8e..757f470b9f27 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java @@ -1053,7 +1053,7 @@ public void testRequestNextInputSplitWithLocalFailover() throws Exception { @Test public void testRequestNextInputSplitWithGlobalFailover() throws Exception { configuration.setInteger(RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_ATTEMPTS, 1); - configuration.setString(RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 s"); + configuration.set(RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_DELAY, Duration.ofSeconds(0)); final Function>, Collection> expectAllRemainingInputSplits = this::flattenCollection; diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFailureRateStrategyITBase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFailureRateStrategyITBase.java index 7cf356ac2657..c29701754dce 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFailureRateStrategyITBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFailureRateStrategyITBase.java @@ -27,6 +27,8 @@ import org.junit.ClassRule; import org.junit.experimental.categories.Category; +import java.time.Duration; + /** * Test cluster configuration with failure-rate recovery. */ @@ -45,8 +47,8 @@ private static Configuration getConfiguration() { Configuration config = new Configuration(); config.setString(RestartStrategyOptions.RESTART_STRATEGY, "failure-rate"); config.setInteger(RestartStrategyOptions.RESTART_STRATEGY_FAILURE_RATE_MAX_FAILURES_PER_INTERVAL, 1); - config.setString(RestartStrategyOptions.RESTART_STRATEGY_FAILURE_RATE_FAILURE_RATE_INTERVAL, "1 second"); - config.setString(RestartStrategyOptions.RESTART_STRATEGY_FAILURE_RATE_DELAY, "0 s"); + config.set(RestartStrategyOptions.RESTART_STRATEGY_FAILURE_RATE_FAILURE_RATE_INTERVAL, Duration.ofSeconds(1)); + config.set(RestartStrategyOptions.RESTART_STRATEGY_FAILURE_RATE_DELAY, Duration.ofSeconds(0)); return config; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFixedDelayRestartStrategyITBase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFixedDelayRestartStrategyITBase.java index eec385906475..45739bb3b9e5 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFixedDelayRestartStrategyITBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFixedDelayRestartStrategyITBase.java @@ -27,6 +27,8 @@ import org.junit.ClassRule; import org.junit.experimental.categories.Category; +import java.time.Duration; + /** * Test cluster configuration with fixed-delay recovery. */ @@ -45,7 +47,7 @@ private static Configuration getConfiguration() { Configuration config = new Configuration(); config.setString(RestartStrategyOptions.RESTART_STRATEGY, "fixed-delay"); config.setInteger(RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_ATTEMPTS, 1); - config.setString(RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_DELAY, "100 ms"); + config.set(RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_DELAY, Duration.ofMillis(100)); return config; } From d8a5d411cdb4e468f71955e66e42fb1744c48987 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Fri, 15 Nov 2019 08:59:55 +0100 Subject: [PATCH 674/746] [FLINK-14750][documentation] Regenerated restart strategy documentation --- .../failure_rate_restart_strategy_configuration.html | 8 ++++---- .../fixed_delay_restart_strategy_configuration.html | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/_includes/generated/failure_rate_restart_strategy_configuration.html b/docs/_includes/generated/failure_rate_restart_strategy_configuration.html index 06f600a64623..5485097bab86 100644 --- a/docs/_includes/generated/failure_rate_restart_strategy_configuration.html +++ b/docs/_includes/generated/failure_rate_restart_strategy_configuration.html @@ -10,14 +10,14 @@
    restart-strategy.failure-rate.delay
    - "1 s" - String + 1000ms + Duration 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" - String + 60000ms + Duration 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" diff --git a/docs/_includes/generated/fixed_delay_restart_strategy_configuration.html b/docs/_includes/generated/fixed_delay_restart_strategy_configuration.html index f3f2a323a1d7..8958f3f1af41 100644 --- a/docs/_includes/generated/fixed_delay_restart_strategy_configuration.html +++ b/docs/_includes/generated/fixed_delay_restart_strategy_configuration.html @@ -16,8 +16,8 @@
    restart-strategy.fixed-delay.delay
    - "1 s" - String + 1000ms + Duration 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" From ec12ec02baf557a7b38db9d5819179d16e13c653 Mon Sep 17 00:00:00 2001 From: Andrey Zagrebin Date: Mon, 18 Nov 2019 12:26:15 +0300 Subject: [PATCH 675/746] [hotfix] Add String type in config docs for taskmanager.memory.framework.off-heap.size --- docs/_includes/generated/task_manager_memory_configuration.html | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/_includes/generated/task_manager_memory_configuration.html b/docs/_includes/generated/task_manager_memory_configuration.html index 0f4ea879f7e6..c4a6ce77595c 100644 --- a/docs/_includes/generated/task_manager_memory_configuration.html +++ b/docs/_includes/generated/task_manager_memory_configuration.html @@ -17,6 +17,7 @@
    taskmanager.memory.framework.off-heap.size
    "64m" + String Framework Off-Heap Memory size for TaskExecutors. This is the size of off-heap memory (JVM direct memory or native memory) reserved for TaskExecutor framework, which will not be allocated to task slots. It will be accounted as part of the JVM max direct memory size limit. From dcc1330375826b779e4902176bb2473704dabb11 Mon Sep 17 00:00:00 2001 From: godfrey he Date: Mon, 18 Nov 2019 23:48:34 +0800 Subject: [PATCH 676/746] [FLINK-14694] [table-planner-blink] Enabled all tests from package o.a.f.table.planner.functions.aggfunctions * enabled tests that were previously not run because they were defined in abstract classes * fixed assertions for BinaryGeneric in those tests --- .../aggfunctions/AggFunctionTestBase.java | 72 +- ...LastValueAggFunctionWithOrderTestBase.java | 23 +- .../FirstValueAggFunctionWithOrderTest.java | 733 +++++++--------- ...FirstValueAggFunctionWithoutOrderTest.java | 508 +++++------ ...ueWithRetractAggFunctionWithOrderTest.java | 742 +++++++--------- ...ithRetractAggFunctionWithoutOrderTest.java | 515 +++++------ .../LastValueAggFunctionWithOrderTest.java | 734 +++++++--------- .../LastValueAggFunctionWithoutOrderTest.java | 510 +++++------ ...ueWithRetractAggFunctionWithOrderTest.java | 754 ++++++++-------- ...ithRetractAggFunctionWithoutOrderTest.java | 516 +++++------ .../ListAggWsWithRetractAggFunctionTest.java | 5 +- .../MaxWithRetractAggFunctionTest.java | 816 +++++++----------- .../MinWithRetractAggFunctionTest.java | 808 +++++++---------- 13 files changed, 2923 insertions(+), 3813 deletions(-) diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/AggFunctionTestBase.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/AggFunctionTestBase.java index 6b46e567dc8d..62023598c0a9 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/AggFunctionTestBase.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/AggFunctionTestBase.java @@ -18,12 +18,19 @@ package org.apache.flink.table.planner.functions.aggfunctions; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.table.api.TableException; +import org.apache.flink.table.dataformat.BinaryGeneric; +import org.apache.flink.table.dataformat.GenericRow; import org.apache.flink.table.functions.AggregateFunction; import org.apache.flink.table.planner.functions.aggfunctions.MaxWithRetractAggFunction.MaxWithRetractAccumulator; import org.apache.flink.table.planner.functions.aggfunctions.MinWithRetractAggFunction.MinWithRetractAccumulator; import org.apache.flink.table.planner.functions.utils.UserDefinedFunctionUtils; +import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo; +import org.apache.flink.table.runtime.typeutils.BinaryGenericSerializer; import org.apache.flink.util.Preconditions; import org.junit.Test; @@ -35,7 +42,9 @@ import java.util.Collections; import java.util.List; +import static org.apache.flink.table.utils.BinaryGenericAsserter.equivalent; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; /** * Base class for aggregate function test. @@ -45,6 +54,29 @@ */ public abstract class AggFunctionTestBase { + /** + * Spec for parameterized aggregate function tests. + */ + protected static class AggFunctionTestSpec { + final AggregateFunction aggregator; + final List> inputValueSets; + final List expectedResults; + + public AggFunctionTestSpec( + AggregateFunction aggregator, + List> inputValueSets, + List expectedResults) { + this.aggregator = aggregator; + this.inputValueSets = inputValueSets; + this.expectedResults = expectedResults; + } + + @Override + public String toString() { + return aggregator.getClass().getSimpleName(); + } + } + protected abstract List> getInputValueSets(); protected abstract List getExpectedResults(); @@ -77,20 +109,20 @@ public void testAccumulateAndRetractWithoutMerge() T expected = expectedResults.get(i); ACC acc = accumulateValues(inputValues); T result = aggregator.getValue(acc); - validateResult(expected, result); + validateResult(expected, result, aggregator.getAccumulatorType()); if (UserDefinedFunctionUtils.ifMethodExistInFunction("retract", aggregator)) { retractValues(acc, inputValues); ACC expectedAcc = aggregator.createAccumulator(); // The two accumulators should be exactly same - validateResult(expectedAcc, acc); + validateResult(expectedAcc, acc, aggregator.getAccumulatorType()); } } } @Test - public void testAggregateWithMerge() throws NoSuchMethodException, InvocationTargetException, - IllegalAccessException { + public void testAggregateWithMerge() + throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { AggregateFunction aggregator = getAggregator(); if (UserDefinedFunctionUtils.ifMethodExistInFunction("merge", aggregator)) { Method mergeFunc = aggregator.getClass().getMethod("merge", getAccClass(), Iterable.class); @@ -115,14 +147,14 @@ public void testAggregateWithMerge() throws NoSuchMethodException, InvocationTar mergeFunc.invoke(aggregator, (Object) acc, accumulators); T result = aggregator.getValue(acc); - validateResult(expected, result); + validateResult(expected, result, aggregator.getResultType()); // 2. verify merge with accumulate & retract if (UserDefinedFunctionUtils.ifMethodExistInFunction("retract", aggregator)) { retractValues(acc, inputValues); ACC expectedAcc = aggregator.createAccumulator(); // The two accumulators should be exactly same - validateResult(expectedAcc, acc); + validateResult(expectedAcc, acc, aggregator.getAccumulatorType()); } } @@ -138,13 +170,14 @@ public void testAggregateWithMerge() throws NoSuchMethodException, InvocationTar mergeFunc.invoke(aggregator, (Object) acc, accumulators); T result = aggregator.getValue(acc); - validateResult(expected, result); + validateResult(expected, result, aggregator.getResultType()); } } } @Test - public void testMergeReservedAccumulator() throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { + public void testMergeReservedAccumulator() + throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { AggregateFunction aggregator = getAggregator(); boolean hasMerge = UserDefinedFunctionUtils.ifMethodExistInFunction("merge", aggregator); boolean hasRetract = UserDefinedFunctionUtils.ifMethodExistInFunction("retract", aggregator); @@ -180,7 +213,7 @@ public void testMergeReservedAccumulator() throws NoSuchMethodException, Invocat // getValue T result = aggregator.getValue(accWithSubset); - validateResult(expectedValue, result); + validateResult(expectedValue, result, aggregator.getResultType()); } } @@ -202,12 +235,12 @@ public void testResetAccumulator() throws NoSuchMethodException, InvocationTarge resetAccFunc.invoke(aggregator, (Object) acc); ACC expectedAcc = aggregator.createAccumulator(); //The accumulator after reset should be exactly same as the new accumulator - validateResult(expectedAcc, acc); + validateResult(expectedAcc, acc, aggregator.getAccumulatorType()); } } } - protected void validateResult(E expected, E result) { + protected void validateResult(E expected, E result, TypeInformation typeInfo) { if (expected instanceof BigDecimal && result instanceof BigDecimal) { // BigDecimal.equals() value and scale but we are only interested in value. assertEquals(0, ((BigDecimal) expected).compareTo((BigDecimal) result)); @@ -223,11 +256,28 @@ protected void validateResult(E expected, E result) { MaxWithRetractAccumulator r = (MaxWithRetractAccumulator) result; assertEquals(e.max, r.max); assertEquals(e.mapSize, r.mapSize); + } else if (expected instanceof BinaryGeneric && result instanceof BinaryGeneric) { + TypeSerializer serializer = typeInfo.createSerializer(new ExecutionConfig()); + assertThat( + (BinaryGeneric) result, + equivalent((BinaryGeneric) expected, new BinaryGenericSerializer(serializer))); + } else if (expected instanceof GenericRow && result instanceof GenericRow) { + validateGenericRow((GenericRow) expected, (GenericRow) result, (BaseRowTypeInfo) typeInfo); } else { assertEquals(expected, result); } } + private void validateGenericRow(GenericRow expected, GenericRow result, BaseRowTypeInfo typeInfo) { + assertEquals(expected.getArity(), result.getArity()); + + for (int i = 0; i < expected.getArity(); ++i) { + Object expectedObj = expected.getField(i); + Object resultObj = result.getField(i); + validateResult(expectedObj, resultObj, typeInfo.getTypeAt(i)); + } + } + protected ACC accumulateValues(List values) throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { AggregateFunction aggregator = getAggregator(); diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/FirstLastValueAggFunctionWithOrderTestBase.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/FirstLastValueAggFunctionWithOrderTestBase.java index 1b6bfeed307c..06f84a5fadd5 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/FirstLastValueAggFunctionWithOrderTestBase.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/FirstLastValueAggFunctionWithOrderTestBase.java @@ -36,6 +36,23 @@ */ public abstract class FirstLastValueAggFunctionWithOrderTestBase extends AggFunctionTestBase { + /** + * An AggFunctionTestSpec with input order. + */ + protected static class AggFunctionWithOrderTestSpec extends AggFunctionTestSpec { + + final List> inputOrderSets; + + public AggFunctionWithOrderTestSpec( + AggregateFunction aggregator, + List> inputOrderSets, + List> inputValueSets, + List expectedResults) { + super(aggregator, inputValueSets, expectedResults); + this.inputOrderSets = inputOrderSets; + } + } + protected Method getAccumulateFunc() throws NoSuchMethodException { return getAggregator().getClass().getMethod("accumulate", getAccClass(), Object.class, Long.class); } @@ -68,13 +85,13 @@ public void testAccumulateAndRetractWithoutMerge() T expected = expectedResults.get(i); GenericRow acc = accumulateValues(inputValues, inputOrders); T result = aggregator.getValue(acc); - validateResult(expected, result); + validateResult(expected, result, aggregator.getResultType()); if (UserDefinedFunctionUtils.ifMethodExistInFunction("retract", aggregator)) { retractValues(acc, inputValues, inputOrders); GenericRow expectedAcc = aggregator.createAccumulator(); // The two accumulators should be exactly same - validateResult(expectedAcc, acc); + validateResult(expectedAcc, acc, aggregator.getAccumulatorType()); } } } @@ -103,7 +120,7 @@ public void testResetAccumulator() throws NoSuchMethodException, InvocationTarge resetAccFunc.invoke(aggregator, (Object) acc); GenericRow expectedAcc = aggregator.createAccumulator(); //The accumulator after reset should be exactly same as the new accumulator - validateResult(expectedAcc, acc); + validateResult(expectedAcc, acc, aggregator.getAccumulatorType()); } } } diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/FirstValueAggFunctionWithOrderTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/FirstValueAggFunctionWithOrderTest.java index f72aa1be6d99..dbd9b8391cce 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/FirstValueAggFunctionWithOrderTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/FirstValueAggFunctionWithOrderTest.java @@ -33,434 +33,357 @@ import org.apache.flink.table.planner.functions.aggfunctions.FirstValueAggFunction.StringFirstValueAggFunction; import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + import java.util.Arrays; import java.util.List; +import java.util.function.Function; /** * Test case for built-in FirstValue aggregate function. * This class tests `accumulate` method with order argument. */ -public abstract class FirstValueAggFunctionWithOrderTest extends FirstLastValueAggFunctionWithOrderTestBase { - /** - * Test FirstValueAggFunction for number type. - */ - public abstract static class NumberFirstValueAggFunctionWithOrderTest - extends FirstValueAggFunctionWithOrderTest { - protected abstract T getValue(String v); +@RunWith(Parameterized.class) +public class FirstValueAggFunctionWithOrderTest extends FirstLastValueAggFunctionWithOrderTestBase { - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - getValue("1"), - null, - getValue("-99"), - getValue("3"), - null, - getValue("3"), - getValue("2"), - getValue("-99") - ), - Arrays.asList( - null, - null, - null, - null - ), - Arrays.asList( - null, - getValue("10"), - null, - getValue("5") - ) - ); - } + @Parameterized.Parameter + public AggFunctionWithOrderTestSpec aggFunctionTestSpec; - @Override - protected List> getInputOrderSets() { - return Arrays.asList( - Arrays.asList( - 10L, - 2L, - 5L, - 6L, - 11L, - 3L, - 7L, - 5L - ), - Arrays.asList( - 8L, - 6L, - 9L, - 5L - ), - Arrays.asList( - null, - 6L, - 4L, - 3L - ) - ); - } + private static final int DECIMAL_PRECISION = 20; + private static final int DECIMAL_SCALE = 6; - @Override - protected List getExpectedResults() { - return Arrays.asList( - getValue("3"), - null, - getValue("5") - ); - } + @Override + protected List> getInputOrderSets() { + return aggFunctionTestSpec.inputOrderSets; } - /** - * Test for ByteFirstValueAggFunction. - */ - public static class ByteFirstValueAggFunctionWithOrderTest - extends NumberFirstValueAggFunctionWithOrderTest { - - @Override - protected Byte getValue(String v) { - return Byte.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new ByteFirstValueAggFunction(); - } + @Override + protected List> getInputValueSets() { + return aggFunctionTestSpec.inputValueSets; } - /** - * Test for ShortFirstValueAggFunction. - */ - public static class ShortFirstValueAggFunctionWithOrderTest - extends NumberFirstValueAggFunctionWithOrderTest { - - @Override - protected Short getValue(String v) { - return Short.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new ShortFirstValueAggFunction(); - } + @Override + protected List getExpectedResults() { + return aggFunctionTestSpec.expectedResults; } - /** - * Test for IntFirstValueAggFunction. - */ - public static class IntFirstValueAggFunctionWithOrderTest - extends NumberFirstValueAggFunctionWithOrderTest { - - @Override - protected Integer getValue(String v) { - return Integer.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new IntFirstValueAggFunction(); - } - } - - /** - * Test for LongFirstValueAggFunction. - */ - public static class LongFirstValueAggFunctionWithOrderTest - extends NumberFirstValueAggFunctionWithOrderTest { - - @Override - protected Long getValue(String v) { - return Long.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new LongFirstValueAggFunction(); - } - } - - /** - * Test for FloatFirstValueAggFunction. - */ - public static class FloatFirstValueAggFunctionWithOrderTest - extends NumberFirstValueAggFunctionWithOrderTest { - - @Override - protected Float getValue(String v) { - return Float.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new FloatFirstValueAggFunction(); - } + @Override + protected AggregateFunction getAggregator() { + return aggFunctionTestSpec.aggregator; } - /** - * Test for DoubleFirstValueAggFunction. - */ - public static class DoubleFirstValueAggFunctionWithOrderTest - extends NumberFirstValueAggFunctionWithOrderTest { - - @Override - protected Double getValue(String v) { - return Double.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new DoubleFirstValueAggFunction(); - } + @Parameterized.Parameters(name = "{index}: {0}") + public static List testData() { + return Arrays.asList( + /** + * Test for ByteFirstValueAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new ByteFirstValueAggFunction(), + numberInputOrderSets(), + numberInputValueSets(Byte::valueOf), + numberExpectedResults(Byte::valueOf) + ), + /** + * Test for ShortFirstValueAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new ShortFirstValueAggFunction(), + numberInputOrderSets(), + numberInputValueSets(Short::valueOf), + numberExpectedResults(Short::valueOf) + ), + /** + * Test for IntFirstValueAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new IntFirstValueAggFunction(), + numberInputOrderSets(), + numberInputValueSets(Integer::valueOf), + numberExpectedResults(Integer::valueOf) + ), + /** + * Test for LongFirstValueAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new LongFirstValueAggFunction(), + numberInputOrderSets(), + numberInputValueSets(Long::valueOf), + numberExpectedResults(Long::valueOf) + ), + /** + * Test for FloatFirstValueAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new FloatFirstValueAggFunction(), + numberInputOrderSets(), + numberInputValueSets(Float::valueOf), + numberExpectedResults(Float::valueOf) + ), + /** + * Test for DoubleFirstValueAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new DoubleFirstValueAggFunction(), + numberInputOrderSets(), + numberInputValueSets(Double::valueOf), + numberExpectedResults(Double::valueOf) + ), + /** + * Test for BooleanFirstValueAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new BooleanFirstValueAggFunction(), + Arrays.asList( + Arrays.asList( + 6L, + 2L, + 3L + ), + Arrays.asList( + 1L, + 2L, + 3L + ), + Arrays.asList( + 10L, + 2L, + 5L, + 11L, + 3L, + 7L, + 5L + ), + Arrays.asList( + 6L, + 9L, + 5L + ), + Arrays.asList( + 4L, + 3L + ) + ), + Arrays.asList( + Arrays.asList( + false, + false, + false + ), + Arrays.asList( + true, + true, + true + ), + Arrays.asList( + true, + false, + null, + true, + false, + true, + null + ), + Arrays.asList( + null, + null, + null + ), + Arrays.asList( + null, + true + ) + ), + Arrays.asList( + false, + true, + false, + null, + true + ) + ), + /** + * Test for DecimalFirstValueAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new DecimalFirstValueAggFunction(DecimalTypeInfo.of(DECIMAL_PRECISION, DECIMAL_SCALE)), + Arrays.asList( + Arrays.asList( + 10L, + 2L, + 1L, + 5L, + null, + 3L, + 1L, + 5L, + 2L + ), + Arrays.asList( + 6L, + 5L, + null, + 8L, + null + ), + Arrays.asList( + 8L, + 6L + ) + ), + Arrays.asList( + Arrays.asList( + Decimal.castFrom("1", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("1000.000001", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-1", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-999.998999", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-999.999", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("999.999", DECIMAL_PRECISION, DECIMAL_SCALE) + ), + Arrays.asList( + null, + null, + null, + null, + null + ), + Arrays.asList( + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE) + ) + ), + Arrays.asList( + Decimal.castFrom("-1", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE) + ) + ), + /** + * Test for StringFirstValueAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new StringFirstValueAggFunction(), + Arrays.asList( + Arrays.asList( + 10L, + 2L, + 5L, + null, + 3L, + 1L, + 5L + ), + Arrays.asList( + 6L, + 5L + ), + Arrays.asList( + 8L, + 6L + ), + Arrays.asList( + 6L, + 4L, + 3L + ) + ), + Arrays.asList( + Arrays.asList( + BinaryString.fromString("abc"), + BinaryString.fromString("def"), + BinaryString.fromString("ghi"), + null, + BinaryString.fromString("jkl"), + null, + BinaryString.fromString("zzz") + ), + Arrays.asList( + null, + null + ), + Arrays.asList( + null, + BinaryString.fromString("a") + ), + Arrays.asList( + BinaryString.fromString("x"), + null, + BinaryString.fromString("e") + ) + ), + Arrays.asList( + BinaryString.fromString("def"), + null, + BinaryString.fromString("a"), + BinaryString.fromString("e") + ) + ) + ); } - /** - * Test for BooleanFirstValueAggFunction. - */ - public static class BooleanFirstValueAggFunctionWithOrderTest - extends FirstValueAggFunctionWithOrderTest { - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - false, - false, - false - ), - Arrays.asList( - true, - true, - true - ), - Arrays.asList( - true, - false, - null, - true, - false, - true, - null - ), - Arrays.asList( - null, - null, - null - ), - Arrays.asList( - null, - true - )); - } - - @Override - protected List> getInputOrderSets() { - return Arrays.asList( - Arrays.asList( - 6L, - 2L, - 3L - ), - Arrays.asList( - 1L, - 2L, - 3L - ), - Arrays.asList( - 10L, - 2L, - 5L, - 11L, - 3L, - 7L, - 5L - ), - Arrays.asList( - 6L, - 9L, - 5L - ), - Arrays.asList( - 4L, - 3L - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - false, - true, - false, - null, - true - ); - } - - @Override - protected AggregateFunction getAggregator() { - return new BooleanFirstValueAggFunction(); - } + private static List> numberInputOrderSets() { + return Arrays.asList( + Arrays.asList( + 10L, + 2L, + 5L, + 6L, + 11L, + 3L, + 7L, + 5L + ), + Arrays.asList( + 8L, + 6L, + 9L, + 5L + ), + Arrays.asList( + null, + 6L, + 4L, + 3L + ) + ); } - /** - * Test for DecimalFirstValueAggFunction. - */ - public static class DecimalFirstValueAggFunctionWithOrderTest - extends FirstValueAggFunctionWithOrderTest { - - private int precision = 20; - private int scale = 6; - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - Decimal.castFrom("1", precision, scale), - Decimal.castFrom("1000.000001", precision, scale), - Decimal.castFrom("-1", precision, scale), - Decimal.castFrom("-999.998999", precision, scale), - null, - Decimal.castFrom("0", precision, scale), - Decimal.castFrom("-999.999", precision, scale), - null, - Decimal.castFrom("999.999", precision, scale) - ), - Arrays.asList( - null, - null, - null, - null, - null - ), - Arrays.asList( - null, - Decimal.castFrom("0", precision, scale) - ) - ); - } - - @Override - protected List> getInputOrderSets() { - return Arrays.asList( - Arrays.asList( - 10L, - 2L, - 1L, - 5L, - null, - 3L, - 1L, - 5L, - 2L - ), - Arrays.asList( - 6L, - 5L, - null, - 8L, - null - ), - Arrays.asList( - 8L, - 6L - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - Decimal.castFrom("-1", precision, scale), - null, - Decimal.castFrom("0", precision, scale) - ); - } - - @Override - protected AggregateFunction getAggregator() { - return new DecimalFirstValueAggFunction(DecimalTypeInfo.of(precision, scale)); - } + private static List> numberInputValueSets(Function strToValueFun) { + return Arrays.asList( + Arrays.asList( + strToValueFun.apply("1"), + null, + strToValueFun.apply("-99"), + strToValueFun.apply("3"), + null, + strToValueFun.apply("3"), + strToValueFun.apply("2"), + strToValueFun.apply("-99") + ), + Arrays.asList( + null, + null, + null, + null + ), + Arrays.asList( + null, + strToValueFun.apply("10"), + null, + strToValueFun.apply("5") + ) + ); } - /** - * Test for StringFirstValueAggFunction. - */ - public static class StringFirstValueAggFunctionWithOrderTest - extends FirstValueAggFunctionWithOrderTest { - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - BinaryString.fromString("abc"), - BinaryString.fromString("def"), - BinaryString.fromString("ghi"), - null, - BinaryString.fromString("jkl"), - null, - BinaryString.fromString("zzz") - ), - Arrays.asList( - null, - null - ), - Arrays.asList( - null, - BinaryString.fromString("a") - ), - Arrays.asList( - BinaryString.fromString("x"), - null, - BinaryString.fromString("e") - ) - ); - } - - @Override - protected List> getInputOrderSets() { - return Arrays.asList( - Arrays.asList( - 10L, - 2L, - 5L, - null, - 3L, - 1L, - 5L - ), - Arrays.asList( - 6L, - 5L - ), - Arrays.asList( - 8L, - 6L - ), - Arrays.asList( - 6L, - 4L, - 3L - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - BinaryString.fromString("def"), - null, - BinaryString.fromString("a"), - BinaryString.fromString("e") - ); - } - - @Override - protected AggregateFunction getAggregator() { - return new StringFirstValueAggFunction(); - } + private static List numberExpectedResults(Function strToValueFun) { + return Arrays.asList( + strToValueFun.apply("3"), + null, + strToValueFun.apply("5") + ); } } diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/FirstValueAggFunctionWithoutOrderTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/FirstValueAggFunctionWithoutOrderTest.java index d72b38fb3302..3bfe52a30099 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/FirstValueAggFunctionWithoutOrderTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/FirstValueAggFunctionWithoutOrderTest.java @@ -33,319 +33,243 @@ import org.apache.flink.table.planner.functions.aggfunctions.FirstValueAggFunction.StringFirstValueAggFunction; import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + import java.util.Arrays; import java.util.List; +import java.util.function.Function; /** * Test case for built-in FirstValue aggregate function. * This class tests `accumulate` method without order argument. */ -public abstract class FirstValueAggFunctionWithoutOrderTest extends AggFunctionTestBase { - - @Override - protected Class getAccClass() { - return GenericRow.class; - } - - /** - * Test FirstValueAggFunction for number type. - */ - public abstract static class NumberFirstValueAggFunctionWithoutOrderTest - extends FirstValueAggFunctionWithoutOrderTest { - protected abstract T getValue(String v); - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - getValue("1"), - null, - getValue("-99"), - getValue("3"), - null - ), - Arrays.asList( - null, - null, - null, - null - ), - Arrays.asList( - null, - getValue("10"), - null, - getValue("3") - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - getValue("1"), - null, - getValue("10") - ); - } - } +@RunWith(Parameterized.class) +public class FirstValueAggFunctionWithoutOrderTest extends AggFunctionTestBase { - /** - * Test for ByteFirstValueAggFunction. - */ - public static class ByteFirstValueAggFunctionWithoutOrderTest - extends NumberFirstValueAggFunctionWithoutOrderTest { + @Parameterized.Parameter + public AggFunctionTestSpec aggFunctionTestSpec; - @Override - protected Byte getValue(String v) { - return Byte.valueOf(v); - } + private static final int DECIMAL_PRECISION = 20; + private static final int DECIMAL_SCALE = 6; - @Override - protected AggregateFunction getAggregator() { - return new ByteFirstValueAggFunction(); - } - } - - /** - * Test for ShortFirstValueAggFunction. - */ - public static class ShortFirstValueAggFunctionWithoutOrderTest - extends NumberFirstValueAggFunctionWithoutOrderTest { - - @Override - protected Short getValue(String v) { - return Short.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new ShortFirstValueAggFunction(); - } - } - - /** - * Test for IntFirstValueAggFunction. - */ - public static class IntFirstValueAggFunctionWithoutOrderTest - extends NumberFirstValueAggFunctionWithoutOrderTest { - - @Override - protected Integer getValue(String v) { - return Integer.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new IntFirstValueAggFunction(); - } + @Override + protected List> getInputValueSets() { + return aggFunctionTestSpec.inputValueSets; } - /** - * Test for LongFirstValueAggFunction. - */ - public static class LongFirstValueAggFunctionWithoutOrderTest - extends NumberFirstValueAggFunctionWithoutOrderTest { - - @Override - protected Long getValue(String v) { - return Long.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new LongFirstValueAggFunction(); - } + @Override + protected List getExpectedResults() { + return aggFunctionTestSpec.expectedResults; } - /** - * Test for FloatFirstValueAggFunction. - */ - public static class FloatFirstValueAggFunctionWithoutOrderTest - extends NumberFirstValueAggFunctionWithoutOrderTest { - - @Override - protected Float getValue(String v) { - return Float.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new FloatFirstValueAggFunction(); - } + @Override + protected AggregateFunction getAggregator() { + return aggFunctionTestSpec.aggregator; } - /** - * Test for DoubleFirstValueAggFunction. - */ - public static class DoubleFirstValueAggFunctionWithoutOrderTest - extends NumberFirstValueAggFunctionWithoutOrderTest { - - @Override - protected Double getValue(String v) { - return Double.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new DoubleFirstValueAggFunction(); - } + @Override + protected Class getAccClass() { + return GenericRow.class; } - /** - * Test for BooleanFirstValueAggFunction. - */ - public static class BooleanFirstValueAggFunctionWithoutOrderTest extends - FirstValueAggFunctionWithoutOrderTest { - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - false, - false, - false - ), - Arrays.asList( - true, - true, - true - ), - Arrays.asList( - true, - false, - null, - true, - false, - true, - null - ), - Arrays.asList( - null, - null, - null - ), - Arrays.asList( - null, - true - )); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - false, - true, - true, - null, - true - ); - } - - @Override - protected AggregateFunction getAggregator() { - return new BooleanFirstValueAggFunction(); - } + @Parameterized.Parameters(name = "{index}: {0}") + public static List testData() { + return Arrays.asList( + /** + * Test for ByteFirstValueAggFunction. + */ + new AggFunctionTestSpec<>( + new ByteFirstValueAggFunction(), + numberInputValueSets(Byte::valueOf), + numberExpectedResults(Byte::valueOf) + ), + /** + * Test for ShortFirstValueAggFunction. + */ + new AggFunctionTestSpec<>( + new ShortFirstValueAggFunction(), + numberInputValueSets(Short::valueOf), + numberExpectedResults(Short::valueOf) + ), + /** + * Test for IntFirstValueAggFunction. + */ + new AggFunctionTestSpec<>( + new IntFirstValueAggFunction(), + numberInputValueSets(Integer::valueOf), + numberExpectedResults(Integer::valueOf) + ), + /** + * Test for LongFirstValueAggFunction. + */ + new AggFunctionTestSpec<>( + new LongFirstValueAggFunction(), + numberInputValueSets(Long::valueOf), + numberExpectedResults(Long::valueOf) + ), + /** + * Test for FloatFirstValueAggFunction. + */ + new AggFunctionTestSpec<>( + new FloatFirstValueAggFunction(), + numberInputValueSets(Float::valueOf), + numberExpectedResults(Float::valueOf) + ), + /** + * Test for DoubleFirstValueAggFunction. + */ + new AggFunctionTestSpec<>( + new DoubleFirstValueAggFunction(), + numberInputValueSets(Double::valueOf), + numberExpectedResults(Double::valueOf) + ), + /** + * Test for BooleanFirstValueAggFunction. + */ + new AggFunctionTestSpec<>( + new BooleanFirstValueAggFunction(), + Arrays.asList( + Arrays.asList( + false, + false, + false + ), + Arrays.asList( + true, + true, + true + ), + Arrays.asList( + true, + false, + null, + true, + false, + true, + null + ), + Arrays.asList( + null, + null, + null + ), + Arrays.asList( + null, + true + ) + ), + Arrays.asList( + false, + true, + true, + null, + true + ) + ), + /** + * Test for DecimalFirstValueAggFunction. + */ + new AggFunctionTestSpec<>( + new DecimalFirstValueAggFunction(DecimalTypeInfo.of(DECIMAL_PRECISION, DECIMAL_SCALE)), + Arrays.asList( + Arrays.asList( + Decimal.castFrom("1", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("1000.000001", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-1", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-999.998999", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-999.999", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("999.999", DECIMAL_PRECISION, DECIMAL_SCALE) + ), + Arrays.asList( + null, + null, + null, + null, + null + ), + Arrays.asList( + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE) + ) + ), + Arrays.asList( + Decimal.castFrom("1", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE) + ) + ), + /** + * Test for StringFirstValueAggFunction. + */ + new AggFunctionTestSpec<>( + new StringFirstValueAggFunction(), + Arrays.asList( + Arrays.asList( + BinaryString.fromString("abc"), + BinaryString.fromString("def"), + BinaryString.fromString("ghi"), + null, + BinaryString.fromString("jkl"), + null, + BinaryString.fromString("zzz") + ), + Arrays.asList( + null, + null + ), + Arrays.asList( + null, + BinaryString.fromString("a") + ), + Arrays.asList( + BinaryString.fromString("x"), + null, + BinaryString.fromString("e") + ) + ), + Arrays.asList( + BinaryString.fromString("abc"), + null, + BinaryString.fromString("a"), + BinaryString.fromString("x") + ) + ) + ); } - /** - * Test for DecimalFirstValueAggFunction. - */ - public static class DecimalFirstValueAggFunctionWithoutOrderTest extends - FirstValueAggFunctionWithoutOrderTest { - - private int precision = 20; - private int scale = 6; - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - Decimal.castFrom("1", precision, scale), - Decimal.castFrom("1000.000001", precision, scale), - Decimal.castFrom("-1", precision, scale), - Decimal.castFrom("-999.998999", precision, scale), - null, - Decimal.castFrom("0", precision, scale), - Decimal.castFrom("-999.999", precision, scale), - null, - Decimal.castFrom("999.999", precision, scale) - ), - Arrays.asList( - null, - null, - null, - null, - null - ), - Arrays.asList( - null, - Decimal.castFrom("0", precision, scale) - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - Decimal.castFrom("1", precision, scale), - null, - Decimal.castFrom("0", precision, scale) - ); - } - - @Override - protected AggregateFunction getAggregator() { - return new DecimalFirstValueAggFunction(DecimalTypeInfo.of(precision, scale)); - } + private static List> numberInputValueSets(Function strToValueFun) { + return Arrays.asList( + Arrays.asList( + strToValueFun.apply("1"), + null, + strToValueFun.apply("-99"), + strToValueFun.apply("3"), + null + ), + Arrays.asList( + null, + null, + null, + null + ), + Arrays.asList( + null, + strToValueFun.apply("10"), + null, + strToValueFun.apply("3") + ) + ); } - /** - * Test for StringFirstValueAggFunction. - */ - public static class StringFirstValueAggFunctionWithoutOrderTest extends - FirstValueAggFunctionWithoutOrderTest { - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - BinaryString.fromString("abc"), - BinaryString.fromString("def"), - BinaryString.fromString("ghi"), - null, - BinaryString.fromString("jkl"), - null, - BinaryString.fromString("zzz") - ), - Arrays.asList( - null, - null - ), - Arrays.asList( - null, - BinaryString.fromString("a") - ), - Arrays.asList( - BinaryString.fromString("x"), - null, - BinaryString.fromString("e") - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - BinaryString.fromString("abc"), - null, - BinaryString.fromString("a"), - BinaryString.fromString("x") - ); - } - - @Override - protected AggregateFunction getAggregator() { - return new StringFirstValueAggFunction(); - } + private static List numberExpectedResults(Function strToValueFun) { + return Arrays.asList( + strToValueFun.apply("1"), + null, + strToValueFun.apply("10") + ); } } diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/FirstValueWithRetractAggFunctionWithOrderTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/FirstValueWithRetractAggFunctionWithOrderTest.java index e685e02353db..caf148579017 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/FirstValueWithRetractAggFunctionWithOrderTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/FirstValueWithRetractAggFunctionWithOrderTest.java @@ -33,442 +33,364 @@ import org.apache.flink.table.planner.functions.aggfunctions.FirstValueWithRetractAggFunction.StringFirstValueWithRetractAggFunction; import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + import java.lang.reflect.Method; import java.util.Arrays; import java.util.List; +import java.util.function.Function; /** * Test case for built-in FirstValue with retract aggregate function. * This class tests `accumulate` method with order argument. */ -public abstract class FirstValueWithRetractAggFunctionWithOrderTest - extends FirstLastValueAggFunctionWithOrderTestBase { - - @Override - protected Method getRetractFunc() throws NoSuchMethodException { - return getAggregator().getClass().getMethod("retract", getAccClass(), Object.class, Long.class); - } - - /** - * Test FirstValueWithRetractAggFunction for number type. - */ - public abstract static class NumberFirstValueWithRetractAggFunctionWithOrderTest - extends FirstValueWithRetractAggFunctionWithOrderTest { - protected abstract T getValue(String v); - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - getValue("1"), - null, - getValue("-99"), - getValue("3"), - null, - getValue("3"), - getValue("2"), - getValue("-99") - ), - Arrays.asList( - null, - null, - null, - null - ), - Arrays.asList( - null, - getValue("10"), - null, - getValue("5") - ) - ); - } - - @Override - protected List> getInputOrderSets() { - return Arrays.asList( - Arrays.asList( - 10L, - 2L, - 5L, - 6L, - 11L, - 3L, - 7L, - 5L - ), - Arrays.asList( - 8L, - 6L, - 9L, - 5L - ), - Arrays.asList( - null, - 6L, - 4L, - 3L - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - getValue("3"), - null, - getValue("5") - ); - } - } +@RunWith(Parameterized.class) +public class FirstValueWithRetractAggFunctionWithOrderTest extends FirstLastValueAggFunctionWithOrderTestBase { - /** - * Test for ByteFirstValueWithRetractAggFunction. - */ - public static class ByteFirstValueWithRetractAggFunctionWithOrderTest - extends NumberFirstValueWithRetractAggFunctionWithOrderTest { + @Parameterized.Parameter + public AggFunctionWithOrderTestSpec aggFunctionTestSpec; - @Override - protected Byte getValue(String v) { - return Byte.valueOf(v); - } + private static final int DECIMAL_PRECISION = 20; + private static final int DECIMAL_SCALE = 6; - @Override - protected AggregateFunction getAggregator() { - return new ByteFirstValueWithRetractAggFunction(); - } + @Override + protected List> getInputOrderSets() { + return aggFunctionTestSpec.inputOrderSets; } - /** - * Test for ShortFirstValueWithRetractAggFunction. - */ - public static class ShortFirstValueWithRetractAggFunctionWithOrderTest - extends NumberFirstValueWithRetractAggFunctionWithOrderTest { - - @Override - protected Short getValue(String v) { - return Short.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new ShortFirstValueWithRetractAggFunction(); - } + @Override + protected List> getInputValueSets() { + return aggFunctionTestSpec.inputValueSets; } - /** - * Test for IntFirstValueWithRetractAggFunction. - */ - public static class IntFirstValueWithRetractAggFunctionWithOrderTest - extends NumberFirstValueWithRetractAggFunctionWithOrderTest { - - @Override - protected Integer getValue(String v) { - return Integer.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new IntFirstValueWithRetractAggFunction(); - } + @Override + protected List getExpectedResults() { + return aggFunctionTestSpec.expectedResults; } - /** - * Test for LongFirstValueWithRetractAggFunction. - */ - public static class LongFirstValueWithRetractAggFunctionWithOrderTest - extends NumberFirstValueWithRetractAggFunctionWithOrderTest { - - @Override - protected Long getValue(String v) { - return Long.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new LongFirstValueWithRetractAggFunction(); - } + @Override + protected AggregateFunction getAggregator() { + return aggFunctionTestSpec.aggregator; } - /** - * Test for FloatFirstValueWithRetractAggFunction. - */ - public static class FloatFirstValueWithRetractAggFunctionWithOrderTest - extends NumberFirstValueWithRetractAggFunctionWithOrderTest { - - @Override - protected Float getValue(String v) { - return Float.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new FloatFirstValueWithRetractAggFunction(); - } + @Override + protected Method getRetractFunc() throws NoSuchMethodException { + return getAggregator().getClass().getMethod("retract", getAccClass(), Object.class, Long.class); } - /** - * Test for DoubleFirstValueWithRetractAggFunction. - */ - public static class DoubleFirstValueWithRetractAggFunctionWithOrderTest - extends NumberFirstValueWithRetractAggFunctionWithOrderTest { - - @Override - protected Double getValue(String v) { - return Double.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new DoubleFirstValueWithRetractAggFunction(); - } + @Parameterized.Parameters(name = "{index}: {0}") + public static List testData() { + return Arrays.asList( + /** + * Test for ByteFirstValueWithRetractAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new ByteFirstValueWithRetractAggFunction(), + numberInputOrderSets(), + numberInputValueSets(Byte::valueOf), + numberExpectedResults(Byte::valueOf) + ), + /** + * Test for ShortFirstValueWithRetractAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new ShortFirstValueWithRetractAggFunction(), + numberInputOrderSets(), + numberInputValueSets(Short::valueOf), + numberExpectedResults(Short::valueOf) + ), + /** + * Test for IntFirstValueWithRetractAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new IntFirstValueWithRetractAggFunction(), + numberInputOrderSets(), + numberInputValueSets(Integer::valueOf), + numberExpectedResults(Integer::valueOf) + ), + /** + * Test for LongFirstValueWithRetractAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new LongFirstValueWithRetractAggFunction(), + numberInputOrderSets(), + numberInputValueSets(Long::valueOf), + numberExpectedResults(Long::valueOf) + ), + /** + * Test for FloatFirstValueWithRetractAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new FloatFirstValueWithRetractAggFunction(), + numberInputOrderSets(), + numberInputValueSets(Float::valueOf), + numberExpectedResults(Float::valueOf) + ), + /** + * Test for DoubleFirstValueWithRetractAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new DoubleFirstValueWithRetractAggFunction(), + numberInputOrderSets(), + numberInputValueSets(Double::valueOf), + numberExpectedResults(Double::valueOf) + ), + /** + * Test for BooleanFirstValueWithRetractAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new BooleanFirstValueWithRetractAggFunction(), + Arrays.asList( + Arrays.asList( + 6L, + 2L, + 3L + ), + Arrays.asList( + 1L, + 2L, + 3L + ), + Arrays.asList( + 10L, + 2L, + 5L, + 11L, + 3L, + 7L, + 5L + ), + Arrays.asList( + 6L, + 9L, + 5L + ), + Arrays.asList( + 4L, + 3L + ) + ), + Arrays.asList( + Arrays.asList( + false, + false, + false + ), + Arrays.asList( + true, + true, + true + ), + Arrays.asList( + true, + false, + null, + true, + false, + true, + null + ), + Arrays.asList( + null, + null, + null + ), + Arrays.asList( + null, + true + ) + ), + Arrays.asList( + false, + true, + false, + null, + true + ) + ), + /** + * Test for DecimalFirstValueWithRetractAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new DecimalFirstValueWithRetractAggFunction( + DecimalTypeInfo.of(DECIMAL_PRECISION, DECIMAL_SCALE)), + Arrays.asList( + Arrays.asList( + 10L, + 2L, + 1L, + 5L, + null, + 3L, + 1L, + 5L, + 2L + ), + Arrays.asList( + 6L, + 5L, + null, + 8L, + null + ), + Arrays.asList( + 8L, + 6L + ) + ), + Arrays.asList( + Arrays.asList( + Decimal.castFrom("1", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("1000.000001", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-1", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-999.998999", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-999.999", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("999.999", DECIMAL_PRECISION, DECIMAL_SCALE) + ), + Arrays.asList( + null, + null, + null, + null, + null + ), + Arrays.asList( + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE) + ) + ), + Arrays.asList( + Decimal.castFrom("-1", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE) + ) + ), + /** + * Test for StringFirstValueWithRetractAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new StringFirstValueWithRetractAggFunction(), + Arrays.asList( + Arrays.asList( + 10L, + 2L, + 5L, + null, + 3L, + 1L, + 5L + ), + Arrays.asList( + 6L, + 5L + ), + Arrays.asList( + 8L, + 6L + ), + Arrays.asList( + 6L, + 4L, + 3L + ) + ), + Arrays.asList( + Arrays.asList( + BinaryString.fromString("abc"), + BinaryString.fromString("def"), + BinaryString.fromString("ghi"), + null, + BinaryString.fromString("jkl"), + null, + BinaryString.fromString("zzz") + ), + Arrays.asList( + null, + null + ), + Arrays.asList( + null, + BinaryString.fromString("a") + ), + Arrays.asList( + BinaryString.fromString("x"), + null, + BinaryString.fromString("e") + ) + ), + Arrays.asList( + BinaryString.fromString("def"), + null, + BinaryString.fromString("a"), + BinaryString.fromString("e") + ) + ) + ); } - /** - * Test for BooleanFirstValueWithRetractAggFunction. - */ - public static class BooleanFirstValueWithRetractAggFunctionWithOrderTest - extends FirstValueWithRetractAggFunctionWithOrderTest { - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - false, - false, - false - ), - Arrays.asList( - true, - true, - true - ), - Arrays.asList( - true, - false, - null, - true, - false, - true, - null - ), - Arrays.asList( - null, - null, - null - ), - Arrays.asList( - null, - true - )); - } - - @Override - protected List> getInputOrderSets() { - return Arrays.asList( - Arrays.asList( - 6L, - 2L, - 3L - ), - Arrays.asList( - 1L, - 2L, - 3L - ), - Arrays.asList( - 10L, - 2L, - 5L, - 11L, - 3L, - 7L, - 5L - ), - Arrays.asList( - 6L, - 9L, - 5L - ), - Arrays.asList( - 4L, - 3L - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - false, - true, - false, - null, - true - ); - } - - @Override - protected AggregateFunction getAggregator() { - return new BooleanFirstValueWithRetractAggFunction(); - } + private static List> numberInputOrderSets() { + return Arrays.asList( + Arrays.asList( + 10L, + 2L, + 5L, + 6L, + 11L, + 3L, + 7L, + 5L + ), + Arrays.asList( + 8L, + 6L, + 9L, + 5L + ), + Arrays.asList( + null, + 6L, + 4L, + 3L + ) + ); } - /** - * Test for DecimalFirstValueWithRetractAggFunction. - */ - public static class DecimalFirstValueWithRetractAggFunctionWithOrderTest - extends FirstValueWithRetractAggFunctionWithOrderTest { - - private int precision = 20; - private int scale = 6; - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - Decimal.castFrom("1", precision, scale), - Decimal.castFrom("1000.000001", precision, scale), - Decimal.castFrom("-1", precision, scale), - Decimal.castFrom("-999.998999", precision, scale), - null, - Decimal.castFrom("0", precision, scale), - Decimal.castFrom("-999.999", precision, scale), - null, - Decimal.castFrom("999.999", precision, scale) - ), - Arrays.asList( - null, - null, - null, - null, - null - ), - Arrays.asList( - null, - Decimal.castFrom("0", precision, scale) - ) - ); - } - - @Override - protected List> getInputOrderSets() { - return Arrays.asList( - Arrays.asList( - 10L, - 2L, - 1L, - 5L, - null, - 3L, - 1L, - 5L, - 2L - ), - Arrays.asList( - 6L, - 5L, - null, - 8L, - null - ), - Arrays.asList( - 8L, - 6L - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - Decimal.castFrom("-1", precision, scale), - null, - Decimal.castFrom("0", precision, scale) - ); - } - - @Override - protected AggregateFunction getAggregator() { - return new DecimalFirstValueWithRetractAggFunction(DecimalTypeInfo.of(precision, scale)); - } + private static List> numberInputValueSets(Function strToValueFun) { + return Arrays.asList( + Arrays.asList( + strToValueFun.apply("1"), + null, + strToValueFun.apply("-99"), + strToValueFun.apply("3"), + null, + strToValueFun.apply("3"), + strToValueFun.apply("2"), + strToValueFun.apply("-99") + ), + Arrays.asList( + null, + null, + null, + null + ), + Arrays.asList( + null, + strToValueFun.apply("10"), + null, + strToValueFun.apply("5") + ) + ); } - /** - * Test for StringFirstValueWithRetractAggFunction. - */ - public static class StringFirstValueWithRetractAggFunctionWithOrderTest - extends FirstValueWithRetractAggFunctionWithOrderTest { - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - BinaryString.fromString("abc"), - BinaryString.fromString("def"), - BinaryString.fromString("ghi"), - null, - BinaryString.fromString("jkl"), - null, - BinaryString.fromString("zzz") - ), - Arrays.asList( - null, - null - ), - Arrays.asList( - null, - BinaryString.fromString("a") - ), - Arrays.asList( - BinaryString.fromString("x"), - null, - BinaryString.fromString("e") - ) - ); - } - - @Override - protected List> getInputOrderSets() { - return Arrays.asList( - Arrays.asList( - 10L, - 2L, - 5L, - null, - 3L, - 1L, - 5L - ), - Arrays.asList( - 6L, - 5L - ), - Arrays.asList( - 8L, - 6L - ), - Arrays.asList( - 6L, - 4L, - 3L - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - BinaryString.fromString("def"), - null, - BinaryString.fromString("a"), - BinaryString.fromString("e") - ); - } - - @Override - protected AggregateFunction getAggregator() { - return new StringFirstValueWithRetractAggFunction(); - } + private static List numberExpectedResults(Function strToValueFun) { + return Arrays.asList( + strToValueFun.apply("3"), + null, + strToValueFun.apply("5") + ); } } diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/FirstValueWithRetractAggFunctionWithoutOrderTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/FirstValueWithRetractAggFunctionWithoutOrderTest.java index d14c31237d61..386f80302e67 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/FirstValueWithRetractAggFunctionWithoutOrderTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/FirstValueWithRetractAggFunctionWithoutOrderTest.java @@ -33,325 +33,250 @@ import org.apache.flink.table.planner.functions.aggfunctions.FirstValueWithRetractAggFunction.StringFirstValueWithRetractAggFunction; import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + import java.lang.reflect.Method; import java.util.Arrays; import java.util.List; +import java.util.function.Function; /** * Test case for built-in FirstValue with retract aggregate function. * This class tests `accumulate` method without order argument. */ -public abstract class FirstValueWithRetractAggFunctionWithoutOrderTest extends AggFunctionTestBase { - - @Override - protected Class getAccClass() { - return GenericRow.class; - } +@RunWith(Parameterized.class) +public class FirstValueWithRetractAggFunctionWithoutOrderTest extends AggFunctionTestBase { - @Override - protected Method getRetractFunc() throws NoSuchMethodException { - return getAggregator().getClass().getMethod("retract", getAccClass(), Object.class); - } - - /** - * Test FirstValueWithRetractAggFunction for number type. - */ - public abstract static class NumberFirstValueWithRetractAggFunctionWithoutOrderTest - extends FirstValueWithRetractAggFunctionWithoutOrderTest { - protected abstract T getValue(String v); + @Parameterized.Parameter + public AggFunctionTestSpec aggFunctionTestSpec; - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - getValue("1"), - null, - getValue("-99"), - getValue("3"), - null - ), - Arrays.asList( - null, - null, - null, - null - ), - Arrays.asList( - null, - getValue("10"), - null, - getValue("3") - ) - ); - } + private static final int DECIMAL_PRECISION = 20; + private static final int DECIMAL_SCALE = 6; - @Override - protected List getExpectedResults() { - return Arrays.asList( - getValue("1"), - null, - getValue("10") - ); - } - } - - /** - * Test for ByteFirstValueWithRetractAggFunction. - */ - public static class ByteFirstValueWithRetractAggFunctionWithoutOrderTest - extends NumberFirstValueWithRetractAggFunctionWithoutOrderTest { - - @Override - protected Byte getValue(String v) { - return Byte.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new ByteFirstValueWithRetractAggFunction(); - } - } - - /** - * Test for ShortFirstValueWithRetractAggFunction. - */ - public static class ShortFirstValueWithRetractAggFunctionWithoutOrderTest - extends NumberFirstValueWithRetractAggFunctionWithoutOrderTest { - - @Override - protected Short getValue(String v) { - return Short.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new ShortFirstValueWithRetractAggFunction(); - } + @Override + protected List> getInputValueSets() { + return aggFunctionTestSpec.inputValueSets; } - /** - * Test for IntFirstValueWithRetractAggFunction. - */ - public static class IntFirstValueWithRetractAggFunctionWithoutOrderTest - extends NumberFirstValueWithRetractAggFunctionWithoutOrderTest { - - @Override - protected Integer getValue(String v) { - return Integer.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new IntFirstValueWithRetractAggFunction(); - } + @Override + protected List getExpectedResults() { + return aggFunctionTestSpec.expectedResults; } - /** - * Test for LongFirstValueWithRetractAggFunction. - */ - public static class LongFirstValueWithRetractAggFunctionWithoutOrderTest - extends NumberFirstValueWithRetractAggFunctionWithoutOrderTest { - - @Override - protected Long getValue(String v) { - return Long.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new LongFirstValueWithRetractAggFunction(); - } + @Override + protected AggregateFunction getAggregator() { + return aggFunctionTestSpec.aggregator; } - /** - * Test for FloatFirstValueWithRetractAggFunction. - */ - public static class FloatFirstValueWithRetractAggFunctionWithoutOrderTest - extends NumberFirstValueWithRetractAggFunctionWithoutOrderTest { - - @Override - protected Float getValue(String v) { - return Float.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new FloatFirstValueWithRetractAggFunction(); - } + @Override + protected Class getAccClass() { + return GenericRow.class; } - /** - * Test for DoubleFirstValueWithRetractAggFunction. - */ - public static class DoubleFirstValueWithRetractAggFunctionWithoutOrderTest - extends NumberFirstValueWithRetractAggFunctionWithoutOrderTest { - - @Override - protected Double getValue(String v) { - return Double.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new DoubleFirstValueWithRetractAggFunction(); - } + @Override + protected Method getRetractFunc() throws NoSuchMethodException { + return getAggregator().getClass().getMethod("retract", getAccClass(), Object.class); } - /** - * Test for BooleanFirstValueWithRetractAggFunction. - */ - public static class BooleanFirstValueWithRetractAggFunctionWithoutOrderTest extends - FirstValueWithRetractAggFunctionWithoutOrderTest { - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - false, - false, - false - ), - Arrays.asList( - true, - true, - true - ), - Arrays.asList( - true, - false, - null, - true, - false, - true, - null - ), - Arrays.asList( - null, - null, - null - ), - Arrays.asList( - null, - true - )); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - false, - true, - true, - null, - true - ); - } - - @Override - protected AggregateFunction getAggregator() { - return new BooleanFirstValueWithRetractAggFunction(); - } + @Parameterized.Parameters(name = "{index}: {0}") + public static List testData() { + return Arrays.asList( + /** + * Test for ByteFirstValueWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new ByteFirstValueWithRetractAggFunction(), + numberInputValueSets(Byte::valueOf), + numberExpectedResults(Byte::valueOf) + ), + /** + * Test for ShortFirstValueWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new ShortFirstValueWithRetractAggFunction(), + numberInputValueSets(Short::valueOf), + numberExpectedResults(Short::valueOf) + ), + /** + * Test for IntFirstValueWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new IntFirstValueWithRetractAggFunction(), + numberInputValueSets(Integer::valueOf), + numberExpectedResults(Integer::valueOf) + ), + /** + * Test for LongFirstValueWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new LongFirstValueWithRetractAggFunction(), + numberInputValueSets(Long::valueOf), + numberExpectedResults(Long::valueOf) + ), + /** + * Test for FloatFirstValueWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new FloatFirstValueWithRetractAggFunction(), + numberInputValueSets(Float::valueOf), + numberExpectedResults(Float::valueOf) + ), + /** + * Test for DoubleFirstValueWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new DoubleFirstValueWithRetractAggFunction(), + numberInputValueSets(Double::valueOf), + numberExpectedResults(Double::valueOf) + ), + /** + * Test for BooleanFirstValueWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new BooleanFirstValueWithRetractAggFunction(), + Arrays.asList( + Arrays.asList( + false, + false, + false + ), + Arrays.asList( + true, + true, + true + ), + Arrays.asList( + true, + false, + null, + true, + false, + true, + null + ), + Arrays.asList( + null, + null, + null + ), + Arrays.asList( + null, + true + )), + Arrays.asList( + false, + true, + true, + null, + true + ) + ), + /** + * Test for DecimalFirstValueWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new DecimalFirstValueWithRetractAggFunction( + DecimalTypeInfo.of(DECIMAL_PRECISION, DECIMAL_SCALE)), + Arrays.asList( + Arrays.asList( + Decimal.castFrom("1", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("1000.000001", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-1", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-999.998999", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-999.999", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("999.999", DECIMAL_PRECISION, DECIMAL_SCALE) + ), + Arrays.asList( + null, + null, + null, + null, + null + ), + Arrays.asList( + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE) + ) + ), + Arrays.asList( + Decimal.castFrom("1", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE) + ) + ), + /** + * Test for StringLastValueWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new StringFirstValueWithRetractAggFunction(), + Arrays.asList( + Arrays.asList( + BinaryString.fromString("abc"), + BinaryString.fromString("def"), + BinaryString.fromString("ghi"), + null, + BinaryString.fromString("jkl"), + null, + BinaryString.fromString("zzz") + ), + Arrays.asList( + null, + null + ), + Arrays.asList( + null, + BinaryString.fromString("a") + ), + Arrays.asList( + BinaryString.fromString("x"), + null, + BinaryString.fromString("e") + ) + ), + Arrays.asList( + BinaryString.fromString("abc"), + null, + BinaryString.fromString("a"), + BinaryString.fromString("x") + ) + ) + + ); } - /** - * Test for DecimalFirstValueWithRetractAggFunction. - */ - public static class DecimalFirstValueWithRetractAggFunctionWithoutOrderTest extends - FirstValueWithRetractAggFunctionWithoutOrderTest { - - private int precision = 20; - private int scale = 6; - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - Decimal.castFrom("1", precision, scale), - Decimal.castFrom("1000.000001", precision, scale), - Decimal.castFrom("-1", precision, scale), - Decimal.castFrom("-999.998999", precision, scale), - null, - Decimal.castFrom("0", precision, scale), - Decimal.castFrom("-999.999", precision, scale), - null, - Decimal.castFrom("999.999", precision, scale) - ), - Arrays.asList( - null, - null, - null, - null, - null - ), - Arrays.asList( - null, - Decimal.castFrom("0", precision, scale) - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - Decimal.castFrom("1", precision, scale), - null, - Decimal.castFrom("0", precision, scale) - ); - } - - @Override - protected AggregateFunction getAggregator() { - return new DecimalFirstValueWithRetractAggFunction(DecimalTypeInfo.of(precision, scale)); - } + private static List> numberInputValueSets(Function strToValueFun) { + return Arrays.asList( + Arrays.asList( + strToValueFun.apply("1"), + null, + strToValueFun.apply("-99"), + strToValueFun.apply("3"), + null + ), + Arrays.asList( + null, + null, + null, + null + ), + Arrays.asList( + null, + strToValueFun.apply("10"), + null, + strToValueFun.apply("3") + ) + ); } - /** - * Test for StringFirstValueWithRetractAggFunction. - */ - public static class StringFirstValueWithRetractAggFunctionWithoutOrderTest extends - FirstValueWithRetractAggFunctionWithoutOrderTest { - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - BinaryString.fromString("abc"), - BinaryString.fromString("def"), - BinaryString.fromString("ghi"), - null, - BinaryString.fromString("jkl"), - null, - BinaryString.fromString("zzz") - ), - Arrays.asList( - null, - null - ), - Arrays.asList( - null, - BinaryString.fromString("a") - ), - Arrays.asList( - BinaryString.fromString("x"), - null, - BinaryString.fromString("e") - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - BinaryString.fromString("abc"), - null, - BinaryString.fromString("a"), - BinaryString.fromString("x") - ); - } - - @Override - protected AggregateFunction getAggregator() { - return new StringFirstValueWithRetractAggFunction(); - } + private static List numberExpectedResults(Function strToValueFun) { + return Arrays.asList( + strToValueFun.apply("1"), + null, + strToValueFun.apply("10") + ); } } diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/LastValueAggFunctionWithOrderTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/LastValueAggFunctionWithOrderTest.java index ebcb50e79537..1ffad8a62851 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/LastValueAggFunctionWithOrderTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/LastValueAggFunctionWithOrderTest.java @@ -33,435 +33,357 @@ import org.apache.flink.table.planner.functions.aggfunctions.LastValueAggFunction.StringLastValueAggFunction; import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + import java.util.Arrays; import java.util.List; +import java.util.function.Function; /** * Test case for built-in LastValue aggregate function. * This class tests `accumulate` method with order argument. */ -public abstract class LastValueAggFunctionWithOrderTest extends FirstLastValueAggFunctionWithOrderTestBase { - - /** - * Test LastValueAggFunction for number type. - */ - public abstract static class NumberLastValueAggFunctionWithOrderTest - extends LastValueAggFunctionWithOrderTest { - protected abstract T getValue(String v); +@RunWith(Parameterized.class) +public class LastValueAggFunctionWithOrderTest extends FirstLastValueAggFunctionWithOrderTestBase { - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - getValue("1"), - null, - getValue("-99"), - getValue("3"), - null, - getValue("3"), - getValue("2"), - getValue("-99") - ), - Arrays.asList( - null, - null, - null, - null - ), - Arrays.asList( - null, - getValue("10"), - null, - getValue("5") - ) - ); - } + @Parameterized.Parameter + public AggFunctionWithOrderTestSpec aggFunctionTestSpec; - @Override - protected List> getInputOrderSets() { - return Arrays.asList( - Arrays.asList( - 10L, - 2L, - 5L, - 6L, - 11L, - 3L, - 17L, - 5L - ), - Arrays.asList( - 8L, - 6L, - 9L, - 5L - ), - Arrays.asList( - null, - 6L, - 4L, - 3L - ) - ); - } + private static final int DECIMAL_PRECISION = 20; + private static final int DECIMAL_SCALE = 6; - @Override - protected List getExpectedResults() { - return Arrays.asList( - getValue("2"), - null, - getValue("10") - ); - } + @Override + protected List> getInputOrderSets() { + return aggFunctionTestSpec.inputOrderSets; } - /** - * Test for ByteLastValueAggFunction. - */ - public static class ByteLastValueAggFunctionWithOrderTest - extends NumberLastValueAggFunctionWithOrderTest { - - @Override - protected Byte getValue(String v) { - return Byte.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new ByteLastValueAggFunction(); - } + @Override + protected List> getInputValueSets() { + return aggFunctionTestSpec.inputValueSets; } - /** - * Test for ShortLastValueAggFunction. - */ - public static class ShortLastValueAggFunctionWithOrderTest - extends NumberLastValueAggFunctionWithOrderTest { - - @Override - protected Short getValue(String v) { - return Short.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new ShortLastValueAggFunction(); - } + @Override + protected List getExpectedResults() { + return aggFunctionTestSpec.expectedResults; } - /** - * Test for IntLastValueAggFunction. - */ - public static class IntLastValueAggFunctionWithOrderTest - extends NumberLastValueAggFunctionWithOrderTest { - - @Override - protected Integer getValue(String v) { - return Integer.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new IntLastValueAggFunction(); - } + @Override + protected AggregateFunction getAggregator() { + return aggFunctionTestSpec.aggregator; } - /** - * Test for LongLastValueAggFunction. - */ - public static class LongLastValueAggFunctionWithOrderTest - extends NumberLastValueAggFunctionWithOrderTest { - - @Override - protected Long getValue(String v) { - return Long.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new LongLastValueAggFunction(); - } + @Parameterized.Parameters(name = "{index}: {0}") + public static List testData() { + return Arrays.asList( + /** + * Test for ByteLastValueAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new ByteLastValueAggFunction(), + numberInputOrderSets(), + numberInputValueSets(Byte::valueOf), + numberExpectedResults(Byte::valueOf) + ), + /** + * Test for ShortLastValueAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new ShortLastValueAggFunction(), + numberInputOrderSets(), + numberInputValueSets(Short::valueOf), + numberExpectedResults(Short::valueOf) + ), + /** + * Test for IntLastValueAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new IntLastValueAggFunction(), + numberInputOrderSets(), + numberInputValueSets(Integer::valueOf), + numberExpectedResults(Integer::valueOf) + ), + /** + * Test for LongLastValueAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new LongLastValueAggFunction(), + numberInputOrderSets(), + numberInputValueSets(Long::valueOf), + numberExpectedResults(Long::valueOf) + ), + /** + * Test for FloatLastValueAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new FloatLastValueAggFunction(), + numberInputOrderSets(), + numberInputValueSets(Float::valueOf), + numberExpectedResults(Float::valueOf) + ), + /** + * Test for DoubleLastValueAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new DoubleLastValueAggFunction(), + numberInputOrderSets(), + numberInputValueSets(Double::valueOf), + numberExpectedResults(Double::valueOf) + ), + /** + * Test for BooleanLastValueAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new BooleanLastValueAggFunction(), + Arrays.asList( + Arrays.asList( + 6L, + 2L, + 3L + ), + Arrays.asList( + 1L, + 2L, + 3L + ), + Arrays.asList( + 10L, + 2L, + 5L, + 3L, + 11L, + 7L, + 5L + ), + Arrays.asList( + 6L, + 9L, + 5L + ), + Arrays.asList( + 4L, + 3L + ) + ), + Arrays.asList( + Arrays.asList( + false, + false, + false + ), + Arrays.asList( + true, + true, + true + ), + Arrays.asList( + true, + false, + null, + true, + false, + true, + null + ), + Arrays.asList( + null, + null, + null + ), + Arrays.asList( + null, + true + ) + ), + Arrays.asList( + false, + true, + false, + null, + true + ) + ), + /** + * Test for DecimalLastValueAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new DecimalLastValueAggFunction(DecimalTypeInfo.of(DECIMAL_PRECISION, DECIMAL_SCALE)), + Arrays.asList( + Arrays.asList( + 10L, + 2L, + 1L, + 5L, + null, + 3L, + 1L, + 5L, + 2L + ), + Arrays.asList( + 6L, + 5L, + null, + 8L, + null + ), + Arrays.asList( + 8L, + 6L + ) + ), + Arrays.asList( + Arrays.asList( + Decimal.castFrom("1", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("1000.000001", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-1", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-999.998999", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-999.999", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("999.999", DECIMAL_PRECISION, DECIMAL_SCALE) + ), + Arrays.asList( + null, + null, + null, + null, + null + ), + Arrays.asList( + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE) + ) + ), + Arrays.asList( + Decimal.castFrom("1", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE) + ) + ), + /** + * Test for StringLastValueAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new StringLastValueAggFunction(), + Arrays.asList( + Arrays.asList( + 10L, + 2L, + 5L, + null, + 3L, + 1L, + 15L + ), + Arrays.asList( + 6L, + 5L + ), + Arrays.asList( + 8L, + 6L + ), + Arrays.asList( + 6L, + 4L, + 3L + ) + ), + Arrays.asList( + Arrays.asList( + BinaryString.fromString("abc"), + BinaryString.fromString("def"), + BinaryString.fromString("ghi"), + null, + BinaryString.fromString("jkl"), + null, + BinaryString.fromString("zzz") + ), + Arrays.asList( + null, + null + ), + Arrays.asList( + null, + BinaryString.fromString("a") + ), + Arrays.asList( + BinaryString.fromString("x"), + null, + BinaryString.fromString("e") + ) + ), + Arrays.asList( + BinaryString.fromString("zzz"), + null, + BinaryString.fromString("a"), + BinaryString.fromString("x") + ) + ) + ); } - /** - * Test for FloatLastValueAggFunction. - */ - public static class FloatLastValueAggFunctionWithOrderTest - extends NumberLastValueAggFunctionWithOrderTest { - - @Override - protected Float getValue(String v) { - return Float.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new FloatLastValueAggFunction(); - } + private static List> numberInputOrderSets() { + return Arrays.asList( + Arrays.asList( + 10L, + 2L, + 5L, + 6L, + 11L, + 3L, + 17L, + 5L + ), + Arrays.asList( + 8L, + 6L, + 9L, + 5L + ), + Arrays.asList( + null, + 6L, + 4L, + 3L + ) + ); } - /** - * Test for DoubleLastValueAggFunction. - */ - public static class DoubleLastValueAggFunctionWithOrderTest - extends NumberLastValueAggFunctionWithOrderTest { - - @Override - protected Double getValue(String v) { - return Double.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new DoubleLastValueAggFunction(); - } + private static List> numberInputValueSets(Function strToValueFun) { + return Arrays.asList( + Arrays.asList( + strToValueFun.apply("1"), + null, + strToValueFun.apply("-99"), + strToValueFun.apply("3"), + null, + strToValueFun.apply("3"), + strToValueFun.apply("2"), + strToValueFun.apply("-99") + ), + Arrays.asList( + null, + null, + null, + null + ), + Arrays.asList( + null, + strToValueFun.apply("10"), + null, + strToValueFun.apply("5") + ) + ); } - /** - * Test for BooleanLastValueAggFunction. - */ - public static class BooleanLastValueAggFunctionWithOrderTest - extends LastValueAggFunctionWithOrderTest { - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - false, - false, - false - ), - Arrays.asList( - true, - true, - true - ), - Arrays.asList( - true, - false, - null, - true, - false, - true, - null - ), - Arrays.asList( - null, - null, - null - ), - Arrays.asList( - null, - true - )); - } - - @Override - protected List> getInputOrderSets() { - return Arrays.asList( - Arrays.asList( - 6L, - 2L, - 3L - ), - Arrays.asList( - 1L, - 2L, - 3L - ), - Arrays.asList( - 10L, - 2L, - 5L, - 3L, - 11L, - 7L, - 5L - ), - Arrays.asList( - 6L, - 9L, - 5L - ), - Arrays.asList( - 4L, - 3L - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - false, - true, - false, - null, - true - ); - } - - @Override - protected AggregateFunction getAggregator() { - return new BooleanLastValueAggFunction(); - } - } - - /** - * Test for DecimalLastValueAggFunction. - */ - public static class DecimalLastValueAggFunctionWithOrderTest - extends LastValueAggFunctionWithOrderTest { - - private int precision = 20; - private int scale = 6; - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - Decimal.castFrom("1", precision, scale), - Decimal.castFrom("1000.000001", precision, scale), - Decimal.castFrom("-1", precision, scale), - Decimal.castFrom("-999.998999", precision, scale), - null, - Decimal.castFrom("0", precision, scale), - Decimal.castFrom("-999.999", precision, scale), - null, - Decimal.castFrom("999.999", precision, scale) - ), - Arrays.asList( - null, - null, - null, - null, - null - ), - Arrays.asList( - null, - Decimal.castFrom("0", precision, scale) - ) - ); - } - - @Override - protected List> getInputOrderSets() { - return Arrays.asList( - Arrays.asList( - 10L, - 2L, - 1L, - 5L, - null, - 3L, - 1L, - 5L, - 2L - ), - Arrays.asList( - 6L, - 5L, - null, - 8L, - null - ), - Arrays.asList( - 8L, - 6L - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - Decimal.castFrom("1", precision, scale), - null, - Decimal.castFrom("0", precision, scale) - ); - } - - @Override - protected AggregateFunction getAggregator() { - return new DecimalLastValueAggFunction(DecimalTypeInfo.of(precision, scale)); - } - } - - /** - * Test for StringLastValueAggFunction. - */ - public static class StringLastValueAggFunctionWithOrderTest - extends LastValueAggFunctionWithOrderTest { - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - BinaryString.fromString("abc"), - BinaryString.fromString("def"), - BinaryString.fromString("ghi"), - null, - BinaryString.fromString("jkl"), - null, - BinaryString.fromString("zzz") - ), - Arrays.asList( - null, - null - ), - Arrays.asList( - null, - BinaryString.fromString("a") - ), - Arrays.asList( - BinaryString.fromString("x"), - null, - BinaryString.fromString("e") - ) - ); - } - - @Override - protected List> getInputOrderSets() { - return Arrays.asList( - Arrays.asList( - 10L, - 2L, - 5L, - null, - 3L, - 1L, - 15L - ), - Arrays.asList( - 6L, - 5L - ), - Arrays.asList( - 8L, - 6L - ), - Arrays.asList( - 6L, - 4L, - 3L - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - BinaryString.fromString("zzz"), - null, - BinaryString.fromString("a"), - BinaryString.fromString("x") - ); - } - - @Override - protected AggregateFunction getAggregator() { - return new StringLastValueAggFunction(); - } + private static List numberExpectedResults(Function strToValueFun) { + return Arrays.asList( + strToValueFun.apply("2"), + null, + strToValueFun.apply("10") + ); } } diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/LastValueAggFunctionWithoutOrderTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/LastValueAggFunctionWithoutOrderTest.java index 74e42ca1e876..96d199e959ef 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/LastValueAggFunctionWithoutOrderTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/LastValueAggFunctionWithoutOrderTest.java @@ -33,320 +33,244 @@ import org.apache.flink.table.planner.functions.aggfunctions.LastValueAggFunction.StringLastValueAggFunction; import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + import java.util.Arrays; import java.util.List; +import java.util.function.Function; /** * Test case for built-in LastValue aggregate function. * This class tests `accumulate` method without order argument. */ -public abstract class LastValueAggFunctionWithoutOrderTest extends AggFunctionTestBase { - - @Override - protected Class getAccClass() { - return GenericRow.class; - } - - /** - * Test LastValueAggFunction for number type. - */ - public abstract static class NumberLastValueAggFunctionWithoutOrderTest - extends LastValueAggFunctionWithoutOrderTest { - protected abstract T getValue(String v); - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - getValue("1"), - null, - getValue("-99"), - getValue("3"), - null - ), - Arrays.asList( - null, - null, - null, - null - ), - Arrays.asList( - null, - getValue("10"), - null, - getValue("3") - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - getValue("3"), - null, - getValue("3") - ); - } - } +@RunWith(Parameterized.class) +public class LastValueAggFunctionWithoutOrderTest extends AggFunctionTestBase { - /** - * Test for ByteLastValueAggFunction. - */ - public static class ByteLastValueAggFunctionWithoutOrderTest - extends NumberLastValueAggFunctionWithoutOrderTest { + @Parameterized.Parameter + public AggFunctionTestSpec aggFunctionTestSpec; - @Override - protected Byte getValue(String v) { - return Byte.valueOf(v); - } + private static final int DECIMAL_PRECISION = 20; + private static final int DECIMAL_SCALE = 6; - @Override - protected AggregateFunction getAggregator() { - return new ByteLastValueAggFunction(); - } - } - - /** - * Test for ShortLastValueAggFunction. - */ - public static class ShortLastValueAggFunctionWithoutOrderTest - extends NumberLastValueAggFunctionWithoutOrderTest { - - @Override - protected Short getValue(String v) { - return Short.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new ShortLastValueAggFunction(); - } - } - - /** - * Test for IntLastValueAggFunction. - */ - public static class IntLastValueAggFunctionWithoutOrderTest - extends NumberLastValueAggFunctionWithoutOrderTest { - - @Override - protected Integer getValue(String v) { - return Integer.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new IntLastValueAggFunction(); - } + @Override + protected List> getInputValueSets() { + return aggFunctionTestSpec.inputValueSets; } - /** - * Test for LongLastValueAggFunction. - */ - public static class LongLastValueAggFunctionWithoutOrderTest - extends NumberLastValueAggFunctionWithoutOrderTest { - - @Override - protected Long getValue(String v) { - return Long.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new LongLastValueAggFunction(); - } + @Override + protected List getExpectedResults() { + return aggFunctionTestSpec.expectedResults; } - /** - * Test for FloatLastValueAggFunction. - */ - public static class FloatLastValueAggFunctionWithoutOrderTest - extends NumberLastValueAggFunctionWithoutOrderTest { - - @Override - protected Float getValue(String v) { - return Float.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new FloatLastValueAggFunction(); - } + @Override + protected AggregateFunction getAggregator() { + return aggFunctionTestSpec.aggregator; } - /** - * Test for DoubleLastValueAggFunction. - */ - public static class DoubleLastValueAggFunctionWithoutOrderTest - extends NumberLastValueAggFunctionWithoutOrderTest { - - @Override - protected Double getValue(String v) { - return Double.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new DoubleLastValueAggFunction(); - } + @Override + protected Class getAccClass() { + return GenericRow.class; } - /** - * Test for BooleanLastValueAggFunction. - */ - public static class BooleanLastValueAggFunctionWithoutOrderTest extends - LastValueAggFunctionWithoutOrderTest { - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - false, - false, - false - ), - Arrays.asList( - true, - true, - true - ), - Arrays.asList( - true, - false, - null, - true, - false, - true, - null - ), - Arrays.asList( - null, - null, - null - ), - Arrays.asList( - null, - true - )); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - false, - true, - true, - null, - true - ); - } - - @Override - protected AggregateFunction getAggregator() { - return new BooleanLastValueAggFunction(); - } + @Parameterized.Parameters(name = "{index}: {0}") + public static List testData() { + return Arrays.asList( + /** + * Test for ByteLastValueAggFunction. + */ + new AggFunctionTestSpec<>( + new ByteLastValueAggFunction(), + numberInputValueSets(Byte::valueOf), + numberExpectedResults(Byte::valueOf) + ), + /** + * Test for ShortLastValueAggFunction. + */ + new AggFunctionTestSpec<>( + new ShortLastValueAggFunction(), + numberInputValueSets(Short::valueOf), + numberExpectedResults(Short::valueOf) + ), + /** + * Test for IntLastValueAggFunction. + */ + new AggFunctionTestSpec<>( + new IntLastValueAggFunction(), + numberInputValueSets(Integer::valueOf), + numberExpectedResults(Integer::valueOf) + ), + /** + * Test for LongLastValueAggFunction. + */ + new AggFunctionTestSpec<>( + new LongLastValueAggFunction(), + numberInputValueSets(Long::valueOf), + numberExpectedResults(Long::valueOf) + ), + /** + * Test for FloatLastValueAggFunction. + */ + new AggFunctionTestSpec<>( + new FloatLastValueAggFunction(), + numberInputValueSets(Float::valueOf), + numberExpectedResults(Float::valueOf) + ), + /** + * Test for DoubleLastValueAggFunction. + */ + new AggFunctionTestSpec<>( + new DoubleLastValueAggFunction(), + numberInputValueSets(Double::valueOf), + numberExpectedResults(Double::valueOf) + ), + /** + * Test for BooleanLastValueAggFunction. + */ + new AggFunctionTestSpec<>( + new BooleanLastValueAggFunction(), + Arrays.asList( + Arrays.asList( + false, + false, + false + ), + Arrays.asList( + true, + true, + true + ), + Arrays.asList( + true, + false, + null, + true, + false, + true, + null + ), + Arrays.asList( + null, + null, + null + ), + Arrays.asList( + null, + true + ) + ), + Arrays.asList( + false, + true, + true, + null, + true + ) + ), + /** + * Test for DecimalLastValueAggFunction. + */ + new AggFunctionTestSpec<>( + new DecimalLastValueAggFunction(DecimalTypeInfo.of(DECIMAL_PRECISION, DECIMAL_SCALE)), + Arrays.asList( + Arrays.asList( + Decimal.castFrom("1", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("1000.000001", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-1", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-999.998999", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-999.999", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("999.999", DECIMAL_PRECISION, DECIMAL_SCALE) + ), + Arrays.asList( + null, + null, + null, + null, + null + ), + Arrays.asList( + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE) + ) + ), + Arrays.asList( + Decimal.castFrom("999.999", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE) + ) + ), + /** + * Test for StringLastValueAggFunction. + */ + new AggFunctionTestSpec<>( + new StringLastValueAggFunction(), + Arrays.asList( + Arrays.asList( + BinaryString.fromString("abc"), + BinaryString.fromString("def"), + BinaryString.fromString("ghi"), + null, + BinaryString.fromString("jkl"), + null, + BinaryString.fromString("zzz") + ), + Arrays.asList( + null, + null + ), + Arrays.asList( + null, + BinaryString.fromString("a"), + null + ), + Arrays.asList( + BinaryString.fromString("x"), + null, + BinaryString.fromString("e") + ) + ), + Arrays.asList( + BinaryString.fromString("zzz"), + null, + BinaryString.fromString("a"), + BinaryString.fromString("e") + ) + ) + ); } - /** - * Test for DecimalLastValueAggFunction. - */ - public static class DecimalLastValueAggFunctionWithoutOrderTest extends - LastValueAggFunctionWithoutOrderTest { - - private int precision = 20; - private int scale = 6; - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - Decimal.castFrom("1", precision, scale), - Decimal.castFrom("1000.000001", precision, scale), - Decimal.castFrom("-1", precision, scale), - Decimal.castFrom("-999.998999", precision, scale), - null, - Decimal.castFrom("0", precision, scale), - Decimal.castFrom("-999.999", precision, scale), - null, - Decimal.castFrom("999.999", precision, scale) - ), - Arrays.asList( - null, - null, - null, - null, - null - ), - Arrays.asList( - null, - Decimal.castFrom("0", precision, scale) - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - Decimal.castFrom("999.999", precision, scale), - null, - Decimal.castFrom("0", precision, scale) - ); - } - - @Override - protected AggregateFunction getAggregator() { - return new DecimalLastValueAggFunction(DecimalTypeInfo.of(precision, scale)); - } + private static List> numberInputValueSets(Function strToValueFun) { + return Arrays.asList( + Arrays.asList( + strToValueFun.apply("1"), + null, + strToValueFun.apply("-99"), + strToValueFun.apply("3"), + null + ), + Arrays.asList( + null, + null, + null, + null + ), + Arrays.asList( + null, + strToValueFun.apply("10"), + null, + strToValueFun.apply("3") + ) + ); } - /** - * Test for StringLastValueAggFunction. - */ - public static class StringLastValueAggFunctionWithoutOrderTest extends - LastValueAggFunctionWithoutOrderTest { - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - BinaryString.fromString("abc"), - BinaryString.fromString("def"), - BinaryString.fromString("ghi"), - null, - BinaryString.fromString("jkl"), - null, - BinaryString.fromString("zzz") - ), - Arrays.asList( - null, - null - ), - Arrays.asList( - null, - BinaryString.fromString("a"), - null - ), - Arrays.asList( - BinaryString.fromString("x"), - null, - BinaryString.fromString("e") - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - BinaryString.fromString("zzz"), - null, - BinaryString.fromString("a"), - BinaryString.fromString("e") - ); - } - - @Override - protected AggregateFunction getAggregator() { - return new StringLastValueAggFunction(); - } + private static List numberExpectedResults(Function strToValueFun) { + return Arrays.asList( + strToValueFun.apply("3"), + null, + strToValueFun.apply("3") + ); } } diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/LastValueWithRetractAggFunctionWithOrderTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/LastValueWithRetractAggFunctionWithOrderTest.java index 303bb130eca7..861953e3a52f 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/LastValueWithRetractAggFunctionWithOrderTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/LastValueWithRetractAggFunctionWithOrderTest.java @@ -33,448 +33,372 @@ import org.apache.flink.table.planner.functions.aggfunctions.LastValueWithRetractAggFunction.StringLastValueWithRetractAggFunction; import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + import java.lang.reflect.Method; import java.util.Arrays; import java.util.List; +import java.util.function.Function; /** * Test case for built-in LastValue with retract aggregate function. * This class tests `accumulate` method with order argument. */ -public abstract class LastValueWithRetractAggFunctionWithOrderTest - extends FirstLastValueAggFunctionWithOrderTestBase { - - @Override - protected Method getRetractFunc() throws NoSuchMethodException { - return getAggregator().getClass().getMethod("retract", getAccClass(), Object.class, Long.class); - } +@RunWith(Parameterized.class) +public class LastValueWithRetractAggFunctionWithOrderTest extends FirstLastValueAggFunctionWithOrderTestBase { - /** - * Test LastValueWithRetractAggFunction for number type. - */ - public abstract static class NumberLastValueWithRetractAggFunctionWithOrderTest - extends LastValueWithRetractAggFunctionWithOrderTest { - protected abstract T getValue(String v); + @Parameterized.Parameter + public AggFunctionWithOrderTestSpec aggFunctionTestSpec; - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - getValue("1"), - null, - getValue("-99"), - getValue("3"), - null, - getValue("3"), - getValue("2"), - getValue("-99") - ), - Arrays.asList( - null, - null, - null, - null - ), - Arrays.asList( - null, - getValue("10"), - null, - getValue("5") - ) - ); - } + private static final int DECIMAL_PRECISION = 20; + private static final int DECIMAL_SCALE = 6; - @Override - protected List> getInputOrderSets() { - return Arrays.asList( - Arrays.asList( - 10L, - 2L, - 5L, - 6L, - 11L, - 13L, - 7L, - 5L - ), - Arrays.asList( - 8L, - 6L, - 9L, - 5L - ), - Arrays.asList( - null, - 6L, - 4L, - 3L - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - getValue("3"), - null, - getValue("10") - ); - } + @Override + protected List> getInputOrderSets() { + return aggFunctionTestSpec.inputOrderSets; } - /** - * Test for ByteLastValueWithRetractAggFunction. - */ - public static class ByteLastValueWithRetractAggFunctionWithOrderTest - extends NumberLastValueWithRetractAggFunctionWithOrderTest { - - @Override - protected Byte getValue(String v) { - return Byte.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new ByteLastValueWithRetractAggFunction(); - } + @Override + protected List> getInputValueSets() { + return aggFunctionTestSpec.inputValueSets; } - /** - * Test for ShortLastValueWithRetractAggFunction. - */ - public static class ShortLastValueWithRetractAggFunctionWithOrderTest - extends NumberLastValueWithRetractAggFunctionWithOrderTest { - - @Override - protected Short getValue(String v) { - return Short.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new ShortLastValueWithRetractAggFunction(); - } + @Override + protected List getExpectedResults() { + return aggFunctionTestSpec.expectedResults; } - /** - * Test for IntLastValueWithRetractAggFunction. - */ - public static class IntLastValueWithRetractAggFunctionWithOrderTest - extends NumberLastValueWithRetractAggFunctionWithOrderTest { - - @Override - protected Integer getValue(String v) { - return Integer.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new IntLastValueWithRetractAggFunction(); - } + @Override + protected AggregateFunction getAggregator() { + return aggFunctionTestSpec.aggregator; } - /** - * Test for LongLastValueWithRetractAggFunction. - */ - public static class LongLastValueWithRetractAggFunctionWithOrderTest - extends NumberLastValueWithRetractAggFunctionWithOrderTest { - - @Override - protected Long getValue(String v) { - return Long.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new LongLastValueWithRetractAggFunction(); - } + @Override + protected Method getRetractFunc() throws NoSuchMethodException { + return getAggregator().getClass().getMethod("retract", getAccClass(), Object.class, Long.class); } - /** - * Test for FloatLastValueWithRetractAggFunction. - */ - public static class FloatLastValueWithRetractAggFunctionWithOrderTest - extends NumberLastValueWithRetractAggFunctionWithOrderTest { - - @Override - protected Float getValue(String v) { - return Float.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new FloatLastValueWithRetractAggFunction(); - } + @Parameterized.Parameters(name = "{index}: {0}") + public static List testData() { + return Arrays.asList( + /** + * Test for ByteLastValueWithRetractAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new ByteLastValueWithRetractAggFunction(), + numberInputOrderSets(), + numberInputValueSets(Byte::valueOf), + numberExpectedResults(Byte::valueOf) + ), + /** + * Test for ShortLastValueWithRetractAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new ShortLastValueWithRetractAggFunction(), + numberInputOrderSets(), + numberInputValueSets(Short::valueOf), + numberExpectedResults(Short::valueOf) + ), + /** + * Test for IntLastValueWithRetractAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new IntLastValueWithRetractAggFunction(), + numberInputOrderSets(), + numberInputValueSets(Integer::valueOf), + numberExpectedResults(Integer::valueOf) + ), + /** + * Test for LongLastValueWithRetractAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new LongLastValueWithRetractAggFunction(), + numberInputOrderSets(), + numberInputValueSets(Long::valueOf), + numberExpectedResults(Long::valueOf) + ), + /** + * Test for FloatLastValueWithRetractAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new FloatLastValueWithRetractAggFunction(), + numberInputOrderSets(), + numberInputValueSets(Float::valueOf), + numberExpectedResults(Float::valueOf) + ), + /** + * Test for DoubleLastValueWithRetractAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new DoubleLastValueWithRetractAggFunction(), + numberInputOrderSets(), + numberInputValueSets(Double::valueOf), + numberExpectedResults(Double::valueOf) + ), + /** + * Test for BooleanLastValueWithRetractAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new BooleanLastValueWithRetractAggFunction(), + Arrays.asList( + Arrays.asList( + 6L, + 2L, + 3L + ), + Arrays.asList( + 1L, + 2L, + 3L + ), + Arrays.asList( + 10L, + 2L, + 5L, + 11L, + 3L, + 7L, + 5L + ), + Arrays.asList( + 6L, + 9L, + 5L + ), + Arrays.asList( + 4L, + 3L + ) + ), + Arrays.asList( + Arrays.asList( + false, + false, + false + ), + Arrays.asList( + true, + true, + true + ), + Arrays.asList( + true, + false, + null, + true, + false, + true, + null + ), + Arrays.asList( + null, + null, + null + ), + Arrays.asList( + null, + true + ) + ), + Arrays.asList( + false, + true, + true, + null, + true + ) + ), + /** + * Test for DecimalLastValueWithRetractAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new DecimalLastValueWithRetractAggFunction( + DecimalTypeInfo.of(DECIMAL_PRECISION, DECIMAL_SCALE)), + Arrays.asList( + Arrays.asList( + 10L, + 2L, + 1L, + 5L, + null, + 3L, + 1L, + 5L, + 2L + ), + Arrays.asList( + 6L, + 5L, + null, + 8L, + null + ), + Arrays.asList( + 8L, + 6L + ) + ), + Arrays.asList( + Arrays.asList( + Decimal.castFrom("1", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("1000.000001", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-1", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-999.998999", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-999.999", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("999.999", DECIMAL_PRECISION, DECIMAL_SCALE) + ), + Arrays.asList( + null, + null, + null, + null, + null + ), + Arrays.asList( + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE) + ) + ), + Arrays.asList( + Decimal.castFrom("1", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE) + ) + ), + /** + * Test for StringLastValueWithRetractAggFunction. + */ + new AggFunctionWithOrderTestSpec<>( + new StringLastValueWithRetractAggFunction(), + Arrays.asList( + Arrays.asList( + 10L, + 2L, + 5L, + null, + 3L, + 1L, + 5L, + 10L, + 15L, + 11L + ), + Arrays.asList( + 6L, + 5L + ), + Arrays.asList( + 8L, + 6L + ), + Arrays.asList( + 6L, + 4L, + 3L + ) + ), + Arrays.asList( + Arrays.asList( + BinaryString.fromString("abc"), + BinaryString.fromString("def"), + BinaryString.fromString("ghi"), + null, + BinaryString.fromString("jkl"), + null, + BinaryString.fromString("zzz"), + BinaryString.fromString("abc"), + BinaryString.fromString("def"), + BinaryString.fromString("abc") + ), + Arrays.asList( + null, + null + ), + Arrays.asList( + null, + BinaryString.fromString("a") + ), + Arrays.asList( + BinaryString.fromString("x"), + null, + BinaryString.fromString("e") + ) + ), + Arrays.asList( + BinaryString.fromString("def"), + null, + BinaryString.fromString("a"), + BinaryString.fromString("x") + ) + ) + ); } - /** - * Test for DoubleLastValueWithRetractAggFunction. - */ - public static class DoubleLastValueWithRetractAggFunctionWithOrderTest - extends NumberLastValueWithRetractAggFunctionWithOrderTest { - - @Override - protected Double getValue(String v) { - return Double.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new DoubleLastValueWithRetractAggFunction(); - } + private static List> numberInputOrderSets() { + return Arrays.asList( + Arrays.asList( + 10L, + 2L, + 5L, + 6L, + 11L, + 13L, + 7L, + 5L + ), + Arrays.asList( + 8L, + 6L, + 9L, + 5L + ), + Arrays.asList( + null, + 6L, + 4L, + 3L + ) + ); } - /** - * Test for BooleanLastValueWithRetractAggFunction. - */ - public static class BooleanLastValueWithRetractAggFunctionWithOrderTest - extends LastValueWithRetractAggFunctionWithOrderTest { - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - false, - false, - false - ), - Arrays.asList( - true, - true, - true - ), - Arrays.asList( - true, - false, - null, - true, - false, - true, - null - ), - Arrays.asList( - null, - null, - null - ), - Arrays.asList( - null, - true - )); - } - - @Override - protected List> getInputOrderSets() { - return Arrays.asList( - Arrays.asList( - 6L, - 2L, - 3L - ), - Arrays.asList( - 1L, - 2L, - 3L - ), - Arrays.asList( - 10L, - 2L, - 5L, - 11L, - 3L, - 7L, - 5L - ), - Arrays.asList( - 6L, - 9L, - 5L - ), - Arrays.asList( - 4L, - 3L - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - false, - true, - true, - null, - true - ); - } - - @Override - protected AggregateFunction getAggregator() { - return new BooleanLastValueWithRetractAggFunction(); - } + private static List> numberInputValueSets(Function strToValueFun) { + return Arrays.asList( + Arrays.asList( + strToValueFun.apply("1"), + null, + strToValueFun.apply("-99"), + strToValueFun.apply("3"), + null, + strToValueFun.apply("3"), + strToValueFun.apply("2"), + strToValueFun.apply("-99") + ), + Arrays.asList( + null, + null, + null, + null + ), + Arrays.asList( + null, + strToValueFun.apply("10"), + null, + strToValueFun.apply("5") + ) + ); } - /** - * Test for DecimalLastValueWithRetractAggFunction. - */ - public static class DecimalLastValueWithRetractAggFunctionWithOrderTest - extends LastValueWithRetractAggFunctionWithOrderTest { - - private int precision = 20; - private int scale = 6; - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - Decimal.castFrom("1", precision, scale), - Decimal.castFrom("1000.000001", precision, scale), - Decimal.castFrom("-1", precision, scale), - Decimal.castFrom("-999.998999", precision, scale), - null, - Decimal.castFrom("0", precision, scale), - Decimal.castFrom("-999.999", precision, scale), - null, - Decimal.castFrom("999.999", precision, scale) - ), - Arrays.asList( - null, - null, - null, - null, - null - ), - Arrays.asList( - null, - Decimal.castFrom("0", precision, scale) - ) - ); - } - - @Override - protected List> getInputOrderSets() { - return Arrays.asList( - Arrays.asList( - 10L, - 2L, - 1L, - 5L, - null, - 3L, - 1L, - 5L, - 2L - ), - Arrays.asList( - 6L, - 5L, - null, - 8L, - null - ), - Arrays.asList( - 8L, - 6L - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - Decimal.castFrom("1", precision, scale), - null, - Decimal.castFrom("0", precision, scale) - ); - } - - @Override - protected AggregateFunction getAggregator() { - return new DecimalLastValueWithRetractAggFunction(DecimalTypeInfo.of(precision, scale)); - } + private static List numberExpectedResults(Function strToValueFun) { + return Arrays.asList( + strToValueFun.apply("3"), + null, + strToValueFun.apply("10") + ); } +} - /** - * Test for StringLastValueWithRetractAggFunction. - */ - public static class StringLastValueWithRetractAggFunctionWithOrderTest - extends LastValueWithRetractAggFunctionWithOrderTest { - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - BinaryString.fromString("abc"), - BinaryString.fromString("def"), - BinaryString.fromString("ghi"), - null, - BinaryString.fromString("jkl"), - null, - BinaryString.fromString("zzz"), - BinaryString.fromString("abc"), - BinaryString.fromString("def"), - BinaryString.fromString("abc") - ), - Arrays.asList( - null, - null - ), - Arrays.asList( - null, - BinaryString.fromString("a") - ), - Arrays.asList( - BinaryString.fromString("x"), - null, - BinaryString.fromString("e") - ) - ); - } - - @Override - protected List> getInputOrderSets() { - return Arrays.asList( - Arrays.asList( - 10L, - 2L, - 5L, - null, - 3L, - 1L, - 5L, - 10L, - 15L, - 11L - ), - Arrays.asList( - 6L, - 5L - ), - Arrays.asList( - 8L, - 6L - ), - Arrays.asList( - 6L, - 4L, - 3L - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - BinaryString.fromString("def"), - null, - BinaryString.fromString("a"), - BinaryString.fromString("x") - ); - } - @Override - protected AggregateFunction getAggregator() { - return new StringLastValueWithRetractAggFunction(); - } - } -} diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/LastValueWithRetractAggFunctionWithoutOrderTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/LastValueWithRetractAggFunctionWithoutOrderTest.java index 6face9767c2d..774f82e00514 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/LastValueWithRetractAggFunctionWithoutOrderTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/LastValueWithRetractAggFunctionWithoutOrderTest.java @@ -33,325 +33,251 @@ import org.apache.flink.table.planner.functions.aggfunctions.LastValueWithRetractAggFunction.StringLastValueWithRetractAggFunction; import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + import java.lang.reflect.Method; import java.util.Arrays; import java.util.List; +import java.util.function.Function; /** * Test case for built-in LastValue with retract aggregate function. * This class tests `accumulate` method without order argument. */ -public abstract class LastValueWithRetractAggFunctionWithoutOrderTest extends AggFunctionTestBase { - - @Override - protected Class getAccClass() { - return GenericRow.class; - } +@RunWith(Parameterized.class) +public class LastValueWithRetractAggFunctionWithoutOrderTest extends AggFunctionTestBase { - @Override - protected Method getRetractFunc() throws NoSuchMethodException { - return getAggregator().getClass().getMethod("retract", getAccClass(), Object.class); - } - - /** - * Test LastValueWithRetractAggFunction for number type. - */ - public abstract static class NumberLastValueWithRetractAggFunctionWithoutOrderTest - extends LastValueWithRetractAggFunctionWithoutOrderTest { - protected abstract T getValue(String v); + @Parameterized.Parameter + public AggFunctionTestSpec aggFunctionTestSpec; - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - getValue("1"), - null, - getValue("-99"), - getValue("3"), - null - ), - Arrays.asList( - null, - null, - null, - null - ), - Arrays.asList( - null, - getValue("10"), - null, - getValue("3") - ) - ); - } + private static final int DECIMAL_PRECISION = 20; + private static final int DECIMAL_SCALE = 6; - @Override - protected List getExpectedResults() { - return Arrays.asList( - getValue("3"), - null, - getValue("3") - ); - } - } - - /** - * Test for ByteLastValueWithRetractAggFunction. - */ - public static class ByteLastValueWithRetractAggFunctionWithoutOrderTest - extends NumberLastValueWithRetractAggFunctionWithoutOrderTest { - - @Override - protected Byte getValue(String v) { - return Byte.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new ByteLastValueWithRetractAggFunction(); - } - } - - /** - * Test for ShortLastValueWithRetractAggFunction. - */ - public static class ShortLastValueWithRetractAggFunctionWithoutOrderTest - extends NumberLastValueWithRetractAggFunctionWithoutOrderTest { - - @Override - protected Short getValue(String v) { - return Short.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new ShortLastValueWithRetractAggFunction(); - } + @Override + protected List> getInputValueSets() { + return aggFunctionTestSpec.inputValueSets; } - /** - * Test for IntLastValueWithRetractAggFunction. - */ - public static class IntLastValueWithRetractAggFunctionWithoutOrderTest - extends NumberLastValueWithRetractAggFunctionWithoutOrderTest { - - @Override - protected Integer getValue(String v) { - return Integer.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new IntLastValueWithRetractAggFunction(); - } + @Override + protected List getExpectedResults() { + return aggFunctionTestSpec.expectedResults; } - /** - * Test for LongLastValueWithRetractAggFunction. - */ - public static class LongLastValueWithRetractAggFunctionWithoutOrderTest - extends NumberLastValueWithRetractAggFunctionWithoutOrderTest { - - @Override - protected Long getValue(String v) { - return Long.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new LongLastValueWithRetractAggFunction(); - } + @Override + protected AggregateFunction getAggregator() { + return aggFunctionTestSpec.aggregator; } - /** - * Test for FloatLastValueWithRetractAggFunction. - */ - public static class FloatLastValueWithRetractAggFunctionWithoutOrderTest - extends NumberLastValueWithRetractAggFunctionWithoutOrderTest { - - @Override - protected Float getValue(String v) { - return Float.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new FloatLastValueWithRetractAggFunction(); - } + @Override + protected Class getAccClass() { + return GenericRow.class; } - /** - * Test for DoubleLastValueWithRetractAggFunction. - */ - public static class DoubleLastValueWithRetractAggFunctionWithoutOrderTest - extends NumberLastValueWithRetractAggFunctionWithoutOrderTest { - - @Override - protected Double getValue(String v) { - return Double.valueOf(v); - } - - @Override - protected AggregateFunction getAggregator() { - return new DoubleLastValueWithRetractAggFunction(); - } + @Override + protected Method getRetractFunc() throws NoSuchMethodException { + return getAggregator().getClass().getMethod("retract", getAccClass(), Object.class); } - /** - * Test for BooleanLastValueWithRetractAggFunction. - */ - public static class BooleanLastValueWithRetractAggFunctionWithoutOrderTest extends - LastValueWithRetractAggFunctionWithoutOrderTest { - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - false, - false, - false - ), - Arrays.asList( - true, - true, - true - ), - Arrays.asList( - true, - false, - null, - true, - false, - true, - null - ), - Arrays.asList( - null, - null, - null - ), - Arrays.asList( - null, - true - )); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - false, - true, - true, - null, - true - ); - } - - @Override - protected AggregateFunction getAggregator() { - return new BooleanLastValueWithRetractAggFunction(); - } + @Parameterized.Parameters(name = "{index}: {0}") + public static List testData() { + return Arrays.asList( + /** + * Test for ByteLastValueWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new ByteLastValueWithRetractAggFunction(), + numberInputValueSets(Byte::valueOf), + numberExpectedResults(Byte::valueOf) + ), + /** + * Test for ShortLastValueWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new ShortLastValueWithRetractAggFunction(), + numberInputValueSets(Short::valueOf), + numberExpectedResults(Short::valueOf) + ), + /** + * Test for IntLastValueWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new IntLastValueWithRetractAggFunction(), + numberInputValueSets(Integer::valueOf), + numberExpectedResults(Integer::valueOf) + ), + /** + * Test for LongLastValueWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new LongLastValueWithRetractAggFunction(), + numberInputValueSets(Long::valueOf), + numberExpectedResults(Long::valueOf) + ), + /** + * Test for FloatLastValueWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new FloatLastValueWithRetractAggFunction(), + numberInputValueSets(Float::valueOf), + numberExpectedResults(Float::valueOf) + ), + /** + * Test for DoubleLastValueWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new DoubleLastValueWithRetractAggFunction(), + numberInputValueSets(Double::valueOf), + numberExpectedResults(Double::valueOf) + ), + /** + * Test for BooleanLastValueWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new BooleanLastValueWithRetractAggFunction(), + Arrays.asList( + Arrays.asList( + false, + false, + false + ), + Arrays.asList( + true, + true, + true + ), + Arrays.asList( + true, + false, + null, + true, + false, + true, + null + ), + Arrays.asList( + null, + null, + null + ), + Arrays.asList( + null, + true + ) + ), + Arrays.asList( + false, + true, + true, + null, + true + ) + ), + /** + * Test for DecimalLastValueWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new DecimalLastValueWithRetractAggFunction( + DecimalTypeInfo.of(DECIMAL_PRECISION, DECIMAL_SCALE)), + Arrays.asList( + Arrays.asList( + Decimal.castFrom("1", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("1000.000001", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-1", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-999.998999", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-999.999", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("999.999", DECIMAL_PRECISION, DECIMAL_SCALE) + ), + Arrays.asList( + null, + null, + null, + null, + null + ), + Arrays.asList( + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE) + ) + ), + Arrays.asList( + Decimal.castFrom("999.999", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE) + ) + ), + /** + * Test for StringLastValueWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new StringLastValueWithRetractAggFunction(), + Arrays.asList( + Arrays.asList( + BinaryString.fromString("abc"), + BinaryString.fromString("def"), + BinaryString.fromString("ghi"), + null, + BinaryString.fromString("jkl"), + null, + BinaryString.fromString("zzz") + ), + Arrays.asList( + null, + null + ), + Arrays.asList( + null, + BinaryString.fromString("a") + ), + Arrays.asList( + BinaryString.fromString("x"), + null, + BinaryString.fromString("e") + ) + ), + Arrays.asList( + BinaryString.fromString("zzz"), + null, + BinaryString.fromString("a"), + BinaryString.fromString("e") + ) + ) + + ); } - /** - * Test for DecimalLastValueWithRetractAggFunction. - */ - public static class DecimalLastValueWithRetractAggFunctionWithoutOrderTest extends - LastValueWithRetractAggFunctionWithoutOrderTest { - - private int precision = 20; - private int scale = 6; - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - Decimal.castFrom("1", precision, scale), - Decimal.castFrom("1000.000001", precision, scale), - Decimal.castFrom("-1", precision, scale), - Decimal.castFrom("-999.998999", precision, scale), - null, - Decimal.castFrom("0", precision, scale), - Decimal.castFrom("-999.999", precision, scale), - null, - Decimal.castFrom("999.999", precision, scale) - ), - Arrays.asList( - null, - null, - null, - null, - null - ), - Arrays.asList( - null, - Decimal.castFrom("0", precision, scale) - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - Decimal.castFrom("999.999", precision, scale), - null, - Decimal.castFrom("0", precision, scale) - ); - } - - @Override - protected AggregateFunction getAggregator() { - return new DecimalLastValueWithRetractAggFunction(DecimalTypeInfo.of(precision, scale)); - } + private static List> numberInputValueSets(Function strToValueFun) { + return Arrays.asList( + Arrays.asList( + strToValueFun.apply("1"), + null, + strToValueFun.apply("-99"), + strToValueFun.apply("3"), + null + ), + Arrays.asList( + null, + null, + null, + null + ), + Arrays.asList( + null, + strToValueFun.apply("10"), + null, + strToValueFun.apply("3") + ) + ); } - /** - * Test for StringLastValueWithRetractAggFunction. - */ - public static class StringLastValueWithRetractAggFunctionWithoutOrderTest extends - LastValueWithRetractAggFunctionWithoutOrderTest { - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - BinaryString.fromString("abc"), - BinaryString.fromString("def"), - BinaryString.fromString("ghi"), - null, - BinaryString.fromString("jkl"), - null, - BinaryString.fromString("zzz") - ), - Arrays.asList( - null, - null - ), - Arrays.asList( - null, - BinaryString.fromString("a") - ), - Arrays.asList( - BinaryString.fromString("x"), - null, - BinaryString.fromString("e") - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - BinaryString.fromString("zzz"), - null, - BinaryString.fromString("a"), - BinaryString.fromString("e") - ); - } - - @Override - protected AggregateFunction getAggregator() { - return new StringLastValueWithRetractAggFunction(); - } + private static List numberExpectedResults(Function strToValueFun) { + return Arrays.asList( + strToValueFun.apply("3"), + null, + strToValueFun.apply("3") + ); } } diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/ListAggWsWithRetractAggFunctionTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/ListAggWsWithRetractAggFunctionTest.java index 67ee8ffa6b28..3f845ce7e276 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/ListAggWsWithRetractAggFunctionTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/ListAggWsWithRetractAggFunctionTest.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.functions.aggfunctions; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.table.dataformat.BinaryString; import org.apache.flink.table.functions.AggregateFunction; @@ -105,14 +106,14 @@ protected Class getAccClass() { } @Override - protected void validateResult(E expected, E result) { + protected void validateResult(E expected, E result, TypeInformation typeInfo) { if (expected instanceof ListAggWsWithRetractAccumulator && result instanceof ListAggWsWithRetractAccumulator) { ListAggWsWithRetractAccumulator e = (ListAggWsWithRetractAccumulator) expected; ListAggWsWithRetractAccumulator r = (ListAggWsWithRetractAccumulator) result; assertEquals(e.list, r.list); assertEquals(e.list, r.list); } else { - super.validateResult(expected, result); + super.validateResult(expected, result, typeInfo); } } diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/MaxWithRetractAggFunctionTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/MaxWithRetractAggFunctionTest.java index 2bd0a7e3380d..9f930e4308bb 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/MaxWithRetractAggFunctionTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/MaxWithRetractAggFunctionTest.java @@ -36,529 +36,351 @@ import org.apache.flink.table.planner.functions.aggfunctions.MaxWithRetractAggFunction.TimestampMaxWithRetractAggFunction; import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + import java.lang.reflect.Method; import java.sql.Date; import java.sql.Time; import java.sql.Timestamp; import java.util.Arrays; import java.util.List; +import java.util.function.Function; /** * Test case for built-in Max with retraction aggregate function. */ -public abstract class MaxWithRetractAggFunctionTest extends AggFunctionTestBase> { - - @Override - protected Class getAccClass() { - return MaxWithRetractAccumulator.class; - } - - @Override - protected Method getRetractFunc() throws NoSuchMethodException { - return getAggregator().getClass().getMethod("retract", getAccClass(), Object.class); - } - - /** - * Test MaxWithRetractAggFunction for number type. - */ - public abstract static class NumberMaxWithRetractAggFunctionTest extends MaxWithRetractAggFunctionTest { - protected abstract T getMinValue(); - - protected abstract T getMaxValue(); - - protected abstract T getValue(String v); - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - getValue("1"), - null, - getMaxValue(), - getValue("-99"), - getValue("3"), - getValue("56"), - getValue("0"), - getMinValue(), - getValue("-20"), - getValue("17"), - null - ), - Arrays.asList( - null, - null, - null, - null, - null, - null - ), - Arrays.asList( - null, - getValue("10") - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - getMaxValue(), - null, - getValue("10") - ); - } - } - - /** - * Test for ByteMaxWithRetractAggFunction. - */ - public static class ByteMaxWithRetractAggFunctionTest extends NumberMaxWithRetractAggFunctionTest { - - @Override - protected Byte getMinValue() { - return Byte.MIN_VALUE + 1; - } - - @Override - protected Byte getMaxValue() { - return Byte.MAX_VALUE - 1; - } +@RunWith(Parameterized.class) +public class MaxWithRetractAggFunctionTest extends AggFunctionTestBase> { - @Override - protected Byte getValue(String v) { - return Byte.valueOf(v); - } + @Parameterized.Parameter + public AggFunctionTestSpec> aggFunctionTestSpec; - @Override - protected AggregateFunction> getAggregator() { - return new ByteMaxWithRetractAggFunction(); - } - } - - /** - * Test for ShortMaxWithRetractAggFunction. - */ - public static class ShortMaxWithRetractAggFunctionTest extends NumberMaxWithRetractAggFunctionTest { - - @Override - protected Short getMinValue() { - return Short.MIN_VALUE + 1; - } - - @Override - protected Short getMaxValue() { - return Short.MAX_VALUE - 1; - } - - @Override - protected Short getValue(String v) { - return Short.valueOf(v); - } - - @Override - protected AggregateFunction> getAggregator() { - return new ShortMaxWithRetractAggFunction(); - } - } - - /** - * Test for IntMaxWithRetractAggFunction. - */ - public static class IntMaxWithRetractAggFunctionTest extends NumberMaxWithRetractAggFunctionTest { - - @Override - protected Integer getMinValue() { - return Integer.MIN_VALUE + 1; - } - - @Override - protected Integer getMaxValue() { - return Integer.MAX_VALUE - 1; - } - - @Override - protected Integer getValue(String v) { - return Integer.valueOf(v); - } - - @Override - protected AggregateFunction> getAggregator() { - return new IntMaxWithRetractAggFunction(); - } - } - - /** - * Test for LongMaxWithRetractAggFunction. - */ - public static class LongMaxWithRetractAggFunctionTest extends NumberMaxWithRetractAggFunctionTest { - - @Override - protected Long getMinValue() { - return Long.MIN_VALUE + 1; - } - - @Override - protected Long getMaxValue() { - return Long.MAX_VALUE - 1; - } - - @Override - protected Long getValue(String v) { - return Long.valueOf(v); - } - - @Override - protected AggregateFunction> getAggregator() { - return new LongMaxWithRetractAggFunction(); - } - } + private static final int DECIMAL_PRECISION = 20; + private static final int DECIMAL_SCALE = 6; - /** - * Test for FloatMaxWithRetractAggFunction. - */ - public static class FloatMaxWithRetractAggFunctionTest extends NumberMaxWithRetractAggFunctionTest { - - @Override - protected Float getMinValue() { - return -Float.MAX_VALUE / 2; - } - - @Override - protected Float getMaxValue() { - return Float.MAX_VALUE / 2; - } - - @Override - protected Float getValue(String v) { - return Float.valueOf(v); - } - - @Override - protected AggregateFunction> getAggregator() { - return new FloatMaxWithRetractAggFunction(); - } + @Override + protected List> getInputValueSets() { + return aggFunctionTestSpec.inputValueSets; } - /** - * Test for DoubleMaxWithRetractAggFunction. - */ - public static class DoubleMaxWithRetractAggFunctionTest extends NumberMaxWithRetractAggFunctionTest { - - @Override - protected Double getMinValue() { - return -Double.MAX_VALUE / 2; - } - - @Override - protected Double getMaxValue() { - return Double.MAX_VALUE / 2; - } - - @Override - protected Double getValue(String v) { - return Double.valueOf(v); - } - - @Override - protected AggregateFunction> getAggregator() { - return new DoubleMaxWithRetractAggFunction(); - } + @Override + protected List getExpectedResults() { + return aggFunctionTestSpec.expectedResults; } - /** - * Test for BooleanMaxWithRetractAggFunction. - */ - public static class BooleanMaxWithRetractAggFunctionTest extends MaxWithRetractAggFunctionTest { - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - false, - false, - false - ), - Arrays.asList( - true, - true, - true - ), - Arrays.asList( - true, - false, - null, - true, - false, - true, - null - ), - Arrays.asList( - null, - null, - null - ), - Arrays.asList( - null, - true - )); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - false, - true, - true, - null, - true - ); - } - - @Override - protected AggregateFunction> getAggregator() { - return new BooleanMaxWithRetractAggFunction(); - } - - @Override - protected Class getAccClass() { - return MaxWithRetractAccumulator.class; - } - - @Override - protected Method getRetractFunc() throws NoSuchMethodException { - return getAggregator().getClass().getMethod("retract", getAccClass(), Object.class); - } + @Override + protected AggregateFunction> getAggregator() { + return aggFunctionTestSpec.aggregator; } - /** - * Test for DecimalMaxWithRetractAggFunction. - */ - public static class DecimalMaxWithRetractAggFunctionTest extends MaxWithRetractAggFunctionTest { - - private int precision = 20; - private int scale = 6; - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - Decimal.castFrom("1", precision, scale), - Decimal.castFrom("1000.000001", precision, scale), - Decimal.castFrom("-1", precision, scale), - Decimal.castFrom("-999.998999", precision, scale), - null, - Decimal.castFrom("0", precision, scale), - Decimal.castFrom("-999.999", precision, scale), - null, - Decimal.castFrom("999.999", precision, scale) - ), - Arrays.asList( - null, - null, - null, - null, - null - ), - Arrays.asList( - null, - Decimal.castFrom("0", precision, scale) - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - Decimal.castFrom("1000.000001", precision, scale), - null, - Decimal.castFrom("0", precision, scale) - ); - } - - @Override - protected AggregateFunction> getAggregator() { - return new DecimalMaxWithRetractAggFunction(DecimalTypeInfo.of(precision, scale)); - } + @Override + protected Class getAccClass() { + return MaxWithRetractAccumulator.class; } - /** - * Test for StringMaxWithRetractAggFunction. - */ - public static class StringMaxWithRetractAggFunctionTest extends MaxWithRetractAggFunctionTest { - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - BinaryString.fromString("abc"), - BinaryString.fromString("def"), - BinaryString.fromString("ghi"), - null, - BinaryString.fromString("jkl"), - null, - BinaryString.fromString("zzz") - ), - Arrays.asList( - null, - null - ), - Arrays.asList( - null, - BinaryString.fromString("a") - ), - Arrays.asList( - BinaryString.fromString("x"), - null, - BinaryString.fromString("e") - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - BinaryString.fromString("zzz"), - null, - BinaryString.fromString("a"), - BinaryString.fromString("x") - ); - } - - @Override - protected AggregateFunction> getAggregator() { - return new StringMaxWithRetractAggFunction(); - } + @Override + protected Method getRetractFunc() throws NoSuchMethodException { + return getAggregator().getClass().getMethod("retract", getAccClass(), Object.class); } - /** - * Test for TimestampMaxWithRetractAggFunction. - */ - public static class TimestampMaxWithRetractAggFunctionTest extends MaxWithRetractAggFunctionTest { - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - new Timestamp(0), - new Timestamp(1000), - new Timestamp(100), - null, - new Timestamp(10) - ), - Arrays.asList( - null, - null, - null, - null, - null - ), - Arrays.asList( - null, - new Timestamp(1) - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - new Timestamp(1000), - null, - new Timestamp(1) - ); - } - - @Override - protected AggregateFunction> getAggregator() { - return new TimestampMaxWithRetractAggFunction(); - } + @Parameterized.Parameters(name = "{index}: {0}") + public static List testData() { + return Arrays.asList( + /** + * Test for ByteMaxWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new ByteMaxWithRetractAggFunction(), + numberInputValueSets((byte) (Byte.MIN_VALUE + 1), (byte) (Byte.MAX_VALUE - 1), Byte::valueOf), + numberExpectedResults((byte) (Byte.MAX_VALUE - 1), Byte::valueOf) + ), + /** + * Test for ShortMaxWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new ShortMaxWithRetractAggFunction(), + numberInputValueSets( + (short) (Short.MIN_VALUE + 1), (short) (Short.MAX_VALUE - 1), Short::valueOf), + numberExpectedResults((short) (Short.MAX_VALUE - 1), Short::valueOf) + ), + /** + * Test for IntMaxWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new IntMaxWithRetractAggFunction(), + numberInputValueSets(Integer.MIN_VALUE + 1, Integer.MAX_VALUE - 1, Integer::valueOf), + numberExpectedResults(Integer.MAX_VALUE - 1, Integer::valueOf) + ), + /** + * Test for LongMaxWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new LongMaxWithRetractAggFunction(), + numberInputValueSets(Long.MIN_VALUE + 1L, Long.MAX_VALUE - 1L, Long::valueOf), + numberExpectedResults(Long.MAX_VALUE - 1L, Long::valueOf) + ), + /** + * Test for FloatMaxWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new FloatMaxWithRetractAggFunction(), + numberInputValueSets((-Float.MAX_VALUE / 2), (Float.MAX_VALUE / 2), Float::valueOf), + numberExpectedResults((Float.MAX_VALUE / 2), Float::valueOf) + ), + /** + * Test for DoubleMaxWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new DoubleMaxWithRetractAggFunction(), + numberInputValueSets((-Double.MAX_VALUE / 2), (Double.MAX_VALUE / 2), Double::valueOf), + numberExpectedResults((Double.MAX_VALUE / 2), Double::valueOf) + ), + /** + * Test for BooleanMaxWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new BooleanMaxWithRetractAggFunction(), + Arrays.asList( + Arrays.asList( + false, + false, + false + ), + Arrays.asList( + true, + true, + true + ), + Arrays.asList( + true, + false, + null, + true, + false, + true, + null + ), + Arrays.asList( + null, + null, + null + ), + Arrays.asList( + null, + true + ) + ), + Arrays.asList( + false, + true, + true, + null, + true + ) + ), + /** + * Test for DecimalMaxWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new DecimalMaxWithRetractAggFunction(DecimalTypeInfo.of(DECIMAL_PRECISION, DECIMAL_SCALE)), + Arrays.asList( + Arrays.asList( + Decimal.castFrom("1", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("1000.000001", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-1", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-999.998999", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE), + Decimal.castFrom("-999.999", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("999.999", DECIMAL_PRECISION, DECIMAL_SCALE) + ), + Arrays.asList( + null, + null, + null, + null, + null + ), + Arrays.asList( + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE) + ) + ), + Arrays.asList( + Decimal.castFrom("1000.000001", DECIMAL_PRECISION, DECIMAL_SCALE), + null, + Decimal.castFrom("0", DECIMAL_PRECISION, DECIMAL_SCALE) + ) + ), + /** + * Test for StringMaxWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new StringMaxWithRetractAggFunction(), + Arrays.asList( + Arrays.asList( + BinaryString.fromString("abc"), + BinaryString.fromString("def"), + BinaryString.fromString("ghi"), + null, + BinaryString.fromString("jkl"), + null, + BinaryString.fromString("zzz") + ), + Arrays.asList( + null, + null + ), + Arrays.asList( + null, + BinaryString.fromString("a") + ), + Arrays.asList( + BinaryString.fromString("x"), + null, + BinaryString.fromString("e") + ) + ), + Arrays.asList( + BinaryString.fromString("zzz"), + null, + BinaryString.fromString("a"), + BinaryString.fromString("x") + ) + ), + /** + * Test for TimestampMaxWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new TimestampMaxWithRetractAggFunction(), + Arrays.asList( + Arrays.asList( + new Timestamp(0), + new Timestamp(1000), + new Timestamp(100), + null, + new Timestamp(10) + ), + Arrays.asList( + null, + null, + null, + null, + null + ), + Arrays.asList( + null, + new Timestamp(1) + ) + ), + Arrays.asList( + new Timestamp(1000), + null, + new Timestamp(1) + ) + ), + /** + * Test for DateMaxWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new DateMaxWithRetractAggFunction(), + Arrays.asList( + Arrays.asList( + new Date(0), + new Date(1000), + new Date(100), + null, + new Date(10) + ), + Arrays.asList( + null, + null, + null, + null, + null + ), + Arrays.asList( + null, + new Date(1) + ) + ), + Arrays.asList( + new Date(1000), + null, + new Date(1) + ) + ), + /** + * Test for TimeMaxWithRetractAggFunction. + */ + new AggFunctionTestSpec<>( + new TimeMaxWithRetractAggFunction(), + Arrays.asList( + Arrays.asList( + new Time(0), + new Time(1000), + new Time(100), + null, + new Time(10) + ), + Arrays.asList( + null, + null, + null, + null, + null + ), + Arrays.asList( + null, + new Time(1) + ) + ), + Arrays.asList( + new Time(1000), + null, + new Time(1) + ) + ) + ); } - /** - * Test for DateMaxWithRetractAggFunction. - */ - public static class DateMaxWithRetractAggFunctionTest extends MaxWithRetractAggFunctionTest { - - @Override - protected List> getInputValueSets() { - return Arrays.asList( - Arrays.asList( - new Date(0), - new Date(1000), - new Date(100), - null, - new Date(10) - ), - Arrays.asList( - null, - null, - null, - null, - null - ), - Arrays.asList( - null, - new Date(1) - ) - ); - } - - @Override - protected List getExpectedResults() { - return Arrays.asList( - new Date(1000), - null, - new Date(1) - ); - } - - @Override - protected AggregateFunction> getAggregator() { - return new DateMaxWithRetractAggFunction(); - } + private static List> numberInputValueSets(N minValue, N maxValue, Function strToValueFun) { + return Arrays.asList( + Arrays.asList( + strToValueFun.apply("1"), + null, + maxValue, + strToValueFun.apply("-99"), + strToValueFun.apply("3"), + strToValueFun.apply("56"), + strToValueFun.apply("0"), + minValue, + strToValueFun.apply("-20"), + strToValueFun.apply("17"), + null + ), + Arrays.asList( + null, + null, + null, + null, + null, + null + ), + Arrays.asList( + null, + strToValueFun.apply("10") + ) + ); } - /** - * Test for TimeMaxWithRetractAggFunction. - */ - public static class TimeMaxWithRetractAggFunctionTest extends MaxWithRetractAggFunctionTest